From 804949d519e2caa293a409d84b4e6190c1105444 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 8 Feb 2015 14:55:07 -0800 Subject: [PATCH 001/817] [SQL] Set sessionState in QueryExecution. This PR sets the SessionState in HiveContext's QueryExecution. So, we can make sure that SessionState.get can return the SessionState every time. Author: Yin Huai Closes #4445 from yhuai/setSessionState and squashes the following commits: 769c9f1 [Yin Huai] Remove unused import. 439f329 [Yin Huai] Try again. 427a0c9 [Yin Huai] Set SessionState everytime when we create a QueryExecution in HiveContext. a3b7793 [Yin Huai] Set sessionState when dealing with CreateTableAsSelect. --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ad37b7d0e6f59..2c00659496972 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -424,6 +424,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends super.QueryExecution(logicalPlan) { + // Like what we do in runHive, makes sure the session represented by the + // `sessionState` field is activated. + if (SessionState.get() != sessionState) { + SessionState.start(sessionState) + } /** * Returns the result as a hive compatible sequence of strings. For native commands, the From 5c299c58fb9a5434a40be82150d4725bba805adf Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 8 Feb 2015 16:26:20 -0800 Subject: [PATCH 002/817] [SPARK-5598][MLLIB] model save/load for ALS following #4233. jkbradley Author: Xiangrui Meng Closes #4422 from mengxr/SPARK-5598 and squashes the following commits: a059394 [Xiangrui Meng] SaveLoad not extending Loader 14b7ea6 [Xiangrui Meng] address comments f487cb2 [Xiangrui Meng] add unit tests 62fc43c [Xiangrui Meng] implement save/load for MFM --- .../spark/mllib/recommendation/ALS.scala | 2 +- .../MatrixFactorizationModel.scala | 82 ++++++++++++++++++- .../MatrixFactorizationModelSuite.scala | 19 +++++ 3 files changed, 100 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 4bb28d1b1e071..caacab943030b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.recommendation import org.apache.spark.Logging -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.recommendation.{ALS => NewALS} import org.apache.spark.rdd.RDD 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 ed2f8b41bcae5..9ff06ac362a31 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,13 +17,17 @@ package org.apache.spark.mllib.recommendation +import java.io.IOException import java.lang.{Integer => JavaInteger} +import org.apache.hadoop.fs.Path import org.jblas.DoubleMatrix -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.storage.StorageLevel /** @@ -41,7 +45,8 @@ import org.apache.spark.storage.StorageLevel class MatrixFactorizationModel( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) extends Serializable with Logging { + val productFeatures: RDD[(Int, Array[Double])]) + extends Saveable with Serializable with Logging { require(rank > 0) validateFeatures("User", userFeatures) @@ -125,6 +130,12 @@ class MatrixFactorizationModel( recommend(productFeatures.lookup(product).head, userFeatures, num) .map(t => Rating(t._1, product, t._2)) + protected override val formatVersion: String = "1.0" + + override def save(sc: SparkContext, path: String): Unit = { + MatrixFactorizationModel.SaveLoadV1_0.save(this, path) + } + private def recommend( recommendToFeatures: Array[Double], recommendableFeatures: RDD[(Int, Array[Double])], @@ -136,3 +147,70 @@ class MatrixFactorizationModel( scored.top(num)(Ordering.by(_._2)) } } + +object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { + + import org.apache.spark.mllib.util.Loader._ + + override def load(sc: SparkContext, path: String): MatrixFactorizationModel = { + val (loadedClassName, formatVersion, metadata) = loadMetadata(sc, path) + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, formatVersion) match { + case (className, "1.0") if className == classNameV1_0 => + SaveLoadV1_0.load(sc, path) + case _ => + throw new IOException("MatrixFactorizationModel.load did not recognize model with" + + s"(class: $loadedClassName, version: $formatVersion). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } + + private[recommendation] + object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private[recommendation] + val thisClassName = "org.apache.spark.mllib.recommendation.MatrixFactorizationModel" + + /** + * Saves a [[MatrixFactorizationModel]], where user features are saved under `data/users` and + * product features are saved under `data/products`. + */ + def save(model: MatrixFactorizationModel, path: String): Unit = { + val sc = model.userFeatures.sparkContext + val sqlContext = new SQLContext(sc) + import sqlContext.implicits.createDataFrame + val metadata = (thisClassName, thisFormatVersion, model.rank) + val metadataRDD = sc.parallelize(Seq(metadata), 1).toDataFrame("class", "version", "rank") + metadataRDD.toJSON.saveAsTextFile(metadataPath(path)) + model.userFeatures.toDataFrame("id", "features").saveAsParquetFile(userPath(path)) + model.productFeatures.toDataFrame("id", "features").saveAsParquetFile(productPath(path)) + } + + def load(sc: SparkContext, path: String): MatrixFactorizationModel = { + val sqlContext = new SQLContext(sc) + val (className, formatVersion, metadata) = loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + val rank = metadata.select("rank").first().getInt(0) + val userFeatures = sqlContext.parquetFile(userPath(path)) + .map { case Row(id: Int, features: Seq[Double]) => + (id, features.toArray) + } + val productFeatures = sqlContext.parquetFile(productPath(path)) + .map { case Row(id: Int, features: Seq[Double]) => + (id, features.toArray) + } + new MatrixFactorizationModel(rank, userFeatures, productFeatures) + } + + private def userPath(path: String): String = { + new Path(dataPath(path), "user").toUri.toString + } + + private def productPath(path: String): String = { + new Path(dataPath(path), "product").toUri.toString + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala index b9caecc904a23..9801e87576744 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils class MatrixFactorizationModelSuite extends FunSuite with MLlibTestSparkContext { @@ -53,4 +54,22 @@ class MatrixFactorizationModelSuite extends FunSuite with MLlibTestSparkContext new MatrixFactorizationModel(rank, userFeatures, prodFeatures1) } } + + test("save/load") { + val model = new MatrixFactorizationModel(rank, userFeatures, prodFeatures) + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + def collect(features: RDD[(Int, Array[Double])]): Set[(Int, Seq[Double])] = { + features.mapValues(_.toSeq).collect().toSet + } + try { + model.save(sc, path) + val newModel = MatrixFactorizationModel.load(sc, path) + assert(newModel.rank === rank) + assert(collect(newModel.userFeatures) === collect(userFeatures)) + assert(collect(newModel.productFeatures) === collect(prodFeatures)) + } finally { + Utils.deleteRecursively(tempDir) + } + } } From 56aff4bd6c7c9d18f4f962025708f20a4a82dcf0 Mon Sep 17 00:00:00 2001 From: Sam Halliday Date: Sun, 8 Feb 2015 16:34:26 -0800 Subject: [PATCH 003/817] SPARK-5665 [DOCS] Update netlib-java documentation I am the author of netlib-java and I found this documentation to be out of date. Some main points: 1. Breeze has not depended on jBLAS for some time 2. netlib-java provides a pure JVM implementation as the fallback (the original docs did not appear to be aware of this, claiming that gfortran was necessary) 3. The licensing issue is not just about LGPL: optimised natives have proprietary licenses. Building with the LGPL flag turned on really doesn't help you get past this. 4. I really think it's best to direct people to my detailed setup guide instead of trying to compress it into one sentence. It is different for each architecture, each OS, and for each backend. I hope this helps to clear things up :smile: Author: Sam Halliday Author: Sam Halliday Closes #4448 from fommil/patch-1 and squashes the following commits: 18cda11 [Sam Halliday] remove link to skillsmatters at request of @mengxr a35e4a9 [Sam Halliday] reword netlib-java/breeze docs --- docs/mllib-guide.md | 41 ++++++++++++++++++++++++----------------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 7779fbc9c49e4..3d32d03e35c62 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -56,25 +56,32 @@ See the **[spark.ml programming guide](ml-guide.html)** for more information on # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), -which depends on [netlib-java](https://github.com/fommil/netlib-java), -and [jblas](https://github.com/mikiobraun/jblas). -`netlib-java` and `jblas` depend on native Fortran routines. -You need to install the +MLlib uses the linear algebra package +[Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java) for optimised +numerical processing. If natives are not available at runtime, you +will see a warning message and a pure JVM implementation will be used +instead. + +To learn more about the benefits and background of system optimised +natives, you may wish to watch Sam Halliday's ScalaX talk on +[High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/)). + +Due to licensing issues with runtime proprietary binaries, we do not +include `netlib-java`'s native proxies by default. To configure +`netlib-java` / Breeze to use system optimised binaries, include +`com.github.fommil.netlib:all:1.1.2` (or build Spark with +`-Pnetlib-lgpl`) as a dependency of your project and read the +[netlib-java](https://github.com/fommil/netlib-java) documentation for +your platform's additional installation instructions. + +MLlib also uses [jblas](https://github.com/mikiobraun/jblas) which +will require you to install the [gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not already present on your nodes. -MLlib will throw a linking error if it cannot detect these libraries automatically. -Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's -dependency set under default settings. -If no native library is available at runtime, you will see a warning message. -To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or -include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. -If you want to use optimized BLAS/LAPACK libraries such as -[OpenBLAS](http://www.openblas.net/), please link its shared libraries to -`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively. -BLAS/LAPACK libraries on worker nodes should be built without multithreading. - -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. + +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) +version 1.4 or newer. --- From a052ed42501fee3641348337505b6176426653c4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 8 Feb 2015 18:56:51 -0800 Subject: [PATCH 004/817] [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in tabular format. An example: ``` year month AVG('Adj Close) MAX('Adj Close) 1980 12 0.503218 0.595103 1981 01 0.523289 0.570307 1982 02 0.436504 0.475256 1983 03 0.410516 0.442194 1984 04 0.450090 0.483521 ``` Author: Reynold Xin Closes #4416 from rxin/SPARK-5643 and squashes the following commits: d0e0d6e [Reynold Xin] [SQL] Minor update to data source and statistics documentation. 269da83 [Reynold Xin] Updated isLocal comment. 2cf3c27 [Reynold Xin] Moved logic into optimizer. 1a04d8b [Reynold Xin] [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in columnar format. --- .../sql/catalyst/optimizer/Optimizer.scala | 18 +++++- .../catalyst/plans/logical/LogicalPlan.scala | 7 ++- .../ConvertToLocalRelationSuite.scala | 57 +++++++++++++++++++ .../org/apache/spark/sql/DataFrame.scala | 21 ++++++- .../org/apache/spark/sql/DataFrameImpl.scala | 41 +++++++++++-- .../apache/spark/sql/IncomputableColumn.scala | 6 +- .../spark/sql/execution/basicOperators.scala | 7 +-- .../apache/spark/sql/sources/interfaces.scala | 15 +++-- 8 files changed, 151 insertions(+), 21 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8c8f2896eb99b..3bc48c95c5653 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -50,7 +50,9 @@ object DefaultOptimizer extends Optimizer { CombineFilters, PushPredicateThroughProject, PushPredicateThroughJoin, - ColumnPruning) :: Nil + ColumnPruning) :: + Batch("LocalRelation", FixedPoint(100), + ConvertToLocalRelation) :: Nil } /** @@ -610,3 +612,17 @@ object DecimalAggregates extends Rule[LogicalPlan] { DecimalType(prec + 4, scale + 4)) } } + +/** + * Converts local operations (i.e. ones that don't require data exchange) on LocalRelation to + * another LocalRelation. + * + * This is relatively simple as it currently handles only a single case: Project. + */ +object ConvertToLocalRelation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Project(projectList, LocalRelation(output, data)) => + val projection = new InterpretedProjection(projectList, output) + LocalRelation(projectList.map(_.toAttribute), data.map(projection)) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 8d30528328946..7cf4b81274906 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -29,12 +29,15 @@ import org.apache.spark.sql.catalyst.trees /** * Estimates of various statistics. The default estimation logic simply lazily multiplies the * corresponding statistic produced by the children. To override this behavior, override - * `statistics` and assign it an overriden version of `Statistics`. + * `statistics` and assign it an overridden version of `Statistics`. * - * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the + * '''NOTE''': concrete and/or overridden versions of statistics fields should pay attention to the * performance of the implementations. The reason is that estimations might get triggered in * performance-critical processes, such as query plan planning. * + * Note that we are using a BigInt here since it is easy to overflow a 64-bit integer in + * cardinality estimation (e.g. cartesian joins). + * * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it * defaults to the product of children's `sizeInBytes`. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala new file mode 100644 index 0000000000000..cf42d43823399 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + + +class ConvertToLocalRelationSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("LocalRelation", FixedPoint(100), + ConvertToLocalRelation) :: Nil + } + + test("Project on LocalRelation should be turned into a single LocalRelation") { + val testRelation = LocalRelation( + LocalRelation('a.int, 'b.int).output, + Row(1, 2) :: + Row(4, 5) :: Nil) + + val correctAnswer = LocalRelation( + LocalRelation('a1.int, 'b1.int).output, + Row(1, 3) :: + Row(4, 6) :: Nil) + + val projectOnLocal = testRelation.select( + UnresolvedAttribute("a").as("a1"), + (UnresolvedAttribute("b") + 1).as("b1")) + + val optimized = Optimize(projectOnLocal.analyze) + + comparePlans(optimized, correctAnswer) + } + +} 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 8ad6526f872e5..17ea3cde8e50e 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 @@ -102,7 +102,7 @@ trait DataFrame extends RDDApi[Row] { * }}} */ @scala.annotation.varargs - def toDataFrame(colName: String, colNames: String*): DataFrame + def toDataFrame(colNames: String*): DataFrame /** Returns the schema of this [[DataFrame]]. */ def schema: StructType @@ -116,6 +116,25 @@ trait DataFrame extends RDDApi[Row] { /** Prints the schema to the console in a nice tree format. */ def printSchema(): Unit + /** + * Returns true if the `collect` and `take` methods can be run locally + * (without any Spark executors). + */ + def isLocal: Boolean + + /** + * Displays the [[DataFrame]] in a tabular form. For example: + * {{{ + * year month AVG('Adj Close) MAX('Adj Close) + * 1980 12 0.503218 0.595103 + * 1981 01 0.523289 0.570307 + * 1982 02 0.436504 0.475256 + * 1983 03 0.410516 0.442194 + * 1984 04 0.450090 0.483521 + * }}} + */ + def show(): Unit + /** * Cartesian join with another [[DataFrame]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 789bcf6184b3e..fa05a5dcac6bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -90,14 +90,13 @@ private[sql] class DataFrameImpl protected[sql]( } } - override def toDataFrame(colName: String, colNames: String*): DataFrame = { - val newNames = colName +: colNames - require(schema.size == newNames.size, + override def toDataFrame(colNames: String*): DataFrame = { + require(schema.size == colNames.size, "The number of columns doesn't match.\n" + "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + newNames.mkString(", ")) + "New column names: " + colNames.mkString(", ")) - val newCols = schema.fieldNames.zip(newNames).map { case (oldName, newName) => + val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => apply(oldName).as(newName) } select(newCols :_*) @@ -113,6 +112,38 @@ private[sql] class DataFrameImpl protected[sql]( override def printSchema(): Unit = println(schema.treeString) + override def isLocal: Boolean = { + logicalPlan.isInstanceOf[LocalRelation] + } + + override def show(): Unit = { + val data = take(20) + val numCols = schema.fieldNames.length + + // For cells that are beyond 20 characters, replace it with the first 17 and "..." + val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row => + row.toSeq.map { cell => + val str = if (cell == null) "null" else cell.toString + if (str.length > 20) str.substring(0, 17) + "..." else str + } : Seq[String] + } + + // Compute the width of each column + val colWidths = Array.fill(numCols)(0) + for (row <- rows) { + for ((cell, i) <- row.zipWithIndex) { + colWidths(i) = math.max(colWidths(i), cell.length) + } + } + + // Pad the cells and print them + println(rows.map { row => + row.zipWithIndex.map { case (cell, i) => + String.format(s"%-${colWidths(i)}s", cell) + }.mkString(" ") + }.mkString("\n")) + } + override def join(right: DataFrame): DataFrame = { Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 6043fb4dee01d..782f6e28eebb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -48,7 +48,7 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten protected[sql] override def logicalPlan: LogicalPlan = err() - override def toDataFrame(colName: String, colNames: String*): DataFrame = err() + override def toDataFrame(colNames: String*): DataFrame = err() override def schema: StructType = err() @@ -58,6 +58,10 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def printSchema(): Unit = err() + override def show(): Unit = err() + + override def isLocal: Boolean = false + override def join(right: DataFrame): DataFrame = err() override def join(right: DataFrame, joinExprs: Column): DataFrame = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 66aed5d5113d1..4dc506c21ab9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.execution -import scala.collection.mutable.ArrayBuffer -import scala.reflect.runtime.universe.TypeTag - import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDD, ShuffledRDD} @@ -40,7 +37,7 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends @transient lazy val buildProjection = newMutableProjection(projectList, child.output) - def execute() = child.execute().mapPartitions { iter => + override def execute() = child.execute().mapPartitions { iter => val resuableProjection = buildProjection() iter.map(resuableProjection) } @@ -55,7 +52,7 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { @transient lazy val conditionEvaluator = newPredicate(condition, child.output) - def execute() = child.execute().mapPartitions { iter => + override def execute() = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } } 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 a640ba57e0885..5eecc303ef72b 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 @@ -87,13 +87,13 @@ trait CreatableRelationProvider { /** * ::DeveloperApi:: - * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must - * be able to produce the schema of their data in the form of a [[StructType]] Concrete + * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must + * be able to produce the schema of their data in the form of a [[StructType]]. Concrete * implementation should inherit from one of the descendant `Scan` classes, which define various * abstract methods for execution. * * BaseRelations must also define a equality function that only returns true when the two - * instances will return the same data. This equality function is used when determining when + * instances will return the same data. This equality function is used when determining when * it is safe to substitute cached results for a given relation. */ @DeveloperApi @@ -102,13 +102,16 @@ abstract class BaseRelation { def schema: StructType /** - * Returns an estimated size of this relation in bytes. This information is used by the planner + * Returns an estimated size of this relation in bytes. This information is used by the planner * to decided when it is safe to broadcast a relation and can be overridden by sources that * know the size ahead of time. By default, the system will assume that tables are too - * large to broadcast. This method will be called multiple times during query planning + * large to broadcast. This method will be called multiple times during query planning * and thus should not perform expensive operations for each invocation. + * + * Note that it is always better to overestimate size than underestimate, because underestimation + * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). */ - def sizeInBytes = sqlContext.conf.defaultSizeInBytes + def sizeInBytes: Long = sqlContext.conf.defaultSizeInBytes } /** From c17161189d57f2e3a8d3550ea59a68edf487c8b7 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 8 Feb 2015 21:07:36 -0800 Subject: [PATCH 005/817] [SPARK-5660][MLLIB] Make Matrix apply public This is #4447 with `override`. Closes #4447 Author: Joseph K. Bradley Author: Xiangrui Meng Closes #4462 from mengxr/SPARK-5660 and squashes the following commits: f82c8d6 [Xiangrui Meng] add override to matrix.apply 91cedde [Joseph K. Bradley] made matrix apply public --- .../main/scala/org/apache/spark/mllib/linalg/Matrices.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 84f8ac2e0d9cd..c8a97b8c53d9b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -50,7 +50,7 @@ sealed trait Matrix extends Serializable { private[mllib] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ - private[mllib] def apply(i: Int, j: Int): Double + def apply(i: Int, j: Int): Double /** Return the index for the (i, j)-th element in the backing array. */ private[mllib] def index(i: Int, j: Int): Int @@ -163,7 +163,7 @@ class DenseMatrix( private[mllib] def apply(i: Int): Double = values(i) - private[mllib] def apply(i: Int, j: Int): Double = values(index(i, j)) + override def apply(i: Int, j: Int): Double = values(index(i, j)) private[mllib] def index(i: Int, j: Int): Int = { if (!isTransposed) i + numRows * j else j + numCols * i @@ -398,7 +398,7 @@ class SparseMatrix( } } - private[mllib] def apply(i: Int, j: Int): Double = { + override def apply(i: Int, j: Int): Double = { val ind = index(i, j) if (ind < 0) 0.0 else values(ind) } From 4396dfb37f433ef186e3e0a09db9906986ec940b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 8 Feb 2015 21:08:50 -0800 Subject: [PATCH 006/817] SPARK-4405 [MLLIB] Matrices.* construction methods should check for rows x cols overflow Check that size of dense matrix array is not beyond Int.MaxValue in Matrices.* methods. jkbradley this should be an easy one. Review and/or merge as you see fit. Author: Sean Owen Closes #4461 from srowen/SPARK-4405 and squashes the following commits: c67574e [Sean Owen] Check that size of dense matrix array is not beyond Int.MaxValue in Matrices.* methods --- .../org/apache/spark/mllib/linalg/Matrices.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index c8a97b8c53d9b..89b38679b7494 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -256,8 +256,11 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros */ - def zeros(numRows: Int, numCols: Int): DenseMatrix = + def zeros(numRows: Int, numCols: Int): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + } /** * Generate a `DenseMatrix` consisting of ones. @@ -265,8 +268,11 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones */ - def ones(numRows: Int, numCols: Int): DenseMatrix = + def ones(numRows: Int, numCols: Int): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + } /** * Generate an Identity Matrix in `DenseMatrix` format. @@ -291,6 +297,8 @@ object DenseMatrix { * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) } @@ -302,6 +310,8 @@ object DenseMatrix { * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) } From 4575c5643a82818bf64f9648314bdc2fdc12febb Mon Sep 17 00:00:00 2001 From: Hung Lin Date: Sun, 8 Feb 2015 22:36:42 -0800 Subject: [PATCH 007/817] [SPARK-5472][SQL] Fix Scala code style Fix Scala code style. Author: Hung Lin Closes #4464 from hunglin/SPARK-5472 and squashes the following commits: ef7a3b3 [Hung Lin] SPARK-5472: fix scala style --- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 42 +++++++++---------- .../apache/spark/sql/jdbc/JDBCRelation.scala | 35 +++++++++------- 2 files changed, 41 insertions(+), 36 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index a2f94675fb5a3..0bec32cca1325 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -17,13 +17,10 @@ package org.apache.spark.sql.jdbc -import java.sql.{Connection, DatabaseMetaData, DriverManager, ResultSet, ResultSetMetaData, SQLException} -import scala.collection.mutable.ArrayBuffer +import java.sql.{Connection, DriverManager, ResultSet, ResultSetMetaData, SQLException} import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD -import org.apache.spark.util.NextIterator -import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow} import org.apache.spark.sql.types._ import org.apache.spark.sql.sources._ @@ -100,7 +97,7 @@ private[sql] object JDBCRDD extends Logging { try { val rsmd = rs.getMetaData val ncols = rsmd.getColumnCount - var fields = new Array[StructField](ncols); + val fields = new Array[StructField](ncols) var i = 0 while (i < ncols) { val columnName = rsmd.getColumnName(i + 1) @@ -176,23 +173,27 @@ private[sql] object JDBCRDD extends Logging { * * @return An RDD representing "SELECT requiredColumns FROM fqTable". */ - def scanTable(sc: SparkContext, - schema: StructType, - driver: String, - url: String, - fqTable: String, - requiredColumns: Array[String], - filters: Array[Filter], - parts: Array[Partition]): RDD[Row] = { + def scanTable( + sc: SparkContext, + schema: StructType, + driver: String, + url: String, + fqTable: String, + requiredColumns: Array[String], + filters: Array[Filter], + parts: Array[Partition]): RDD[Row] = { + val prunedSchema = pruneSchema(schema, requiredColumns) - return new JDBCRDD(sc, - getConnector(driver, url), - prunedSchema, - fqTable, - requiredColumns, - filters, - parts) + return new + JDBCRDD( + sc, + getConnector(driver, url), + prunedSchema, + fqTable, + requiredColumns, + filters, + parts) } } @@ -412,6 +413,5 @@ private[sql] class JDBCRDD( gotNext = false nextValue } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index e09125e406ba2..66ad38eb7c45b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -96,7 +96,8 @@ private[sql] class DefaultSource extends RelationProvider { if (driver != null) Class.forName(driver) - if ( partitionColumn != null + if ( + partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { sys.error("Partitioning incompletely specified") } @@ -104,30 +105,34 @@ private[sql] class DefaultSource extends RelationProvider { val partitionInfo = if (partitionColumn == null) { null } else { - JDBCPartitioningInfo(partitionColumn, - lowerBound.toLong, upperBound.toLong, - numPartitions.toInt) + JDBCPartitioningInfo( + partitionColumn, + lowerBound.toLong, + upperBound.toLong, + numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) JDBCRelation(url, table, parts)(sqlContext) } } -private[sql] case class JDBCRelation(url: String, - table: String, - parts: Array[Partition])( - @transient val sqlContext: SQLContext) - extends PrunedFilteredScan { +private[sql] case class JDBCRelation( + url: String, + table: String, + parts: Array[Partition])(@transient val sqlContext: SQLContext) extends PrunedFilteredScan { override val schema = JDBCRDD.resolveTable(url, table) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { val driver: String = DriverManager.getDriver(url).getClass.getCanonicalName - JDBCRDD.scanTable(sqlContext.sparkContext, - schema, - driver, url, - table, - requiredColumns, filters, - parts) + JDBCRDD.scanTable( + sqlContext.sparkContext, + schema, + driver, + url, + table, + requiredColumns, + filters, + parts) } } From 855d12ac0a9cdade4cd2cc64c4e7209478be6690 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 8 Feb 2015 23:40:36 -0800 Subject: [PATCH 008/817] [SPARK-5539][MLLIB] LDA guide This is the LDA user guide from jkbradley with Java and Scala code example. Author: Xiangrui Meng Author: Joseph K. Bradley Closes #4465 from mengxr/lda-guide and squashes the following commits: 6dcb7d1 [Xiangrui Meng] update java example in the user guide 76169ff [Xiangrui Meng] update java example 36c3ae2 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into lda-guide c2a1efe [Joseph K. Bradley] Added LDA programming guide, plus Java example (which is in the guide and probably should be removed). --- data/mllib/sample_lda_data.txt | 12 ++ docs/mllib-clustering.md | 129 +++++++++++++++++- .../spark/examples/mllib/JavaLDAExample.java | 75 ++++++++++ 3 files changed, 215 insertions(+), 1 deletion(-) create mode 100644 data/mllib/sample_lda_data.txt create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java diff --git a/data/mllib/sample_lda_data.txt b/data/mllib/sample_lda_data.txt new file mode 100644 index 0000000000000..2e76702ca9d67 --- /dev/null +++ b/data/mllib/sample_lda_data.txt @@ -0,0 +1,12 @@ +1 2 6 0 2 3 1 1 0 0 3 +1 3 0 1 3 0 0 2 0 0 1 +1 4 1 0 0 4 9 0 1 2 0 +2 1 0 3 0 0 5 0 2 3 9 +3 1 1 9 3 0 2 0 0 1 3 +4 2 0 3 4 5 1 1 1 4 0 +2 1 0 3 0 0 5 0 2 2 9 +1 1 1 9 2 1 2 0 0 1 3 +4 4 0 3 4 2 1 3 0 0 0 +2 8 2 0 3 0 2 0 2 7 2 +1 1 1 9 0 2 2 0 0 3 3 +4 1 0 0 4 5 1 3 0 1 0 diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 1e9ef345b7435..99ed6b60e3f00 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -55,7 +55,7 @@ has the following parameters: Power iteration clustering is a scalable and efficient algorithm for clustering points given pointwise mutual affinity values. Internally the algorithm: -* accepts a [Graph](https://spark.apache.org/docs/0.9.2/api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points. +* accepts a [Graph](api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points. * calculates the principal eigenvalue and eigenvector * Clusters each of the input points according to their principal eigenvector component value @@ -71,6 +71,35 @@ Example outputs for a dataset inspired by the paper - but with five clusters ins

+### Latent Dirichlet Allocation (LDA) + +[Latent Dirichlet Allocation (LDA)](http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation) +is a topic model which infers topics from a collection of text documents. +LDA can be thought of as a clustering algorithm as follows: + +* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset. +* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts. +* Rather than estimating a clustering using a traditional distance, LDA uses a function based + on a statistical model of how text documents are generated. + +LDA takes in a collection of documents as vectors of word counts. +It learns clustering using [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +on the likelihood function. After fitting on the documents, LDA provides: + +* Topics: Inferred topics, each of which is a probability distribution over terms (words). +* Topic distributions for documents: For each document in the training set, LDA gives a probability distribution over topics. + +LDA takes the following parameters: + +* `k`: Number of topics (i.e., cluster centers) +* `maxIterations`: Limit on the number of iterations of EM used for learning +* `docConcentration`: Hyperparameter for prior over documents' distributions over topics. Currently must be > 1, where larger values encourage smoother inferred distributions. +* `topicConcentration`: Hyperparameter for prior over topics' distributions over terms (words). Currently must be > 1, where larger values encourage smoother inferred distributions. +* `checkpointInterval`: If using checkpointing (set in the Spark configuration), this parameter specifies the frequency with which checkpoints will be created. If `maxIterations` is large, using checkpointing can help reduce shuffle file sizes on disk and help with failure recovery. + +*Note*: LDA is a new feature with some missing functionality. In particular, it does not yet +support prediction on new documents, and it does not have a Python API. These will be added in the future. + ### Examples #### k-means @@ -293,6 +322,104 @@ for i in range(2): +#### Latent Dirichlet Allocation (LDA) Example + +In the following example, we load word count vectors representing a corpus of documents. +We then use [LDA](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) +to infer three topics from the documents. The number of desired clusters is passed +to the algorithm. We then output the topics, represented as probability distributions over words. + +
+
+ +{% highlight scala %} +import org.apache.spark.mllib.clustering.LDA +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data +val data = sc.textFile("data/mllib/sample_lda_data.txt") +val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))) +// Index documents with unique IDs +val corpus = parsedData.zipWithIndex.map(_.swap).cache() + +// Cluster the documents into three topics using LDA +val ldaModel = new LDA().setK(3).run(corpus) + +// Output topics. Each is a distribution over words (matching word count vectors) +println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize + " words):") +val topics = ldaModel.topicsMatrix +for (topic <- Range(0, 3)) { + print("Topic " + topic + ":") + for (word <- Range(0, ldaModel.vocabSize)) { print(" " + topics(word, topic)); } + println() +} +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.clustering.DistributedLDAModel; +import org.apache.spark.mllib.clustering.LDA; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SparkConf; + +public class JavaLDAExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("LDA Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/sample_lda_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public Vector call(String s) { + String[] sarray = s.trim().split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) + values[i] = Double.parseDouble(sarray[i]); + return Vectors.dense(values); + } + } + ); + // Index documents with unique IDs + JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map( + new Function, Tuple2>() { + public Tuple2 call(Tuple2 doc_id) { + return doc_id.swap(); + } + } + )); + corpus.cache(); + + // Cluster the documents into three topics using LDA + DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus); + + // Output topics. Each is a distribution over words (matching word count vectors) + System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() + + " words):"); + Matrix topics = ldaModel.topicsMatrix(); + for (int topic = 0; topic < 3; topic++) { + System.out.print("Topic " + topic + ":"); + for (int word = 0; word < ldaModel.vocabSize(); word++) { + System.out.print(" " + topics.apply(word, topic)); + } + System.out.println(); + } + } +} +{% endhighlight %} +
+ +
+ + In order to run the above application, follow the instructions provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) section of the Spark diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java new file mode 100644 index 0000000000000..f394ff2084463 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.clustering.DistributedLDAModel; +import org.apache.spark.mllib.clustering.LDA; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SparkConf; + +public class JavaLDAExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("LDA Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/sample_lda_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public Vector call(String s) { + String[] sarray = s.trim().split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) + values[i] = Double.parseDouble(sarray[i]); + return Vectors.dense(values); + } + } + ); + // Index documents with unique IDs + JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map( + new Function, Tuple2>() { + public Tuple2 call(Tuple2 doc_id) { + return doc_id.swap(); + } + } + )); + corpus.cache(); + + // Cluster the documents into three topics using LDA + DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus); + + // Output topics. Each is a distribution over words (matching word count vectors) + System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() + + " words):"); + Matrix topics = ldaModel.topicsMatrix(); + for (int topic = 0; topic < 3; topic++) { + System.out.print("Topic " + topic + ":"); + for (int word = 0; word < ldaModel.vocabSize(); word++) { + System.out.print(" " + topics.apply(word, topic)); + } + System.out.println(); + } + } +} From 4dfe180fc893bee1146161f8b2a6efd4d6d2bb8c Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 9 Feb 2015 09:44:53 +0000 Subject: [PATCH 009/817] [SPARK-5473] [EC2] Expose SSH failures after status checks pass If there is some fatal problem with launching a cluster, `spark-ec2` just hangs without giving the user useful feedback on what the problem is. This PR exposes the output of the SSH calls to the user if the SSH test fails during cluster launch for any reason but the instance status checks are all green. It also removes the growing trail of dots while waiting in favor of a fixed 3 dots. For example: ``` $ ./ec2/spark-ec2 -k key -i /incorrect/path/identity.pem --instance-type m3.medium --slaves 1 --zone us-east-1c launch "spark-test" Setting up security groups... Searching for existing cluster spark-test... Spark AMI: ami-35b1885c Launching instances... Launched 1 slaves in us-east-1c, regid = r-7dadd096 Launched master in us-east-1c, regid = r-fcadd017 Waiting for cluster to enter 'ssh-ready' state... Warning: SSH connection error. (This could be temporary.) Host: 127.0.0.1 SSH return code: 255 SSH output: Warning: Identity file /incorrect/path/identity.pem not accessible: No such file or directory. Warning: Permanently added '127.0.0.1' (RSA) to the list of known hosts. Permission denied (publickey). ``` This should give users enough information when some unrecoverable error occurs during launch so they can know to abort the launch. This will help avoid situations like the ones reported [here on Stack Overflow](http://stackoverflow.com/q/28002443/) and [here on the user list](http://mail-archives.apache.org/mod_mbox/spark-user/201501.mbox/%3C1422323829398-21381.postn3.nabble.com%3E), where the users couldn't tell what the problem was because it was being hidden by `spark-ec2`. This is a usability improvement that should be backported to 1.2. Resolves [SPARK-5473](https://issues.apache.org/jira/browse/SPARK-5473). Author: Nicholas Chammas Closes #4262 from nchammas/expose-ssh-failure and squashes the following commits: 8bda6ed [Nicholas Chammas] default to print SSH output 2b92534 [Nicholas Chammas] show SSH output after status check pass --- ec2/spark_ec2.py | 36 ++++++++++++++++++++++++------------ 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 725b1e47e0cea..87b2112fe4628 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -34,6 +34,7 @@ import sys import tarfile import tempfile +import textwrap import time import urllib2 import warnings @@ -681,21 +682,32 @@ def setup_spark_cluster(master, opts): print "Ganglia started at http://%s:5080/ganglia" % master -def is_ssh_available(host, opts): +def is_ssh_available(host, opts, print_ssh_output=True): """ Check if SSH is available on a host. """ - try: - with open(os.devnull, 'w') as devnull: - ret = subprocess.check_call( - ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3', - '%s@%s' % (opts.user, host), stringify_command('true')], - stdout=devnull, - stderr=devnull - ) - return ret == 0 - except subprocess.CalledProcessError as e: - return False + s = subprocess.Popen( + ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3', + '%s@%s' % (opts.user, host), stringify_command('true')], + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT # we pipe stderr through stdout to preserve output order + ) + cmd_output = s.communicate()[0] # [1] is stderr, which we redirected to stdout + + if s.returncode != 0 and print_ssh_output: + # extra leading newline is for spacing in wait_for_cluster_state() + print textwrap.dedent("""\n + Warning: SSH connection error. (This could be temporary.) + Host: {h} + SSH return code: {r} + SSH output: {o} + """).format( + h=host, + r=s.returncode, + o=cmd_output.strip() + ) + + return s.returncode == 0 def is_cluster_ssh_available(cluster_instances, opts): From 0793ee1b4dea1f4b0df749e8ad7c1ab70b512faf Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 9 Feb 2015 10:12:12 +0000 Subject: [PATCH 010/817] SPARK-2149. [MLLIB] Univariate kernel density estimation Author: Sandy Ryza Closes #1093 from sryza/sandy-spark-2149 and squashes the following commits: 5f06b33 [Sandy Ryza] More review comments 0f73060 [Sandy Ryza] Respond to Sean's review comments 0dfa005 [Sandy Ryza] SPARK-2149. Univariate kernel density estimation --- .../spark/mllib/stat/KernelDensity.scala | 71 +++++++++++++++++++ .../apache/spark/mllib/stat/Statistics.scala | 14 ++++ .../spark/mllib/stat/KernelDensitySuite.scala | 47 ++++++++++++ 3 files changed, 132 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala new file mode 100644 index 0000000000000..0deef11b4511a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.apache.spark.rdd.RDD + +private[stat] object KernelDensity { + /** + * Given a set of samples from a distribution, estimates its density at the set of given points. + * Uses a Gaussian kernel with the given standard deviation. + */ + def estimate(samples: RDD[Double], standardDeviation: Double, + evaluationPoints: Array[Double]): Array[Double] = { + if (standardDeviation <= 0.0) { + throw new IllegalArgumentException("Standard deviation must be positive") + } + + // This gets used in each Gaussian PDF computation, so compute it up front + val logStandardDeviationPlusHalfLog2Pi = + Math.log(standardDeviation) + 0.5 * Math.log(2 * Math.PI) + + val (points, count) = samples.aggregate((new Array[Double](evaluationPoints.length), 0))( + (x, y) => { + var i = 0 + while (i < evaluationPoints.length) { + x._1(i) += normPdf(y, standardDeviation, logStandardDeviationPlusHalfLog2Pi, + evaluationPoints(i)) + i += 1 + } + (x._1, i) + }, + (x, y) => { + var i = 0 + while (i < evaluationPoints.length) { + x._1(i) += y._1(i) + i += 1 + } + (x._1, x._2 + y._2) + }) + + var i = 0 + while (i < points.length) { + points(i) /= count + i += 1 + } + points + } + + private def normPdf(mean: Double, standardDeviation: Double, + logStandardDeviationPlusHalfLog2Pi: Double, x: Double): Double = { + val x0 = x - mean + val x1 = x0 / standardDeviation + val logDensity = -0.5 * x1 * x1 - logStandardDeviationPlusHalfLog2Pi + Math.exp(logDensity) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index b3fad0c52d655..32561620ac914 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -149,4 +149,18 @@ object Statistics { def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { ChiSqTest.chiSquaredFeatures(data) } + + /** + * Given an empirical distribution defined by the input RDD of samples, estimate its density at + * each of the given evaluation points using a Gaussian kernel. + * + * @param samples The samples RDD used to define the empirical distribution. + * @param standardDeviation The standard deviation of the kernel Gaussians. + * @param evaluationPoints The points at which to estimate densities. + * @return An array the same size as evaluationPoints with the density at each point. + */ + def kernelDensity(samples: RDD[Double], standardDeviation: Double, + evaluationPoints: Iterable[Double]): Array[Double] = { + KernelDensity.estimate(samples, standardDeviation, evaluationPoints.toArray) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala new file mode 100644 index 0000000000000..f6a1e19f50296 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.scalatest.FunSuite + +import org.apache.commons.math3.distribution.NormalDistribution + +import org.apache.spark.mllib.util.LocalClusterSparkContext + +class KernelDensitySuite extends FunSuite with LocalClusterSparkContext { + test("kernel density single sample") { + val rdd = sc.parallelize(Array(5.0)) + val evaluationPoints = Array(5.0, 6.0) + val densities = KernelDensity.estimate(rdd, 3.0, evaluationPoints) + val normal = new NormalDistribution(5.0, 3.0) + val acceptableErr = 1e-6 + assert(densities(0) - normal.density(5.0) < acceptableErr) + assert(densities(0) - normal.density(6.0) < acceptableErr) + } + + test("kernel density multiple samples") { + val rdd = sc.parallelize(Array(5.0, 10.0)) + val evaluationPoints = Array(5.0, 6.0) + val densities = KernelDensity.estimate(rdd, 3.0, evaluationPoints) + val normal1 = new NormalDistribution(5.0, 3.0) + val normal2 = new NormalDistribution(10.0, 3.0) + val acceptableErr = 1e-6 + assert(densities(0) - (normal1.density(5.0) + normal2.density(5.0)) / 2 < acceptableErr) + assert(densities(0) - (normal1.density(6.0) + normal2.density(6.0)) / 2 < acceptableErr) + } +} From de7806048ac49a8bfdf44d8f87bc11cea1dfb242 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 9 Feb 2015 10:33:57 -0800 Subject: [PATCH 011/817] SPARK-4267 [YARN] Failing to launch jobs on Spark on YARN with Hadoop 2.5.0 or later Before passing to YARN, escape arguments in "extraJavaOptions" args, in order to correctly handle cases like -Dfoo="one two three". Also standardize how these args are handled and ensure that individual args are treated as stand-alone args, not one string. vanzin andrewor14 Author: Sean Owen Closes #4452 from srowen/SPARK-4267.2 and squashes the following commits: c8297d2 [Sean Owen] Before passing to YARN, escape arguments in "extraJavaOptions" args, in order to correctly handle cases like -Dfoo="one two three". Also standardize how these args are handled and ensure that individual args are treated as stand-alone args, not one string. --- .../org/apache/spark/deploy/yarn/Client.scala | 9 +++++---- .../spark/deploy/yarn/ExecutorRunnable.scala | 17 +++++++++-------- .../spark/deploy/yarn/YarnClusterSuite.scala | 6 ++++-- 3 files changed, 18 insertions(+), 14 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index e7005094b5f3c..8afc1ccdad732 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -435,10 +435,11 @@ private[spark] class Client( // Include driver-specific java options if we are launching a driver if (isClusterMode) { - sparkConf.getOption("spark.driver.extraJavaOptions") + val driverOpts = sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Seq.empty) - .foreach(opts => javaOpts += opts) + driverOpts.foreach { opts => + javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) + } val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), sys.props.get("spark.driver.libraryPath")).flatten if (libraryPaths.nonEmpty) { @@ -460,7 +461,7 @@ private[spark] class Client( val msg = s"$amOptsKey is not allowed to alter memory settings (was '$opts')." throw new SparkException(msg) } - javaOpts ++= Utils.splitCommandString(opts) + javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 408cf09b9bdfa..7cd8c5f0f9204 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -128,14 +128,15 @@ class ExecutorRunnable( // Set the JVM memory val executorMemoryString = executorMemory + "m" - javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " + javaOpts += "-Xms" + executorMemoryString + javaOpts += "-Xmx" + executorMemoryString // Set extra Java options for the executor, if defined sys.props.get("spark.executor.extraJavaOptions").foreach { opts => - javaOpts += opts + javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) } sys.env.get("SPARK_JAVA_OPTS").foreach { opts => - javaOpts += opts + javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) } sys.props.get("spark.executor.extraLibraryPath").foreach { p => prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) @@ -173,11 +174,11 @@ class ExecutorRunnable( // The options are based on // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use // %20the%20Concurrent%20Low%20Pause%20Collector|outline - javaOpts += " -XX:+UseConcMarkSweepGC " - javaOpts += " -XX:+CMSIncrementalMode " - javaOpts += " -XX:+CMSIncrementalPacing " - javaOpts += " -XX:CMSIncrementalDutyCycleMin=0 " - javaOpts += " -XX:CMSIncrementalDutyCycle=10 " + javaOpts += "-XX:+UseConcMarkSweepGC" + javaOpts += "-XX:+CMSIncrementalMode" + javaOpts += "-XX:+CMSIncrementalPacing" + javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" + javaOpts += "-XX:CMSIncrementalDutyCycle=10" } */ diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index eda40efc4c77f..e39de82740b1d 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -75,6 +75,8 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit private var oldConf: Map[String, String] = _ override def beforeAll() { + super.beforeAll() + tempDir = Utils.createTempDir() val logConfDir = new File(tempDir, "log4j") @@ -129,8 +131,8 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit sys.props += ("spark.executor.instances" -> "1") sys.props += ("spark.driver.extraClassPath" -> childClasspath) sys.props += ("spark.executor.extraClassPath" -> childClasspath) - - super.beforeAll() + sys.props += ("spark.executor.extraJavaOptions" -> "-Dfoo=\"one two three\"") + sys.props += ("spark.driver.extraJavaOptions" -> "-Dfoo=\"one two three\"") } override def afterAll() { From afb131637d96e1e5e07eb8abf24e32e7f3b2304d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 9 Feb 2015 11:42:52 -0800 Subject: [PATCH 012/817] [SPARK-5678] Convert DataFrame to pandas.DataFrame and Series ``` pyspark.sql.DataFrame.to_pandas = to_pandas(self) unbound pyspark.sql.DataFrame method Collect all the rows and return a `pandas.DataFrame`. >>> df.to_pandas() # doctest: +SKIP age name 0 2 Alice 1 5 Bob pyspark.sql.Column.to_pandas = to_pandas(self) unbound pyspark.sql.Column method Return a pandas.Series from the column >>> df.age.to_pandas() # doctest: +SKIP 0 2 1 5 dtype: int64 ``` Not tests by jenkins (they depends on pandas) Author: Davies Liu Closes #4476 from davies/to_pandas and squashes the following commits: 6276fb6 [Davies Liu] Convert DataFrame to pandas.DataFrame and Series --- python/pyspark/sql.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index e55f285a778c4..6a6dfbc5851b8 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2284,6 +2284,18 @@ def addColumn(self, colName, col): """ return self.select('*', col.alias(colName)) + def to_pandas(self): + """ + Collect all the rows and return a `pandas.DataFrame`. + + >>> df.to_pandas() # doctest: +SKIP + age name + 0 2 Alice + 1 5 Bob + """ + import pandas as pd + return pd.DataFrame.from_records(self.collect(), columns=self.columns) + # Having SchemaRDD for backward compatibility (for docs) class SchemaRDD(DataFrame): @@ -2551,6 +2563,19 @@ def cast(self, dataType): jc = self._jc.cast(jdt) return Column(jc, self.sql_ctx) + def to_pandas(self): + """ + Return a pandas.Series from the column + + >>> df.age.to_pandas() # doctest: +SKIP + 0 2 + 1 5 + dtype: int64 + """ + import pandas as pd + data = [c for c, in self.collect()] + return pd.Series(data) + def _aggregate_func(name, doc=""): """ Create a function for aggregator by name""" From dae216147f2247fd722fb0909da74fe71cf2fa8b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 9 Feb 2015 11:45:12 -0800 Subject: [PATCH 013/817] [SPARK-5664][BUILD] Restore stty settings when exiting from SBT's spark-shell For launching spark-shell from SBT. Author: Liang-Chi Hsieh Closes #4451 from viirya/restore_stty and squashes the following commits: fdfc480 [Liang-Chi Hsieh] Restore stty settings when exit (for launching spark-shell from SBT). --- build/sbt | 28 ++++++++++++++++++++++++++++ build/sbt-launch-lib.bash | 2 +- 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/build/sbt b/build/sbt index 28ebb64f7197c..cc3203d79bccd 100755 --- a/build/sbt +++ b/build/sbt @@ -125,4 +125,32 @@ loadConfigFile() { [[ -f "$etc_sbt_opts_file" ]] && set -- $(loadConfigFile "$etc_sbt_opts_file") "$@" [[ -f "$sbt_opts_file" ]] && set -- $(loadConfigFile "$sbt_opts_file") "$@" +exit_status=127 +saved_stty="" + +restoreSttySettings() { + stty $saved_stty + saved_stty="" +} + +onExit() { + if [[ "$saved_stty" != "" ]]; then + restoreSttySettings + fi + exit $exit_status +} + +saveSttySettings() { + saved_stty=$(stty -g 2>/dev/null) + if [[ ! $? ]]; then + saved_stty="" + fi +} + +saveSttySettings +trap onExit INT + run "$@" + +exit_status=$? +onExit diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 5e0c640fa5919..504be48b358fa 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -81,7 +81,7 @@ execRunner () { echo "" } - exec "$@" + "$@" } addJava () { From 6fe70d8432314f0b7290a66f114306f61e0a87cc Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 9 Feb 2015 13:20:14 -0800 Subject: [PATCH 014/817] [SPARK-5691] Fixing wrong data structure lookup for dupe app registratio... In Master's registerApplication method, it checks if the application had already registered by examining the addressToWorker hash map. In reality, it should refer to the addressToApp data structure, as this is what really tracks which apps have been registered. Author: mcheah Closes #4477 from mccheah/spark-5691 and squashes the following commits: efdc573 [mcheah] [SPARK-5691] Fixing wrong data structure lookup for dupe app registration --- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index b8b1a25abff2e..53e453990f8c7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -671,7 +671,7 @@ private[spark] class Master( def registerApplication(app: ApplicationInfo): Unit = { val appAddress = app.driver.path.address - if (addressToWorker.contains(appAddress)) { + if (addressToApp.contains(appAddress)) { logInfo("Attempted to re-register application at same address: " + appAddress) return } From 0765af9b21e9204c410c7a849c7201bc3eda8cc3 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Feb 2015 14:17:14 -0800 Subject: [PATCH 015/817] [SPARK-4905][STREAMING] FlumeStreamSuite fix. Using String constructor instead of CharsetDecoder to see if it fixes the issue of empty strings in Flume test output. Author: Hari Shreedharan Closes #4371 from harishreedharan/Flume-stream-attempted-fix and squashes the following commits: 550d363 [Hari Shreedharan] Fix imports. 8695950 [Hari Shreedharan] Use Charsets.UTF_8 instead of "UTF-8" in String constructors. af3ba14 [Hari Shreedharan] [SPARK-4905][STREAMING] FlumeStreamSuite fix. --- .../apache/spark/streaming/flume/FlumeStreamSuite.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index f333e3891b5f0..322de7bf2fed8 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -19,13 +19,13 @@ package org.apache.spark.streaming.flume import java.net.{InetSocketAddress, ServerSocket} import java.nio.ByteBuffer -import java.nio.charset.Charset import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.concurrent.duration._ import scala.language.postfixOps +import com.google.common.base.Charsets import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.flume.source.avro @@ -108,7 +108,7 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L val inputEvents = input.map { item => val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(item.getBytes("UTF-8"))) + event.setBody(ByteBuffer.wrap(item.getBytes(Charsets.UTF_8))) event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) event } @@ -138,14 +138,13 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L status should be (avro.Status.OK) } - val decoder = Charset.forName("UTF-8").newDecoder() eventually(timeout(10 seconds), interval(100 milliseconds)) { val outputEvents = outputBuffer.flatten.map { _.event } outputEvents.foreach { event => event.getHeaders.get("test") should be("header") } - val output = outputEvents.map(event => decoder.decode(event.getBody()).toString) + val output = outputEvents.map(event => new String(event.getBody.array(), Charsets.UTF_8)) output should be (input) } } From f48199eb354d6ec8675c2c1f96c3005064058d66 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 9 Feb 2015 14:51:46 -0800 Subject: [PATCH 016/817] [SPARK-5675][SQL] XyzType companion object should subclass XyzType Otherwise, the following will always return false in Java. ```scala dataType instanceof StringType ``` Author: Reynold Xin Closes #4463 from rxin/type-companion-object and squashes the following commits: 04d5d8d [Reynold Xin] Comment. 976e11e [Reynold Xin] [SPARK-5675][SQL]StringType case object should be subclass of StringType class --- .../apache/spark/sql/types/dataTypes.scala | 85 ++++++++++++++++--- 1 file changed, 73 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 91efe320546a7..2abb1caee9cd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -240,10 +240,16 @@ abstract class DataType { * @group dataType */ @DeveloperApi -case object NullType extends DataType { +class NullType private() extends DataType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "NullType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. override def defaultSize: Int = 1 } +case object NullType extends NullType + + protected[sql] object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -292,7 +298,10 @@ protected[sql] abstract class NativeType extends DataType { * @group dataType */ @DeveloperApi -case object StringType extends NativeType with PrimitiveType { +class StringType private() extends NativeType with PrimitiveType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "StringType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] @@ -303,6 +312,8 @@ case object StringType extends NativeType with PrimitiveType { override def defaultSize: Int = 4096 } +case object StringType extends StringType + /** * :: DeveloperApi :: @@ -313,7 +324,10 @@ case object StringType extends NativeType with PrimitiveType { * @group dataType */ @DeveloperApi -case object BinaryType extends NativeType with PrimitiveType { +class BinaryType private() extends NativeType with PrimitiveType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Array[Byte] @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = new Ordering[JvmType] { @@ -332,6 +346,8 @@ case object BinaryType extends NativeType with PrimitiveType { override def defaultSize: Int = 4096 } +case object BinaryType extends BinaryType + /** * :: DeveloperApi :: @@ -341,7 +357,10 @@ case object BinaryType extends NativeType with PrimitiveType { *@group dataType */ @DeveloperApi -case object BooleanType extends NativeType with PrimitiveType { +class BooleanType private() extends NativeType with PrimitiveType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Boolean @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] @@ -352,6 +371,8 @@ case object BooleanType extends NativeType with PrimitiveType { override def defaultSize: Int = 1 } +case object BooleanType extends BooleanType + /** * :: DeveloperApi :: @@ -362,7 +383,10 @@ case object BooleanType extends NativeType with PrimitiveType { * @group dataType */ @DeveloperApi -case object TimestampType extends NativeType { +class TimestampType private() extends NativeType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Timestamp @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -377,6 +401,8 @@ case object TimestampType extends NativeType { override def defaultSize: Int = 12 } +case object TimestampType extends TimestampType + /** * :: DeveloperApi :: @@ -387,7 +413,10 @@ case object TimestampType extends NativeType { * @group dataType */ @DeveloperApi -case object DateType extends NativeType { +class DateType private() extends NativeType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DateType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -400,6 +429,8 @@ case object DateType extends NativeType { override def defaultSize: Int = 4 } +case object DateType extends DateType + abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for @@ -438,7 +469,10 @@ protected[sql] sealed abstract class IntegralType extends NumericType { * @group dataType */ @DeveloperApi -case object LongType extends IntegralType { +class LongType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "LongType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Long @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Long]] @@ -453,6 +487,8 @@ case object LongType extends IntegralType { override def simpleString = "bigint" } +case object LongType extends LongType + /** * :: DeveloperApi :: @@ -462,7 +498,10 @@ case object LongType extends IntegralType { * @group dataType */ @DeveloperApi -case object IntegerType extends IntegralType { +class IntegerType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Int]] @@ -477,6 +516,8 @@ case object IntegerType extends IntegralType { override def simpleString = "int" } +case object IntegerType extends IntegerType + /** * :: DeveloperApi :: @@ -486,7 +527,10 @@ case object IntegerType extends IntegralType { * @group dataType */ @DeveloperApi -case object ShortType extends IntegralType { +class ShortType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Short @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Short]] @@ -501,6 +545,8 @@ case object ShortType extends IntegralType { override def simpleString = "smallint" } +case object ShortType extends ShortType + /** * :: DeveloperApi :: @@ -510,7 +556,10 @@ case object ShortType extends IntegralType { * @group dataType */ @DeveloperApi -case object ByteType extends IntegralType { +class ByteType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Byte @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Byte]] @@ -525,6 +574,8 @@ case object ByteType extends IntegralType { override def simpleString = "tinyint" } +case object ByteType extends ByteType + /** Matcher for any expressions that evaluate to [[FractionalType]]s */ protected[sql] object FractionalType { @@ -630,7 +681,10 @@ object DecimalType { * @group dataType */ @DeveloperApi -case object DoubleType extends FractionalType { +class DoubleType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Double @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Double]] @@ -644,6 +698,8 @@ case object DoubleType extends FractionalType { override def defaultSize: Int = 8 } +case object DoubleType extends DoubleType + /** * :: DeveloperApi :: @@ -653,7 +709,10 @@ case object DoubleType extends FractionalType { * @group dataType */ @DeveloperApi -case object FloatType extends FractionalType { +class FloatType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. private[sql] type JvmType = Float @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = implicitly[Numeric[Float]] @@ -667,6 +726,8 @@ case object FloatType extends FractionalType { override def defaultSize: Int = 4 } +case object FloatType extends FloatType + object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ From b884daa58084d4f42e2318894067565b94e07f9d Mon Sep 17 00:00:00 2001 From: Florian Verhein Date: Mon, 9 Feb 2015 23:47:07 +0000 Subject: [PATCH 017/817] [SPARK-5611] [EC2] Allow spark-ec2 repo and branch to be set on CLI of spark_ec2.py and by extension, the ami-list Useful for using alternate spark-ec2 repos or branches. Author: Florian Verhein Closes #4385 from florianverhein/master and squashes the following commits: 7e2b4be [Florian Verhein] [SPARK-5611] [EC2] typo 8b653dc [Florian Verhein] [SPARK-5611] [EC2] Enforce only supporting spark-ec2 forks from github, log improvement bc4b0ed [Florian Verhein] [SPARK-5611] allow spark-ec2 repos with different names 8b5c551 [Florian Verhein] improve option naming, fix logging, fix lint failing, add guard to enforce spark-ec2 7724308 [Florian Verhein] [SPARK-5611] [EC2] fixes b42b68c [Florian Verhein] [SPARK-5611] [EC2] Allow spark-ec2 repo and branch to be set on CLI of spark_ec2.py --- ec2/spark_ec2.py | 37 ++++++++++++++++++++++++++++++++----- 1 file changed, 32 insertions(+), 5 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 87b2112fe4628..3e4c49c0e1db6 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -62,10 +62,10 @@ DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" -MESOS_SPARK_EC2_BRANCH = "branch-1.3" -# A URL prefix from which to fetch AMI information -AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) +# Default location to get the spark-ec2 scripts (and ami-list) from +DEFAULT_SPARK_EC2_GITHUB_REPO = "https://github.com/mesos/spark-ec2" +DEFAULT_SPARK_EC2_BRANCH = "branch-1.3" def setup_boto(): @@ -147,6 +147,14 @@ def parse_args(): "--spark-git-repo", default=DEFAULT_SPARK_GITHUB_REPO, help="Github repo from which to checkout supplied commit hash (default: %default)") + parser.add_option( + "--spark-ec2-git-repo", + default=DEFAULT_SPARK_EC2_GITHUB_REPO, + help="Github repo from which to checkout spark-ec2 (default: %default)") + parser.add_option( + "--spark-ec2-git-branch", + default=DEFAULT_SPARK_EC2_BRANCH, + help="Github repo branch of spark-ec2 to use (default: %default)") parser.add_option( "--hadoop-major-version", default="1", help="Major version of Hadoop (default: %default)") @@ -333,7 +341,12 @@ def get_spark_ami(opts): print >> stderr,\ "Don't recognize %s, assuming type is pvm" % opts.instance_type - ami_path = "%s/%s/%s" % (AMI_PREFIX, opts.region, instance_type) + # URL prefix from which to fetch AMI information + ami_prefix = "{r}/{b}/ami-list".format( + r=opts.spark_ec2_git_repo.replace("https://github.com", "https://raw.github.com", 1), + b=opts.spark_ec2_git_branch) + + ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) try: ami = urllib2.urlopen(ami_path).read().strip() print "Spark AMI: " + ami @@ -650,12 +663,15 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten + print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( + r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch) ssh( host=master, opts=opts, command="rm -rf spark-ec2" + " && " - + "git clone https://github.com/mesos/spark-ec2.git -b {b}".format(b=MESOS_SPARK_EC2_BRANCH) + + "git clone {r} -b {b} spark-ec2".format(r=opts.spark_ec2_git_repo, + b=opts.spark_ec2_git_branch) ) print "Deploying files to master..." @@ -1038,6 +1054,17 @@ def real_main(): print >> stderr, "ebs-vol-num cannot be greater than 8" sys.exit(1) + # Prevent breaking ami_prefix (/, .git and startswith checks) + # Prevent forks with non spark-ec2 names for now. + if opts.spark_ec2_git_repo.endswith("/") or \ + opts.spark_ec2_git_repo.endswith(".git") or \ + not opts.spark_ec2_git_repo.startswith("https://github.com") or \ + not opts.spark_ec2_git_repo.endswith("spark-ec2"): + print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \ + "trailing / or .git. " \ + "Furthermore, we currently only support forks named spark-ec2." + sys.exit(1) + try: conn = ec2.connect_to_region(opts.region) except Exception as e: From 68b25cf695e0fce9e465288d5a053e540a3fccb4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 9 Feb 2015 16:02:56 -0800 Subject: [PATCH 018/817] [SQL] Add some missing DataFrame functions. - as with a `Symbol` - distinct - sqlContext.emptyDataFrame - move add/remove col out of RDDApi section Author: Michael Armbrust Closes #4437 from marmbrus/dfMissingFuncs and squashes the following commits: 2004023 [Michael Armbrust] Add missing functions --- .../scala/org/apache/spark/sql/Column.scala | 9 +++++ .../org/apache/spark/sql/DataFrame.scala | 12 +++++-- .../org/apache/spark/sql/DataFrameImpl.scala | 34 +++++++++++-------- .../apache/spark/sql/IncomputableColumn.scala | 10 +++--- .../scala/org/apache/spark/sql/RDDApi.scala | 2 ++ .../org/apache/spark/sql/SQLContext.scala | 5 ++- 6 files changed, 51 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 878b2b0556de7..1011bf0bb5ef4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -549,6 +549,15 @@ trait Column extends DataFrame { */ override def as(alias: String): Column = exprToColumn(Alias(expr, alias)()) + /** + * Gives the column an alias. + * {{{ + * // Renames colA to colB in select output. + * df.select($"colA".as('colB)) + * }}} + */ + override def as(alias: Symbol): Column = exprToColumn(Alias(expr, alias.name)()) + /** * Casts the column to a different data type. * {{{ 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 17ea3cde8e50e..6abfb7853cf1c 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 @@ -156,7 +156,7 @@ trait DataFrame extends RDDApi[Row] { def join(right: DataFrame, joinExprs: Column): DataFrame /** - * Join with another [[DataFrame]], usin g the given join expression. The following performs + * Join with another [[DataFrame]], using the given join expression. The following performs * a full outer join between `df1` and `df2`. * * {{{ @@ -233,7 +233,12 @@ trait DataFrame extends RDDApi[Row] { /** * Returns a new [[DataFrame]] with an alias set. */ - def as(name: String): DataFrame + def as(alias: String): DataFrame + + /** + * (Scala-specific) Returns a new [[DataFrame]] with an alias set. + */ + def as(alias: Symbol): DataFrame /** * Selects a set of expressions. @@ -516,6 +521,9 @@ trait DataFrame extends RDDApi[Row] { */ override def repartition(numPartitions: Int): DataFrame + /** Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. */ + override def distinct: DataFrame + override def persist(): this.type override def persist(newLevel: StorageLevel): this.type diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index fa05a5dcac6bf..73393295ab0a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -196,7 +196,9 @@ private[sql] class DataFrameImpl protected[sql]( }.toSeq :_*) } - override def as(name: String): DataFrame = Subquery(name, logicalPlan) + override def as(alias: String): DataFrame = Subquery(alias, logicalPlan) + + override def as(alias: Symbol): DataFrame = Subquery(alias.name, logicalPlan) override def select(cols: Column*): DataFrame = { val exprs = cols.zipWithIndex.map { @@ -215,7 +217,19 @@ private[sql] class DataFrameImpl protected[sql]( override def selectExpr(exprs: String*): DataFrame = { select(exprs.map { expr => Column(new SqlParser().parseExpression(expr)) - } :_*) + }: _*) + } + + override def addColumn(colName: String, col: Column): DataFrame = { + select(Column("*"), col.as(colName)) + } + + override def renameColumn(existingName: String, newName: String): DataFrame = { + val colNames = schema.map { field => + val name = field.name + if (name == existingName) Column(name).as(newName) else Column(name) + } + select(colNames :_*) } override def filter(condition: Column): DataFrame = { @@ -264,18 +278,8 @@ private[sql] class DataFrameImpl protected[sql]( } ///////////////////////////////////////////////////////////////////////////// - - override def addColumn(colName: String, col: Column): DataFrame = { - select(Column("*"), col.as(colName)) - } - - override def renameColumn(existingName: String, newName: String): DataFrame = { - val colNames = schema.map { field => - val name = field.name - if (name == existingName) Column(name).as(newName) else Column(name) - } - select(colNames :_*) - } + // RDD API + ///////////////////////////////////////////////////////////////////////////// override def head(n: Int): Array[Row] = limit(n).collect() @@ -307,6 +311,8 @@ private[sql] class DataFrameImpl protected[sql]( sqlContext.applySchema(rdd.repartition(numPartitions), schema) } + override def distinct: DataFrame = Distinct(logicalPlan) + override def persist(): this.type = { sqlContext.cacheManager.cacheQuery(this) this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 782f6e28eebb0..0600dcc226b4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -86,6 +86,10 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def selectExpr(exprs: String*): DataFrame = err() + override def addColumn(colName: String, col: Column): DataFrame = err() + + override def renameColumn(existingName: String, newName: String): DataFrame = err() + override def filter(condition: Column): DataFrame = err() override def filter(conditionExpr: String): DataFrame = err() @@ -110,10 +114,6 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten ///////////////////////////////////////////////////////////////////////////// - override def addColumn(colName: String, col: Column): DataFrame = err() - - override def renameColumn(existingName: String, newName: String): DataFrame = err() - override def head(n: Int): Array[Row] = err() override def head(): Row = err() @@ -140,6 +140,8 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def repartition(numPartitions: Int): DataFrame = err() + override def distinct: DataFrame = err() + override def persist(): this.type = err() override def persist(newLevel: StorageLevel): this.type = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index 38e6382f171d5..df866fd1ad8ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -60,4 +60,6 @@ private[sql] trait RDDApi[T] { def first(): T def repartition(numPartitions: Int): DataFrame + + def distinct: DataFrame } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index bf3990671029e..97e3777f933e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, NoRelation} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} @@ -130,6 +130,9 @@ class SQLContext(@transient val sparkContext: SparkContext) */ val experimental: ExperimentalMethods = new ExperimentalMethods(this) + /** Returns a [[DataFrame]] with no rows or columns. */ + lazy val emptyDataFrame = DataFrame(this, NoRelation) + /** * A collection of methods for registering user-defined functions (UDF). * From 5f0b30e59cc6a3017168189d3aaf09402699dc3b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 9 Feb 2015 16:20:42 -0800 Subject: [PATCH 019/817] [SQL] Code cleanup. I added an unnecessary line of code in https://github.com/apache/spark/commit/13531dd97c08563e53dacdaeaf1102bdd13ef825. My bad. Let's delete it. Author: Yin Huai Closes #4482 from yhuai/unnecessaryCode and squashes the following commits: 3645af0 [Yin Huai] Code cleanup. --- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index c23575fe96898..036efa84d7c85 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -351,9 +351,6 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |) """.stripMargin) - new Path("/Users/yhuai/Desktop/whatever") - - val expectedPath = catalog.hiveDefaultTableFilePath("ctasJsonTable") val filesystemPath = new Path(expectedPath) val fs = filesystemPath.getFileSystem(sparkContext.hadoopConfiguration) From b8080aa86d55e0467fd4328f10a2f0d6605e6cc6 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 9 Feb 2015 16:23:12 -0800 Subject: [PATCH 020/817] [SPARK-5696] [SQL] [HOTFIX] Asks HiveThriftServer2 to re-initialize log4j using Hive configurations In this way, log4j configurations overriden by jets3t-0.9.2.jar can be again overriden by Hive default log4j configurations. This might not be the best solution for this issue since it requires users to use `hive-log4j.properties` rather than `log4j.properties` to initialize `HiveThriftServer2` logging configurations, which can be confusing. The main purpose of this PR is to fix Jenkins PR build. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4484) Author: Cheng Lian Closes #4484 from liancheng/spark-5696 and squashes the following commits: df83956 [Cheng Lian] Hot fix: asks HiveThriftServer2 to re-initialize log4j using Hive configurations --- .../apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 6e07df18b0e15..525777aa454c4 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.common.LogUtils import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} @@ -54,6 +55,8 @@ object HiveThriftServer2 extends Logging { System.exit(-1) } + LogUtils.initHiveLog4j() + logInfo("Starting SparkContext") SparkSQLEnv.init() From 2a36292534a1e9f7a501e88f69bfc3a09fb62cb3 Mon Sep 17 00:00:00 2001 From: Lu Yan Date: Mon, 9 Feb 2015 16:25:38 -0800 Subject: [PATCH 021/817] [SPARK-5614][SQL] Predicate pushdown through Generate. Now in Catalyst's rules, predicates can not be pushed through "Generate" nodes. Further more, partition pruning in HiveTableScan can not be applied on those queries involves "Generate". This makes such queries very inefficient. In practice, it finds patterns like ```scala Filter(predicate, Generate(generator, _, _, _, grandChild)) ``` and splits the predicate into 2 parts by referencing the generated column from Generate node or not. And a new Filter will be created for those conjuncts can be pushed beneath Generate node. If nothing left for the original Filter, it will be removed. For example, physical plan for query ```sql select len, bk from s_server lateral view explode(len_arr) len_table as len where len > 5 and day = '20150102'; ``` where 'day' is a partition column in metastore is like this in current version of Spark SQL: > Project [len, bk] > > Filter ((len > "5") && "(day = "20150102")") > > Generate explode(len_arr), true, false > > HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), None But theoretically the plan should be like this > Project [len, bk] > > Filter (len > "5") > > Generate explode(len_arr), true, false > > HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), Some(day = "20150102") Where partition pruning predicates can be pushed to HiveTableScan nodes. Author: Lu Yan Closes #4394 from ianluyan/ppd and squashes the following commits: a67dce9 [Lu Yan] Fix English grammar. 7cea911 [Lu Yan] Revised based on @marmbrus's opinions ffc59fc [Lu Yan] [SPARK-5614][SQL] Predicate pushdown through Generate. --- .../sql/catalyst/optimizer/Optimizer.scala | 25 ++++++++ .../optimizer/FilterPushdownSuite.scala | 63 ++++++++++++++++++- 2 files changed, 87 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3bc48c95c5653..fd58b9681ea24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -50,6 +50,7 @@ object DefaultOptimizer extends Optimizer { CombineFilters, PushPredicateThroughProject, PushPredicateThroughJoin, + PushPredicateThroughGenerate, ColumnPruning) :: Batch("LocalRelation", FixedPoint(100), ConvertToLocalRelation) :: Nil @@ -455,6 +456,30 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { } } +/** + * Push [[Filter]] operators through [[Generate]] operators. Parts of the predicate that reference + * attributes generated in [[Generate]] will remain above, and the rest should be pushed beneath. + */ +object PushPredicateThroughGenerate extends Rule[LogicalPlan] with PredicateHelper { + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case filter @ Filter(condition, + generate @ Generate(generator, join, outer, alias, grandChild)) => + // Predicates that reference attributes produced by the `Generate` operator cannot + // be pushed below the operator. + val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { + conjunct => conjunct.references subsetOf grandChild.outputSet + } + if (pushDown.nonEmpty) { + val pushDownPredicate = pushDown.reduce(And) + val withPushdown = generate.copy(child = Filter(pushDownPredicate, grandChild)) + stayUp.reduceOption(And).map(Filter(_, withPushdown)).getOrElse(withPushdown) + } else { + filter + } + } +} + /** * Pushes down [[Filter]] operators where the `condition` can be * evaluated using only the attributes of the left or right side of a join. Other diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index ebb123c1f909e..1158b5dfc6147 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.expressions.Explode import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types.IntegerType class FilterPushdownSuite extends PlanTest { @@ -34,7 +36,8 @@ class FilterPushdownSuite extends PlanTest { Batch("Filter Pushdown", Once, CombineFilters, PushPredicateThroughProject, - PushPredicateThroughJoin) :: Nil + PushPredicateThroughJoin, + PushPredicateThroughGenerate) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -411,4 +414,62 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) } + + val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + + test("generate: predicate referenced no generated column") { + val originalQuery = { + testRelationWithArrayType + .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .where(('b >= 5) && ('a > 6)) + } + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = { + testRelationWithArrayType + .where(('b >= 5) && ('a > 6)) + .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")).analyze + } + + comparePlans(optimized, correctAnswer) + } + + test("generate: part of conjuncts referenced generated column") { + val generator = Explode(Seq("c"), 'c_arr) + val originalQuery = { + testRelationWithArrayType + .generate(generator, true, false, Some("arr")) + .where(('b >= 5) && ('c > 6)) + } + val optimized = Optimize(originalQuery.analyze) + val referenceResult = { + testRelationWithArrayType + .where('b >= 5) + .generate(generator, true, false, Some("arr")) + .where('c > 6).analyze + } + + // Since newly generated columns get different ids every time being analyzed + // e.g. comparePlans(originalQuery.analyze, originalQuery.analyze) fails. + // So we check operators manually here. + // Filter("c" > 6) + assertResult(classOf[Filter])(optimized.getClass) + assertResult(1)(optimized.asInstanceOf[Filter].condition.references.size) + assertResult("c"){ + optimized.asInstanceOf[Filter].condition.references.toSeq(0).name + } + + // the rest part + comparePlans(optimized.children(0), referenceResult.children(0)) + } + + test("generate: all conjuncts referenced generated column") { + val originalQuery = { + testRelationWithArrayType + .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .where(('c > 6) || ('b > 5)).analyze + } + val optimized = Optimize(originalQuery) + + comparePlans(optimized, originalQuery) + } } From 0ee53ebce9944722e76b2b28fae79d9956be9f17 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 9 Feb 2015 16:39:34 -0800 Subject: [PATCH 022/817] [SPARK-2096][SQL] support dot notation on array of struct ~~The rule is simple: If you want `a.b` work, then `a` must be some level of nested array of struct(level 0 means just a StructType). And the result of `a.b` is same level of nested array of b-type. An optimization is: the resolve chain looks like `Attribute -> GetItem -> GetField -> GetField ...`, so we could transmit the nested array information between `GetItem` and `GetField` to avoid repeated computation of `innerDataType` and `containsNullList` of that nested array.~~ marmbrus Could you take a look? to evaluate `a.b`, if `a` is array of struct, then `a.b` means get field `b` on each element of `a`, and return a result of array. Author: Wenchen Fan Closes #2405 from cloud-fan/nested-array-dot and squashes the following commits: 08a228a [Wenchen Fan] support dot notation on array of struct --- .../sql/catalyst/analysis/Analyzer.scala | 30 +++++++++------- .../catalyst/expressions/complexTypes.scala | 34 ++++++++++++++++--- .../sql/catalyst/optimizer/Optimizer.scala | 3 +- .../ExpressionEvaluationSuite.scala | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 6 ++-- 5 files changed, 53 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0b59ed1739566..fb2ff014cef07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -22,8 +22,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{ArrayType, StructField, StructType, IntegerType} /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -311,18 +310,25 @@ class Analyzer(catalog: Catalog, * desired fields are found. */ protected def resolveGetField(expr: Expression, fieldName: String): Expression = { + def findField(fields: Array[StructField]): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + sys.error( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + sys.error(s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } expr.dataType match { case StructType(fields) => - val actualField = fields.filter(f => resolver(f.name, fieldName)) - if (actualField.length == 0) { - sys.error( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (actualField.length == 1) { - val field = actualField(0) - GetField(expr, field, fields.indexOf(field)) - } else { - sys.error(s"Ambiguous reference to fields ${actualField.mkString(", ")}") - } + val ordinal = findField(fields) + StructGetField(expr, fields(ordinal), ordinal) + case ArrayType(StructType(fields), containsNull) => + val ordinal = findField(fields) + ArrayGetField(expr, fields(ordinal), ordinal, containsNull) case otherType => sys.error(s"GetField is not valid on fields of type $otherType") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 66e2e5c4bafce..68051a2a2007e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -70,22 +70,48 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { } } + +trait GetField extends UnaryExpression { + self: Product => + + type EvaluatedType = Any + override def foldable = child.foldable + override def toString = s"$child.${field.name}" + + def field: StructField +} + /** * Returns the value of fields in the Struct `child`. */ -case class GetField(child: Expression, field: StructField, ordinal: Int) extends UnaryExpression { - type EvaluatedType = Any +case class StructGetField(child: Expression, field: StructField, ordinal: Int) extends GetField { def dataType = field.dataType override def nullable = child.nullable || field.nullable - override def foldable = child.foldable override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Row] if (baseValue == null) null else baseValue(ordinal) } +} - override def toString = s"$child.${field.name}" +/** + * Returns the array of value of fields in the Array of Struct `child`. + */ +case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, containsNull: Boolean) + extends GetField { + + def dataType = ArrayType(field.dataType, containsNull) + override def nullable = child.nullable + + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Seq[Row]] + if (baseValue == null) null else { + baseValue.map { row => + if (row == null) null else row(ordinal) + } + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index fd58b9681ea24..0da081ed1a6e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -209,7 +209,8 @@ object NullPropagation extends Rule[LogicalPlan] { case e @ IsNotNull(c) if !c.nullable => Literal(true, BooleanType) case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) - case e @ GetField(Literal(null, _), _, _) => Literal(null, e.dataType) + case e @ StructGetField(Literal(null, _), _, _) => Literal(null, e.dataType) + case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal(null, e.dataType) case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) case e @ Count(expr) if !expr.nullable => Count(Literal(1)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 7cf6c80194f6c..dcfd8b28cb02a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -851,7 +851,7 @@ class ExpressionEvaluationSuite extends FunSuite { expr.dataType match { case StructType(fields) => val field = fields.find(_.name == fieldName).get - GetField(expr, field, fields.indexOf(field)) + StructGetField(expr, field, fields.indexOf(field)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 926ba68828ee8..7870cf9b0a868 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -342,21 +342,19 @@ class JsonSuite extends QueryTest { ) } - ignore("Complex field and type inferring (Ignored)") { + test("GetField operation on complex data type") { val jsonDF = jsonRDD(complexFieldAndType1) jsonDF.registerTempTable("jsonTable") - // Right now, "field1" and "field2" are treated as aliases. We should fix it. checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), Row(true, "str1") ) - // Right now, the analyzer cannot resolve arrayOfStruct.field1 and arrayOfStruct.field2. // Getting all values of a specific field from an array of structs. checkAnswer( sql("select arrayOfStruct.field1, arrayOfStruct.field2 from jsonTable"), - Row(Seq(true, false), Seq("str1", null)) + Row(Seq(true, false, null), Seq("str1", null, null)) ) } From d08e7c2b498584609cb3c7922eaaa2a0d115603f Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Mon, 9 Feb 2015 16:40:26 -0800 Subject: [PATCH 023/817] [SPARK-5648][SQL] support "alter ... unset tblproperties("key")" make hivecontext support "alter ... unset tblproperties("key")" like : alter view viewName unset tblproperties("k") alter table tableName unset tblproperties("k") Author: DoingDone9 <799203320@qq.com> Closes #4424 from DoingDone9/unset and squashes the following commits: 6dd8bee [DoingDone9] support "alter ... unset tblproperties("key")" --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 2a4b88092179f..f51af62d3340b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -103,6 +103,7 @@ private[hive] object HiveQl { "TOK_CREATEINDEX", "TOK_DROPDATABASE", "TOK_DROPINDEX", + "TOK_DROPTABLE_PROPERTIES", "TOK_MSCK", "TOK_ALTERVIEW_ADDPARTS", @@ -111,6 +112,7 @@ private[hive] object HiveQl { "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", "TOK_CREATEVIEW", + "TOK_DROPVIEW_PROPERTIES", "TOK_DROPVIEW", "TOK_EXPORT", From 3ec3ad295ddd1435da68251b7479ffb60aec7037 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 9 Feb 2015 16:52:05 -0800 Subject: [PATCH 024/817] [SPARK-5699] [SQL] [Tests] Runs hive-thriftserver tests whenever SQL code is modified [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4486) Author: Cheng Lian Closes #4486 from liancheng/spark-5699 and squashes the following commits: 538001d [Cheng Lian] Runs hive-thriftserver tests whenever SQL code is modified --- dev/run-tests | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 2257a566bb1bb..483958757a2dd 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -36,7 +36,7 @@ function handle_error () { } -# Build against the right verison of Hadoop. +# Build against the right version of Hadoop. { if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then @@ -77,7 +77,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" fi } -# Only run Hive tests if there are sql changes. +# Only run Hive tests if there are SQL changes. # Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master @@ -183,7 +183,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string # will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") fi From d302c4800bf2f74eceb731169ddf1766136b7398 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 9 Feb 2015 17:33:29 -0800 Subject: [PATCH 025/817] [SPARK-5698] Do not let user request negative # of executors Otherwise we might crash the ApplicationMaster. Why? Please see https://issues.apache.org/jira/browse/SPARK-5698. sryza I believe this is also relevant in your patch #4168. Author: Andrew Or Closes #4483 from andrewor14/da-negative and squashes the following commits: 53ed955 [Andrew Or] Throw IllegalArgumentException instead 0e89fd5 [Andrew Or] Check against negative requests --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 9d2fb4f3b4729..f9ca93432bf41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -314,6 +314,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + if (numAdditionalExecutors < 0) { + throw new IllegalArgumentException( + "Attempted to request a negative number of additional executor(s) " + + s"$numAdditionalExecutors from the cluster manager. Please specify a positive number!") + } logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") logDebug(s"Number of pending executors is now $numPendingExecutors") numPendingExecutors += numAdditionalExecutors From 08488c175f2e8532cb6aab84da2abd9ad57179cc Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 9 Feb 2015 20:49:22 -0800 Subject: [PATCH 026/817] [SPARK-5469] restructure pyspark.sql into multiple files All the DataTypes moved into pyspark.sql.types The changes can be tracked by `--find-copies-harder -M25` ``` davieslocalhost:~/work/spark/python$ git diff --find-copies-harder -M25 --numstat master.. 2 5 python/docs/pyspark.ml.rst 0 3 python/docs/pyspark.mllib.rst 10 2 python/docs/pyspark.sql.rst 1 1 python/pyspark/mllib/linalg.py 21 14 python/pyspark/{mllib => sql}/__init__.py 14 2108 python/pyspark/{sql.py => sql/context.py} 10 1772 python/pyspark/{sql.py => sql/dataframe.py} 7 6 python/pyspark/{sql_tests.py => sql/tests.py} 8 1465 python/pyspark/{sql.py => sql/types.py} 4 2 python/run-tests 1 1 sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala ``` Also `git blame -C -C python/pyspark/sql/context.py` to track the history. Author: Davies Liu Closes #4479 from davies/sql and squashes the following commits: 1b5f0a5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into sql 2b2b983 [Davies Liu] restructure pyspark.sql --- python/docs/pyspark.ml.rst | 7 +- python/docs/pyspark.mllib.rst | 3 - python/docs/pyspark.sql.rst | 12 +- python/pyspark/mllib/linalg.py | 2 +- python/pyspark/sql.py | 2736 ----------------- python/pyspark/sql/__init__.py | 42 + python/pyspark/sql/context.py | 642 ++++ python/pyspark/sql/dataframe.py | 974 ++++++ python/pyspark/{sql_tests.py => sql/tests.py} | 13 +- python/pyspark/sql/types.py | 1279 ++++++++ python/run-tests | 6 +- .../spark/sql/test/ExamplePointUDT.scala | 2 +- 12 files changed, 2962 insertions(+), 2756 deletions(-) delete mode 100644 python/pyspark/sql.py create mode 100644 python/pyspark/sql/__init__.py create mode 100644 python/pyspark/sql/context.py create mode 100644 python/pyspark/sql/dataframe.py rename python/pyspark/{sql_tests.py => sql/tests.py} (96%) create mode 100644 python/pyspark/sql/types.py diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index f10d1339a9a8f..4da6d4a74a299 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -1,11 +1,8 @@ pyspark.ml package ===================== -Submodules ----------- - -pyspark.ml module ------------------ +Module Context +-------------- .. automodule:: pyspark.ml :members: diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index 4548b8739ed91..21f66ca344a3c 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -1,9 +1,6 @@ pyspark.mllib package ===================== -Submodules ----------- - pyspark.mllib.classification module ----------------------------------- diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index 65b3650ae10ab..80c6f02a9df41 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -1,10 +1,18 @@ pyspark.sql module ================== -Module contents ---------------- +Module Context +-------------- .. automodule:: pyspark.sql :members: :undoc-members: :show-inheritance: + + +pyspark.sql.types module +------------------------ +.. automodule:: pyspark.sql.types + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 7f21190ed8c25..597012b1c967c 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,7 +29,7 @@ import numpy as np -from pyspark.sql import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ +from pyspark.sql.types import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ IntegerType, ByteType diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py deleted file mode 100644 index 6a6dfbc5851b8..0000000000000 --- a/python/pyspark/sql.py +++ /dev/null @@ -1,2736 +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. -# - -""" -public classes of Spark SQL: - - - L{SQLContext} - Main entry point for SQL functionality. - - L{DataFrame} - A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In - addition to normal RDD operations, DataFrames also support SQL. - - L{GroupedData} - - L{Column} - Column is a DataFrame with a single column. - - L{Row} - A Row of data returned by a Spark SQL query. - - L{HiveContext} - Main entry point for accessing data stored in Apache Hive.. -""" - -import sys -import itertools -import decimal -import datetime -import keyword -import warnings -import json -import re -import random -import os -from tempfile import NamedTemporaryFile -from array import array -from operator import itemgetter -from itertools import imap - -from py4j.protocol import Py4JError -from py4j.java_collections import ListConverter, MapConverter - -from pyspark.context import SparkContext -from pyspark.rdd import RDD, _prepare_for_python_RDD -from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \ - CloudPickleSerializer, UTF8Deserializer -from pyspark.storagelevel import StorageLevel -from pyspark.traceback_utils import SCCallSiteSync - - -__all__ = [ - "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", - "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", - "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "DataFrame", "GroupedData", "Column", "Row", "Dsl", - "SchemaRDD"] - - -class DataType(object): - - """Spark SQL DataType""" - - def __repr__(self): - return self.__class__.__name__ - - def __hash__(self): - return hash(str(self)) - - def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.__dict__ == other.__dict__) - - def __ne__(self, other): - return not self.__eq__(other) - - @classmethod - def typeName(cls): - return cls.__name__[:-4].lower() - - def jsonValue(self): - return self.typeName() - - def json(self): - return json.dumps(self.jsonValue(), - separators=(',', ':'), - sort_keys=True) - - -class PrimitiveTypeSingleton(type): - - """Metaclass for PrimitiveType""" - - _instances = {} - - def __call__(cls): - if cls not in cls._instances: - cls._instances[cls] = super(PrimitiveTypeSingleton, cls).__call__() - return cls._instances[cls] - - -class PrimitiveType(DataType): - - """Spark SQL PrimitiveType""" - - __metaclass__ = PrimitiveTypeSingleton - - def __eq__(self, other): - # because they should be the same object - return self is other - - -class NullType(PrimitiveType): - - """Spark SQL NullType - - The data type representing None, used for the types which has not - been inferred. - """ - - -class StringType(PrimitiveType): - - """Spark SQL StringType - - The data type representing string values. - """ - - -class BinaryType(PrimitiveType): - - """Spark SQL BinaryType - - The data type representing bytearray values. - """ - - -class BooleanType(PrimitiveType): - - """Spark SQL BooleanType - - The data type representing bool values. - """ - - -class DateType(PrimitiveType): - - """Spark SQL DateType - - The data type representing datetime.date values. - """ - - -class TimestampType(PrimitiveType): - - """Spark SQL TimestampType - - The data type representing datetime.datetime values. - """ - - -class DecimalType(DataType): - - """Spark SQL DecimalType - - The data type representing decimal.Decimal values. - """ - - def __init__(self, precision=None, scale=None): - self.precision = precision - self.scale = scale - self.hasPrecisionInfo = precision is not None - - def jsonValue(self): - if self.hasPrecisionInfo: - return "decimal(%d,%d)" % (self.precision, self.scale) - else: - return "decimal" - - def __repr__(self): - if self.hasPrecisionInfo: - return "DecimalType(%d,%d)" % (self.precision, self.scale) - else: - return "DecimalType()" - - -class DoubleType(PrimitiveType): - - """Spark SQL DoubleType - - The data type representing float values. - """ - - -class FloatType(PrimitiveType): - - """Spark SQL FloatType - - The data type representing single precision floating-point values. - """ - - -class ByteType(PrimitiveType): - - """Spark SQL ByteType - - The data type representing int values with 1 singed byte. - """ - - -class IntegerType(PrimitiveType): - - """Spark SQL IntegerType - - The data type representing int values. - """ - - -class LongType(PrimitiveType): - - """Spark SQL LongType - - The data type representing long values. If the any value is - beyond the range of [-9223372036854775808, 9223372036854775807], - please use DecimalType. - """ - - -class ShortType(PrimitiveType): - - """Spark SQL ShortType - - The data type representing int values with 2 signed bytes. - """ - - -class ArrayType(DataType): - - """Spark SQL ArrayType - - The data type representing list values. An ArrayType object - comprises two fields, elementType (a DataType) and containsNull (a bool). - The field of elementType is used to specify the type of array elements. - The field of containsNull is used to specify if the array has None values. - - """ - - def __init__(self, elementType, containsNull=True): - """Creates an ArrayType - - :param elementType: the data type of elements. - :param containsNull: indicates whether the list contains None values. - - >>> ArrayType(StringType) == ArrayType(StringType, True) - True - >>> ArrayType(StringType, False) == ArrayType(StringType) - False - """ - self.elementType = elementType - self.containsNull = containsNull - - def __repr__(self): - return "ArrayType(%s,%s)" % (self.elementType, - str(self.containsNull).lower()) - - def jsonValue(self): - return {"type": self.typeName(), - "elementType": self.elementType.jsonValue(), - "containsNull": self.containsNull} - - @classmethod - def fromJson(cls, json): - return ArrayType(_parse_datatype_json_value(json["elementType"]), - json["containsNull"]) - - -class MapType(DataType): - - """Spark SQL MapType - - The data type representing dict values. A MapType object comprises - three fields, keyType (a DataType), valueType (a DataType) and - valueContainsNull (a bool). - - The field of keyType is used to specify the type of keys in the map. - The field of valueType is used to specify the type of values in the map. - The field of valueContainsNull is used to specify if values of this - map has None values. - - For values of a MapType column, keys are not allowed to have None values. - - """ - - def __init__(self, keyType, valueType, valueContainsNull=True): - """Creates a MapType - :param keyType: the data type of keys. - :param valueType: the data type of values. - :param valueContainsNull: indicates whether values contains - null values. - - >>> (MapType(StringType, IntegerType) - ... == MapType(StringType, IntegerType, True)) - True - >>> (MapType(StringType, IntegerType, False) - ... == MapType(StringType, FloatType)) - False - """ - self.keyType = keyType - self.valueType = valueType - self.valueContainsNull = valueContainsNull - - def __repr__(self): - return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, - str(self.valueContainsNull).lower()) - - def jsonValue(self): - return {"type": self.typeName(), - "keyType": self.keyType.jsonValue(), - "valueType": self.valueType.jsonValue(), - "valueContainsNull": self.valueContainsNull} - - @classmethod - def fromJson(cls, json): - return MapType(_parse_datatype_json_value(json["keyType"]), - _parse_datatype_json_value(json["valueType"]), - json["valueContainsNull"]) - - -class StructField(DataType): - - """Spark SQL StructField - - Represents a field in a StructType. - A StructField object comprises three fields, name (a string), - dataType (a DataType) and nullable (a bool). The field of name - is the name of a StructField. The field of dataType specifies - the data type of a StructField. - - The field of nullable specifies if values of a StructField can - contain None values. - - """ - - def __init__(self, name, dataType, nullable=True, metadata=None): - """Creates a StructField - :param name: the name of this field. - :param dataType: the data type of this field. - :param nullable: indicates whether values of this field - can be null. - :param metadata: metadata of this field, which is a map from string - to simple type that can be serialized to JSON - automatically - - >>> (StructField("f1", StringType, True) - ... == StructField("f1", StringType, True)) - True - >>> (StructField("f1", StringType, True) - ... == StructField("f2", StringType, True)) - False - """ - self.name = name - self.dataType = dataType - self.nullable = nullable - self.metadata = metadata or {} - - def __repr__(self): - return "StructField(%s,%s,%s)" % (self.name, self.dataType, - str(self.nullable).lower()) - - def jsonValue(self): - return {"name": self.name, - "type": self.dataType.jsonValue(), - "nullable": self.nullable, - "metadata": self.metadata} - - @classmethod - def fromJson(cls, json): - return StructField(json["name"], - _parse_datatype_json_value(json["type"]), - json["nullable"], - json["metadata"]) - - -class StructType(DataType): - - """Spark SQL StructType - - The data type representing rows. - A StructType object comprises a list of L{StructField}. - - """ - - def __init__(self, fields): - """Creates a StructType - - >>> struct1 = StructType([StructField("f1", StringType, True)]) - >>> struct2 = StructType([StructField("f1", StringType, True)]) - >>> struct1 == struct2 - True - >>> struct1 = StructType([StructField("f1", StringType, True)]) - >>> struct2 = StructType([StructField("f1", StringType, True), - ... [StructField("f2", IntegerType, False)]]) - >>> struct1 == struct2 - False - """ - self.fields = fields - - def __repr__(self): - return ("StructType(List(%s))" % - ",".join(str(field) for field in self.fields)) - - def jsonValue(self): - return {"type": self.typeName(), - "fields": [f.jsonValue() for f in self.fields]} - - @classmethod - def fromJson(cls, json): - return StructType([StructField.fromJson(f) for f in json["fields"]]) - - -class UserDefinedType(DataType): - """ - .. note:: WARN: Spark Internal Use Only - SQL User-Defined Type (UDT). - """ - - @classmethod - def typeName(cls): - return cls.__name__.lower() - - @classmethod - def sqlType(cls): - """ - Underlying SQL storage type for this UDT. - """ - raise NotImplementedError("UDT must implement sqlType().") - - @classmethod - def module(cls): - """ - The Python module of the UDT. - """ - raise NotImplementedError("UDT must implement module().") - - @classmethod - def scalaUDT(cls): - """ - The class name of the paired Scala UDT. - """ - raise NotImplementedError("UDT must have a paired Scala UDT.") - - def serialize(self, obj): - """ - Converts the a user-type object into a SQL datum. - """ - raise NotImplementedError("UDT must implement serialize().") - - def deserialize(self, datum): - """ - Converts a SQL datum into a user-type object. - """ - raise NotImplementedError("UDT must implement deserialize().") - - def json(self): - return json.dumps(self.jsonValue(), separators=(',', ':'), sort_keys=True) - - def jsonValue(self): - schema = { - "type": "udt", - "class": self.scalaUDT(), - "pyClass": "%s.%s" % (self.module(), type(self).__name__), - "sqlType": self.sqlType().jsonValue() - } - return schema - - @classmethod - def fromJson(cls, json): - pyUDT = json["pyClass"] - split = pyUDT.rfind(".") - pyModule = pyUDT[:split] - pyClass = pyUDT[split+1:] - m = __import__(pyModule, globals(), locals(), [pyClass], -1) - UDT = getattr(m, pyClass) - return UDT() - - def __eq__(self, other): - return type(self) == type(other) - - -_all_primitive_types = dict((v.typeName(), v) - for v in globals().itervalues() - if type(v) is PrimitiveTypeSingleton and - v.__base__ == PrimitiveType) - - -_all_complex_types = dict((v.typeName(), v) - for v in [ArrayType, MapType, StructType]) - - -def _parse_datatype_json_string(json_string): - """Parses the given data type JSON string. - >>> def check_datatype(datatype): - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) - ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) - ... return datatype == python_datatype - >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) - True - >>> # Simple ArrayType. - >>> simple_arraytype = ArrayType(StringType(), True) - >>> check_datatype(simple_arraytype) - True - >>> # Simple MapType. - >>> simple_maptype = MapType(StringType(), LongType()) - >>> check_datatype(simple_maptype) - True - >>> # Simple StructType. - >>> simple_structtype = StructType([ - ... StructField("a", DecimalType(), False), - ... StructField("b", BooleanType(), True), - ... StructField("c", LongType(), True), - ... StructField("d", BinaryType(), False)]) - >>> check_datatype(simple_structtype) - True - >>> # Complex StructType. - >>> complex_structtype = StructType([ - ... StructField("simpleArray", simple_arraytype, True), - ... StructField("simpleMap", simple_maptype, True), - ... StructField("simpleStruct", simple_structtype, True), - ... StructField("boolean", BooleanType(), False), - ... StructField("withMeta", DoubleType(), False, {"name": "age"})]) - >>> check_datatype(complex_structtype) - True - >>> # Complex ArrayType. - >>> complex_arraytype = ArrayType(complex_structtype, True) - >>> check_datatype(complex_arraytype) - True - >>> # Complex MapType. - >>> complex_maptype = MapType(complex_structtype, - ... complex_arraytype, False) - >>> check_datatype(complex_maptype) - True - >>> check_datatype(ExamplePointUDT()) - True - >>> structtype_with_udt = StructType([StructField("label", DoubleType(), False), - ... StructField("point", ExamplePointUDT(), False)]) - >>> check_datatype(structtype_with_udt) - True - """ - return _parse_datatype_json_value(json.loads(json_string)) - - -_FIXED_DECIMAL = re.compile("decimal\\((\\d+),(\\d+)\\)") - - -def _parse_datatype_json_value(json_value): - if type(json_value) is unicode: - if json_value in _all_primitive_types.keys(): - return _all_primitive_types[json_value]() - elif json_value == u'decimal': - return DecimalType() - elif _FIXED_DECIMAL.match(json_value): - m = _FIXED_DECIMAL.match(json_value) - return DecimalType(int(m.group(1)), int(m.group(2))) - else: - raise ValueError("Could not parse datatype: %s" % json_value) - else: - tpe = json_value["type"] - if tpe in _all_complex_types: - return _all_complex_types[tpe].fromJson(json_value) - elif tpe == 'udt': - return UserDefinedType.fromJson(json_value) - else: - raise ValueError("not supported type: %s" % tpe) - - -# Mapping Python types to Spark SQL DataType -_type_mappings = { - type(None): NullType, - bool: BooleanType, - int: IntegerType, - long: LongType, - float: DoubleType, - str: StringType, - unicode: StringType, - bytearray: BinaryType, - decimal.Decimal: DecimalType, - datetime.date: DateType, - datetime.datetime: TimestampType, - datetime.time: TimestampType, -} - - -def _infer_type(obj): - """Infer the DataType from obj - - >>> p = ExamplePoint(1.0, 2.0) - >>> _infer_type(p) - ExamplePointUDT - """ - if obj is None: - raise ValueError("Can not infer type for None") - - if hasattr(obj, '__UDT__'): - return obj.__UDT__ - - dataType = _type_mappings.get(type(obj)) - if dataType is not None: - return dataType() - - if isinstance(obj, dict): - for key, value in obj.iteritems(): - if key is not None and value is not None: - return MapType(_infer_type(key), _infer_type(value), True) - else: - return MapType(NullType(), NullType(), True) - elif isinstance(obj, (list, array)): - for v in obj: - if v is not None: - return ArrayType(_infer_type(obj[0]), True) - else: - return ArrayType(NullType(), True) - else: - try: - return _infer_schema(obj) - except ValueError: - raise ValueError("not supported type: %s" % type(obj)) - - -def _infer_schema(row): - """Infer the schema from dict/namedtuple/object""" - if isinstance(row, dict): - items = sorted(row.items()) - - elif isinstance(row, tuple): - if hasattr(row, "_fields"): # namedtuple - items = zip(row._fields, tuple(row)) - elif hasattr(row, "__FIELDS__"): # Row - items = zip(row.__FIELDS__, tuple(row)) - elif all(isinstance(x, tuple) and len(x) == 2 for x in row): - items = row - else: - raise ValueError("Can't infer schema from tuple") - - elif hasattr(row, "__dict__"): # object - items = sorted(row.__dict__.items()) - - else: - raise ValueError("Can not infer schema for type: %s" % type(row)) - - fields = [StructField(k, _infer_type(v), True) for k, v in items] - return StructType(fields) - - -def _need_python_to_sql_conversion(dataType): - """ - Checks whether we need python to sql conversion for the given type. - For now, only UDTs need this conversion. - - >>> _need_python_to_sql_conversion(DoubleType()) - False - >>> schema0 = StructType([StructField("indices", ArrayType(IntegerType(), False), False), - ... StructField("values", ArrayType(DoubleType(), False), False)]) - >>> _need_python_to_sql_conversion(schema0) - False - >>> _need_python_to_sql_conversion(ExamplePointUDT()) - True - >>> schema1 = ArrayType(ExamplePointUDT(), False) - >>> _need_python_to_sql_conversion(schema1) - True - >>> schema2 = StructType([StructField("label", DoubleType(), False), - ... StructField("point", ExamplePointUDT(), False)]) - >>> _need_python_to_sql_conversion(schema2) - True - """ - if isinstance(dataType, StructType): - return any([_need_python_to_sql_conversion(f.dataType) for f in dataType.fields]) - elif isinstance(dataType, ArrayType): - return _need_python_to_sql_conversion(dataType.elementType) - elif isinstance(dataType, MapType): - return _need_python_to_sql_conversion(dataType.keyType) or \ - _need_python_to_sql_conversion(dataType.valueType) - elif isinstance(dataType, UserDefinedType): - return True - else: - return False - - -def _python_to_sql_converter(dataType): - """ - Returns a converter that converts a Python object into a SQL datum for the given type. - - >>> conv = _python_to_sql_converter(DoubleType()) - >>> conv(1.0) - 1.0 - >>> conv = _python_to_sql_converter(ArrayType(DoubleType(), False)) - >>> conv([1.0, 2.0]) - [1.0, 2.0] - >>> conv = _python_to_sql_converter(ExamplePointUDT()) - >>> conv(ExamplePoint(1.0, 2.0)) - [1.0, 2.0] - >>> schema = StructType([StructField("label", DoubleType(), False), - ... StructField("point", ExamplePointUDT(), False)]) - >>> conv = _python_to_sql_converter(schema) - >>> conv((1.0, ExamplePoint(1.0, 2.0))) - (1.0, [1.0, 2.0]) - """ - if not _need_python_to_sql_conversion(dataType): - return lambda x: x - - if isinstance(dataType, StructType): - names, types = zip(*[(f.name, f.dataType) for f in dataType.fields]) - converters = map(_python_to_sql_converter, types) - - def converter(obj): - if isinstance(obj, dict): - return tuple(c(obj.get(n)) for n, c in zip(names, converters)) - elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): - return tuple(c(v) for c, v in zip(converters, obj)) - elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs - d = dict(obj) - return tuple(c(d.get(n)) for n, c in zip(names, converters)) - else: - return tuple(c(v) for c, v in zip(converters, obj)) - else: - raise ValueError("Unexpected tuple %r with type %r" % (obj, dataType)) - return converter - elif isinstance(dataType, ArrayType): - element_converter = _python_to_sql_converter(dataType.elementType) - return lambda a: [element_converter(v) for v in a] - elif isinstance(dataType, MapType): - key_converter = _python_to_sql_converter(dataType.keyType) - value_converter = _python_to_sql_converter(dataType.valueType) - return lambda m: dict([(key_converter(k), value_converter(v)) for k, v in m.items()]) - elif isinstance(dataType, UserDefinedType): - return lambda obj: dataType.serialize(obj) - else: - raise ValueError("Unexpected type %r" % dataType) - - -def _has_nulltype(dt): - """ Return whether there is NullType in `dt` or not """ - if isinstance(dt, StructType): - return any(_has_nulltype(f.dataType) for f in dt.fields) - elif isinstance(dt, ArrayType): - return _has_nulltype((dt.elementType)) - elif isinstance(dt, MapType): - return _has_nulltype(dt.keyType) or _has_nulltype(dt.valueType) - else: - return isinstance(dt, NullType) - - -def _merge_type(a, b): - if isinstance(a, NullType): - return b - elif isinstance(b, NullType): - return a - elif type(a) is not type(b): - # TODO: type cast (such as int -> long) - raise TypeError("Can not merge type %s and %s" % (a, b)) - - # same type - if isinstance(a, StructType): - nfs = dict((f.name, f.dataType) for f in b.fields) - fields = [StructField(f.name, _merge_type(f.dataType, nfs.get(f.name, NullType()))) - for f in a.fields] - names = set([f.name for f in fields]) - for n in nfs: - if n not in names: - fields.append(StructField(n, nfs[n])) - return StructType(fields) - - elif isinstance(a, ArrayType): - return ArrayType(_merge_type(a.elementType, b.elementType), True) - - elif isinstance(a, MapType): - return MapType(_merge_type(a.keyType, b.keyType), - _merge_type(a.valueType, b.valueType), - True) - else: - return a - - -def _create_converter(dataType): - """Create an converter to drop the names of fields in obj """ - if isinstance(dataType, ArrayType): - conv = _create_converter(dataType.elementType) - return lambda row: map(conv, row) - - elif isinstance(dataType, MapType): - kconv = _create_converter(dataType.keyType) - vconv = _create_converter(dataType.valueType) - return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) - - elif isinstance(dataType, NullType): - return lambda x: None - - elif not isinstance(dataType, StructType): - return lambda x: x - - # dataType must be StructType - names = [f.name for f in dataType.fields] - converters = [_create_converter(f.dataType) for f in dataType.fields] - - def convert_struct(obj): - if obj is None: - return - - if isinstance(obj, tuple): - if hasattr(obj, "_fields"): - d = dict(zip(obj._fields, obj)) - elif hasattr(obj, "__FIELDS__"): - d = dict(zip(obj.__FIELDS__, obj)) - elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): - d = dict(obj) - else: - raise ValueError("unexpected tuple: %s" % str(obj)) - - elif isinstance(obj, dict): - d = obj - elif hasattr(obj, "__dict__"): # object - d = obj.__dict__ - else: - raise ValueError("Unexpected obj: %s" % obj) - - return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) - - return convert_struct - - -_BRACKETS = {'(': ')', '[': ']', '{': '}'} - - -def _split_schema_abstract(s): - """ - split the schema abstract into fields - - >>> _split_schema_abstract("a b c") - ['a', 'b', 'c'] - >>> _split_schema_abstract("a(a b)") - ['a(a b)'] - >>> _split_schema_abstract("a b[] c{a b}") - ['a', 'b[]', 'c{a b}'] - >>> _split_schema_abstract(" ") - [] - """ - - r = [] - w = '' - brackets = [] - for c in s: - if c == ' ' and not brackets: - if w: - r.append(w) - w = '' - else: - w += c - if c in _BRACKETS: - brackets.append(c) - elif c in _BRACKETS.values(): - if not brackets or c != _BRACKETS[brackets.pop()]: - raise ValueError("unexpected " + c) - - if brackets: - raise ValueError("brackets not closed: %s" % brackets) - if w: - r.append(w) - return r - - -def _parse_field_abstract(s): - """ - Parse a field in schema abstract - - >>> _parse_field_abstract("a") - StructField(a,None,true) - >>> _parse_field_abstract("b(c d)") - StructField(b,StructType(...c,None,true),StructField(d... - >>> _parse_field_abstract("a[]") - StructField(a,ArrayType(None,true),true) - >>> _parse_field_abstract("a{[]}") - StructField(a,MapType(None,ArrayType(None,true),true),true) - """ - if set(_BRACKETS.keys()) & set(s): - idx = min((s.index(c) for c in _BRACKETS if c in s)) - name = s[:idx] - return StructField(name, _parse_schema_abstract(s[idx:]), True) - else: - return StructField(s, None, True) - - -def _parse_schema_abstract(s): - """ - parse abstract into schema - - >>> _parse_schema_abstract("a b c") - StructType...a...b...c... - >>> _parse_schema_abstract("a[b c] b{}") - StructType...a,ArrayType...b...c...b,MapType... - >>> _parse_schema_abstract("c{} d{a b}") - StructType...c,MapType...d,MapType...a...b... - >>> _parse_schema_abstract("a b(t)").fields[1] - StructField(b,StructType(List(StructField(t,None,true))),true) - """ - s = s.strip() - if not s: - return - - elif s.startswith('('): - return _parse_schema_abstract(s[1:-1]) - - elif s.startswith('['): - return ArrayType(_parse_schema_abstract(s[1:-1]), True) - - elif s.startswith('{'): - return MapType(None, _parse_schema_abstract(s[1:-1])) - - parts = _split_schema_abstract(s) - fields = [_parse_field_abstract(p) for p in parts] - return StructType(fields) - - -def _infer_schema_type(obj, dataType): - """ - Fill the dataType with types inferred from obj - - >>> schema = _parse_schema_abstract("a b c d") - >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10)) - >>> _infer_schema_type(row, schema) - StructType...IntegerType...DoubleType...StringType...DateType... - >>> row = [[1], {"key": (1, 2.0)}] - >>> schema = _parse_schema_abstract("a[] b{c d}") - >>> _infer_schema_type(row, schema) - StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType... - """ - if dataType is None: - return _infer_type(obj) - - if not obj: - return NullType() - - if isinstance(dataType, ArrayType): - eType = _infer_schema_type(obj[0], dataType.elementType) - return ArrayType(eType, True) - - elif isinstance(dataType, MapType): - k, v = obj.iteritems().next() - return MapType(_infer_schema_type(k, dataType.keyType), - _infer_schema_type(v, dataType.valueType)) - - elif isinstance(dataType, StructType): - fs = dataType.fields - assert len(fs) == len(obj), \ - "Obj(%s) have different length with fields(%s)" % (obj, fs) - fields = [StructField(f.name, _infer_schema_type(o, f.dataType), True) - for o, f in zip(obj, fs)] - return StructType(fields) - - else: - raise ValueError("Unexpected dataType: %s" % dataType) - - -_acceptable_types = { - BooleanType: (bool,), - ByteType: (int, long), - ShortType: (int, long), - IntegerType: (int, long), - LongType: (int, long), - FloatType: (float,), - DoubleType: (float,), - DecimalType: (decimal.Decimal,), - StringType: (str, unicode), - BinaryType: (bytearray,), - DateType: (datetime.date,), - TimestampType: (datetime.datetime,), - ArrayType: (list, tuple, array), - MapType: (dict,), - StructType: (tuple, list), -} - - -def _verify_type(obj, dataType): - """ - Verify the type of obj against dataType, raise an exception if - they do not match. - - >>> _verify_type(None, StructType([])) - >>> _verify_type("", StringType()) - >>> _verify_type(0, IntegerType()) - >>> _verify_type(range(3), ArrayType(ShortType())) - >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - TypeError:... - >>> _verify_type({}, MapType(StringType(), IntegerType())) - >>> _verify_type((), StructType([])) - >>> _verify_type([], StructType([])) - >>> _verify_type([1], StructType([])) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - ValueError:... - >>> _verify_type(ExamplePoint(1.0, 2.0), ExamplePointUDT()) - >>> _verify_type([1.0, 2.0], ExamplePointUDT()) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - ValueError:... - """ - # all objects are nullable - if obj is None: - return - - if isinstance(dataType, UserDefinedType): - if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): - raise ValueError("%r is not an instance of type %r" % (obj, dataType)) - _verify_type(dataType.serialize(obj), dataType.sqlType()) - return - - _type = type(dataType) - assert _type in _acceptable_types, "unkown datatype: %s" % dataType - - # subclass of them can not be deserialized in JVM - if type(obj) not in _acceptable_types[_type]: - raise TypeError("%s can not accept object in type %s" - % (dataType, type(obj))) - - if isinstance(dataType, ArrayType): - for i in obj: - _verify_type(i, dataType.elementType) - - elif isinstance(dataType, MapType): - for k, v in obj.iteritems(): - _verify_type(k, dataType.keyType) - _verify_type(v, dataType.valueType) - - elif isinstance(dataType, StructType): - if len(obj) != len(dataType.fields): - raise ValueError("Length of object (%d) does not match with" - "length of fields (%d)" % (len(obj), len(dataType.fields))) - for v, f in zip(obj, dataType.fields): - _verify_type(v, f.dataType) - - -_cached_cls = {} - - -def _restore_object(dataType, obj): - """ Restore object during unpickling. """ - # use id(dataType) as key to speed up lookup in dict - # Because of batched pickling, dataType will be the - # same object in most cases. - k = id(dataType) - cls = _cached_cls.get(k) - if cls is None: - # use dataType as key to avoid create multiple class - cls = _cached_cls.get(dataType) - if cls is None: - cls = _create_cls(dataType) - _cached_cls[dataType] = cls - _cached_cls[k] = cls - return cls(obj) - - -def _create_object(cls, v): - """ Create an customized object with class `cls`. """ - # datetime.date would be deserialized as datetime.datetime - # from java type, so we need to set it back. - if cls is datetime.date and isinstance(v, datetime.datetime): - return v.date() - return cls(v) if v is not None else v - - -def _create_getter(dt, i): - """ Create a getter for item `i` with schema """ - cls = _create_cls(dt) - - def getter(self): - return _create_object(cls, self[i]) - - return getter - - -def _has_struct_or_date(dt): - """Return whether `dt` is or has StructType/DateType in it""" - if isinstance(dt, StructType): - return True - elif isinstance(dt, ArrayType): - return _has_struct_or_date(dt.elementType) - elif isinstance(dt, MapType): - return _has_struct_or_date(dt.keyType) or _has_struct_or_date(dt.valueType) - elif isinstance(dt, DateType): - return True - elif isinstance(dt, UserDefinedType): - return True - return False - - -def _create_properties(fields): - """Create properties according to fields""" - ps = {} - for i, f in enumerate(fields): - name = f.name - if (name.startswith("__") and name.endswith("__") - or keyword.iskeyword(name)): - warnings.warn("field name %s can not be accessed in Python," - "use position to access it instead" % name) - if _has_struct_or_date(f.dataType): - # delay creating object until accessing it - getter = _create_getter(f.dataType, i) - else: - getter = itemgetter(i) - ps[name] = property(getter) - return ps - - -def _create_cls(dataType): - """ - Create an class by dataType - - The created class is similar to namedtuple, but can have nested schema. - - >>> schema = _parse_schema_abstract("a b c") - >>> row = (1, 1.0, "str") - >>> schema = _infer_schema_type(row, schema) - >>> obj = _create_cls(schema)(row) - >>> import pickle - >>> pickle.loads(pickle.dumps(obj)) - Row(a=1, b=1.0, c='str') - - >>> row = [[1], {"key": (1, 2.0)}] - >>> schema = _parse_schema_abstract("a[] b{c d}") - >>> schema = _infer_schema_type(row, schema) - >>> obj = _create_cls(schema)(row) - >>> pickle.loads(pickle.dumps(obj)) - Row(a=[1], b={'key': Row(c=1, d=2.0)}) - >>> pickle.loads(pickle.dumps(obj.a)) - [1] - >>> pickle.loads(pickle.dumps(obj.b)) - {'key': Row(c=1, d=2.0)} - """ - - if isinstance(dataType, ArrayType): - cls = _create_cls(dataType.elementType) - - def List(l): - if l is None: - return - return [_create_object(cls, v) for v in l] - - return List - - elif isinstance(dataType, MapType): - kcls = _create_cls(dataType.keyType) - vcls = _create_cls(dataType.valueType) - - def Dict(d): - if d is None: - return - return dict((_create_object(kcls, k), _create_object(vcls, v)) for k, v in d.items()) - - return Dict - - elif isinstance(dataType, DateType): - return datetime.date - - elif isinstance(dataType, UserDefinedType): - return lambda datum: dataType.deserialize(datum) - - elif not isinstance(dataType, StructType): - # no wrapper for primitive types - return lambda x: x - - class Row(tuple): - - """ Row in DataFrame """ - __DATATYPE__ = dataType - __FIELDS__ = tuple(f.name for f in dataType.fields) - __slots__ = () - - # create property for fast access - locals().update(_create_properties(dataType.fields)) - - def asDict(self): - """ Return as a dict """ - return dict((n, getattr(self, n)) for n in self.__FIELDS__) - - def __repr__(self): - # call collect __repr__ for nested objects - return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) - for n in self.__FIELDS__)) - - def __reduce__(self): - return (_restore_object, (self.__DATATYPE__, tuple(self))) - - return Row - - -class SQLContext(object): - - """Main entry point for Spark SQL functionality. - - A SQLContext can be used create L{DataFrame}, register L{DataFrame} as - tables, execute SQL over tables, cache tables, and read parquet files. - """ - - def __init__(self, sparkContext, sqlContext=None): - """Create a new SQLContext. - - :param sparkContext: The SparkContext to wrap. - :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new - SQLContext in the JVM, instead we make all calls to this object. - - >>> df = sqlCtx.inferSchema(rdd) - >>> sqlCtx.inferSchema(df) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - TypeError:... - - >>> bad_rdd = sc.parallelize([1,2,3]) - >>> sqlCtx.inferSchema(bad_rdd) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - ValueError:... - - >>> from datetime import datetime - >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, - ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), - ... time=datetime(2014, 8, 1, 14, 1, 5))]) - >>> df = sqlCtx.inferSchema(allTypes) - >>> df.registerTempTable("allTypes") - >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' - ... 'from allTypes where b and i > 0').collect() - [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] - >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, - ... x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] - """ - self._sc = sparkContext - self._jsc = self._sc._jsc - self._jvm = self._sc._jvm - self._scala_SQLContext = sqlContext - - @property - def _ssql_ctx(self): - """Accessor for the JVM Spark SQL context. - - Subclasses can override this property to provide their own - JVM Contexts. - """ - if self._scala_SQLContext is None: - self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) - return self._scala_SQLContext - - def registerFunction(self, name, f, returnType=StringType()): - """Registers a lambda function as a UDF so it can be used in SQL statements. - - In addition to a name and the function itself, the return type can be optionally specified. - When the return type is not given it default to a string and conversion will automatically - be done. For any other return type, the produced object must match the specified type. - - >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) - >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() - [Row(c0=u'4')] - >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() - [Row(c0=4)] - """ - func = lambda _, it: imap(lambda x: f(*x), it) - ser = AutoBatchedSerializer(PickleSerializer()) - command = (func, None, ser, ser) - pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) - self._ssql_ctx.udf().registerPython(name, - bytearray(pickled_cmd), - env, - includes, - self._sc.pythonExec, - bvars, - self._sc._javaAccumulator, - returnType.json()) - - def inferSchema(self, rdd, samplingRatio=None): - """Infer and apply a schema to an RDD of L{Row}. - - When samplingRatio is specified, the schema is inferred by looking - at the types of each row in the sampled dataset. Otherwise, the - first 100 rows of the RDD are inspected. Nested collections are - supported, which can include array, dict, list, Row, tuple, - namedtuple, or object. - - Each row could be L{pyspark.sql.Row} object or namedtuple or objects. - Using top level dicts is deprecated, as dict is used to represent Maps. - - If a single column has multiple distinct inferred types, it may cause - runtime exceptions. - - >>> rdd = sc.parallelize( - ... [Row(field1=1, field2="row1"), - ... Row(field1=2, field2="row2"), - ... Row(field1=3, field2="row3")]) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.collect()[0] - Row(field1=1, field2=u'row1') - - >>> NestedRow = Row("f1", "f2") - >>> nestedRdd1 = sc.parallelize([ - ... NestedRow(array('i', [1, 2]), {"row1": 1.0}), - ... NestedRow(array('i', [2, 3]), {"row2": 2.0})]) - >>> df = sqlCtx.inferSchema(nestedRdd1) - >>> df.collect() - [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})] - - >>> nestedRdd2 = sc.parallelize([ - ... NestedRow([[1, 2], [2, 3]], [1, 2]), - ... NestedRow([[2, 3], [3, 4]], [2, 3])]) - >>> df = sqlCtx.inferSchema(nestedRdd2) - >>> df.collect() - [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] - - >>> from collections import namedtuple - >>> CustomRow = namedtuple('CustomRow', 'field1 field2') - >>> rdd = sc.parallelize( - ... [CustomRow(field1=1, field2="row1"), - ... CustomRow(field1=2, field2="row2"), - ... CustomRow(field1=3, field2="row3")]) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.collect()[0] - Row(field1=1, field2=u'row1') - """ - - if isinstance(rdd, DataFrame): - raise TypeError("Cannot apply schema to DataFrame") - - first = rdd.first() - if not first: - raise ValueError("The first row in RDD is empty, " - "can not infer schema") - if type(first) is dict: - warnings.warn("Using RDD of dict to inferSchema is deprecated," - "please use pyspark.sql.Row instead") - - if samplingRatio is None: - schema = _infer_schema(first) - if _has_nulltype(schema): - for row in rdd.take(100)[1:]: - schema = _merge_type(schema, _infer_schema(row)) - if not _has_nulltype(schema): - break - else: - warnings.warn("Some of types cannot be determined by the " - "first 100 rows, please try again with sampling") - else: - if samplingRatio > 0.99: - rdd = rdd.sample(False, float(samplingRatio)) - schema = rdd.map(_infer_schema).reduce(_merge_type) - - converter = _create_converter(schema) - rdd = rdd.map(converter) - return self.applySchema(rdd, schema) - - def applySchema(self, rdd, schema): - """ - Applies the given schema to the given RDD of L{tuple} or L{list}. - - These tuples or lists can contain complex nested structures like - lists, maps or nested rows. - - The schema should be a StructType. - - It is important that the schema matches the types of the objects - in each row or exceptions could be thrown at runtime. - - >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) - >>> schema = StructType([StructField("field1", IntegerType(), False), - ... StructField("field2", StringType(), False)]) - >>> df = sqlCtx.applySchema(rdd2, schema) - >>> sqlCtx.registerRDDAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT * from table1") - >>> df2.collect() - [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] - - >>> from datetime import date, datetime - >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, - ... date(2010, 1, 1), - ... datetime(2010, 1, 1, 1, 1, 1), - ... {"a": 1}, (2,), [1, 2, 3], None)]) - >>> schema = StructType([ - ... StructField("byte1", ByteType(), False), - ... StructField("byte2", ByteType(), False), - ... StructField("short1", ShortType(), False), - ... StructField("short2", ShortType(), False), - ... StructField("int", IntegerType(), False), - ... StructField("float", FloatType(), False), - ... StructField("date", DateType(), False), - ... StructField("time", TimestampType(), False), - ... StructField("map", - ... MapType(StringType(), IntegerType(), False), False), - ... StructField("struct", - ... StructType([StructField("b", ShortType(), False)]), False), - ... StructField("list", ArrayType(ByteType(), False), False), - ... StructField("null", DoubleType(), True)]) - >>> df = sqlCtx.applySchema(rdd, schema) - >>> results = df.map( - ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date, - ... x.time, x.map["a"], x.struct.b, x.list, x.null)) - >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE - (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), - datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) - - >>> df.registerTempTable("table2") - >>> sqlCtx.sql( - ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + - ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + - ... "float + 1.5 as float FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] - - >>> rdd = sc.parallelize([(127, -32768, 1.0, - ... datetime(2010, 1, 1, 1, 1, 1), - ... {"a": 1}, (2,), [1, 2, 3])]) - >>> abstract = "byte short float time map{} struct(b) list[]" - >>> schema = _parse_schema_abstract(abstract) - >>> typedSchema = _infer_schema_type(rdd.first(), schema) - >>> df = sqlCtx.applySchema(rdd, typedSchema) - >>> df.collect() - [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])] - """ - - if isinstance(rdd, DataFrame): - raise TypeError("Cannot apply schema to DataFrame") - - if not isinstance(schema, StructType): - raise TypeError("schema should be StructType") - - # take the first few rows to verify schema - rows = rdd.take(10) - # Row() cannot been deserialized by Pyrolite - if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': - rdd = rdd.map(tuple) - rows = rdd.take(10) - - for row in rows: - _verify_type(row, schema) - - # convert python objects to sql data - converter = _python_to_sql_converter(schema) - rdd = rdd.map(converter) - - jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) - df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) - return DataFrame(df, self) - - def registerRDDAsTable(self, rdd, tableName): - """Registers the given RDD as a temporary table in the catalog. - - Temporary tables exist only during the lifetime of this instance of - SQLContext. - - >>> df = sqlCtx.inferSchema(rdd) - >>> sqlCtx.registerRDDAsTable(df, "table1") - """ - if (rdd.__class__ is DataFrame): - df = rdd._jdf - self._ssql_ctx.registerRDDAsTable(df, tableName) - else: - raise ValueError("Can only register DataFrame as table") - - def parquetFile(self, *paths): - """Loads a Parquet file, returning the result as a L{DataFrame}. - - >>> import tempfile, shutil - >>> parquetFile = tempfile.mkdtemp() - >>> shutil.rmtree(parquetFile) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) - >>> sorted(df.collect()) == sorted(df2.collect()) - True - """ - gateway = self._sc._gateway - jpath = paths[0] - jpaths = gateway.new_array(gateway.jvm.java.lang.String, len(paths) - 1) - for i in range(1, len(paths)): - jpaths[i] = paths[i] - jdf = self._ssql_ctx.parquetFile(jpath, jpaths) - return DataFrame(jdf, self) - - def jsonFile(self, path, schema=None, samplingRatio=1.0): - """ - Loads a text file storing one JSON object per line as a - L{DataFrame}. - - If the schema is provided, applies the given schema to this - JSON dataset. - - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. - - >>> import tempfile, shutil - >>> jsonFile = tempfile.mkdtemp() - >>> shutil.rmtree(jsonFile) - >>> ofn = open(jsonFile, 'w') - >>> for json in jsonStrings: - ... print>>ofn, json - >>> ofn.close() - >>> df1 = sqlCtx.jsonFile(jsonFile) - >>> sqlCtx.registerRDDAsTable(df1, "table1") - >>> df2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table1") - >>> for r in df2.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema()) - >>> sqlCtx.registerRDDAsTable(df3, "table2") - >>> df4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table2") - >>> for r in df4.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> schema = StructType([ - ... StructField("field2", StringType(), True), - ... StructField("field3", - ... StructType([ - ... StructField("field5", - ... ArrayType(IntegerType(), False), True)]), False)]) - >>> df5 = sqlCtx.jsonFile(jsonFile, schema) - >>> sqlCtx.registerRDDAsTable(df5, "table3") - >>> df6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, " - ... "field3.field5[0] as f3 from table3") - >>> df6.collect() - [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] - """ - if schema is None: - df = self._ssql_ctx.jsonFile(path, samplingRatio) - else: - scala_datatype = self._ssql_ctx.parseDataType(schema.json()) - df = self._ssql_ctx.jsonFile(path, scala_datatype) - return DataFrame(df, self) - - def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): - """Loads an RDD storing one JSON object per string as a L{DataFrame}. - - If the schema is provided, applies the given schema to this - JSON dataset. - - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. - - >>> df1 = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerRDDAsTable(df1, "table1") - >>> df2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table1") - >>> for r in df2.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> df3 = sqlCtx.jsonRDD(json, df1.schema()) - >>> sqlCtx.registerRDDAsTable(df3, "table2") - >>> df4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table2") - >>> for r in df4.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> schema = StructType([ - ... StructField("field2", StringType(), True), - ... StructField("field3", - ... StructType([ - ... StructField("field5", - ... ArrayType(IntegerType(), False), True)]), False)]) - >>> df5 = sqlCtx.jsonRDD(json, schema) - >>> sqlCtx.registerRDDAsTable(df5, "table3") - >>> df6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, " - ... "field3.field5[0] as f3 from table3") - >>> df6.collect() - [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] - - >>> sqlCtx.jsonRDD(sc.parallelize(['{}', - ... '{"key0": {"key1": "value1"}}'])).collect() - [Row(key0=None), Row(key0=Row(key1=u'value1'))] - >>> sqlCtx.jsonRDD(sc.parallelize(['{"key0": null}', - ... '{"key0": {"key1": "value1"}}'])).collect() - [Row(key0=None), Row(key0=Row(key1=u'value1'))] - """ - - def func(iterator): - for x in iterator: - if not isinstance(x, basestring): - x = unicode(x) - if isinstance(x, unicode): - x = x.encode("utf-8") - yield x - keyed = rdd.mapPartitions(func) - keyed._bypass_serializer = True - jrdd = keyed._jrdd.map(self._jvm.BytesToString()) - if schema is None: - df = self._ssql_ctx.jsonRDD(jrdd.rdd(), samplingRatio) - else: - scala_datatype = self._ssql_ctx.parseDataType(schema.json()) - df = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) - return DataFrame(df, self) - - def sql(self, sqlQuery): - """Return a L{DataFrame} representing the result of the given query. - - >>> df = sqlCtx.inferSchema(rdd) - >>> sqlCtx.registerRDDAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") - >>> df2.collect() - [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] - """ - return DataFrame(self._ssql_ctx.sql(sqlQuery), self) - - def table(self, tableName): - """Returns the specified table as a L{DataFrame}. - - >>> df = sqlCtx.inferSchema(rdd) - >>> sqlCtx.registerRDDAsTable(df, "table1") - >>> df2 = sqlCtx.table("table1") - >>> sorted(df.collect()) == sorted(df2.collect()) - True - """ - return DataFrame(self._ssql_ctx.table(tableName), self) - - def cacheTable(self, tableName): - """Caches the specified table in-memory.""" - self._ssql_ctx.cacheTable(tableName) - - def uncacheTable(self, tableName): - """Removes the specified table from the in-memory cache.""" - self._ssql_ctx.uncacheTable(tableName) - - -class HiveContext(SQLContext): - - """A variant of Spark SQL that integrates with data stored in Hive. - - Configuration for Hive is read from hive-site.xml on the classpath. - It supports running both SQL and HiveQL commands. - """ - - def __init__(self, sparkContext, hiveContext=None): - """Create a new HiveContext. - - :param sparkContext: The SparkContext to wrap. - :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new - HiveContext in the JVM, instead we make all calls to this object. - """ - SQLContext.__init__(self, sparkContext) - - if hiveContext: - self._scala_HiveContext = hiveContext - - @property - def _ssql_ctx(self): - try: - if not hasattr(self, '_scala_HiveContext'): - self._scala_HiveContext = self._get_hive_ctx() - return self._scala_HiveContext - except Py4JError as e: - raise Exception("You must build Spark with Hive. " - "Export 'SPARK_HIVE=true' and run " - "build/sbt assembly", e) - - def _get_hive_ctx(self): - return self._jvm.HiveContext(self._jsc.sc()) - - -def _create_row(fields, values): - row = Row(*values) - row.__FIELDS__ = fields - return row - - -class Row(tuple): - - """ - A row in L{DataFrame}. The fields in it can be accessed like attributes. - - Row can be used to create a row object by using named arguments, - the fields will be sorted by names. - - >>> row = Row(name="Alice", age=11) - >>> row - Row(age=11, name='Alice') - >>> row.name, row.age - ('Alice', 11) - - Row also can be used to create another Row like class, then it - could be used to create Row objects, such as - - >>> Person = Row("name", "age") - >>> Person - - >>> Person("Alice", 11) - Row(name='Alice', age=11) - """ - - def __new__(self, *args, **kwargs): - if args and kwargs: - raise ValueError("Can not use both args " - "and kwargs to create Row") - if args: - # create row class or objects - return tuple.__new__(self, args) - - elif kwargs: - # create row objects - names = sorted(kwargs.keys()) - values = tuple(kwargs[n] for n in names) - row = tuple.__new__(self, values) - row.__FIELDS__ = names - return row - - else: - raise ValueError("No args or kwargs") - - def asDict(self): - """ - Return as an dict - """ - if not hasattr(self, "__FIELDS__"): - raise TypeError("Cannot convert a Row class into dict") - return dict(zip(self.__FIELDS__, self)) - - # let obect acs like class - def __call__(self, *args): - """create new Row object""" - return _create_row(self, args) - - def __getattr__(self, item): - if item.startswith("__"): - raise AttributeError(item) - try: - # it will be slow when it has many fields, - # but this will not be used in normal cases - idx = self.__FIELDS__.index(item) - return self[idx] - except IndexError: - raise AttributeError(item) - - def __reduce__(self): - if hasattr(self, "__FIELDS__"): - return (_create_row, (self.__FIELDS__, tuple(self))) - else: - return tuple.__reduce__(self) - - def __repr__(self): - if hasattr(self, "__FIELDS__"): - return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) - else: - return "" % ", ".join(self) - - -class DataFrame(object): - - """A collection of rows that have the same columns. - - A :class:`DataFrame` is equivalent to a relational table in Spark SQL, - and can be created using various functions in :class:`SQLContext`:: - - people = sqlContext.parquetFile("...") - - Once created, it can be manipulated using the various domain-specific-language - (DSL) functions defined in: :class:`DataFrame`, :class:`Column`. - - To select a column from the data frame, use the apply method:: - - ageCol = people.age - - Note that the :class:`Column` type can also be manipulated - through its various functions:: - - # The following creates a new column that increases everybody's age by 10. - people.age + 10 - - - A more concrete example:: - - # To create DataFrame using SQLContext - people = sqlContext.parquetFile("...") - department = sqlContext.parquetFile("...") - - people.filter(people.age > 30).join(department, people.deptId == department.id)) \ - .groupBy(department.name, "gender").agg({"salary": "avg", "age": "max"}) - """ - - def __init__(self, jdf, sql_ctx): - self._jdf = jdf - self.sql_ctx = sql_ctx - self._sc = sql_ctx and sql_ctx._sc - self.is_cached = False - - @property - def rdd(self): - """ - Return the content of the :class:`DataFrame` as an :class:`RDD` - of :class:`Row` s. - """ - if not hasattr(self, '_lazy_rdd'): - jrdd = self._jdf.javaToPython() - rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) - schema = self.schema() - - def applySchema(it): - cls = _create_cls(schema) - return itertools.imap(cls, it) - - self._lazy_rdd = rdd.mapPartitions(applySchema) - - return self._lazy_rdd - - def toJSON(self, use_unicode=False): - """Convert a DataFrame into a MappedRDD of JSON documents; one document per row. - - >>> df1 = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerRDDAsTable(df1, "table1") - >>> df2 = sqlCtx.sql( "SELECT * from table1") - >>> df2.toJSON().take(1)[0] == '{"field1":1,"field2":"row1","field3":{"field4":11}}' - True - >>> df3 = sqlCtx.sql( "SELECT field3.field4 from table1") - >>> df3.toJSON().collect() == ['{"field4":11}', '{"field4":22}', '{"field4":33}'] - True - """ - rdd = self._jdf.toJSON() - return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) - - def saveAsParquetFile(self, path): - """Save the contents as a Parquet file, preserving the schema. - - Files that are written out using this method can be read back in as - a DataFrame using the L{SQLContext.parquetFile} method. - - >>> import tempfile, shutil - >>> parquetFile = tempfile.mkdtemp() - >>> shutil.rmtree(parquetFile) - >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) - >>> sorted(df2.collect()) == sorted(df.collect()) - True - """ - self._jdf.saveAsParquetFile(path) - - def registerTempTable(self, name): - """Registers this RDD as a temporary table using the given name. - - The lifetime of this temporary table is tied to the L{SQLContext} - that was used to create this DataFrame. - - >>> df.registerTempTable("people") - >>> df2 = sqlCtx.sql("select * from people") - >>> sorted(df.collect()) == sorted(df2.collect()) - True - """ - self._jdf.registerTempTable(name) - - def registerAsTable(self, name): - """DEPRECATED: use registerTempTable() instead""" - warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) - self.registerTempTable(name) - - def insertInto(self, tableName, overwrite=False): - """Inserts the contents of this DataFrame into the specified table. - - Optionally overwriting any existing data. - """ - self._jdf.insertInto(tableName, overwrite) - - def saveAsTable(self, tableName): - """Creates a new table with the contents of this DataFrame.""" - self._jdf.saveAsTable(tableName) - - def schema(self): - """Returns the schema of this DataFrame (represented by - a L{StructType}). - - >>> df.schema() - StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) - """ - return _parse_datatype_json_string(self._jdf.schema().json()) - - def printSchema(self): - """Prints out the schema in the tree format. - - >>> df.printSchema() - root - |-- age: integer (nullable = true) - |-- name: string (nullable = true) - - """ - print (self._jdf.schema().treeString()) - - def count(self): - """Return the number of elements in this RDD. - - Unlike the base RDD implementation of count, this implementation - leverages the query optimizer to compute the count on the DataFrame, - which supports features such as filter pushdown. - - >>> df.count() - 2L - """ - return self._jdf.count() - - def collect(self): - """Return a list that contains all of the rows. - - Each object in the list is a Row, the fields can be accessed as - attributes. - - >>> df.collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] - """ - with SCCallSiteSync(self._sc) as css: - bytesInJava = self._jdf.javaToPython().collect().iterator() - tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir) - tempFile.close() - self._sc._writeToFile(bytesInJava, tempFile.name) - # Read the data into Python and deserialize it: - with open(tempFile.name, 'rb') as tempFile: - rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile)) - os.unlink(tempFile.name) - cls = _create_cls(self.schema()) - return [cls(r) for r in rs] - - def limit(self, num): - """Limit the result count to the number specified. - - >>> df.limit(1).collect() - [Row(age=2, name=u'Alice')] - >>> df.limit(0).collect() - [] - """ - jdf = self._jdf.limit(num) - return DataFrame(jdf, self.sql_ctx) - - def take(self, num): - """Take the first num rows of the RDD. - - Each object in the list is a Row, the fields can be accessed as - attributes. - - >>> df.take(2) - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] - """ - return self.limit(num).collect() - - def map(self, f): - """ Return a new RDD by applying a function to each Row, it's a - shorthand for df.rdd.map() - - >>> df.map(lambda p: p.name).collect() - [u'Alice', u'Bob'] - """ - return self.rdd.map(f) - - def mapPartitions(self, f, preservesPartitioning=False): - """ - Return a new RDD by applying a function to each partition. - - >>> rdd = sc.parallelize([1, 2, 3, 4], 4) - >>> def f(iterator): yield 1 - >>> rdd.mapPartitions(f).sum() - 4 - """ - return self.rdd.mapPartitions(f, preservesPartitioning) - - def cache(self): - """ Persist with the default storage level (C{MEMORY_ONLY_SER}). - """ - self.is_cached = True - self._jdf.cache() - return self - - def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): - """ Set the storage level to persist its values across operations - after the first time it is computed. This can only be used to assign - a new storage level if the RDD does not have a storage level set yet. - If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). - """ - self.is_cached = True - javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel) - self._jdf.persist(javaStorageLevel) - return self - - def unpersist(self, blocking=True): - """ Mark it as non-persistent, and remove all blocks for it from - memory and disk. - """ - self.is_cached = False - self._jdf.unpersist(blocking) - return self - - # def coalesce(self, numPartitions, shuffle=False): - # rdd = self._jdf.coalesce(numPartitions, shuffle, None) - # return DataFrame(rdd, self.sql_ctx) - - def repartition(self, numPartitions): - """ Return a new :class:`DataFrame` that has exactly `numPartitions` - partitions. - """ - rdd = self._jdf.repartition(numPartitions, None) - return DataFrame(rdd, self.sql_ctx) - - def sample(self, withReplacement, fraction, seed=None): - """ - Return a sampled subset of this DataFrame. - - >>> df = sqlCtx.inferSchema(rdd) - >>> df.sample(False, 0.5, 97).count() - 2L - """ - assert fraction >= 0.0, "Negative fraction value: %s" % fraction - seed = seed if seed is not None else random.randint(0, sys.maxint) - rdd = self._jdf.sample(withReplacement, fraction, long(seed)) - return DataFrame(rdd, self.sql_ctx) - - # def takeSample(self, withReplacement, num, seed=None): - # """Return a fixed-size sampled subset of this DataFrame. - # - # >>> df = sqlCtx.inferSchema(rdd) - # >>> df.takeSample(False, 2, 97) - # [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')] - # """ - # seed = seed if seed is not None else random.randint(0, sys.maxint) - # with SCCallSiteSync(self.context) as css: - # bytesInJava = self._jdf \ - # .takeSampleToPython(withReplacement, num, long(seed)) \ - # .iterator() - # cls = _create_cls(self.schema()) - # return map(cls, self._collect_iterator_through_file(bytesInJava)) - - @property - def dtypes(self): - """Return all column names and their data types as a list. - - >>> df.dtypes - [('age', 'integer'), ('name', 'string')] - """ - return [(str(f.name), f.dataType.jsonValue()) for f in self.schema().fields] - - @property - def columns(self): - """ Return all column names as a list. - - >>> df.columns - [u'age', u'name'] - """ - return [f.name for f in self.schema().fields] - - def join(self, other, joinExprs=None, joinType=None): - """ - Join with another DataFrame, using the given join expression. - The following performs a full outer join between `df1` and `df2`:: - - :param other: Right side of the join - :param joinExprs: Join expression - :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. - - >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] - """ - - if joinExprs is None: - jdf = self._jdf.join(other._jdf) - else: - assert isinstance(joinExprs, Column), "joinExprs should be Column" - if joinType is None: - jdf = self._jdf.join(other._jdf, joinExprs._jc) - else: - assert isinstance(joinType, basestring), "joinType should be basestring" - jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) - return DataFrame(jdf, self.sql_ctx) - - def sort(self, *cols): - """ Return a new :class:`DataFrame` sorted by the specified column. - - :param cols: The columns or expressions used for sorting - - >>> df.sort(df.age.desc()).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] - >>> df.sortBy(df.age.desc()).collect() - [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] - """ - if not cols: - raise ValueError("should sort by at least one column") - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) - return DataFrame(jdf, self.sql_ctx) - - sortBy = sort - - def head(self, n=None): - """ Return the first `n` rows or the first row if n is None. - - >>> df.head() - Row(age=2, name=u'Alice') - >>> df.head(1) - [Row(age=2, name=u'Alice')] - """ - if n is None: - rs = self.head(1) - return rs[0] if rs else None - return self.take(n) - - def first(self): - """ Return the first row. - - >>> df.first() - Row(age=2, name=u'Alice') - """ - return self.head() - - def __getitem__(self, item): - """ Return the column by given name - - >>> df['age'].collect() - [Row(age=2), Row(age=5)] - >>> df[ ["name", "age"]].collect() - [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] - >>> df[ df.age > 3 ].collect() - [Row(age=5, name=u'Bob')] - """ - if isinstance(item, basestring): - jc = self._jdf.apply(item) - return Column(jc, self.sql_ctx) - elif isinstance(item, Column): - return self.filter(item) - elif isinstance(item, list): - return self.select(*item) - else: - raise IndexError("unexpected index: %s" % item) - - def __getattr__(self, name): - """ Return the column by given name - - >>> df.age.collect() - [Row(age=2), Row(age=5)] - """ - if name.startswith("__"): - raise AttributeError(name) - jc = self._jdf.apply(name) - return Column(jc, self.sql_ctx) - - def select(self, *cols): - """ Selecting a set of expressions. - - >>> df.select().collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] - >>> df.select('*').collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] - >>> df.select('name', 'age').collect() - [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] - >>> df.select(df.name, (df.age + 10).alias('age')).collect() - [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] - """ - if not cols: - cols = ["*"] - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) - return DataFrame(jdf, self.sql_ctx) - - def selectExpr(self, *expr): - """ - Selects a set of SQL expressions. This is a variant of - `select` that accepts SQL expressions. - - >>> df.selectExpr("age * 2", "abs(age)").collect() - [Row(('age * 2)=4, Abs('age)=2), Row(('age * 2)=10, Abs('age)=5)] - """ - jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) - jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) - return DataFrame(jdf, self.sql_ctx) - - def filter(self, condition): - """ Filtering rows using the given condition, which could be - Column expression or string of SQL expression. - - where() is an alias for filter(). - - >>> df.filter(df.age > 3).collect() - [Row(age=5, name=u'Bob')] - >>> df.where(df.age == 2).collect() - [Row(age=2, name=u'Alice')] - - >>> df.filter("age > 3").collect() - [Row(age=5, name=u'Bob')] - >>> df.where("age = 2").collect() - [Row(age=2, name=u'Alice')] - """ - if isinstance(condition, basestring): - jdf = self._jdf.filter(condition) - elif isinstance(condition, Column): - jdf = self._jdf.filter(condition._jc) - else: - raise TypeError("condition should be string or Column") - return DataFrame(jdf, self.sql_ctx) - - where = filter - - def groupBy(self, *cols): - """ Group the :class:`DataFrame` using the specified columns, - so we can run aggregation on them. See :class:`GroupedData` - for all the available aggregate functions. - - >>> df.groupBy().avg().collect() - [Row(AVG(age#0)=3.5)] - >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] - >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] - """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) - return GroupedData(jdf, self.sql_ctx) - - def agg(self, *exprs): - """ Aggregate on the entire :class:`DataFrame` without groups - (shorthand for df.groupBy.agg()). - - >>> df.agg({"age": "max"}).collect() - [Row(MAX(age#0)=5)] - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.min(df.age)).collect() - [Row(MIN(age#0)=2)] - """ - return self.groupBy().agg(*exprs) - - def unionAll(self, other): - """ Return a new DataFrame containing union of rows in this - frame and another frame. - - This is equivalent to `UNION ALL` in SQL. - """ - return DataFrame(self._jdf.unionAll(other._jdf), self.sql_ctx) - - def intersect(self, other): - """ Return a new :class:`DataFrame` containing rows only in - both this frame and another frame. - - This is equivalent to `INTERSECT` in SQL. - """ - return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx) - - def subtract(self, other): - """ Return a new :class:`DataFrame` containing rows in this frame - but not in another frame. - - This is equivalent to `EXCEPT` in SQL. - """ - return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) - - def addColumn(self, colName, col): - """ Return a new :class:`DataFrame` by adding a column. - - >>> df.addColumn('age2', df.age + 2).collect() - [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] - """ - return self.select('*', col.alias(colName)) - - def to_pandas(self): - """ - Collect all the rows and return a `pandas.DataFrame`. - - >>> df.to_pandas() # doctest: +SKIP - age name - 0 2 Alice - 1 5 Bob - """ - import pandas as pd - return pd.DataFrame.from_records(self.collect(), columns=self.columns) - - -# Having SchemaRDD for backward compatibility (for docs) -class SchemaRDD(DataFrame): - """ - SchemaRDD is deprecated, please use DataFrame - """ - - -def dfapi(f): - def _api(self): - name = f.__name__ - jdf = getattr(self._jdf, name)() - return DataFrame(jdf, self.sql_ctx) - _api.__name__ = f.__name__ - _api.__doc__ = f.__doc__ - return _api - - -class GroupedData(object): - - """ - A set of methods for aggregations on a :class:`DataFrame`, - created by DataFrame.groupBy(). - """ - - def __init__(self, jdf, sql_ctx): - self._jdf = jdf - self.sql_ctx = sql_ctx - - def agg(self, *exprs): - """ Compute aggregates by specifying a map from column name - to aggregate methods. - - The available aggregate methods are `avg`, `max`, `min`, - `sum`, `count`. - - :param exprs: list or aggregate columns or a map from column - name to aggregate methods. - - >>> gdf = df.groupBy(df.name) - >>> gdf.agg({"age": "max"}).collect() - [Row(name=u'Bob', MAX(age#0)=5), Row(name=u'Alice', MAX(age#0)=2)] - >>> from pyspark.sql import Dsl - >>> gdf.agg(Dsl.min(df.age)).collect() - [Row(MIN(age#0)=5), Row(MIN(age#0)=2)] - """ - assert exprs, "exprs should not be empty" - if len(exprs) == 1 and isinstance(exprs[0], dict): - jmap = MapConverter().convert(exprs[0], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(jmap) - else: - # Columns - assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - jcols = ListConverter().convert([c._jc for c in exprs[1:]], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) - return DataFrame(jdf, self.sql_ctx) - - @dfapi - def count(self): - """ Count the number of rows for each group. - - >>> df.groupBy(df.age).count().collect() - [Row(age=2, count=1), Row(age=5, count=1)] - """ - - @dfapi - def mean(self): - """Compute the average value for each numeric columns - for each group. This is an alias for `avg`.""" - - @dfapi - def avg(self): - """Compute the average value for each numeric columns - for each group.""" - - @dfapi - def max(self): - """Compute the max value for each numeric columns for - each group. """ - - @dfapi - def min(self): - """Compute the min value for each numeric column for - each group.""" - - @dfapi - def sum(self): - """Compute the sum for each numeric columns for each - group.""" - - -def _create_column_from_literal(literal): - sc = SparkContext._active_spark_context - return sc._jvm.Dsl.lit(literal) - - -def _create_column_from_name(name): - sc = SparkContext._active_spark_context - return sc._jvm.Dsl.col(name) - - -def _to_java_column(col): - if isinstance(col, Column): - jcol = col._jc - else: - jcol = _create_column_from_name(col) - return jcol - - -def _unary_op(name, doc="unary operator"): - """ Create a method for given unary operator """ - def _(self): - jc = getattr(self._jc, name)() - return Column(jc, self.sql_ctx) - _.__doc__ = doc - return _ - - -def _dsl_op(name, doc=''): - def _(self): - jc = getattr(self._sc._jvm.Dsl, name)(self._jc) - return Column(jc, self.sql_ctx) - _.__doc__ = doc - return _ - - -def _bin_op(name, doc="binary operator"): - """ Create a method for given binary operator - """ - def _(self, other): - jc = other._jc if isinstance(other, Column) else other - njc = getattr(self._jc, name)(jc) - return Column(njc, self.sql_ctx) - _.__doc__ = doc - return _ - - -def _reverse_op(name, doc="binary operator"): - """ Create a method for binary operator (this object is on right side) - """ - def _(self, other): - jother = _create_column_from_literal(other) - jc = getattr(jother, name)(self._jc) - return Column(jc, self.sql_ctx) - _.__doc__ = doc - return _ - - -class Column(DataFrame): - - """ - A column in a DataFrame. - - `Column` instances can be created by:: - - # 1. Select a column out of a DataFrame - df.colName - df["colName"] - - # 2. Create from an expression - df.colName + 1 - 1 / df.colName - """ - - def __init__(self, jc, sql_ctx=None): - self._jc = jc - super(Column, self).__init__(jc, sql_ctx) - - # arithmetic operators - __neg__ = _dsl_op("negate") - __add__ = _bin_op("plus") - __sub__ = _bin_op("minus") - __mul__ = _bin_op("multiply") - __div__ = _bin_op("divide") - __mod__ = _bin_op("mod") - __radd__ = _bin_op("plus") - __rsub__ = _reverse_op("minus") - __rmul__ = _bin_op("multiply") - __rdiv__ = _reverse_op("divide") - __rmod__ = _reverse_op("mod") - - # logistic operators - __eq__ = _bin_op("equalTo") - __ne__ = _bin_op("notEqual") - __lt__ = _bin_op("lt") - __le__ = _bin_op("leq") - __ge__ = _bin_op("geq") - __gt__ = _bin_op("gt") - - # `and`, `or`, `not` cannot be overloaded in Python, - # so use bitwise operators as boolean operators - __and__ = _bin_op('and') - __or__ = _bin_op('or') - __invert__ = _dsl_op('not') - __rand__ = _bin_op("and") - __ror__ = _bin_op("or") - - # container operators - __contains__ = _bin_op("contains") - __getitem__ = _bin_op("getItem") - getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") - - # string methods - rlike = _bin_op("rlike") - like = _bin_op("like") - startswith = _bin_op("startsWith") - endswith = _bin_op("endsWith") - - def substr(self, startPos, length): - """ - Return a Column which is a substring of the column - - :param startPos: start position (int or Column) - :param length: length of the substring (int or Column) - - >>> df.name.substr(1, 3).collect() - [Row(col=u'Ali'), Row(col=u'Bob')] - """ - if type(startPos) != type(length): - raise TypeError("Can not mix the type") - if isinstance(startPos, (int, long)): - jc = self._jc.substr(startPos, length) - elif isinstance(startPos, Column): - jc = self._jc.substr(startPos._jc, length._jc) - else: - raise TypeError("Unexpected type: %s" % type(startPos)) - return Column(jc, self.sql_ctx) - - __getslice__ = substr - - # order - asc = _unary_op("asc") - desc = _unary_op("desc") - - isNull = _unary_op("isNull", "True if the current expression is null.") - isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") - - def alias(self, alias): - """Return a alias for this column - - >>> df.age.alias("age2").collect() - [Row(age2=2), Row(age2=5)] - """ - return Column(getattr(self._jc, "as")(alias), self.sql_ctx) - - def cast(self, dataType): - """ Convert the column into type `dataType` - - >>> df.select(df.age.cast("string").alias('ages')).collect() - [Row(ages=u'2'), Row(ages=u'5')] - >>> df.select(df.age.cast(StringType()).alias('ages')).collect() - [Row(ages=u'2'), Row(ages=u'5')] - """ - if self.sql_ctx is None: - sc = SparkContext._active_spark_context - ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) - else: - ssql_ctx = self.sql_ctx._ssql_ctx - if isinstance(dataType, basestring): - jc = self._jc.cast(dataType) - elif isinstance(dataType, DataType): - jdt = ssql_ctx.parseDataType(dataType.json()) - jc = self._jc.cast(jdt) - return Column(jc, self.sql_ctx) - - def to_pandas(self): - """ - Return a pandas.Series from the column - - >>> df.age.to_pandas() # doctest: +SKIP - 0 2 - 1 5 - dtype: int64 - """ - import pandas as pd - data = [c for c, in self.collect()] - return pd.Series(data) - - -def _aggregate_func(name, doc=""): - """ Create a function for aggregator by name""" - def _(col): - sc = SparkContext._active_spark_context - jc = getattr(sc._jvm.Dsl, name)(_to_java_column(col)) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return staticmethod(_) - - -class UserDefinedFunction(object): - def __init__(self, func, returnType): - self.func = func - self.returnType = returnType - self._broadcast = None - self._judf = self._create_judf() - - def _create_judf(self): - f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) - ser = AutoBatchedSerializer(PickleSerializer()) - command = (func, None, ser, ser) - sc = SparkContext._active_spark_context - pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) - ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) - jdt = ssql_ctx.parseDataType(self.returnType.json()) - judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, - includes, sc.pythonExec, broadcast_vars, - sc._javaAccumulator, jdt) - return judf - - def __del__(self): - if self._broadcast is not None: - self._broadcast.unpersist() - self._broadcast = None - - def __call__(self, *cols): - sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) - return Column(jc) - - -class Dsl(object): - """ - A collections of builtin aggregators - """ - DSLS = { - 'lit': 'Creates a :class:`Column` of literal value.', - 'col': 'Returns a :class:`Column` based on the given column name.', - 'column': 'Returns a :class:`Column` based on the given column name.', - 'upper': 'Converts a string expression to upper case.', - 'lower': 'Converts a string expression to upper case.', - 'sqrt': 'Computes the square root of the specified float value.', - 'abs': 'Computes the absolutle value.', - - 'max': 'Aggregate function: returns the maximum value of the expression in a group.', - 'min': 'Aggregate function: returns the minimum value of the expression in a group.', - 'first': 'Aggregate function: returns the first value in a group.', - 'last': 'Aggregate function: returns the last value in a group.', - 'count': 'Aggregate function: returns the number of items in a group.', - 'sum': 'Aggregate function: returns the sum of all values in the expression.', - 'avg': 'Aggregate function: returns the average of the values in a group.', - 'mean': 'Aggregate function: returns the average of the values in a group.', - 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', - } - - for _name, _doc in DSLS.items(): - locals()[_name] = _aggregate_func(_name, _doc) - del _name, _doc - - @staticmethod - def countDistinct(col, *cols): - """ Return a new Column for distinct count of (col, *cols) - - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.countDistinct(df.age, df.name).alias('c')).collect() - [Row(c=2)] - - >>> df.agg(Dsl.countDistinct("age", "name").alias('c')).collect() - [Row(c=2)] - """ - sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = sc._jvm.Dsl.countDistinct(_to_java_column(col), - sc._jvm.PythonUtils.toSeq(jcols)) - return Column(jc) - - @staticmethod - def approxCountDistinct(col, rsd=None): - """ Return a new Column for approxiate distinct count of (col, *cols) - - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.approxCountDistinct(df.age).alias('c')).collect() - [Row(c=2)] - """ - sc = SparkContext._active_spark_context - if rsd is None: - jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col)) - else: - jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col), rsd) - return Column(jc) - - @staticmethod - def udf(f, returnType=StringType()): - """Create a user defined function (UDF) - - >>> slen = Dsl.udf(lambda s: len(s), IntegerType()) - >>> df.select(slen(df.name).alias('slen')).collect() - [Row(slen=5), Row(slen=3)] - """ - return UserDefinedFunction(f, returnType) - - -def _test(): - import doctest - from pyspark.context import SparkContext - # let doctest run in pyspark.sql, so DataTypes can be picklable - import pyspark.sql - from pyspark.sql import Row, SQLContext - from pyspark.sql_tests import ExamplePoint, ExamplePointUDT - globs = pyspark.sql.__dict__.copy() - sc = SparkContext('local[4]', 'PythonTest') - globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) - globs['rdd'] = sc.parallelize( - [Row(field1=1, field2="row1"), - Row(field1=2, field2="row2"), - Row(field1=3, field2="row3")] - ) - rdd2 = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]) - rdd3 = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]) - globs['df'] = sqlCtx.inferSchema(rdd2) - globs['df2'] = sqlCtx.inferSchema(rdd3) - globs['ExamplePoint'] = ExamplePoint - globs['ExamplePointUDT'] = ExamplePointUDT - jsonStrings = [ - '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' - '"field6":[{"field7": "row2"}]}', - '{"field1" : null, "field2": "row3", ' - '"field3":{"field4":33, "field5": []}}' - ] - globs['jsonStrings'] = jsonStrings - globs['json'] = sc.parallelize(jsonStrings) - (failure_count, test_count) = doctest.testmod( - pyspark.sql, globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() - if failure_count: - exit(-1) - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py new file mode 100644 index 0000000000000..0a5ba00393aab --- /dev/null +++ b/python/pyspark/sql/__init__.py @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +public classes of Spark SQL: + + - L{SQLContext} + Main entry point for SQL functionality. + - L{DataFrame} + A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In + addition to normal RDD operations, DataFrames also support SQL. + - L{GroupedData} + - L{Column} + Column is a DataFrame with a single column. + - L{Row} + A Row of data returned by a Spark SQL query. + - L{HiveContext} + Main entry point for accessing data stored in Apache Hive.. +""" + +from pyspark.sql.context import SQLContext, HiveContext +from pyspark.sql.types import Row +from pyspark.sql.dataframe import DataFrame, GroupedData, Column, Dsl, SchemaRDD + +__all__ = [ + 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', + 'Dsl', +] diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py new file mode 100644 index 0000000000000..49f016a9cf2e9 --- /dev/null +++ b/python/pyspark/sql/context.py @@ -0,0 +1,642 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import warnings +import json +from array import array +from itertools import imap + +from py4j.protocol import Py4JError + +from pyspark.rdd import _prepare_for_python_RDD +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.sql.types import StringType, StructType, _verify_type, \ + _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter +from pyspark.sql.dataframe import DataFrame + +__all__ = ["SQLContext", "HiveContext"] + + +class SQLContext(object): + + """Main entry point for Spark SQL functionality. + + A SQLContext can be used create L{DataFrame}, register L{DataFrame} as + tables, execute SQL over tables, cache tables, and read parquet files. + """ + + def __init__(self, sparkContext, sqlContext=None): + """Create a new SQLContext. + + :param sparkContext: The SparkContext to wrap. + :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + SQLContext in the JVM, instead we make all calls to this object. + + >>> df = sqlCtx.inferSchema(rdd) + >>> sqlCtx.inferSchema(df) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + TypeError:... + + >>> bad_rdd = sc.parallelize([1,2,3]) + >>> sqlCtx.inferSchema(bad_rdd) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... + + >>> from datetime import datetime + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), + ... time=datetime(2014, 8, 1, 14, 1, 5))]) + >>> df = sqlCtx.inferSchema(allTypes) + >>> df.registerTempTable("allTypes") + >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + ... 'from allTypes where b and i > 0').collect() + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] + >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, + ... x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + """ + self._sc = sparkContext + self._jsc = self._sc._jsc + self._jvm = self._sc._jvm + self._scala_SQLContext = sqlContext + + @property + def _ssql_ctx(self): + """Accessor for the JVM Spark SQL context. + + Subclasses can override this property to provide their own + JVM Contexts. + """ + if self._scala_SQLContext is None: + self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) + return self._scala_SQLContext + + def registerFunction(self, name, f, returnType=StringType()): + """Registers a lambda function as a UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not given it default to a string and conversion will automatically + be done. For any other return type, the produced object must match the specified type. + + >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + [Row(c0=u'4')] + >>> from pyspark.sql.types import IntegerType + >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + """ + func = lambda _, it: imap(lambda x: f(*x), it) + ser = AutoBatchedSerializer(PickleSerializer()) + command = (func, None, ser, ser) + pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) + self._ssql_ctx.udf().registerPython(name, + bytearray(pickled_cmd), + env, + includes, + self._sc.pythonExec, + bvars, + self._sc._javaAccumulator, + returnType.json()) + + def inferSchema(self, rdd, samplingRatio=None): + """Infer and apply a schema to an RDD of L{Row}. + + When samplingRatio is specified, the schema is inferred by looking + at the types of each row in the sampled dataset. Otherwise, the + first 100 rows of the RDD are inspected. Nested collections are + supported, which can include array, dict, list, Row, tuple, + namedtuple, or object. + + Each row could be L{pyspark.sql.Row} object or namedtuple or objects. + Using top level dicts is deprecated, as dict is used to represent Maps. + + If a single column has multiple distinct inferred types, it may cause + runtime exceptions. + + >>> rdd = sc.parallelize( + ... [Row(field1=1, field2="row1"), + ... Row(field1=2, field2="row2"), + ... Row(field1=3, field2="row3")]) + >>> df = sqlCtx.inferSchema(rdd) + >>> df.collect()[0] + Row(field1=1, field2=u'row1') + + >>> NestedRow = Row("f1", "f2") + >>> nestedRdd1 = sc.parallelize([ + ... NestedRow(array('i', [1, 2]), {"row1": 1.0}), + ... NestedRow(array('i', [2, 3]), {"row2": 2.0})]) + >>> df = sqlCtx.inferSchema(nestedRdd1) + >>> df.collect() + [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})] + + >>> nestedRdd2 = sc.parallelize([ + ... NestedRow([[1, 2], [2, 3]], [1, 2]), + ... NestedRow([[2, 3], [3, 4]], [2, 3])]) + >>> df = sqlCtx.inferSchema(nestedRdd2) + >>> df.collect() + [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] + + >>> from collections import namedtuple + >>> CustomRow = namedtuple('CustomRow', 'field1 field2') + >>> rdd = sc.parallelize( + ... [CustomRow(field1=1, field2="row1"), + ... CustomRow(field1=2, field2="row2"), + ... CustomRow(field1=3, field2="row3")]) + >>> df = sqlCtx.inferSchema(rdd) + >>> df.collect()[0] + Row(field1=1, field2=u'row1') + """ + + if isinstance(rdd, DataFrame): + raise TypeError("Cannot apply schema to DataFrame") + + first = rdd.first() + if not first: + raise ValueError("The first row in RDD is empty, " + "can not infer schema") + if type(first) is dict: + warnings.warn("Using RDD of dict to inferSchema is deprecated," + "please use pyspark.sql.Row instead") + + if samplingRatio is None: + schema = _infer_schema(first) + if _has_nulltype(schema): + for row in rdd.take(100)[1:]: + schema = _merge_type(schema, _infer_schema(row)) + if not _has_nulltype(schema): + break + else: + warnings.warn("Some of types cannot be determined by the " + "first 100 rows, please try again with sampling") + else: + if samplingRatio > 0.99: + rdd = rdd.sample(False, float(samplingRatio)) + schema = rdd.map(_infer_schema).reduce(_merge_type) + + converter = _create_converter(schema) + rdd = rdd.map(converter) + return self.applySchema(rdd, schema) + + def applySchema(self, rdd, schema): + """ + Applies the given schema to the given RDD of L{tuple} or L{list}. + + These tuples or lists can contain complex nested structures like + lists, maps or nested rows. + + The schema should be a StructType. + + It is important that the schema matches the types of the objects + in each row or exceptions could be thrown at runtime. + + >>> from pyspark.sql.types import * + >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) + >>> schema = StructType([StructField("field1", IntegerType(), False), + ... StructField("field2", StringType(), False)]) + >>> df = sqlCtx.applySchema(rdd2, schema) + >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> df2 = sqlCtx.sql("SELECT * from table1") + >>> df2.collect() + [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] + + >>> from datetime import date, datetime + >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + ... date(2010, 1, 1), + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3], None)]) + >>> schema = StructType([ + ... StructField("byte1", ByteType(), False), + ... StructField("byte2", ByteType(), False), + ... StructField("short1", ShortType(), False), + ... StructField("short2", ShortType(), False), + ... StructField("int", IntegerType(), False), + ... StructField("float", FloatType(), False), + ... StructField("date", DateType(), False), + ... StructField("time", TimestampType(), False), + ... StructField("map", + ... MapType(StringType(), IntegerType(), False), False), + ... StructField("struct", + ... StructType([StructField("b", ShortType(), False)]), False), + ... StructField("list", ArrayType(ByteType(), False), False), + ... StructField("null", DoubleType(), True)]) + >>> df = sqlCtx.applySchema(rdd, schema) + >>> results = df.map( + ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date, + ... x.time, x.map["a"], x.struct.b, x.list, x.null)) + >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE + (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), + datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + + >>> df.registerTempTable("table2") + >>> sqlCtx.sql( + ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + + ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + + ... "float + 1.5 as float FROM table2").collect() + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] + + >>> from pyspark.sql.types import _parse_schema_abstract, _infer_schema_type + >>> rdd = sc.parallelize([(127, -32768, 1.0, + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3])]) + >>> abstract = "byte short float time map{} struct(b) list[]" + >>> schema = _parse_schema_abstract(abstract) + >>> typedSchema = _infer_schema_type(rdd.first(), schema) + >>> df = sqlCtx.applySchema(rdd, typedSchema) + >>> df.collect() + [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])] + """ + + if isinstance(rdd, DataFrame): + raise TypeError("Cannot apply schema to DataFrame") + + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + + # take the first few rows to verify schema + rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + + for row in rows: + _verify_type(row, schema) + + # convert python objects to sql data + converter = _python_to_sql_converter(schema) + rdd = rdd.map(converter) + + jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) + df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) + return DataFrame(df, self) + + def registerRDDAsTable(self, rdd, tableName): + """Registers the given RDD as a temporary table in the catalog. + + Temporary tables exist only during the lifetime of this instance of + SQLContext. + + >>> df = sqlCtx.inferSchema(rdd) + >>> sqlCtx.registerRDDAsTable(df, "table1") + """ + if (rdd.__class__ is DataFrame): + df = rdd._jdf + self._ssql_ctx.registerRDDAsTable(df, tableName) + else: + raise ValueError("Can only register DataFrame as table") + + def parquetFile(self, *paths): + """Loads a Parquet file, returning the result as a L{DataFrame}. + + >>> import tempfile, shutil + >>> parquetFile = tempfile.mkdtemp() + >>> shutil.rmtree(parquetFile) + >>> df = sqlCtx.inferSchema(rdd) + >>> df.saveAsParquetFile(parquetFile) + >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> sorted(df.collect()) == sorted(df2.collect()) + True + """ + gateway = self._sc._gateway + jpath = paths[0] + jpaths = gateway.new_array(gateway.jvm.java.lang.String, len(paths) - 1) + for i in range(1, len(paths)): + jpaths[i] = paths[i] + jdf = self._ssql_ctx.parquetFile(jpath, jpaths) + return DataFrame(jdf, self) + + def jsonFile(self, path, schema=None, samplingRatio=1.0): + """ + Loads a text file storing one JSON object per line as a + L{DataFrame}. + + If the schema is provided, applies the given schema to this + JSON dataset. + + Otherwise, it samples the dataset with ratio `samplingRatio` to + determine the schema. + + >>> import tempfile, shutil + >>> jsonFile = tempfile.mkdtemp() + >>> shutil.rmtree(jsonFile) + >>> ofn = open(jsonFile, 'w') + >>> for json in jsonStrings: + ... print>>ofn, json + >>> ofn.close() + >>> df1 = sqlCtx.jsonFile(jsonFile) + >>> sqlCtx.registerRDDAsTable(df1, "table1") + >>> df2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in df2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + + >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema()) + >>> sqlCtx.registerRDDAsTable(df3, "table2") + >>> df4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in df4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + + >>> from pyspark.sql.types import * + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) + >>> df5 = sqlCtx.jsonFile(jsonFile, schema) + >>> sqlCtx.registerRDDAsTable(df5, "table3") + >>> df6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> df6.collect() + [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] + """ + if schema is None: + df = self._ssql_ctx.jsonFile(path, samplingRatio) + else: + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) + df = self._ssql_ctx.jsonFile(path, scala_datatype) + return DataFrame(df, self) + + def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): + """Loads an RDD storing one JSON object per string as a L{DataFrame}. + + If the schema is provided, applies the given schema to this + JSON dataset. + + Otherwise, it samples the dataset with ratio `samplingRatio` to + determine the schema. + + >>> df1 = sqlCtx.jsonRDD(json) + >>> sqlCtx.registerRDDAsTable(df1, "table1") + >>> df2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in df2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + + >>> df3 = sqlCtx.jsonRDD(json, df1.schema()) + >>> sqlCtx.registerRDDAsTable(df3, "table2") + >>> df4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in df4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + + >>> from pyspark.sql.types import * + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) + >>> df5 = sqlCtx.jsonRDD(json, schema) + >>> sqlCtx.registerRDDAsTable(df5, "table3") + >>> df6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> df6.collect() + [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] + + >>> sqlCtx.jsonRDD(sc.parallelize(['{}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] + >>> sqlCtx.jsonRDD(sc.parallelize(['{"key0": null}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] + """ + + def func(iterator): + for x in iterator: + if not isinstance(x, basestring): + x = unicode(x) + if isinstance(x, unicode): + x = x.encode("utf-8") + yield x + keyed = rdd.mapPartitions(func) + keyed._bypass_serializer = True + jrdd = keyed._jrdd.map(self._jvm.BytesToString()) + if schema is None: + df = self._ssql_ctx.jsonRDD(jrdd.rdd(), samplingRatio) + else: + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) + df = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) + return DataFrame(df, self) + + def sql(self, sqlQuery): + """Return a L{DataFrame} representing the result of the given query. + + >>> df = sqlCtx.inferSchema(rdd) + >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> df2.collect() + [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] + """ + return DataFrame(self._ssql_ctx.sql(sqlQuery), self) + + def table(self, tableName): + """Returns the specified table as a L{DataFrame}. + + >>> df = sqlCtx.inferSchema(rdd) + >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> df2 = sqlCtx.table("table1") + >>> sorted(df.collect()) == sorted(df2.collect()) + True + """ + return DataFrame(self._ssql_ctx.table(tableName), self) + + def cacheTable(self, tableName): + """Caches the specified table in-memory.""" + self._ssql_ctx.cacheTable(tableName) + + def uncacheTable(self, tableName): + """Removes the specified table from the in-memory cache.""" + self._ssql_ctx.uncacheTable(tableName) + + +class HiveContext(SQLContext): + + """A variant of Spark SQL that integrates with data stored in Hive. + + Configuration for Hive is read from hive-site.xml on the classpath. + It supports running both SQL and HiveQL commands. + """ + + def __init__(self, sparkContext, hiveContext=None): + """Create a new HiveContext. + + :param sparkContext: The SparkContext to wrap. + :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new + HiveContext in the JVM, instead we make all calls to this object. + """ + SQLContext.__init__(self, sparkContext) + + if hiveContext: + self._scala_HiveContext = hiveContext + + @property + def _ssql_ctx(self): + try: + if not hasattr(self, '_scala_HiveContext'): + self._scala_HiveContext = self._get_hive_ctx() + return self._scala_HiveContext + except Py4JError as e: + raise Exception("You must build Spark with Hive. " + "Export 'SPARK_HIVE=true' and run " + "build/sbt assembly", e) + + def _get_hive_ctx(self): + return self._jvm.HiveContext(self._jsc.sc()) + + +def _create_row(fields, values): + row = Row(*values) + row.__FIELDS__ = fields + return row + + +class Row(tuple): + + """ + A row in L{DataFrame}. The fields in it can be accessed like attributes. + + Row can be used to create a row object by using named arguments, + the fields will be sorted by names. + + >>> row = Row(name="Alice", age=11) + >>> row + Row(age=11, name='Alice') + >>> row.name, row.age + ('Alice', 11) + + Row also can be used to create another Row like class, then it + could be used to create Row objects, such as + + >>> Person = Row("name", "age") + >>> Person + + >>> Person("Alice", 11) + Row(name='Alice', age=11) + """ + + def __new__(self, *args, **kwargs): + if args and kwargs: + raise ValueError("Can not use both args " + "and kwargs to create Row") + if args: + # create row class or objects + return tuple.__new__(self, args) + + elif kwargs: + # create row objects + names = sorted(kwargs.keys()) + values = tuple(kwargs[n] for n in names) + row = tuple.__new__(self, values) + row.__FIELDS__ = names + return row + + else: + raise ValueError("No args or kwargs") + + def asDict(self): + """ + Return as an dict + """ + if not hasattr(self, "__FIELDS__"): + raise TypeError("Cannot convert a Row class into dict") + return dict(zip(self.__FIELDS__, self)) + + # let obect acs like class + def __call__(self, *args): + """create new Row object""" + return _create_row(self, args) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + try: + # it will be slow when it has many fields, + # but this will not be used in normal cases + idx = self.__FIELDS__.index(item) + return self[idx] + except IndexError: + raise AttributeError(item) + + def __reduce__(self): + if hasattr(self, "__FIELDS__"): + return (_create_row, (self.__FIELDS__, tuple(self))) + else: + return tuple.__reduce__(self) + + def __repr__(self): + if hasattr(self, "__FIELDS__"): + return "Row(%s)" % ", ".join("%s=%r" % (k, v) + for k, v in zip(self.__FIELDS__, self)) + else: + return "" % ", ".join(self) + + +def _test(): + import doctest + from pyspark.context import SparkContext + from pyspark.sql import Row, SQLContext + import pyspark.sql.context + globs = pyspark.sql.context.__dict__.copy() + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['rdd'] = sc.parallelize( + [Row(field1=1, field2="row1"), + Row(field1=2, field2="row2"), + Row(field1=3, field2="row3")] + ) + jsonStrings = [ + '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' + '"field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", ' + '"field3":{"field4":33, "field5": []}}' + ] + globs['jsonStrings'] = jsonStrings + globs['json'] = sc.parallelize(jsonStrings) + (failure_count, test_count) = doctest.testmod( + pyspark.sql.context, globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py new file mode 100644 index 0000000000000..cda704eea75f5 --- /dev/null +++ b/python/pyspark/sql/dataframe.py @@ -0,0 +1,974 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import itertools +import warnings +import random +import os +from tempfile import NamedTemporaryFile +from itertools import imap + +from py4j.java_collections import ListConverter, MapConverter + +from pyspark.context import SparkContext +from pyspark.rdd import RDD, _prepare_for_python_RDD +from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \ + UTF8Deserializer +from pyspark.storagelevel import StorageLevel +from pyspark.traceback_utils import SCCallSiteSync +from pyspark.sql.types import * +from pyspark.sql.types import _create_cls, _parse_datatype_json_string + + +__all__ = ["DataFrame", "GroupedData", "Column", "Dsl", "SchemaRDD"] + + +class DataFrame(object): + + """A collection of rows that have the same columns. + + A :class:`DataFrame` is equivalent to a relational table in Spark SQL, + and can be created using various functions in :class:`SQLContext`:: + + people = sqlContext.parquetFile("...") + + Once created, it can be manipulated using the various domain-specific-language + (DSL) functions defined in: :class:`DataFrame`, :class:`Column`. + + To select a column from the data frame, use the apply method:: + + ageCol = people.age + + Note that the :class:`Column` type can also be manipulated + through its various functions:: + + # The following creates a new column that increases everybody's age by 10. + people.age + 10 + + + A more concrete example:: + + # To create DataFrame using SQLContext + people = sqlContext.parquetFile("...") + department = sqlContext.parquetFile("...") + + people.filter(people.age > 30).join(department, people.deptId == department.id)) \ + .groupBy(department.name, "gender").agg({"salary": "avg", "age": "max"}) + """ + + def __init__(self, jdf, sql_ctx): + self._jdf = jdf + self.sql_ctx = sql_ctx + self._sc = sql_ctx and sql_ctx._sc + self.is_cached = False + + @property + def rdd(self): + """ + Return the content of the :class:`DataFrame` as an :class:`RDD` + of :class:`Row` s. + """ + if not hasattr(self, '_lazy_rdd'): + jrdd = self._jdf.javaToPython() + rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) + schema = self.schema() + + def applySchema(it): + cls = _create_cls(schema) + return itertools.imap(cls, it) + + self._lazy_rdd = rdd.mapPartitions(applySchema) + + return self._lazy_rdd + + def toJSON(self, use_unicode=False): + """Convert a DataFrame into a MappedRDD of JSON documents; one document per row. + + >>> df.toJSON().first() + '{"age":2,"name":"Alice"}' + """ + rdd = self._jdf.toJSON() + return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) + + def saveAsParquetFile(self, path): + """Save the contents as a Parquet file, preserving the schema. + + Files that are written out using this method can be read back in as + a DataFrame using the L{SQLContext.parquetFile} method. + + >>> import tempfile, shutil + >>> parquetFile = tempfile.mkdtemp() + >>> shutil.rmtree(parquetFile) + >>> df.saveAsParquetFile(parquetFile) + >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> sorted(df2.collect()) == sorted(df.collect()) + True + """ + self._jdf.saveAsParquetFile(path) + + def registerTempTable(self, name): + """Registers this RDD as a temporary table using the given name. + + The lifetime of this temporary table is tied to the L{SQLContext} + that was used to create this DataFrame. + + >>> df.registerTempTable("people") + >>> df2 = sqlCtx.sql("select * from people") + >>> sorted(df.collect()) == sorted(df2.collect()) + True + """ + self._jdf.registerTempTable(name) + + def registerAsTable(self, name): + """DEPRECATED: use registerTempTable() instead""" + warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) + self.registerTempTable(name) + + def insertInto(self, tableName, overwrite=False): + """Inserts the contents of this DataFrame into the specified table. + + Optionally overwriting any existing data. + """ + self._jdf.insertInto(tableName, overwrite) + + def saveAsTable(self, tableName): + """Creates a new table with the contents of this DataFrame.""" + self._jdf.saveAsTable(tableName) + + def schema(self): + """Returns the schema of this DataFrame (represented by + a L{StructType}). + + >>> df.schema() + StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) + """ + return _parse_datatype_json_string(self._jdf.schema().json()) + + def printSchema(self): + """Prints out the schema in the tree format. + + >>> df.printSchema() + root + |-- age: integer (nullable = true) + |-- name: string (nullable = true) + + """ + print (self._jdf.schema().treeString()) + + def count(self): + """Return the number of elements in this RDD. + + Unlike the base RDD implementation of count, this implementation + leverages the query optimizer to compute the count on the DataFrame, + which supports features such as filter pushdown. + + >>> df.count() + 2L + """ + return self._jdf.count() + + def collect(self): + """Return a list that contains all of the rows. + + Each object in the list is a Row, the fields can be accessed as + attributes. + + >>> df.collect() + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + """ + with SCCallSiteSync(self._sc) as css: + bytesInJava = self._jdf.javaToPython().collect().iterator() + tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir) + tempFile.close() + self._sc._writeToFile(bytesInJava, tempFile.name) + # Read the data into Python and deserialize it: + with open(tempFile.name, 'rb') as tempFile: + rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile)) + os.unlink(tempFile.name) + cls = _create_cls(self.schema()) + return [cls(r) for r in rs] + + def limit(self, num): + """Limit the result count to the number specified. + + >>> df.limit(1).collect() + [Row(age=2, name=u'Alice')] + >>> df.limit(0).collect() + [] + """ + jdf = self._jdf.limit(num) + return DataFrame(jdf, self.sql_ctx) + + def take(self, num): + """Take the first num rows of the RDD. + + Each object in the list is a Row, the fields can be accessed as + attributes. + + >>> df.take(2) + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + """ + return self.limit(num).collect() + + def map(self, f): + """ Return a new RDD by applying a function to each Row, it's a + shorthand for df.rdd.map() + + >>> df.map(lambda p: p.name).collect() + [u'Alice', u'Bob'] + """ + return self.rdd.map(f) + + def mapPartitions(self, f, preservesPartitioning=False): + """ + Return a new RDD by applying a function to each partition. + + >>> rdd = sc.parallelize([1, 2, 3, 4], 4) + >>> def f(iterator): yield 1 + >>> rdd.mapPartitions(f).sum() + 4 + """ + return self.rdd.mapPartitions(f, preservesPartitioning) + + def cache(self): + """ Persist with the default storage level (C{MEMORY_ONLY_SER}). + """ + self.is_cached = True + self._jdf.cache() + return self + + def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): + """ Set the storage level to persist its values across operations + after the first time it is computed. This can only be used to assign + a new storage level if the RDD does not have a storage level set yet. + If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). + """ + self.is_cached = True + javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel) + self._jdf.persist(javaStorageLevel) + return self + + def unpersist(self, blocking=True): + """ Mark it as non-persistent, and remove all blocks for it from + memory and disk. + """ + self.is_cached = False + self._jdf.unpersist(blocking) + return self + + # def coalesce(self, numPartitions, shuffle=False): + # rdd = self._jdf.coalesce(numPartitions, shuffle, None) + # return DataFrame(rdd, self.sql_ctx) + + def repartition(self, numPartitions): + """ Return a new :class:`DataFrame` that has exactly `numPartitions` + partitions. + """ + rdd = self._jdf.repartition(numPartitions, None) + return DataFrame(rdd, self.sql_ctx) + + def sample(self, withReplacement, fraction, seed=None): + """ + Return a sampled subset of this DataFrame. + + >>> df.sample(False, 0.5, 97).count() + 1L + """ + assert fraction >= 0.0, "Negative fraction value: %s" % fraction + seed = seed if seed is not None else random.randint(0, sys.maxint) + rdd = self._jdf.sample(withReplacement, fraction, long(seed)) + return DataFrame(rdd, self.sql_ctx) + + # def takeSample(self, withReplacement, num, seed=None): + # """Return a fixed-size sampled subset of this DataFrame. + # + # >>> df = sqlCtx.inferSchema(rdd) + # >>> df.takeSample(False, 2, 97) + # [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')] + # """ + # seed = seed if seed is not None else random.randint(0, sys.maxint) + # with SCCallSiteSync(self.context) as css: + # bytesInJava = self._jdf \ + # .takeSampleToPython(withReplacement, num, long(seed)) \ + # .iterator() + # cls = _create_cls(self.schema()) + # return map(cls, self._collect_iterator_through_file(bytesInJava)) + + @property + def dtypes(self): + """Return all column names and their data types as a list. + + >>> df.dtypes + [('age', 'integer'), ('name', 'string')] + """ + return [(str(f.name), f.dataType.jsonValue()) for f in self.schema().fields] + + @property + def columns(self): + """ Return all column names as a list. + + >>> df.columns + [u'age', u'name'] + """ + return [f.name for f in self.schema().fields] + + def join(self, other, joinExprs=None, joinType=None): + """ + Join with another DataFrame, using the given join expression. + The following performs a full outer join between `df1` and `df2`:: + + :param other: Right side of the join + :param joinExprs: Join expression + :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. + + >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() + [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + """ + + if joinExprs is None: + jdf = self._jdf.join(other._jdf) + else: + assert isinstance(joinExprs, Column), "joinExprs should be Column" + if joinType is None: + jdf = self._jdf.join(other._jdf, joinExprs._jc) + else: + assert isinstance(joinType, basestring), "joinType should be basestring" + jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) + return DataFrame(jdf, self.sql_ctx) + + def sort(self, *cols): + """ Return a new :class:`DataFrame` sorted by the specified column. + + :param cols: The columns or expressions used for sorting + + >>> df.sort(df.age.desc()).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.sortBy(df.age.desc()).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + """ + if not cols: + raise ValueError("should sort by at least one column") + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + self._sc._gateway._gateway_client) + jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) + return DataFrame(jdf, self.sql_ctx) + + sortBy = sort + + def head(self, n=None): + """ Return the first `n` rows or the first row if n is None. + + >>> df.head() + Row(age=2, name=u'Alice') + >>> df.head(1) + [Row(age=2, name=u'Alice')] + """ + if n is None: + rs = self.head(1) + return rs[0] if rs else None + return self.take(n) + + def first(self): + """ Return the first row. + + >>> df.first() + Row(age=2, name=u'Alice') + """ + return self.head() + + def __getitem__(self, item): + """ Return the column by given name + + >>> df['age'].collect() + [Row(age=2), Row(age=5)] + >>> df[ ["name", "age"]].collect() + [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] + >>> df[ df.age > 3 ].collect() + [Row(age=5, name=u'Bob')] + """ + if isinstance(item, basestring): + jc = self._jdf.apply(item) + return Column(jc, self.sql_ctx) + elif isinstance(item, Column): + return self.filter(item) + elif isinstance(item, list): + return self.select(*item) + else: + raise IndexError("unexpected index: %s" % item) + + def __getattr__(self, name): + """ Return the column by given name + + >>> df.age.collect() + [Row(age=2), Row(age=5)] + """ + if name.startswith("__"): + raise AttributeError(name) + jc = self._jdf.apply(name) + return Column(jc, self.sql_ctx) + + def select(self, *cols): + """ Selecting a set of expressions. + + >>> df.select().collect() + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + >>> df.select('*').collect() + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + >>> df.select('name', 'age').collect() + [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] + >>> df.select(df.name, (df.age + 10).alias('age')).collect() + [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] + """ + if not cols: + cols = ["*"] + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + self._sc._gateway._gateway_client) + jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + return DataFrame(jdf, self.sql_ctx) + + def selectExpr(self, *expr): + """ + Selects a set of SQL expressions. This is a variant of + `select` that accepts SQL expressions. + + >>> df.selectExpr("age * 2", "abs(age)").collect() + [Row(('age * 2)=4, Abs('age)=2), Row(('age * 2)=10, Abs('age)=5)] + """ + jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) + jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) + return DataFrame(jdf, self.sql_ctx) + + def filter(self, condition): + """ Filtering rows using the given condition, which could be + Column expression or string of SQL expression. + + where() is an alias for filter(). + + >>> df.filter(df.age > 3).collect() + [Row(age=5, name=u'Bob')] + >>> df.where(df.age == 2).collect() + [Row(age=2, name=u'Alice')] + + >>> df.filter("age > 3").collect() + [Row(age=5, name=u'Bob')] + >>> df.where("age = 2").collect() + [Row(age=2, name=u'Alice')] + """ + if isinstance(condition, basestring): + jdf = self._jdf.filter(condition) + elif isinstance(condition, Column): + jdf = self._jdf.filter(condition._jc) + else: + raise TypeError("condition should be string or Column") + return DataFrame(jdf, self.sql_ctx) + + where = filter + + def groupBy(self, *cols): + """ Group the :class:`DataFrame` using the specified columns, + so we can run aggregation on them. See :class:`GroupedData` + for all the available aggregate functions. + + >>> df.groupBy().avg().collect() + [Row(AVG(age#0)=3.5)] + >>> df.groupBy('name').agg({'age': 'mean'}).collect() + [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + >>> df.groupBy(df.name).avg().collect() + [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + """ + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + self._sc._gateway._gateway_client) + jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + return GroupedData(jdf, self.sql_ctx) + + def agg(self, *exprs): + """ Aggregate on the entire :class:`DataFrame` without groups + (shorthand for df.groupBy.agg()). + + >>> df.agg({"age": "max"}).collect() + [Row(MAX(age#0)=5)] + >>> from pyspark.sql import Dsl + >>> df.agg(Dsl.min(df.age)).collect() + [Row(MIN(age#0)=2)] + """ + return self.groupBy().agg(*exprs) + + def unionAll(self, other): + """ Return a new DataFrame containing union of rows in this + frame and another frame. + + This is equivalent to `UNION ALL` in SQL. + """ + return DataFrame(self._jdf.unionAll(other._jdf), self.sql_ctx) + + def intersect(self, other): + """ Return a new :class:`DataFrame` containing rows only in + both this frame and another frame. + + This is equivalent to `INTERSECT` in SQL. + """ + return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx) + + def subtract(self, other): + """ Return a new :class:`DataFrame` containing rows in this frame + but not in another frame. + + This is equivalent to `EXCEPT` in SQL. + """ + return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) + + def addColumn(self, colName, col): + """ Return a new :class:`DataFrame` by adding a column. + + >>> df.addColumn('age2', df.age + 2).collect() + [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] + """ + return self.select('*', col.alias(colName)) + + def to_pandas(self): + """ + Collect all the rows and return a `pandas.DataFrame`. + + >>> df.to_pandas() # doctest: +SKIP + age name + 0 2 Alice + 1 5 Bob + """ + import pandas as pd + return pd.DataFrame.from_records(self.collect(), columns=self.columns) + + +# Having SchemaRDD for backward compatibility (for docs) +class SchemaRDD(DataFrame): + """ + SchemaRDD is deprecated, please use DataFrame + """ + + +def dfapi(f): + def _api(self): + name = f.__name__ + jdf = getattr(self._jdf, name)() + return DataFrame(jdf, self.sql_ctx) + _api.__name__ = f.__name__ + _api.__doc__ = f.__doc__ + return _api + + +class GroupedData(object): + + """ + A set of methods for aggregations on a :class:`DataFrame`, + created by DataFrame.groupBy(). + """ + + def __init__(self, jdf, sql_ctx): + self._jdf = jdf + self.sql_ctx = sql_ctx + + def agg(self, *exprs): + """ Compute aggregates by specifying a map from column name + to aggregate methods. + + The available aggregate methods are `avg`, `max`, `min`, + `sum`, `count`. + + :param exprs: list or aggregate columns or a map from column + name to aggregate methods. + + >>> gdf = df.groupBy(df.name) + >>> gdf.agg({"age": "max"}).collect() + [Row(name=u'Bob', MAX(age#0)=5), Row(name=u'Alice', MAX(age#0)=2)] + >>> from pyspark.sql import Dsl + >>> gdf.agg(Dsl.min(df.age)).collect() + [Row(MIN(age#0)=5), Row(MIN(age#0)=2)] + """ + assert exprs, "exprs should not be empty" + if len(exprs) == 1 and isinstance(exprs[0], dict): + jmap = MapConverter().convert(exprs[0], + self.sql_ctx._sc._gateway._gateway_client) + jdf = self._jdf.agg(jmap) + else: + # Columns + assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" + jcols = ListConverter().convert([c._jc for c in exprs[1:]], + self.sql_ctx._sc._gateway._gateway_client) + jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + return DataFrame(jdf, self.sql_ctx) + + @dfapi + def count(self): + """ Count the number of rows for each group. + + >>> df.groupBy(df.age).count().collect() + [Row(age=2, count=1), Row(age=5, count=1)] + """ + + @dfapi + def mean(self): + """Compute the average value for each numeric columns + for each group. This is an alias for `avg`.""" + + @dfapi + def avg(self): + """Compute the average value for each numeric columns + for each group.""" + + @dfapi + def max(self): + """Compute the max value for each numeric columns for + each group. """ + + @dfapi + def min(self): + """Compute the min value for each numeric column for + each group.""" + + @dfapi + def sum(self): + """Compute the sum for each numeric columns for each + group.""" + + +def _create_column_from_literal(literal): + sc = SparkContext._active_spark_context + return sc._jvm.Dsl.lit(literal) + + +def _create_column_from_name(name): + sc = SparkContext._active_spark_context + return sc._jvm.Dsl.col(name) + + +def _to_java_column(col): + if isinstance(col, Column): + jcol = col._jc + else: + jcol = _create_column_from_name(col) + return jcol + + +def _unary_op(name, doc="unary operator"): + """ Create a method for given unary operator """ + def _(self): + jc = getattr(self._jc, name)() + return Column(jc, self.sql_ctx) + _.__doc__ = doc + return _ + + +def _dsl_op(name, doc=''): + def _(self): + jc = getattr(self._sc._jvm.Dsl, name)(self._jc) + return Column(jc, self.sql_ctx) + _.__doc__ = doc + return _ + + +def _bin_op(name, doc="binary operator"): + """ Create a method for given binary operator + """ + def _(self, other): + jc = other._jc if isinstance(other, Column) else other + njc = getattr(self._jc, name)(jc) + return Column(njc, self.sql_ctx) + _.__doc__ = doc + return _ + + +def _reverse_op(name, doc="binary operator"): + """ Create a method for binary operator (this object is on right side) + """ + def _(self, other): + jother = _create_column_from_literal(other) + jc = getattr(jother, name)(self._jc) + return Column(jc, self.sql_ctx) + _.__doc__ = doc + return _ + + +class Column(DataFrame): + + """ + A column in a DataFrame. + + `Column` instances can be created by:: + + # 1. Select a column out of a DataFrame + df.colName + df["colName"] + + # 2. Create from an expression + df.colName + 1 + 1 / df.colName + """ + + def __init__(self, jc, sql_ctx=None): + self._jc = jc + super(Column, self).__init__(jc, sql_ctx) + + # arithmetic operators + __neg__ = _dsl_op("negate") + __add__ = _bin_op("plus") + __sub__ = _bin_op("minus") + __mul__ = _bin_op("multiply") + __div__ = _bin_op("divide") + __mod__ = _bin_op("mod") + __radd__ = _bin_op("plus") + __rsub__ = _reverse_op("minus") + __rmul__ = _bin_op("multiply") + __rdiv__ = _reverse_op("divide") + __rmod__ = _reverse_op("mod") + + # logistic operators + __eq__ = _bin_op("equalTo") + __ne__ = _bin_op("notEqual") + __lt__ = _bin_op("lt") + __le__ = _bin_op("leq") + __ge__ = _bin_op("geq") + __gt__ = _bin_op("gt") + + # `and`, `or`, `not` cannot be overloaded in Python, + # so use bitwise operators as boolean operators + __and__ = _bin_op('and') + __or__ = _bin_op('or') + __invert__ = _dsl_op('not') + __rand__ = _bin_op("and") + __ror__ = _bin_op("or") + + # container operators + __contains__ = _bin_op("contains") + __getitem__ = _bin_op("getItem") + getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") + + # string methods + rlike = _bin_op("rlike") + like = _bin_op("like") + startswith = _bin_op("startsWith") + endswith = _bin_op("endsWith") + + def substr(self, startPos, length): + """ + Return a Column which is a substring of the column + + :param startPos: start position (int or Column) + :param length: length of the substring (int or Column) + + >>> df.name.substr(1, 3).collect() + [Row(col=u'Ali'), Row(col=u'Bob')] + """ + if type(startPos) != type(length): + raise TypeError("Can not mix the type") + if isinstance(startPos, (int, long)): + jc = self._jc.substr(startPos, length) + elif isinstance(startPos, Column): + jc = self._jc.substr(startPos._jc, length._jc) + else: + raise TypeError("Unexpected type: %s" % type(startPos)) + return Column(jc, self.sql_ctx) + + __getslice__ = substr + + # order + asc = _unary_op("asc") + desc = _unary_op("desc") + + isNull = _unary_op("isNull", "True if the current expression is null.") + isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") + + def alias(self, alias): + """Return a alias for this column + + >>> df.age.alias("age2").collect() + [Row(age2=2), Row(age2=5)] + """ + return Column(getattr(self._jc, "as")(alias), self.sql_ctx) + + def cast(self, dataType): + """ Convert the column into type `dataType` + + >>> df.select(df.age.cast("string").alias('ages')).collect() + [Row(ages=u'2'), Row(ages=u'5')] + >>> df.select(df.age.cast(StringType()).alias('ages')).collect() + [Row(ages=u'2'), Row(ages=u'5')] + """ + if self.sql_ctx is None: + sc = SparkContext._active_spark_context + ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) + else: + ssql_ctx = self.sql_ctx._ssql_ctx + if isinstance(dataType, basestring): + jc = self._jc.cast(dataType) + elif isinstance(dataType, DataType): + jdt = ssql_ctx.parseDataType(dataType.json()) + jc = self._jc.cast(jdt) + return Column(jc, self.sql_ctx) + + def to_pandas(self): + """ + Return a pandas.Series from the column + + >>> df.age.to_pandas() # doctest: +SKIP + 0 2 + 1 5 + dtype: int64 + """ + import pandas as pd + data = [c for c, in self.collect()] + return pd.Series(data) + + +def _aggregate_func(name, doc=""): + """ Create a function for aggregator by name""" + def _(col): + sc = SparkContext._active_spark_context + jc = getattr(sc._jvm.Dsl, name)(_to_java_column(col)) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return staticmethod(_) + + +class UserDefinedFunction(object): + def __init__(self, func, returnType): + self.func = func + self.returnType = returnType + self._broadcast = None + self._judf = self._create_judf() + + def _create_judf(self): + f = self.func # put it in closure `func` + func = lambda _, it: imap(lambda x: f(*x), it) + ser = AutoBatchedSerializer(PickleSerializer()) + command = (func, None, ser, ser) + sc = SparkContext._active_spark_context + pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) + ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) + jdt = ssql_ctx.parseDataType(self.returnType.json()) + judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, + includes, sc.pythonExec, broadcast_vars, + sc._javaAccumulator, jdt) + return judf + + def __del__(self): + if self._broadcast is not None: + self._broadcast.unpersist() + self._broadcast = None + + def __call__(self, *cols): + sc = SparkContext._active_spark_context + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + sc._gateway._gateway_client) + jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) + return Column(jc) + + +class Dsl(object): + """ + A collections of builtin aggregators + """ + DSLS = { + 'lit': 'Creates a :class:`Column` of literal value.', + 'col': 'Returns a :class:`Column` based on the given column name.', + 'column': 'Returns a :class:`Column` based on the given column name.', + 'upper': 'Converts a string expression to upper case.', + 'lower': 'Converts a string expression to upper case.', + 'sqrt': 'Computes the square root of the specified float value.', + 'abs': 'Computes the absolutle value.', + + 'max': 'Aggregate function: returns the maximum value of the expression in a group.', + 'min': 'Aggregate function: returns the minimum value of the expression in a group.', + 'first': 'Aggregate function: returns the first value in a group.', + 'last': 'Aggregate function: returns the last value in a group.', + 'count': 'Aggregate function: returns the number of items in a group.', + 'sum': 'Aggregate function: returns the sum of all values in the expression.', + 'avg': 'Aggregate function: returns the average of the values in a group.', + 'mean': 'Aggregate function: returns the average of the values in a group.', + 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', + } + + for _name, _doc in DSLS.items(): + locals()[_name] = _aggregate_func(_name, _doc) + del _name, _doc + + @staticmethod + def countDistinct(col, *cols): + """ Return a new Column for distinct count of (col, *cols) + + >>> from pyspark.sql import Dsl + >>> df.agg(Dsl.countDistinct(df.age, df.name).alias('c')).collect() + [Row(c=2)] + + >>> df.agg(Dsl.countDistinct("age", "name").alias('c')).collect() + [Row(c=2)] + """ + sc = SparkContext._active_spark_context + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + sc._gateway._gateway_client) + jc = sc._jvm.Dsl.countDistinct(_to_java_column(col), + sc._jvm.PythonUtils.toSeq(jcols)) + return Column(jc) + + @staticmethod + def approxCountDistinct(col, rsd=None): + """ Return a new Column for approxiate distinct count of (col, *cols) + + >>> from pyspark.sql import Dsl + >>> df.agg(Dsl.approxCountDistinct(df.age).alias('c')).collect() + [Row(c=2)] + """ + sc = SparkContext._active_spark_context + if rsd is None: + jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col)) + else: + jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col), rsd) + return Column(jc) + + @staticmethod + def udf(f, returnType=StringType()): + """Create a user defined function (UDF) + + >>> slen = Dsl.udf(lambda s: len(s), IntegerType()) + >>> df.select(slen(df.name).alias('slen')).collect() + [Row(slen=5), Row(slen=3)] + """ + return UserDefinedFunction(f, returnType) + + +def _test(): + import doctest + from pyspark.context import SparkContext + from pyspark.sql import Row, SQLContext + import pyspark.sql.dataframe + globs = pyspark.sql.dataframe.__dict__.copy() + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlCtx'] = sqlCtx = SQLContext(sc) + rdd2 = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]) + rdd3 = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]) + globs['df'] = sqlCtx.inferSchema(rdd2) + globs['df2'] = sqlCtx.inferSchema(rdd3) + (failure_count, test_count) = doctest.testmod( + pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/sql_tests.py b/python/pyspark/sql/tests.py similarity index 96% rename from python/pyspark/sql_tests.py rename to python/pyspark/sql/tests.py index d314f46e8d2d5..d25c6365ed067 100644 --- a/python/pyspark/sql_tests.py +++ b/python/pyspark/sql/tests.py @@ -34,8 +34,10 @@ else: import unittest -from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ - UserDefinedType, DoubleType + +from pyspark.sql import SQLContext, Column +from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, StructField, \ + UserDefinedType, DoubleType, LongType from pyspark.tests import ReusedPySparkTestCase @@ -220,7 +222,7 @@ def test_convert_row_to_dict(self): self.assertEqual(1.0, row.asDict()['d']['key'].c) def test_infer_schema_with_udt(self): - from pyspark.sql_tests import ExamplePoint, ExamplePointUDT + from pyspark.sql.tests import ExamplePoint, ExamplePointUDT row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) rdd = self.sc.parallelize([row]) df = self.sqlCtx.inferSchema(rdd) @@ -232,7 +234,7 @@ def test_infer_schema_with_udt(self): self.assertEqual(point, ExamplePoint(1.0, 2.0)) def test_apply_schema_with_udt(self): - from pyspark.sql_tests import ExamplePoint, ExamplePointUDT + from pyspark.sql.tests import ExamplePoint, ExamplePointUDT row = (1.0, ExamplePoint(1.0, 2.0)) rdd = self.sc.parallelize([row]) schema = StructType([StructField("label", DoubleType(), False), @@ -242,7 +244,7 @@ def test_apply_schema_with_udt(self): self.assertEquals(point, ExamplePoint(1.0, 2.0)) def test_parquet_with_udt(self): - from pyspark.sql_tests import ExamplePoint + from pyspark.sql.tests import ExamplePoint row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) rdd = self.sc.parallelize([row]) df0 = self.sqlCtx.inferSchema(rdd) @@ -253,7 +255,6 @@ def test_parquet_with_udt(self): self.assertEquals(point, ExamplePoint(1.0, 2.0)) def test_column_operators(self): - from pyspark.sql import Column, LongType ci = self.df.key cs = self.df.value c = ci == cs diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py new file mode 100644 index 0000000000000..41afefe48ee5e --- /dev/null +++ b/python/pyspark/sql/types.py @@ -0,0 +1,1279 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import decimal +import datetime +import keyword +import warnings +import json +import re +from array import array +from operator import itemgetter + + +__all__ = [ + "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", + "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", + "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", ] + + +class DataType(object): + + """Spark SQL DataType""" + + def __repr__(self): + return self.__class__.__name__ + + def __hash__(self): + return hash(str(self)) + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.__dict__ == other.__dict__) + + def __ne__(self, other): + return not self.__eq__(other) + + @classmethod + def typeName(cls): + return cls.__name__[:-4].lower() + + def jsonValue(self): + return self.typeName() + + def json(self): + return json.dumps(self.jsonValue(), + separators=(',', ':'), + sort_keys=True) + + +class PrimitiveTypeSingleton(type): + + """Metaclass for PrimitiveType""" + + _instances = {} + + def __call__(cls): + if cls not in cls._instances: + cls._instances[cls] = super(PrimitiveTypeSingleton, cls).__call__() + return cls._instances[cls] + + +class PrimitiveType(DataType): + + """Spark SQL PrimitiveType""" + + __metaclass__ = PrimitiveTypeSingleton + + def __eq__(self, other): + # because they should be the same object + return self is other + + +class NullType(PrimitiveType): + + """Spark SQL NullType + + The data type representing None, used for the types which has not + been inferred. + """ + + +class StringType(PrimitiveType): + + """Spark SQL StringType + + The data type representing string values. + """ + + +class BinaryType(PrimitiveType): + + """Spark SQL BinaryType + + The data type representing bytearray values. + """ + + +class BooleanType(PrimitiveType): + + """Spark SQL BooleanType + + The data type representing bool values. + """ + + +class DateType(PrimitiveType): + + """Spark SQL DateType + + The data type representing datetime.date values. + """ + + +class TimestampType(PrimitiveType): + + """Spark SQL TimestampType + + The data type representing datetime.datetime values. + """ + + +class DecimalType(DataType): + + """Spark SQL DecimalType + + The data type representing decimal.Decimal values. + """ + + def __init__(self, precision=None, scale=None): + self.precision = precision + self.scale = scale + self.hasPrecisionInfo = precision is not None + + def jsonValue(self): + if self.hasPrecisionInfo: + return "decimal(%d,%d)" % (self.precision, self.scale) + else: + return "decimal" + + def __repr__(self): + if self.hasPrecisionInfo: + return "DecimalType(%d,%d)" % (self.precision, self.scale) + else: + return "DecimalType()" + + +class DoubleType(PrimitiveType): + + """Spark SQL DoubleType + + The data type representing float values. + """ + + +class FloatType(PrimitiveType): + + """Spark SQL FloatType + + The data type representing single precision floating-point values. + """ + + +class ByteType(PrimitiveType): + + """Spark SQL ByteType + + The data type representing int values with 1 singed byte. + """ + + +class IntegerType(PrimitiveType): + + """Spark SQL IntegerType + + The data type representing int values. + """ + + +class LongType(PrimitiveType): + + """Spark SQL LongType + + The data type representing long values. If the any value is + beyond the range of [-9223372036854775808, 9223372036854775807], + please use DecimalType. + """ + + +class ShortType(PrimitiveType): + + """Spark SQL ShortType + + The data type representing int values with 2 signed bytes. + """ + + +class ArrayType(DataType): + + """Spark SQL ArrayType + + The data type representing list values. An ArrayType object + comprises two fields, elementType (a DataType) and containsNull (a bool). + The field of elementType is used to specify the type of array elements. + The field of containsNull is used to specify if the array has None values. + + """ + + def __init__(self, elementType, containsNull=True): + """Creates an ArrayType + + :param elementType: the data type of elements. + :param containsNull: indicates whether the list contains None values. + + >>> ArrayType(StringType) == ArrayType(StringType, True) + True + >>> ArrayType(StringType, False) == ArrayType(StringType) + False + """ + self.elementType = elementType + self.containsNull = containsNull + + def __repr__(self): + return "ArrayType(%s,%s)" % (self.elementType, + str(self.containsNull).lower()) + + def jsonValue(self): + return {"type": self.typeName(), + "elementType": self.elementType.jsonValue(), + "containsNull": self.containsNull} + + @classmethod + def fromJson(cls, json): + return ArrayType(_parse_datatype_json_value(json["elementType"]), + json["containsNull"]) + + +class MapType(DataType): + + """Spark SQL MapType + + The data type representing dict values. A MapType object comprises + three fields, keyType (a DataType), valueType (a DataType) and + valueContainsNull (a bool). + + The field of keyType is used to specify the type of keys in the map. + The field of valueType is used to specify the type of values in the map. + The field of valueContainsNull is used to specify if values of this + map has None values. + + For values of a MapType column, keys are not allowed to have None values. + + """ + + def __init__(self, keyType, valueType, valueContainsNull=True): + """Creates a MapType + :param keyType: the data type of keys. + :param valueType: the data type of values. + :param valueContainsNull: indicates whether values contains + null values. + + >>> (MapType(StringType, IntegerType) + ... == MapType(StringType, IntegerType, True)) + True + >>> (MapType(StringType, IntegerType, False) + ... == MapType(StringType, FloatType)) + False + """ + self.keyType = keyType + self.valueType = valueType + self.valueContainsNull = valueContainsNull + + def __repr__(self): + return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, + str(self.valueContainsNull).lower()) + + def jsonValue(self): + return {"type": self.typeName(), + "keyType": self.keyType.jsonValue(), + "valueType": self.valueType.jsonValue(), + "valueContainsNull": self.valueContainsNull} + + @classmethod + def fromJson(cls, json): + return MapType(_parse_datatype_json_value(json["keyType"]), + _parse_datatype_json_value(json["valueType"]), + json["valueContainsNull"]) + + +class StructField(DataType): + + """Spark SQL StructField + + Represents a field in a StructType. + A StructField object comprises three fields, name (a string), + dataType (a DataType) and nullable (a bool). The field of name + is the name of a StructField. The field of dataType specifies + the data type of a StructField. + + The field of nullable specifies if values of a StructField can + contain None values. + + """ + + def __init__(self, name, dataType, nullable=True, metadata=None): + """Creates a StructField + :param name: the name of this field. + :param dataType: the data type of this field. + :param nullable: indicates whether values of this field + can be null. + :param metadata: metadata of this field, which is a map from string + to simple type that can be serialized to JSON + automatically + + >>> (StructField("f1", StringType, True) + ... == StructField("f1", StringType, True)) + True + >>> (StructField("f1", StringType, True) + ... == StructField("f2", StringType, True)) + False + """ + self.name = name + self.dataType = dataType + self.nullable = nullable + self.metadata = metadata or {} + + def __repr__(self): + return "StructField(%s,%s,%s)" % (self.name, self.dataType, + str(self.nullable).lower()) + + def jsonValue(self): + return {"name": self.name, + "type": self.dataType.jsonValue(), + "nullable": self.nullable, + "metadata": self.metadata} + + @classmethod + def fromJson(cls, json): + return StructField(json["name"], + _parse_datatype_json_value(json["type"]), + json["nullable"], + json["metadata"]) + + +class StructType(DataType): + + """Spark SQL StructType + + The data type representing rows. + A StructType object comprises a list of L{StructField}. + + """ + + def __init__(self, fields): + """Creates a StructType + + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True)]) + >>> struct1 == struct2 + True + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True), + ... [StructField("f2", IntegerType, False)]]) + >>> struct1 == struct2 + False + """ + self.fields = fields + + def __repr__(self): + return ("StructType(List(%s))" % + ",".join(str(field) for field in self.fields)) + + def jsonValue(self): + return {"type": self.typeName(), + "fields": [f.jsonValue() for f in self.fields]} + + @classmethod + def fromJson(cls, json): + return StructType([StructField.fromJson(f) for f in json["fields"]]) + + +class UserDefinedType(DataType): + """ + .. note:: WARN: Spark Internal Use Only + SQL User-Defined Type (UDT). + """ + + @classmethod + def typeName(cls): + return cls.__name__.lower() + + @classmethod + def sqlType(cls): + """ + Underlying SQL storage type for this UDT. + """ + raise NotImplementedError("UDT must implement sqlType().") + + @classmethod + def module(cls): + """ + The Python module of the UDT. + """ + raise NotImplementedError("UDT must implement module().") + + @classmethod + def scalaUDT(cls): + """ + The class name of the paired Scala UDT. + """ + raise NotImplementedError("UDT must have a paired Scala UDT.") + + def serialize(self, obj): + """ + Converts the a user-type object into a SQL datum. + """ + raise NotImplementedError("UDT must implement serialize().") + + def deserialize(self, datum): + """ + Converts a SQL datum into a user-type object. + """ + raise NotImplementedError("UDT must implement deserialize().") + + def json(self): + return json.dumps(self.jsonValue(), separators=(',', ':'), sort_keys=True) + + def jsonValue(self): + schema = { + "type": "udt", + "class": self.scalaUDT(), + "pyClass": "%s.%s" % (self.module(), type(self).__name__), + "sqlType": self.sqlType().jsonValue() + } + return schema + + @classmethod + def fromJson(cls, json): + pyUDT = json["pyClass"] + split = pyUDT.rfind(".") + pyModule = pyUDT[:split] + pyClass = pyUDT[split+1:] + m = __import__(pyModule, globals(), locals(), [pyClass], -1) + UDT = getattr(m, pyClass) + return UDT() + + def __eq__(self, other): + return type(self) == type(other) + + +_all_primitive_types = dict((v.typeName(), v) + for v in globals().itervalues() + if type(v) is PrimitiveTypeSingleton and + v.__base__ == PrimitiveType) + + +_all_complex_types = dict((v.typeName(), v) + for v in [ArrayType, MapType, StructType]) + + +def _parse_datatype_json_string(json_string): + """Parses the given data type JSON string. + >>> def check_datatype(datatype): + ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) + ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) + ... return datatype == python_datatype + >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) + True + >>> # Simple ArrayType. + >>> simple_arraytype = ArrayType(StringType(), True) + >>> check_datatype(simple_arraytype) + True + >>> # Simple MapType. + >>> simple_maptype = MapType(StringType(), LongType()) + >>> check_datatype(simple_maptype) + True + >>> # Simple StructType. + >>> simple_structtype = StructType([ + ... StructField("a", DecimalType(), False), + ... StructField("b", BooleanType(), True), + ... StructField("c", LongType(), True), + ... StructField("d", BinaryType(), False)]) + >>> check_datatype(simple_structtype) + True + >>> # Complex StructType. + >>> complex_structtype = StructType([ + ... StructField("simpleArray", simple_arraytype, True), + ... StructField("simpleMap", simple_maptype, True), + ... StructField("simpleStruct", simple_structtype, True), + ... StructField("boolean", BooleanType(), False), + ... StructField("withMeta", DoubleType(), False, {"name": "age"})]) + >>> check_datatype(complex_structtype) + True + >>> # Complex ArrayType. + >>> complex_arraytype = ArrayType(complex_structtype, True) + >>> check_datatype(complex_arraytype) + True + >>> # Complex MapType. + >>> complex_maptype = MapType(complex_structtype, + ... complex_arraytype, False) + >>> check_datatype(complex_maptype) + True + >>> check_datatype(ExamplePointUDT()) + True + >>> structtype_with_udt = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> check_datatype(structtype_with_udt) + True + """ + return _parse_datatype_json_value(json.loads(json_string)) + + +_FIXED_DECIMAL = re.compile("decimal\\((\\d+),(\\d+)\\)") + + +def _parse_datatype_json_value(json_value): + if type(json_value) is unicode: + if json_value in _all_primitive_types.keys(): + return _all_primitive_types[json_value]() + elif json_value == u'decimal': + return DecimalType() + elif _FIXED_DECIMAL.match(json_value): + m = _FIXED_DECIMAL.match(json_value) + return DecimalType(int(m.group(1)), int(m.group(2))) + else: + raise ValueError("Could not parse datatype: %s" % json_value) + else: + tpe = json_value["type"] + if tpe in _all_complex_types: + return _all_complex_types[tpe].fromJson(json_value) + elif tpe == 'udt': + return UserDefinedType.fromJson(json_value) + else: + raise ValueError("not supported type: %s" % tpe) + + +# Mapping Python types to Spark SQL DataType +_type_mappings = { + type(None): NullType, + bool: BooleanType, + int: IntegerType, + long: LongType, + float: DoubleType, + str: StringType, + unicode: StringType, + bytearray: BinaryType, + decimal.Decimal: DecimalType, + datetime.date: DateType, + datetime.datetime: TimestampType, + datetime.time: TimestampType, +} + + +def _infer_type(obj): + """Infer the DataType from obj + + >>> p = ExamplePoint(1.0, 2.0) + >>> _infer_type(p) + ExamplePointUDT + """ + if obj is None: + raise ValueError("Can not infer type for None") + + if hasattr(obj, '__UDT__'): + return obj.__UDT__ + + dataType = _type_mappings.get(type(obj)) + if dataType is not None: + return dataType() + + if isinstance(obj, dict): + for key, value in obj.iteritems(): + if key is not None and value is not None: + return MapType(_infer_type(key), _infer_type(value), True) + else: + return MapType(NullType(), NullType(), True) + elif isinstance(obj, (list, array)): + for v in obj: + if v is not None: + return ArrayType(_infer_type(obj[0]), True) + else: + return ArrayType(NullType(), True) + else: + try: + return _infer_schema(obj) + except ValueError: + raise ValueError("not supported type: %s" % type(obj)) + + +def _infer_schema(row): + """Infer the schema from dict/namedtuple/object""" + if isinstance(row, dict): + items = sorted(row.items()) + + elif isinstance(row, tuple): + if hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) + elif hasattr(row, "__FIELDS__"): # Row + items = zip(row.__FIELDS__, tuple(row)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in row): + items = row + else: + raise ValueError("Can't infer schema from tuple") + + elif hasattr(row, "__dict__"): # object + items = sorted(row.__dict__.items()) + + else: + raise ValueError("Can not infer schema for type: %s" % type(row)) + + fields = [StructField(k, _infer_type(v), True) for k, v in items] + return StructType(fields) + + +def _need_python_to_sql_conversion(dataType): + """ + Checks whether we need python to sql conversion for the given type. + For now, only UDTs need this conversion. + + >>> _need_python_to_sql_conversion(DoubleType()) + False + >>> schema0 = StructType([StructField("indices", ArrayType(IntegerType(), False), False), + ... StructField("values", ArrayType(DoubleType(), False), False)]) + >>> _need_python_to_sql_conversion(schema0) + False + >>> _need_python_to_sql_conversion(ExamplePointUDT()) + True + >>> schema1 = ArrayType(ExamplePointUDT(), False) + >>> _need_python_to_sql_conversion(schema1) + True + >>> schema2 = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> _need_python_to_sql_conversion(schema2) + True + """ + if isinstance(dataType, StructType): + return any([_need_python_to_sql_conversion(f.dataType) for f in dataType.fields]) + elif isinstance(dataType, ArrayType): + return _need_python_to_sql_conversion(dataType.elementType) + elif isinstance(dataType, MapType): + return _need_python_to_sql_conversion(dataType.keyType) or \ + _need_python_to_sql_conversion(dataType.valueType) + elif isinstance(dataType, UserDefinedType): + return True + else: + return False + + +def _python_to_sql_converter(dataType): + """ + Returns a converter that converts a Python object into a SQL datum for the given type. + + >>> conv = _python_to_sql_converter(DoubleType()) + >>> conv(1.0) + 1.0 + >>> conv = _python_to_sql_converter(ArrayType(DoubleType(), False)) + >>> conv([1.0, 2.0]) + [1.0, 2.0] + >>> conv = _python_to_sql_converter(ExamplePointUDT()) + >>> conv(ExamplePoint(1.0, 2.0)) + [1.0, 2.0] + >>> schema = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> conv = _python_to_sql_converter(schema) + >>> conv((1.0, ExamplePoint(1.0, 2.0))) + (1.0, [1.0, 2.0]) + """ + if not _need_python_to_sql_conversion(dataType): + return lambda x: x + + if isinstance(dataType, StructType): + names, types = zip(*[(f.name, f.dataType) for f in dataType.fields]) + converters = map(_python_to_sql_converter, types) + + def converter(obj): + if isinstance(obj, dict): + return tuple(c(obj.get(n)) for n, c in zip(names, converters)) + elif isinstance(obj, tuple): + if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): + return tuple(c(v) for c, v in zip(converters, obj)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs + d = dict(obj) + return tuple(c(d.get(n)) for n, c in zip(names, converters)) + else: + return tuple(c(v) for c, v in zip(converters, obj)) + else: + raise ValueError("Unexpected tuple %r with type %r" % (obj, dataType)) + return converter + elif isinstance(dataType, ArrayType): + element_converter = _python_to_sql_converter(dataType.elementType) + return lambda a: [element_converter(v) for v in a] + elif isinstance(dataType, MapType): + key_converter = _python_to_sql_converter(dataType.keyType) + value_converter = _python_to_sql_converter(dataType.valueType) + return lambda m: dict([(key_converter(k), value_converter(v)) for k, v in m.items()]) + elif isinstance(dataType, UserDefinedType): + return lambda obj: dataType.serialize(obj) + else: + raise ValueError("Unexpected type %r" % dataType) + + +def _has_nulltype(dt): + """ Return whether there is NullType in `dt` or not """ + if isinstance(dt, StructType): + return any(_has_nulltype(f.dataType) for f in dt.fields) + elif isinstance(dt, ArrayType): + return _has_nulltype((dt.elementType)) + elif isinstance(dt, MapType): + return _has_nulltype(dt.keyType) or _has_nulltype(dt.valueType) + else: + return isinstance(dt, NullType) + + +def _merge_type(a, b): + if isinstance(a, NullType): + return b + elif isinstance(b, NullType): + return a + elif type(a) is not type(b): + # TODO: type cast (such as int -> long) + raise TypeError("Can not merge type %s and %s" % (a, b)) + + # same type + if isinstance(a, StructType): + nfs = dict((f.name, f.dataType) for f in b.fields) + fields = [StructField(f.name, _merge_type(f.dataType, nfs.get(f.name, NullType()))) + for f in a.fields] + names = set([f.name for f in fields]) + for n in nfs: + if n not in names: + fields.append(StructField(n, nfs[n])) + return StructType(fields) + + elif isinstance(a, ArrayType): + return ArrayType(_merge_type(a.elementType, b.elementType), True) + + elif isinstance(a, MapType): + return MapType(_merge_type(a.keyType, b.keyType), + _merge_type(a.valueType, b.valueType), + True) + else: + return a + + +def _create_converter(dataType): + """Create an converter to drop the names of fields in obj """ + if isinstance(dataType, ArrayType): + conv = _create_converter(dataType.elementType) + return lambda row: map(conv, row) + + elif isinstance(dataType, MapType): + kconv = _create_converter(dataType.keyType) + vconv = _create_converter(dataType.valueType) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) + + elif isinstance(dataType, NullType): + return lambda x: None + + elif not isinstance(dataType, StructType): + return lambda x: x + + # dataType must be StructType + names = [f.name for f in dataType.fields] + converters = [_create_converter(f.dataType) for f in dataType.fields] + + def convert_struct(obj): + if obj is None: + return + + if isinstance(obj, tuple): + if hasattr(obj, "_fields"): + d = dict(zip(obj._fields, obj)) + elif hasattr(obj, "__FIELDS__"): + d = dict(zip(obj.__FIELDS__, obj)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): + d = dict(obj) + else: + raise ValueError("unexpected tuple: %s" % str(obj)) + + elif isinstance(obj, dict): + d = obj + elif hasattr(obj, "__dict__"): # object + d = obj.__dict__ + else: + raise ValueError("Unexpected obj: %s" % obj) + + return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) + + return convert_struct + + +_BRACKETS = {'(': ')', '[': ']', '{': '}'} + + +def _split_schema_abstract(s): + """ + split the schema abstract into fields + + >>> _split_schema_abstract("a b c") + ['a', 'b', 'c'] + >>> _split_schema_abstract("a(a b)") + ['a(a b)'] + >>> _split_schema_abstract("a b[] c{a b}") + ['a', 'b[]', 'c{a b}'] + >>> _split_schema_abstract(" ") + [] + """ + + r = [] + w = '' + brackets = [] + for c in s: + if c == ' ' and not brackets: + if w: + r.append(w) + w = '' + else: + w += c + if c in _BRACKETS: + brackets.append(c) + elif c in _BRACKETS.values(): + if not brackets or c != _BRACKETS[brackets.pop()]: + raise ValueError("unexpected " + c) + + if brackets: + raise ValueError("brackets not closed: %s" % brackets) + if w: + r.append(w) + return r + + +def _parse_field_abstract(s): + """ + Parse a field in schema abstract + + >>> _parse_field_abstract("a") + StructField(a,None,true) + >>> _parse_field_abstract("b(c d)") + StructField(b,StructType(...c,None,true),StructField(d... + >>> _parse_field_abstract("a[]") + StructField(a,ArrayType(None,true),true) + >>> _parse_field_abstract("a{[]}") + StructField(a,MapType(None,ArrayType(None,true),true),true) + """ + if set(_BRACKETS.keys()) & set(s): + idx = min((s.index(c) for c in _BRACKETS if c in s)) + name = s[:idx] + return StructField(name, _parse_schema_abstract(s[idx:]), True) + else: + return StructField(s, None, True) + + +def _parse_schema_abstract(s): + """ + parse abstract into schema + + >>> _parse_schema_abstract("a b c") + StructType...a...b...c... + >>> _parse_schema_abstract("a[b c] b{}") + StructType...a,ArrayType...b...c...b,MapType... + >>> _parse_schema_abstract("c{} d{a b}") + StructType...c,MapType...d,MapType...a...b... + >>> _parse_schema_abstract("a b(t)").fields[1] + StructField(b,StructType(List(StructField(t,None,true))),true) + """ + s = s.strip() + if not s: + return + + elif s.startswith('('): + return _parse_schema_abstract(s[1:-1]) + + elif s.startswith('['): + return ArrayType(_parse_schema_abstract(s[1:-1]), True) + + elif s.startswith('{'): + return MapType(None, _parse_schema_abstract(s[1:-1])) + + parts = _split_schema_abstract(s) + fields = [_parse_field_abstract(p) for p in parts] + return StructType(fields) + + +def _infer_schema_type(obj, dataType): + """ + Fill the dataType with types inferred from obj + + >>> schema = _parse_schema_abstract("a b c d") + >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10)) + >>> _infer_schema_type(row, schema) + StructType...IntegerType...DoubleType...StringType...DateType... + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> _infer_schema_type(row, schema) + StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType... + """ + if dataType is None: + return _infer_type(obj) + + if not obj: + return NullType() + + if isinstance(dataType, ArrayType): + eType = _infer_schema_type(obj[0], dataType.elementType) + return ArrayType(eType, True) + + elif isinstance(dataType, MapType): + k, v = obj.iteritems().next() + return MapType(_infer_schema_type(k, dataType.keyType), + _infer_schema_type(v, dataType.valueType)) + + elif isinstance(dataType, StructType): + fs = dataType.fields + assert len(fs) == len(obj), \ + "Obj(%s) have different length with fields(%s)" % (obj, fs) + fields = [StructField(f.name, _infer_schema_type(o, f.dataType), True) + for o, f in zip(obj, fs)] + return StructType(fields) + + else: + raise ValueError("Unexpected dataType: %s" % dataType) + + +_acceptable_types = { + BooleanType: (bool,), + ByteType: (int, long), + ShortType: (int, long), + IntegerType: (int, long), + LongType: (int, long), + FloatType: (float,), + DoubleType: (float,), + DecimalType: (decimal.Decimal,), + StringType: (str, unicode), + BinaryType: (bytearray,), + DateType: (datetime.date,), + TimestampType: (datetime.datetime,), + ArrayType: (list, tuple, array), + MapType: (dict,), + StructType: (tuple, list), +} + + +def _verify_type(obj, dataType): + """ + Verify the type of obj against dataType, raise an exception if + they do not match. + + >>> _verify_type(None, StructType([])) + >>> _verify_type("", StringType()) + >>> _verify_type(0, IntegerType()) + >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + TypeError:... + >>> _verify_type({}, MapType(StringType(), IntegerType())) + >>> _verify_type((), StructType([])) + >>> _verify_type([], StructType([])) + >>> _verify_type([1], StructType([])) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... + >>> _verify_type(ExamplePoint(1.0, 2.0), ExamplePointUDT()) + >>> _verify_type([1.0, 2.0], ExamplePointUDT()) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... + """ + # all objects are nullable + if obj is None: + return + + if isinstance(dataType, UserDefinedType): + if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): + raise ValueError("%r is not an instance of type %r" % (obj, dataType)) + _verify_type(dataType.serialize(obj), dataType.sqlType()) + return + + _type = type(dataType) + assert _type in _acceptable_types, "unkown datatype: %s" % dataType + + # subclass of them can not be deserialized in JVM + if type(obj) not in _acceptable_types[_type]: + raise TypeError("%s can not accept object in type %s" + % (dataType, type(obj))) + + if isinstance(dataType, ArrayType): + for i in obj: + _verify_type(i, dataType.elementType) + + elif isinstance(dataType, MapType): + for k, v in obj.iteritems(): + _verify_type(k, dataType.keyType) + _verify_type(v, dataType.valueType) + + elif isinstance(dataType, StructType): + if len(obj) != len(dataType.fields): + raise ValueError("Length of object (%d) does not match with" + "length of fields (%d)" % (len(obj), len(dataType.fields))) + for v, f in zip(obj, dataType.fields): + _verify_type(v, f.dataType) + + +_cached_cls = {} + + +def _restore_object(dataType, obj): + """ Restore object during unpickling. """ + # use id(dataType) as key to speed up lookup in dict + # Because of batched pickling, dataType will be the + # same object in most cases. + k = id(dataType) + cls = _cached_cls.get(k) + if cls is None: + # use dataType as key to avoid create multiple class + cls = _cached_cls.get(dataType) + if cls is None: + cls = _create_cls(dataType) + _cached_cls[dataType] = cls + _cached_cls[k] = cls + return cls(obj) + + +def _create_object(cls, v): + """ Create an customized object with class `cls`. """ + # datetime.date would be deserialized as datetime.datetime + # from java type, so we need to set it back. + if cls is datetime.date and isinstance(v, datetime.datetime): + return v.date() + return cls(v) if v is not None else v + + +def _create_getter(dt, i): + """ Create a getter for item `i` with schema """ + cls = _create_cls(dt) + + def getter(self): + return _create_object(cls, self[i]) + + return getter + + +def _has_struct_or_date(dt): + """Return whether `dt` is or has StructType/DateType in it""" + if isinstance(dt, StructType): + return True + elif isinstance(dt, ArrayType): + return _has_struct_or_date(dt.elementType) + elif isinstance(dt, MapType): + return _has_struct_or_date(dt.keyType) or _has_struct_or_date(dt.valueType) + elif isinstance(dt, DateType): + return True + elif isinstance(dt, UserDefinedType): + return True + return False + + +def _create_properties(fields): + """Create properties according to fields""" + ps = {} + for i, f in enumerate(fields): + name = f.name + if (name.startswith("__") and name.endswith("__") + or keyword.iskeyword(name)): + warnings.warn("field name %s can not be accessed in Python," + "use position to access it instead" % name) + if _has_struct_or_date(f.dataType): + # delay creating object until accessing it + getter = _create_getter(f.dataType, i) + else: + getter = itemgetter(i) + ps[name] = property(getter) + return ps + + +def _create_cls(dataType): + """ + Create an class by dataType + + The created class is similar to namedtuple, but can have nested schema. + + >>> schema = _parse_schema_abstract("a b c") + >>> row = (1, 1.0, "str") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> import pickle + >>> pickle.loads(pickle.dumps(obj)) + Row(a=1, b=1.0, c='str') + + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> pickle.loads(pickle.dumps(obj)) + Row(a=[1], b={'key': Row(c=1, d=2.0)}) + >>> pickle.loads(pickle.dumps(obj.a)) + [1] + >>> pickle.loads(pickle.dumps(obj.b)) + {'key': Row(c=1, d=2.0)} + """ + + if isinstance(dataType, ArrayType): + cls = _create_cls(dataType.elementType) + + def List(l): + if l is None: + return + return [_create_object(cls, v) for v in l] + + return List + + elif isinstance(dataType, MapType): + kcls = _create_cls(dataType.keyType) + vcls = _create_cls(dataType.valueType) + + def Dict(d): + if d is None: + return + return dict((_create_object(kcls, k), _create_object(vcls, v)) for k, v in d.items()) + + return Dict + + elif isinstance(dataType, DateType): + return datetime.date + + elif isinstance(dataType, UserDefinedType): + return lambda datum: dataType.deserialize(datum) + + elif not isinstance(dataType, StructType): + # no wrapper for primitive types + return lambda x: x + + class Row(tuple): + + """ Row in DataFrame """ + __DATATYPE__ = dataType + __FIELDS__ = tuple(f.name for f in dataType.fields) + __slots__ = () + + # create property for fast access + locals().update(_create_properties(dataType.fields)) + + def asDict(self): + """ Return as a dict """ + return dict((n, getattr(self, n)) for n in self.__FIELDS__) + + def __repr__(self): + # call collect __repr__ for nested objects + return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) + for n in self.__FIELDS__)) + + def __reduce__(self): + return (_restore_object, (self.__DATATYPE__, tuple(self))) + + return Row + + +def _create_row(fields, values): + row = Row(*values) + row.__FIELDS__ = fields + return row + + +class Row(tuple): + + """ + A row in L{DataFrame}. The fields in it can be accessed like attributes. + + Row can be used to create a row object by using named arguments, + the fields will be sorted by names. + + >>> row = Row(name="Alice", age=11) + >>> row + Row(age=11, name='Alice') + >>> row.name, row.age + ('Alice', 11) + + Row also can be used to create another Row like class, then it + could be used to create Row objects, such as + + >>> Person = Row("name", "age") + >>> Person + + >>> Person("Alice", 11) + Row(name='Alice', age=11) + """ + + def __new__(self, *args, **kwargs): + if args and kwargs: + raise ValueError("Can not use both args " + "and kwargs to create Row") + if args: + # create row class or objects + return tuple.__new__(self, args) + + elif kwargs: + # create row objects + names = sorted(kwargs.keys()) + values = tuple(kwargs[n] for n in names) + row = tuple.__new__(self, values) + row.__FIELDS__ = names + return row + + else: + raise ValueError("No args or kwargs") + + def asDict(self): + """ + Return as an dict + """ + if not hasattr(self, "__FIELDS__"): + raise TypeError("Cannot convert a Row class into dict") + return dict(zip(self.__FIELDS__, self)) + + # let obect acs like class + def __call__(self, *args): + """create new Row object""" + return _create_row(self, args) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + try: + # it will be slow when it has many fields, + # but this will not be used in normal cases + idx = self.__FIELDS__.index(item) + return self[idx] + except IndexError: + raise AttributeError(item) + + def __reduce__(self): + if hasattr(self, "__FIELDS__"): + return (_create_row, (self.__FIELDS__, tuple(self))) + else: + return tuple.__reduce__(self) + + def __repr__(self): + if hasattr(self, "__FIELDS__"): + return "Row(%s)" % ", ".join("%s=%r" % (k, v) + for k, v in zip(self.__FIELDS__, self)) + else: + return "" % ", ".join(self) + + +def _test(): + import doctest + from pyspark.context import SparkContext + # let doctest run in pyspark.sql.types, so DataTypes can be picklable + import pyspark.sql.types + from pyspark.sql import Row, SQLContext + from pyspark.sql.tests import ExamplePoint, ExamplePointUDT + globs = pyspark.sql.types.__dict__.copy() + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['ExamplePoint'] = ExamplePoint + globs['ExamplePointUDT'] = ExamplePointUDT + (failure_count, test_count) = doctest.testmod( + pyspark.sql.types, globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index 649a2c44d187b..58a26dd8ff088 100755 --- a/python/run-tests +++ b/python/run-tests @@ -64,8 +64,10 @@ function run_core_tests() { function run_sql_tests() { echo "Run sql tests ..." - run_test "pyspark/sql.py" - run_test "pyspark/sql_tests.py" + run_test "pyspark/sql/types.py" + run_test "pyspark/sql/context.py" + run_test "pyspark/sql/dataframe.py" + run_test "pyspark/sql/tests.py" } function run_mllib_tests() { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index e6f622e87f7a4..eb045e37bf5a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -37,7 +37,7 @@ private[sql] class ExamplePointUDT extends UserDefinedType[ExamplePoint] { override def sqlType: DataType = ArrayType(DoubleType, false) - override def pyUDT: String = "pyspark.sql_tests.ExamplePointUDT" + override def pyUDT: String = "pyspark.sql.tests.ExamplePointUDT" override def serialize(obj: Any): Seq[Double] = { obj match { From 31d435ecfdc24a788a6e38f4e82767bc275a3283 Mon Sep 17 00:00:00 2001 From: KaiXinXiaoLei Date: Mon, 9 Feb 2015 20:58:58 -0800 Subject: [PATCH 027/817] Add a config option to print DAG. Add a config option "spark.rddDebug.enable" to check whether to print DAG info. When "spark.rddDebug.enable" is true, it will print information about DAG in the log. Author: KaiXinXiaoLei Closes #4257 from KaiXinXiaoLei/DAGprint and squashes the following commits: d9fe42e [KaiXinXiaoLei] change log info c27ee76 [KaiXinXiaoLei] change log info 83c2b32 [KaiXinXiaoLei] change config option adcb14f [KaiXinXiaoLei] change the file. f4e7b9e [KaiXinXiaoLei] add a option to print DAG --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71bdbc9b38ddb..8d3c3d000adf3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1420,6 +1420,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite.shortForm) + if (conf.getBoolean("spark.logLineage", false)) { + logInfo("RDD's recursive dependencies:\n" + rdd.toDebugString) + } dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) progressBar.foreach(_.finishAll()) From 36c4e1d75933dc843acb747b91dc12e75ad1df42 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 9 Feb 2015 21:13:58 -0800 Subject: [PATCH 028/817] SPARK-4900 [MLLIB] MLlib SingularValueDecomposition ARPACK IllegalStateException Fix ARPACK error code mapping, at least. It's not yet clear whether the error is what we expect from ARPACK. If it isn't, not sure if that's to be treated as an MLlib or Breeze issue. Author: Sean Owen Closes #4485 from srowen/SPARK-4900 and squashes the following commits: 7355aa1 [Sean Owen] Fix ARPACK error code mapping --- .../org/apache/spark/mllib/linalg/EigenValueDecomposition.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala index 9d6f97528148e..866936aa4f118 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala @@ -117,7 +117,7 @@ private[mllib] object EigenValueDecomposition { info.`val` match { case 1 => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + " Maximum number of iterations taken. (Refer ARPACK user guide for details)") - case 2 => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + + case 3 => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + " No shifts could be applied. Try to increase NCV. " + "(Refer ARPACK user guide for details)") case _ => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + From 20a6013106b56a1a1cc3e8cda092330ffbe77cc3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 9 Feb 2015 21:17:06 -0800 Subject: [PATCH 029/817] [SPARK-2996] Implement userClassPathFirst for driver, yarn. Yarn's config option `spark.yarn.user.classpath.first` does not work the same way as `spark.files.userClassPathFirst`; Yarn's version is a lot more dangerous, in that it modifies the system classpath, instead of restricting the changes to the user's class loader. So this change implements the behavior of the latter for Yarn, and deprecates the more dangerous choice. To be able to achieve feature-parity, I also implemented the option for drivers (the existing option only applies to executors). So now there are two options, each controlling whether to apply userClassPathFirst to the driver or executors. The old option was deprecated, and aliased to the new one (`spark.executor.userClassPathFirst`). The existing "child-first" class loader also had to be fixed. It didn't handle resources, and it was also doing some things that ended up causing JVM errors depending on how things were being called. Author: Marcelo Vanzin Closes #3233 from vanzin/SPARK-2996 and squashes the following commits: 9cf9cf1 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a1499e2 [Marcelo Vanzin] Remove SPARK_HOME propagation. fa7df88 [Marcelo Vanzin] Remove 'test.resource' file, create it dynamically. a8c69f1 [Marcelo Vanzin] Review feedback. cabf962 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a1b8d7e [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 3f768e3 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 2ce3c7a [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 0e6d6be [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 70d4044 [Marcelo Vanzin] Fix pyspark/yarn-cluster test. 0fe7777 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 0e6ef19 [Marcelo Vanzin] Move class loaders around and make names more meaninful. fe970a7 [Marcelo Vanzin] Review feedback. 25d4fed [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 3cb6498 [Marcelo Vanzin] Call the right loadClass() method on the parent. fbb8ab5 [Marcelo Vanzin] Add locking in loadClass() to avoid deadlocks. 2e6c4b7 [Marcelo Vanzin] Mention new setting in documentation. b6497f9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 a10f379 [Marcelo Vanzin] Some feedback. 3730151 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 f513871 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 44010b6 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 7b57cba [Marcelo Vanzin] Remove now outdated message. 5304d64 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 35949c8 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 54e1a98 [Marcelo Vanzin] Merge branch 'master' into SPARK-2996 d1273b2 [Marcelo Vanzin] Add test file to rat exclude. fa1aafa [Marcelo Vanzin] Remove write check on user jars. 89d8072 [Marcelo Vanzin] Cleanups. a963ea3 [Marcelo Vanzin] Implement spark.driver.userClassPathFirst for standalone cluster mode. 50afa5f [Marcelo Vanzin] Fix Yarn executor command line. 7d14397 [Marcelo Vanzin] Register user jars in executor up front. 7f8603c [Marcelo Vanzin] Fix yarn-cluster mode without userClassPathFirst. 20373f5 [Marcelo Vanzin] Fix ClientBaseSuite. 55c88fa [Marcelo Vanzin] Run all Yarn integration tests via spark-submit. 0b64d92 [Marcelo Vanzin] Add deprecation warning to yarn option. 4a84d87 [Marcelo Vanzin] Fix the child-first class loader. d0394b8 [Marcelo Vanzin] Add "deprecated configs" to SparkConf. 46d8cf2 [Marcelo Vanzin] Update doc with new option, change name to "userClassPathFirst". a314f2d [Marcelo Vanzin] Enable driver class path isolation in SparkSubmit. 91f7e54 [Marcelo Vanzin] [yarn] Enable executor class path isolation. a853e74 [Marcelo Vanzin] Re-work CoarseGrainedExecutorBackend command line arguments. 89522ef [Marcelo Vanzin] Add class path isolation support for Yarn cluster mode. --- .../scala/org/apache/spark/SparkConf.scala | 83 +++++- .../scala/org/apache/spark/TestUtils.scala | 19 +- .../org/apache/spark/deploy/Client.scala | 5 +- .../org/apache/spark/deploy/SparkSubmit.scala | 8 +- .../spark/deploy/master/ui/MasterPage.scala | 2 +- .../deploy/rest/StandaloneRestServer.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 15 +- .../spark/deploy/worker/DriverWrapper.scala | 20 +- .../CoarseGrainedExecutorBackend.scala | 83 +++++- .../org/apache/spark/executor/Executor.scala | 52 ++-- .../executor/ExecutorURLClassLoader.scala | 84 ------ .../cluster/SparkDeploySchedulerBackend.scala | 9 +- .../mesos/CoarseMesosSchedulerBackend.scala | 21 +- .../spark/util/MutableURLClassLoader.scala | 103 +++++++ .../apache/spark/util/ParentClassLoader.scala | 7 +- .../org/apache/spark/SparkConfSuite.scala | 12 + .../spark/deploy/SparkSubmitSuite.scala | 27 ++ .../MutableURLClassLoaderSuite.scala} | 12 +- docs/configuration.md | 31 +- pom.xml | 12 +- project/SparkBuild.scala | 8 +- .../spark/deploy/yarn/ApplicationMaster.scala | 25 +- .../org/apache/spark/deploy/yarn/Client.scala | 133 ++++----- .../spark/deploy/yarn/ExecutorRunnable.scala | 25 +- yarn/src/test/resources/log4j.properties | 4 +- .../spark/deploy/yarn/ClientSuite.scala | 6 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 276 ++++++++++++------ 27 files changed, 736 insertions(+), 348 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala create mode 100644 core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala rename core/src/test/scala/org/apache/spark/{executor/ExecutorURLClassLoaderSuite.scala => util/MutableURLClassLoaderSuite.scala} (90%) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 13aa9960ac33a..0dbd26146cb13 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ import scala.collection.mutable.LinkedHashSet @@ -67,7 +68,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (value == null) { throw new NullPointerException("null value for " + key) } - settings.put(key, value) + settings.put(translateConfKey(key, warn = true), value) this } @@ -139,7 +140,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { - settings.putIfAbsent(key, value) + settings.putIfAbsent(translateConfKey(key, warn = true), value) this } @@ -175,7 +176,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { - Option(settings.get(key)) + Option(settings.get(translateConfKey(key))) } /** Get all parameters as a list of pairs */ @@ -228,7 +229,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { def getAppId: String = get("spark.app.id") /** Does the configuration contain a given parameter? */ - def contains(key: String): Boolean = settings.containsKey(key) + def contains(key: String): Boolean = settings.containsKey(translateConfKey(key)) /** Copy this object */ override def clone: SparkConf = { @@ -285,7 +286,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Validate memory fractions val memoryKeys = Seq( "spark.storage.memoryFraction", - "spark.shuffle.memoryFraction", + "spark.shuffle.memoryFraction", "spark.shuffle.safetyFraction", "spark.storage.unrollFraction", "spark.storage.safetyFraction") @@ -351,9 +352,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { def toDebugString: String = { getAll.sorted.map{case (k, v) => k + "=" + v}.mkString("\n") } + } -private[spark] object SparkConf { +private[spark] object SparkConf extends Logging { + + private val deprecatedConfigs: Map[String, DeprecatedConfig] = { + val configs = Seq( + DeprecatedConfig("spark.files.userClassPathFirst", "spark.executor.userClassPathFirst", + "1.3"), + DeprecatedConfig("spark.yarn.user.classpath.first", null, "1.3", + "Use spark.{driver,executor}.userClassPathFirst instead.")) + configs.map { x => (x.oldName, x) }.toMap + } + /** * Return whether the given config is an akka config (e.g. akka.actor.provider). * Note that this does not include spark-specific akka configs (e.g. spark.akka.timeout). @@ -380,4 +392,63 @@ private[spark] object SparkConf { def isSparkPortConf(name: String): Boolean = { (name.startsWith("spark.") && name.endsWith(".port")) || name.startsWith("spark.port.") } + + /** + * Translate the configuration key if it is deprecated and has a replacement, otherwise just + * returns the provided key. + * + * @param userKey Configuration key from the user / caller. + * @param warn Whether to print a warning if the key is deprecated. Warnings will be printed + * only once for each key. + */ + def translateConfKey(userKey: String, warn: Boolean = false): String = { + deprecatedConfigs.get(userKey) + .map { deprecatedKey => + if (warn) { + deprecatedKey.warn() + } + deprecatedKey.newName.getOrElse(userKey) + }.getOrElse(userKey) + } + + /** + * Holds information about keys that have been deprecated or renamed. + * + * @param oldName Old configuration key. + * @param newName New configuration key, or `null` if key has no replacement, in which case the + * deprecated key will be used (but the warning message will still be printed). + * @param version Version of Spark where key was deprecated. + * @param deprecationMessage Message to include in the deprecation warning; mandatory when + * `newName` is not provided. + */ + private case class DeprecatedConfig( + oldName: String, + _newName: String, + version: String, + deprecationMessage: String = null) { + + private val warned = new AtomicBoolean(false) + val newName = Option(_newName) + + if (newName == null && (deprecationMessage == null || deprecationMessage.isEmpty())) { + throw new IllegalArgumentException("Need new config name or deprecation message.") + } + + def warn(): Unit = { + if (warned.compareAndSet(false, true)) { + if (newName != null) { + val message = Option(deprecationMessage).getOrElse( + s"Please use the alternative '$newName' instead.") + logWarning( + s"The configuration option '$oldName' has been replaced as of Spark $version and " + + s"may be removed in the future. $message") + } else { + logWarning( + s"The configuration option '$oldName' has been deprecated as of Spark $version and " + + s"may be removed in the future. $deprecationMessage") + } + } + } + + } } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index be081c3825566..35b324ba6f573 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -17,12 +17,13 @@ package org.apache.spark -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} import java.net.{URI, URL} import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConversions._ +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.{ByteStreams, Files} import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} @@ -59,6 +60,22 @@ private[spark] object TestUtils { createJar(files1 ++ files2, jarFile) } + /** + * Create a jar file containing multiple files. The `files` map contains a mapping of + * file names in the jar file to their contents. + */ + def createJarWithFiles(files: Map[String, String], dir: File = null): URL = { + val tempDir = Option(dir).getOrElse(Utils.createTempDir()) + val jarFile = File.createTempFile("testJar", ".jar", tempDir) + val jarStream = new JarOutputStream(new FileOutputStream(jarFile)) + files.foreach { case (k, v) => + val entry = new JarEntry(k) + jarStream.putNextEntry(entry) + ByteStreams.copy(new ByteArrayInputStream(v.getBytes(UTF_8)), jarStream) + } + jarStream.close() + jarFile.toURI.toURL + } /** * Create a jar file that contains this set of files. All files will be located at the root diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 38b3da0b13756..237d26fc6bd0e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -68,8 +68,9 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) .map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) val javaOpts = sparkJavaOpts ++ extraJavaOpts - val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, sys.env, classPathEntries, libraryPathEntries, javaOpts) + val command = new Command(mainClass, + Seq("{{WORKER_URL}}", "{{USER_JAR}}", driverArgs.mainClass) ++ driverArgs.driverOptions, + sys.env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 6d213926f3d7b..c4bc5054d61a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -37,7 +37,7 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver} import org.apache.spark.deploy.rest._ import org.apache.spark.executor._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} /** * Whether to submit, kill, or request the status of an application. @@ -467,11 +467,11 @@ object SparkSubmit { } val loader = - if (sysProps.getOrElse("spark.files.userClassPathFirst", "false").toBoolean) { - new ChildExecutorURLClassLoader(new Array[URL](0), + if (sysProps.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { + new ChildFirstURLClassLoader(new Array[URL](0), Thread.currentThread.getContextClassLoader) } else { - new ExecutorURLClassLoader(new Array[URL](0), + new MutableURLClassLoader(new Array[URL](0), Thread.currentThread.getContextClassLoader) } Thread.currentThread.setContextClassLoader(loader) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index b47a081053e77..fd514f07664a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -196,7 +196,7 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {Utils.megabytesToString(driver.desc.mem.toLong)} - {driver.desc.command.arguments(1)} + {driver.desc.command.arguments(2)} } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 2033d67e1f394..6e4486e20fcba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -392,7 +392,7 @@ private class SubmitRequestServlet( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.worker.DriverWrapper", - Seq("{{WORKER_URL}}", mainClass) ++ appArgs, // args to the DriverWrapper + Seq("{{WORKER_URL}}", "{{USER_JAR}}", mainClass) ++ appArgs, // args to the DriverWrapper environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) val actualDriverCores = driverCores.map(_.toInt).getOrElse(DEFAULT_CORES) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 28cab36c7b9e2..b964a09bdb218 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -74,10 +74,15 @@ private[spark] class DriverRunner( val driverDir = createWorkingDirectory() val localJarFilename = downloadUserJar(driverDir) - // Make sure user application jar is on the classpath + def substituteVariables(argument: String): String = argument match { + case "{{WORKER_URL}}" => workerUrl + case "{{USER_JAR}}" => localJarFilename + case other => other + } + // TODO: If we add ability to submit multiple jars they should also be added here val builder = CommandUtils.buildProcessBuilder(driverDesc.command, driverDesc.mem, - sparkHome.getAbsolutePath, substituteVariables, Seq(localJarFilename)) + sparkHome.getAbsolutePath, substituteVariables) launchDriver(builder, driverDir, driverDesc.supervise) } catch { @@ -111,12 +116,6 @@ private[spark] class DriverRunner( } } - /** Replace variables in a command argument passed to us */ - private def substituteVariables(argument: String): String = argument match { - case "{{WORKER_URL}}" => workerUrl - case other => other - } - /** * Creates the working directory for this driver. * Will throw an exception if there are errors preparing the directory. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 05e242e6df702..ab467a5ee8c6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -17,10 +17,12 @@ package org.apache.spark.deploy.worker +import java.io.File + import akka.actor._ import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, Utils} /** * Utility object for launching driver programs such that they share fate with the Worker process. @@ -28,21 +30,31 @@ import org.apache.spark.util.{AkkaUtils, Utils} object DriverWrapper { def main(args: Array[String]) { args.toList match { - case workerUrl :: mainClass :: extraArgs => + case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") + val currentLoader = Thread.currentThread.getContextClassLoader + val userJarUrl = new File(userJar).toURI().toURL() + val loader = + if (sys.props.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { + new ChildFirstURLClassLoader(Array(userJarUrl), currentLoader) + } else { + new MutableURLClassLoader(Array(userJarUrl), currentLoader) + } + Thread.currentThread.setContextClassLoader(loader) + // Delegate to supplied main class - val clazz = Class.forName(args(1)) + val clazz = Class.forName(mainClass, true, loader) val mainMethod = clazz.getMethod("main", classOf[Array[String]]) mainMethod.invoke(null, extraArgs.toArray[String]) actorSystem.shutdown() case _ => - System.err.println("Usage: DriverWrapper [options]") + System.err.println("Usage: DriverWrapper [options]") System.exit(-1) } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3a42f8b157977..dd19e4947db1e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,8 +17,10 @@ package org.apache.spark.executor +import java.net.URL import java.nio.ByteBuffer +import scala.collection.mutable import scala.concurrent.Await import akka.actor.{Actor, ActorSelection, Props} @@ -38,6 +40,7 @@ private[spark] class CoarseGrainedExecutorBackend( executorId: String, hostPort: String, cores: Int, + userClassPath: Seq[URL], env: SparkEnv) extends Actor with ActorLogReceive with ExecutorBackend with Logging { @@ -63,7 +66,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) - executor = new Executor(executorId, hostname, env, isLocal = false) + executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -117,7 +120,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { hostname: String, cores: Int, appId: String, - workerUrl: Option[String]) { + workerUrl: Option[String], + userClassPath: Seq[URL]) { SignalLogger.register(log) @@ -162,7 +166,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val sparkHostPort = hostname + ":" + boundPort env.actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, env), + driverUrl, executorId, sparkHostPort, cores, userClassPath, env), name = "Executor") workerUrl.foreach { url => env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") @@ -172,20 +176,69 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } def main(args: Array[String]) { - args.length match { - case x if x < 5 => - System.err.println( + var driverUrl: String = null + var executorId: String = null + var hostname: String = null + var cores: Int = 0 + var appId: String = null + var workerUrl: Option[String] = None + val userClassPath = new mutable.ListBuffer[URL]() + + var argv = args.toList + while (!argv.isEmpty) { + argv match { + case ("--driver-url") :: value :: tail => + driverUrl = value + argv = tail + case ("--executor-id") :: value :: tail => + executorId = value + argv = tail + case ("--hostname") :: value :: tail => + hostname = value + argv = tail + case ("--cores") :: value :: tail => + cores = value.toInt + argv = tail + case ("--app-id") :: value :: tail => + appId = value + argv = tail + case ("--worker-url") :: value :: tail => // Worker url is used in spark standalone mode to enforce fate-sharing with worker - "Usage: CoarseGrainedExecutorBackend " + - " [] ") - System.exit(1) + workerUrl = Some(value) + argv = tail + case ("--user-class-path") :: value :: tail => + userClassPath += new URL(value) + argv = tail + case Nil => + case tail => + System.err.println(s"Unrecognized options: ${tail.mkString(" ")}") + printUsageAndExit() + } + } - // NB: These arguments are provided by SparkDeploySchedulerBackend (for standalone mode) - // and CoarseMesosSchedulerBackend (for mesos mode). - case 5 => - run(args(0), args(1), args(2), args(3).toInt, args(4), None) - case x if x > 5 => - run(args(0), args(1), args(2), args(3).toInt, args(4), Some(args(5))) + if (driverUrl == null || executorId == null || hostname == null || cores <= 0 || + appId == null) { + printUsageAndExit() } + + run(driverUrl, executorId, hostname, cores, appId, workerUrl, userClassPath) } + + private def printUsageAndExit() = { + System.err.println( + """ + |"Usage: CoarseGrainedExecutorBackend [options] + | + | Options are: + | --driver-url + | --executor-id + | --hostname + | --cores + | --app-id + | --worker-url + | --user-class-path + |""".stripMargin) + System.exit(1) + } + } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5141483d1e745..6b22dcd6f5cbf 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -19,6 +19,7 @@ package org.apache.spark.executor import java.io.File import java.lang.management.ManagementFactory +import java.net.URL import java.nio.ByteBuffer import java.util.concurrent._ @@ -33,7 +34,8 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils} +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, + SparkUncaughtExceptionHandler, AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. @@ -43,6 +45,7 @@ private[spark] class Executor( executorId: String, executorHostname: String, env: SparkEnv, + userClassPath: Seq[URL] = Nil, isLocal: Boolean = false) extends Logging { @@ -288,17 +291,23 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): MutableURLClassLoader = { + // Bootstrap the list of jars with the user class path. + val now = System.currentTimeMillis() + userClassPath.foreach { url => + currentJars(url.getPath().split("/").last) = now + } + val currentLoader = Utils.getContextOrSparkClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. - val urls = currentJars.keySet.map { uri => + val urls = userClassPath.toArray ++ currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL - }.toArray - val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) - userClassPathFirst match { - case true => new ChildExecutorURLClassLoader(urls, currentLoader) - case false => new ExecutorURLClassLoader(urls, currentLoader) + } + if (conf.getBoolean("spark.executor.userClassPathFirst", false)) { + new ChildFirstURLClassLoader(urls, currentLoader) + } else { + new MutableURLClassLoader(urls, currentLoader) } } @@ -311,7 +320,7 @@ private[spark] class Executor( if (classUri != null) { logInfo("Using REPL class URI: " + classUri) val userClassPathFirst: java.lang.Boolean = - conf.getBoolean("spark.files.userClassPathFirst", false) + conf.getBoolean("spark.executor.userClassPathFirst", false) try { val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] @@ -344,18 +353,23 @@ private[spark] class Executor( env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentFiles(name) = timestamp } - for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { - logInfo("Fetching " + name + " with timestamp " + timestamp) - // Fetch file with useCache mode, close cache for local mode. - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, - env.securityManager, hadoopConf, timestamp, useCache = !isLocal) - currentJars(name) = timestamp - // Add it to our class loader + for ((name, timestamp) <- newJars) { val localName = name.split("/").last - val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL - if (!urlClassLoader.getURLs.contains(url)) { - logInfo("Adding " + url + " to class loader") - urlClassLoader.addURL(url) + val currentTimeStamp = currentJars.get(name) + .orElse(currentJars.get(localName)) + .getOrElse(-1L) + if (currentTimeStamp < timestamp) { + logInfo("Fetching " + name + " with timestamp " + timestamp) + // Fetch file with useCache mode, close cache for local mode. + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + env.securityManager, hadoopConf, timestamp, useCache = !isLocal) + currentJars(name) = timestamp + // Add it to our class loader + val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL + if (!urlClassLoader.getURLs.contains(url)) { + logInfo("Adding " + url + " to class loader") + urlClassLoader.addURL(url) + } } } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala deleted file mode 100644 index 8011e75944aac..0000000000000 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ /dev/null @@ -1,84 +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.executor - -import java.net.{URLClassLoader, URL} - -import org.apache.spark.util.ParentClassLoader - -/** - * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. - * We also make changes so user classes can come before the default classes. - */ - -private[spark] trait MutableURLClassLoader extends ClassLoader { - def addURL(url: URL) - def getURLs: Array[URL] -} - -private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends MutableURLClassLoader { - - private object userClassLoader extends URLClassLoader(urls, null){ - override def addURL(url: URL) { - super.addURL(url) - } - override def findClass(name: String): Class[_] = { - val loaded = super.findLoadedClass(name) - if (loaded != null) { - return loaded - } - try { - super.findClass(name) - } catch { - case e: ClassNotFoundException => { - parentClassLoader.loadClass(name) - } - } - } - } - - private val parentClassLoader = new ParentClassLoader(parent) - - override def findClass(name: String): Class[_] = { - try { - userClassLoader.findClass(name) - } catch { - case e: ClassNotFoundException => { - parentClassLoader.loadClass(name) - } - } - } - - def addURL(url: URL) { - userClassLoader.addURL(url) - } - - def getURLs() = { - userClassLoader.getURLs() - } -} - -private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) with MutableURLClassLoader { - - override def addURL(url: URL) { - super.addURL(url) - } -} - diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index d2e1680a5fd1b..40fc6b59cdf7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -52,8 +52,13 @@ private[spark] class SparkDeploySchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{APP_ID}}", - "{{WORKER_URL}}") + val args = Seq( + "--driver-url", driverUrl, + "--executor-id", "{{EXECUTOR_ID}}", + "--hostname", "{{HOSTNAME}}", + "--cores", "{{CORES}}", + "--app-id", "{{APP_ID}}", + "--worker-url", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") .map(Utils.splitCommandString).getOrElse(Seq.empty) val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 0d1c2a916ca7f..90dfe14352a8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -154,18 +154,25 @@ private[spark] class CoarseMesosSchedulerBackend( if (uri == null) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( - prefixEnv, runScript, driverUrl, offer.getSlaveId.getValue, - offer.getHostname, numCores, appId)) + "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" + .format(prefixEnv, runScript) + + s" --driver-url $driverUrl" + + s" --executor-id ${offer.getSlaveId.getValue}" + + s" --hostname ${offer.getHostname}" + + s" --cores $numCores" + + s" --app-id $appId") } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - ("cd %s*; %s " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s") - .format(basename, prefixEnv, driverUrl, offer.getSlaveId.getValue, - offer.getHostname, numCores, appId)) + s"cd $basename*; $prefixEnv " + + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + + s" --driver-url $driverUrl" + + s" --executor-id ${offer.getSlaveId.getValue}" + + s" --hostname ${offer.getHostname}" + + s" --cores $numCores" + + s" --app-id $appId") command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala new file mode 100644 index 0000000000000..d9c7103b2f3bf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.net.{URLClassLoader, URL} +import java.util.Enumeration +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConversions._ + +import org.apache.spark.util.ParentClassLoader + +/** + * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. + */ +private[spark] class MutableURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends URLClassLoader(urls, parent) { + + override def addURL(url: URL): Unit = { + super.addURL(url) + } + + override def getURLs(): Array[URL] = { + super.getURLs() + } + +} + +/** + * A mutable class loader that gives preference to its own URLs over the parent class loader + * when loading classes and resources. + */ +private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader(urls, null) { + + private val parentClassLoader = new ParentClassLoader(parent) + + /** + * Used to implement fine-grained class loading locks similar to what is done by Java 7. This + * prevents deadlock issues when using non-hierarchical class loaders. + * + * Note that due to Java 6 compatibility (and some issues with implementing class loaders in + * Scala), Java 7's `ClassLoader.registerAsParallelCapable` method is not called. + */ + private val locks = new ConcurrentHashMap[String, Object]() + + override def loadClass(name: String, resolve: Boolean): Class[_] = { + var lock = locks.get(name) + if (lock == null) { + val newLock = new Object() + lock = locks.putIfAbsent(name, newLock) + if (lock == null) { + lock = newLock + } + } + + lock.synchronized { + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) + } + } + } + + override def getResource(name: String): URL = { + val url = super.findResource(name) + val res = if (url != null) url else parentClassLoader.getResource(name) + res + } + + override def getResources(name: String): Enumeration[URL] = { + val urls = super.findResources(name) + val res = + if (urls != null && urls.hasMoreElements()) { + urls + } else { + parentClassLoader.getResources(name) + } + res + } + + override def addURL(url: URL) { + super.addURL(url) + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 3abc12681fe9a..6d8d9e8da3678 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * A class loader which makes findClass accesible to the child + * A class loader which makes some protected methods in ClassLoader accesible. */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { @@ -29,4 +29,9 @@ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader( override def loadClass(name: String): Class[_] = { super.loadClass(name) } + + override def loadClass(name: String, resolve: Boolean): Class[_] = { + super.loadClass(name, resolve) + } + } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index e08210ae60d17..ea6b73bc68b34 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -197,6 +197,18 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro serializer.newInstance().serialize(new StringBuffer()) } + test("deprecated config keys") { + val conf = new SparkConf() + .set("spark.files.userClassPathFirst", "true") + .set("spark.yarn.user.classpath.first", "true") + assert(conf.contains("spark.files.userClassPathFirst")) + assert(conf.contains("spark.executor.userClassPathFirst")) + assert(conf.contains("spark.yarn.user.classpath.first")) + assert(conf.getBoolean("spark.files.userClassPathFirst", false)) + assert(conf.getBoolean("spark.executor.userClassPathFirst", false)) + assert(conf.getBoolean("spark.yarn.user.classpath.first", false)) + } + } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 1ddccae1262bc..46d745c4ecbfa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -21,6 +21,8 @@ import java.io._ import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.ByteStreams import org.scalatest.FunSuite import org.scalatest.Matchers import org.scalatest.concurrent.Timeouts @@ -450,6 +452,19 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) } + test("user classpath first in driver") { + val systemJar = TestUtils.createJarWithFiles(Map("test.resource" -> "SYSTEM")) + val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER")) + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + "--conf", "spark.driver.extraClassPath=" + systemJar, + "--conf", "spark.driver.userClassPathFirst=true", + userJar.toString) + runSparkSubmit(args) + } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => val unusedJar = TestUtils.createJarWithClasses(Seq.empty) @@ -541,3 +556,15 @@ object SimpleApplicationTest { } } } + +object UserClasspathFirstTest { + def main(args: Array[String]) { + val ccl = Thread.currentThread().getContextClassLoader() + val resource = ccl.getResourceAsStream("test.resource") + val bytes = ByteStreams.toByteArray(resource) + val contents = new String(bytes, 0, bytes.length, UTF_8) + if (contents != "USER") { + throw new SparkException("Should have read user resource, but instead read: " + contents) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala similarity index 90% rename from core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala rename to core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index b7912c09d1410..31e3b7e7bb71b 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.executor +package org.apache.spark.util import java.net.URLClassLoader @@ -24,7 +24,7 @@ import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} import org.apache.spark.util.Utils -class ExecutorURLClassLoaderSuite extends FunSuite { +class MutableURLClassLoaderSuite extends FunSuite { val urls2 = List(TestUtils.createJarWithClasses( classNames = Seq("FakeClass1", "FakeClass2", "FakeClass3"), @@ -37,7 +37,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite { test("child first") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass2").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") @@ -47,7 +47,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite { test("parent first") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val classLoader = new MutableURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass1").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") @@ -57,7 +57,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite { test("child first can fall back") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) val fakeClass = classLoader.loadClass("FakeClass3").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") @@ -65,7 +65,7 @@ class ExecutorURLClassLoaderSuite extends FunSuite { test("child first can fail") { val parentLoader = new URLClassLoader(urls2, null) - val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val classLoader = new ChildFirstURLClassLoader(urls, parentLoader) intercept[java.lang.ClassNotFoundException] { classLoader.loadClass("FakeClassDoesNotExist").newInstance() } diff --git a/docs/configuration.md b/docs/configuration.md index 00e973c245005..eb0d6d33c97d9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -230,6 +230,15 @@ Apart from these, the following properties are also available, and may be useful Set a special library path to use when launching the driver JVM. + + spark.driver.userClassPathFirst + false + + (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading + classes in the the driver. This feature can be used to mitigate conflicts between Spark's + dependencies and user dependencies. It is currently an experimental feature. + + spark.executor.extraJavaOptions (none) @@ -297,13 +306,11 @@ Apart from these, the following properties are also available, and may be useful - spark.files.userClassPathFirst + spark.executor.userClassPathFirst false - (Experimental) Whether to give user-added jars precedence over Spark's own jars when - loading classes in Executors. This feature can be used to mitigate conflicts between - Spark's dependencies and user dependencies. It is currently an experimental feature. - (Currently, this setting does not work for YARN, see SPARK-2996 for more details). + (Experimental) Same functionality as spark.driver.userClassPathFirst, but + applied to executor instances. @@ -865,8 +872,8 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout 120 - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + Default timeout for all network interactions, in seconds. This config will be used in + place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -911,8 +918,8 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.preferDirectBufs true - (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache - block transfer. For environments where off-heap memory is tightly limited, users may wish to + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to turn this off to force all allocations from Netty to be on-heap. @@ -920,7 +927,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.numConnectionsPerPeer 1 - (Netty only) Connections between hosts are reused in order to reduce connection buildup for + (Netty only) Connections between hosts are reused in order to reduce connection buildup for large clusters. For clusters with many hard disks and few hosts, this may result in insufficient concurrency to saturate all disks, and so users may consider increasing this value. @@ -930,7 +937,7 @@ Apart from these, the following properties are also available, and may be useful 3 (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is - set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC + set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC pauses or transient network connectivity issues. @@ -939,7 +946,7 @@ Apart from these, the following properties are also available, and may be useful 5 (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + is simply maxRetries * retryWait, by default 15 seconds. diff --git a/pom.xml b/pom.xml index f6f176d2004b7..a9e968af25453 100644 --- a/pom.xml +++ b/pom.xml @@ -342,7 +342,7 @@ - + @@ -395,7 +395,7 @@ provided - + org.apache.commons commons-lang3 @@ -1178,13 +1178,19 @@ ${project.build.directory}/surefire-reports -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + + + ${test_classpath} + true ${session.executionRootDirectory} 1 false false - ${test_classpath} true false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 95f8dfa3d270f..8fb1239b4a96b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -411,6 +411,10 @@ object TestSettings { lazy val settings = Seq ( // Fork new JVMs for tests and set Java options for those fork := true, + // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes + // launched by the tests have access to the correct test-time classpath. + envVars in Test += ("SPARK_DIST_CLASSPATH" -> + (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":")), javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", @@ -423,10 +427,6 @@ object TestSettings { javaOptions in Test += "-ea", javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" .split(" ").toSeq, - // This places test scope jars on the classpath of executors during tests. - javaOptions in Test += - "-Dspark.executor.extraClassPath=" + (fullClasspath in Test).value.files. - map(_.getAbsolutePath).mkString(":").stripSuffix(":"), javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), 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 4cc320c5d59b5..a9bf861d160c1 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 @@ -19,9 +19,9 @@ package org.apache.spark.deploy.yarn import scala.util.control.NonFatal -import java.io.IOException +import java.io.{File, IOException} import java.lang.reflect.InvocationTargetException -import java.net.Socket +import java.net.{Socket, URL} import java.util.concurrent.atomic.AtomicReference import akka.actor._ @@ -38,7 +38,8 @@ import org.apache.spark.deploy.{PythonRunner, SparkHadoopUtil} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.YarnSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, + SignalLogger, Utils} /** * Common application master functionality for Spark on Yarn. @@ -244,7 +245,6 @@ private[spark] class ApplicationMaster( host: String, port: String, isClusterMode: Boolean): Unit = { - val driverUrl = AkkaUtils.address( AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, @@ -453,12 +453,24 @@ private[spark] class ApplicationMaster( private def startUserApplication(): Thread = { logInfo("Starting the user application in a separate Thread") System.setProperty("spark.executor.instances", args.numExecutors.toString) + + val classpath = Client.getUserClasspath(sparkConf) + val urls = classpath.map { entry => + new URL("file:" + new File(entry.getPath()).getAbsolutePath()) + } + val userClassLoader = + if (Client.isUserClassPathFirst(sparkConf, isDriver = true)) { + new ChildFirstURLClassLoader(urls, Utils.getContextOrSparkClassLoader) + } else { + new MutableURLClassLoader(urls, Utils.getContextOrSparkClassLoader) + } + if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { System.setProperty("spark.submit.pyFiles", PythonRunner.formatPaths(args.pyFiles).mkString(",")) } - val mainMethod = Class.forName(args.userClass, false, - Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) + val mainMethod = userClassLoader.loadClass(args.userClass) + .getMethod("main", classOf[Array[String]]) val userThread = new Thread { override def run() { @@ -483,6 +495,7 @@ private[spark] class ApplicationMaster( } } } + userThread.setContextClassLoader(userClassLoader) userThread.setName("Driver") userThread.start() userThread diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8afc1ccdad732..46d9df93488cb 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -183,8 +183,7 @@ private[spark] class Client( private[yarn] def copyFileToRemote( destDir: Path, srcPath: Path, - replication: Short, - setPerms: Boolean = false): Path = { + replication: Short): Path = { val destFs = destDir.getFileSystem(hadoopConf) val srcFs = srcPath.getFileSystem(hadoopConf) var destPath = srcPath @@ -193,9 +192,7 @@ private[spark] class Client( logInfo(s"Uploading resource $srcPath -> $destPath") FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) destFs.setReplication(destPath, replication) - if (setPerms) { - destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) - } + destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) } else { logInfo(s"Source and destination file systems are the same. Not copying $srcPath") } @@ -239,23 +236,22 @@ private[spark] class Client( /** * Copy the given main resource to the distributed cache if the scheme is not "local". * Otherwise, set the corresponding key in our SparkConf to handle it downstream. - * Each resource is represented by a 4-tuple of: + * Each resource is represented by a 3-tuple of: * (1) destination resource name, * (2) local path to the resource, - * (3) Spark property key to set if the scheme is not local, and - * (4) whether to set permissions for this resource + * (3) Spark property key to set if the scheme is not local */ List( - (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), - (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), - ("log4j.properties", oldLog4jConf.orNull, null, false) - ).foreach { case (destName, _localPath, confKey, setPermissions) => + (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR), + (APP_JAR, args.userJar, CONF_SPARK_USER_JAR), + ("log4j.properties", oldLog4jConf.orNull, null) + ).foreach { case (destName, _localPath, confKey) => val localPath: String = if (_localPath != null) _localPath.trim() else "" if (!localPath.isEmpty()) { val localURI = new URI(localPath) if (localURI.getScheme != LOCAL_SCHEME) { val src = getQualifiedLocalPath(localURI, hadoopConf) - val destPath = copyFileToRemote(dst, src, replication, setPermissions) + val destPath = copyFileToRemote(dst, src, replication) val destFs = FileSystem.get(destPath.toUri(), hadoopConf) distCacheMgr.addResource(destFs, hadoopConf, destPath, localResources, LocalResourceType.FILE, destName, statCache) @@ -707,7 +703,7 @@ object Client extends Logging { * Return the path to the given application's staging directory. */ private def getAppStagingDir(appId: ApplicationId): String = { - SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + buildPath(SPARK_STAGING, appId.toString()) } /** @@ -783,7 +779,13 @@ object Client extends Logging { /** * Populate the classpath entry in the given environment map. - * This includes the user jar, Spark jar, and any extra application jars. + * + * User jars are generally not added to the JVM's system classpath; those are handled by the AM + * and executor backend. When the deprecated `spark.yarn.user.classpath.first` is used, user jars + * are included in the system classpath, though. The extra class path and other uploaded files are + * always made available through the system class path. + * + * @param args Client arguments (when starting the AM) or null (when starting executors). */ private[yarn] def populateClasspath( args: ClientArguments, @@ -795,48 +797,38 @@ object Client extends Logging { addClasspathEntry( YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env ) - - // Normally the users app.jar is last in case conflicts with spark jars if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { - addUserClasspath(args, sparkConf, env) - addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - populateHadoopClasspath(conf, env) - } else { - addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - populateHadoopClasspath(conf, env) - addUserClasspath(args, sparkConf, env) + val userClassPath = + if (args != null) { + getUserClasspath(Option(args.userJar), Option(args.addJars)) + } else { + getUserClasspath(sparkConf) + } + userClassPath.foreach { x => + addFileToClasspath(x, null, env) + } } - - // Append all jar files under the working directory to the classpath. - addClasspathEntry( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + "*", env - ) + addFileToClasspath(new URI(sparkJar(sparkConf)), SPARK_JAR, env) + populateHadoopClasspath(conf, env) + sys.env.get(ENV_DIST_CLASSPATH).foreach(addClasspathEntry(_, env)) } /** - * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly - * to the classpath. + * Returns a list of URIs representing the user classpath. + * + * @param conf Spark configuration. */ - private def addUserClasspath( - args: ClientArguments, - conf: SparkConf, - env: HashMap[String, String]): Unit = { - - // If `args` is not null, we are launching an AM container. - // Otherwise, we are launching executor containers. - val (mainJar, secondaryJars) = - if (args != null) { - (args.userJar, args.addJars) - } else { - (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) - } + def getUserClasspath(conf: SparkConf): Array[URI] = { + getUserClasspath(conf.getOption(CONF_SPARK_USER_JAR), + conf.getOption(CONF_SPARK_YARN_SECONDARY_JARS)) + } - addFileToClasspath(mainJar, APP_JAR, env) - if (secondaryJars != null) { - secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => - addFileToClasspath(jar, null, env) - } - } + private def getUserClasspath( + mainJar: Option[String], + secondaryJars: Option[String]): Array[URI] = { + val mainUri = mainJar.orElse(Some(APP_JAR)).map(new URI(_)) + val secondaryUris = secondaryJars.map(_.split(",")).toSeq.flatten.map(new URI(_)) + (mainUri ++ secondaryUris).toArray } /** @@ -847,27 +839,19 @@ object Client extends Logging { * * If not a "local:" file and no alternate name, the environment is not modified. * - * @param path Path to add to classpath (optional). + * @param uri URI to add to classpath (optional). * @param fileName Alternate name for the file (optional). * @param env Map holding the environment variables. */ private def addFileToClasspath( - path: String, + uri: URI, fileName: String, env: HashMap[String, String]): Unit = { - if (path != null) { - scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val uri = new URI(path) - if (uri.getScheme == LOCAL_SCHEME) { - addClasspathEntry(uri.getPath, env) - return - } - } - } - if (fileName != null) { - addClasspathEntry( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + fileName, env - ) + if (uri != null && uri.getScheme == LOCAL_SCHEME) { + addClasspathEntry(uri.getPath, env) + } else if (fileName != null) { + addClasspathEntry(buildPath( + YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), fileName), env) } } @@ -963,4 +947,23 @@ object Client extends Logging { new Path(qualifiedURI) } + /** + * Whether to consider jars provided by the user to have precedence over the Spark jars when + * loading user classes. + */ + def isUserClassPathFirst(conf: SparkConf, isDriver: Boolean): Boolean = { + if (isDriver) { + conf.getBoolean("spark.driver.userClassPathFirst", false) + } else { + conf.getBoolean("spark.executor.userClassPathFirst", false) + } + } + + /** + * Joins all the path components using Path.SEPARATOR. + */ + def buildPath(components: String*): String = { + components.mkString(Path.SEPARATOR) + } + } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 7cd8c5f0f9204..6d5b8fda76ab8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.yarn +import java.io.File import java.net.URI import java.nio.ByteBuffer @@ -57,7 +58,7 @@ class ExecutorRunnable( var nmClient: NMClient = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) lazy val env = prepareEnvironment(container) - + def run = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() @@ -185,6 +186,16 @@ class ExecutorRunnable( // For log4j configuration to reference javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri => + val absPath = + if (new File(uri.getPath()).isAbsolute()) { + uri.getPath() + } else { + Client.buildPath(Environment.PWD.$(), uri.getPath()) + } + Seq("--user-class-path", "file:" + absPath) + }.toSeq + val commands = prefixEnv ++ Seq( YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server", @@ -196,11 +207,13 @@ class ExecutorRunnable( "-XX:OnOutOfMemoryError='kill %p'") ++ javaOpts ++ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", - masterAddress.toString, - slaveId.toString, - hostname.toString, - executorCores.toString, - appId, + "--driver-url", masterAddress.toString, + "--executor-id", slaveId.toString, + "--hostname", hostname.toString, + "--cores", executorCores.toString, + "--app-id", appId) ++ + userClassPath ++ + Seq( "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index 287c8e3563503..aab41fa49430f 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -16,7 +16,7 @@ # # Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file +log4j.rootCategory=DEBUG, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=true log4j.appender.file.file=target/unit-tests.log @@ -25,4 +25,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{ # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.apache.hadoop=WARN diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 2bb3dcffd61d9..f8f8129d220e4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -82,6 +82,7 @@ class ClientSuite extends FunSuite with Matchers { test("Local jar URIs") { val conf = new Configuration() val sparkConf = new SparkConf().set(Client.CONF_SPARK_JAR, SPARK) + .set("spark.yarn.user.classpath.first", "true") val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) @@ -98,13 +99,10 @@ class ClientSuite extends FunSuite with Matchers { }) if (classOf[Environment].getMethods().exists(_.getName == "$$")) { cp should contain("{{PWD}}") - cp should contain(s"{{PWD}}${Path.SEPARATOR}*") } else if (Utils.isWindows) { cp should contain("%PWD%") - cp should contain(s"%PWD%${Path.SEPARATOR}*") } else { cp should contain(Environment.PWD.$()) - cp should contain(s"${Environment.PWD.$()}${File.separator}*") } cp should not contain (Client.SPARK_JAR) cp should not contain (Client.APP_JAR) @@ -117,7 +115,7 @@ class ClientSuite extends FunSuite with Matchers { val client = spy(new Client(args, conf, sparkConf)) doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort(), anyBoolean()) + any(classOf[Path]), anyShort()) val tempDir = Utils.createTempDir() try { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index e39de82740b1d..0e37276ba724b 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -17,27 +17,34 @@ package org.apache.spark.deploy.yarn -import java.io.File +import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.util.Properties import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ import scala.collection.mutable -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.ByteStreams import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils +/** + * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN + * applications, and require the Spark assembly to be built before they can be successfully + * run. + */ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging { - // log4j configuration for the Yarn containers, so that their output is collected - // by Yarn instead of trying to overwrite unit-tests.log. + // log4j configuration for the YARN containers, so that their output is collected + // by YARN instead of trying to overwrite unit-tests.log. private val LOG4J_CONF = """ |log4j.rootCategory=DEBUG, console |log4j.appender.console=org.apache.log4j.ConsoleAppender @@ -52,13 +59,11 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit | |from pyspark import SparkConf , SparkContext |if __name__ == "__main__": - | if len(sys.argv) != 3: - | print >> sys.stderr, "Usage: test.py [master] [result file]" + | if len(sys.argv) != 2: + | print >> sys.stderr, "Usage: test.py [result file]" | exit(-1) - | conf = SparkConf() - | conf.setMaster(sys.argv[1]).setAppName("python test in yarn cluster mode") - | sc = SparkContext(conf=conf) - | status = open(sys.argv[2],'w') + | sc = SparkContext(conf=SparkConf()) + | status = open(sys.argv[1],'w') | result = "failure" | rdd = sc.parallelize(range(10)) | cnt = rdd.count() @@ -72,23 +77,17 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ - private var oldConf: Map[String, String] = _ + private var logConfDir: File = _ override def beforeAll() { super.beforeAll() tempDir = Utils.createTempDir() - - val logConfDir = new File(tempDir, "log4j") + logConfDir = new File(tempDir, "log4j") logConfDir.mkdir() val logConfFile = new File(logConfDir, "log4j.properties") - Files.write(LOG4J_CONF, logConfFile, Charsets.UTF_8) - - val childClasspath = logConfDir.getAbsolutePath() + File.pathSeparator + - sys.props("java.class.path") - - oldConf = sys.props.filter { case (k, v) => k.startsWith("spark.") }.toMap + Files.write(LOG4J_CONF, logConfFile, UTF_8) yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) yarnCluster.init(new YarnConfiguration()) @@ -119,99 +118,165 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") - config.foreach { e => - sys.props += ("spark.hadoop." + e.getKey() -> e.getValue()) - } fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) - val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) - sys.props += ("spark.yarn.appMasterEnv.SPARK_HOME" -> sparkHome) - sys.props += ("spark.executorEnv.SPARK_HOME" -> sparkHome) - sys.props += ("spark.yarn.jar" -> ("local:" + fakeSparkJar.getAbsolutePath())) - sys.props += ("spark.executor.instances" -> "1") - sys.props += ("spark.driver.extraClassPath" -> childClasspath) - sys.props += ("spark.executor.extraClassPath" -> childClasspath) - sys.props += ("spark.executor.extraJavaOptions" -> "-Dfoo=\"one two three\"") - sys.props += ("spark.driver.extraJavaOptions" -> "-Dfoo=\"one two three\"") } override def afterAll() { yarnCluster.stop() - sys.props.retain { case (k, v) => !k.startsWith("spark.") } - sys.props ++= oldConf super.afterAll() } test("run Spark in yarn-client mode") { - var result = File.createTempFile("result", null, tempDir) - YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) - checkResult(result) - - // verify log urls are present - YarnClusterDriver.listener.addedExecutorInfos.values.foreach { info => - assert(info.logUrlMap.nonEmpty) - } + testBasicYarnApp(true) } test("run Spark in yarn-cluster mode") { - val main = YarnClusterDriver.getClass.getName().stripSuffix("$") - var result = File.createTempFile("result", null, tempDir) - - val args = Array("--class", main, - "--jar", "file:" + fakeSparkJar.getAbsolutePath(), - "--arg", "yarn-cluster", - "--arg", result.getAbsolutePath(), - "--num-executors", "1") - Client.main(args) - checkResult(result) - - // verify log urls are present. - YarnClusterDriver.listener.addedExecutorInfos.values.foreach { info => - assert(info.logUrlMap.nonEmpty) - } + testBasicYarnApp(false) } test("run Spark in yarn-cluster mode unsuccessfully") { - val main = YarnClusterDriver.getClass.getName().stripSuffix("$") - - // Use only one argument so the driver will fail - val args = Array("--class", main, - "--jar", "file:" + fakeSparkJar.getAbsolutePath(), - "--arg", "yarn-cluster", - "--num-executors", "1") + // Don't provide arguments so the driver will fail. val exception = intercept[SparkException] { - Client.main(args) + runSpark(false, mainClassName(YarnClusterDriver.getClass)) + fail("Spark application should have failed.") } - assert(Utils.exceptionString(exception).contains("Application finished with failed status")) } test("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") - Files.write(TEST_PYFILE, primaryPyFile, Charsets.UTF_8) + Files.write(TEST_PYFILE, primaryPyFile, UTF_8) val pyFile = new File(tempDir, "test2.py") - Files.write(TEST_PYFILE, pyFile, Charsets.UTF_8) + Files.write(TEST_PYFILE, pyFile, UTF_8) var result = File.createTempFile("result", null, tempDir) - val args = Array("--class", "org.apache.spark.deploy.PythonRunner", - "--primary-py-file", primaryPyFile.getAbsolutePath(), - "--py-files", pyFile.getAbsolutePath(), - "--arg", "yarn-cluster", - "--arg", result.getAbsolutePath(), - "--name", "python test in yarn-cluster mode", - "--num-executors", "1") - Client.main(args) + // The sbt assembly does not include pyspark / py4j python dependencies, so we need to + // propagate SPARK_HOME so that those are added to PYTHONPATH. See PythonUtils.scala. + val sparkHome = sys.props("spark.test.home") + val extraConf = Map( + "spark.executorEnv.SPARK_HOME" -> sparkHome, + "spark.yarn.appMasterEnv.SPARK_HOME" -> sparkHome) + + runSpark(false, primaryPyFile.getAbsolutePath(), + sparkArgs = Seq("--py-files", pyFile.getAbsolutePath()), + appArgs = Seq(result.getAbsolutePath()), + extraConf = extraConf) checkResult(result) } + test("user class path first in client mode") { + testUseClassPathFirst(true) + } + + test("user class path first in cluster mode") { + testUseClassPathFirst(false) + } + + private def testBasicYarnApp(clientMode: Boolean): Unit = { + var result = File.createTempFile("result", null, tempDir) + runSpark(clientMode, mainClassName(YarnClusterDriver.getClass), + appArgs = Seq(result.getAbsolutePath())) + checkResult(result) + } + + private def testUseClassPathFirst(clientMode: Boolean): Unit = { + // Create a jar file that contains a different version of "test.resource". + val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir) + val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "OVERRIDDEN"), tempDir) + val driverResult = File.createTempFile("driver", null, tempDir) + val executorResult = File.createTempFile("executor", null, tempDir) + runSpark(clientMode, mainClassName(YarnClasspathTest.getClass), + appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()), + extraClassPath = Seq(originalJar.getPath()), + extraJars = Seq("local:" + userJar.getPath()), + extraConf = Map( + "spark.driver.userClassPathFirst" -> "true", + "spark.executor.userClassPathFirst" -> "true")) + checkResult(driverResult, "OVERRIDDEN") + checkResult(executorResult, "OVERRIDDEN") + } + + private def runSpark( + clientMode: Boolean, + klass: String, + appArgs: Seq[String] = Nil, + sparkArgs: Seq[String] = Nil, + extraClassPath: Seq[String] = Nil, + extraJars: Seq[String] = Nil, + extraConf: Map[String, String] = Map()): Unit = { + val master = if (clientMode) "yarn-client" else "yarn-cluster" + val props = new Properties() + + props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) + + val childClasspath = logConfDir.getAbsolutePath() + + File.pathSeparator + + sys.props("java.class.path") + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator) + props.setProperty("spark.driver.extraClassPath", childClasspath) + props.setProperty("spark.executor.extraClassPath", childClasspath) + + // SPARK-4267: make sure java options are propagated correctly. + props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") + props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") + + yarnCluster.getConfig().foreach { e => + props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) + } + + sys.props.foreach { case (k, v) => + if (k.startsWith("spark.")) { + props.setProperty(k, v) + } + } + + extraConf.foreach { case (k, v) => props.setProperty(k, v) } + + val propsFile = File.createTempFile("spark", ".properties", tempDir) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + props.store(writer, "Spark properties.") + writer.close() + + val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil + val mainArgs = + if (klass.endsWith(".py")) { + Seq(klass) + } else { + Seq("--class", klass, fakeSparkJar.getAbsolutePath()) + } + val argv = + Seq( + new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), + "--master", master, + "--num-executors", "1", + "--properties-file", propsFile.getAbsolutePath()) ++ + extraJarArgs ++ + sparkArgs ++ + mainArgs ++ + appArgs + + Utils.executeAndGetOutput(argv, + extraEnvironment = Map("YARN_CONF_DIR" -> tempDir.getAbsolutePath())) + } + /** * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide * any sort of error when the job process finishes successfully, but the job itself fails. So * the tests enforce that something is written to a file after everything is ok to indicate * that the job succeeded. */ - private def checkResult(result: File) = { - var resultString = Files.toString(result, Charsets.UTF_8) - resultString should be ("success") + private def checkResult(result: File): Unit = { + checkResult(result, "success") + } + + private def checkResult(result: File, expected: String): Unit = { + var resultString = Files.toString(result, UTF_8) + resultString should be (expected) + } + + private def mainClassName(klass: Class[_]): String = { + klass.getName().stripSuffix("$") } } @@ -229,22 +294,22 @@ private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 var listener: SaveExecutorInfo = null - def main(args: Array[String]) = { - if (args.length != 2) { + def main(args: Array[String]): Unit = { + if (args.length != 1) { System.err.println( s""" |Invalid command line: ${args.mkString(" ")} | - |Usage: YarnClusterDriver [master] [result file] + |Usage: YarnClusterDriver [result file] """.stripMargin) System.exit(1) } listener = new SaveExecutorInfo - val sc = new SparkContext(new SparkConf().setMaster(args(0)) + val sc = new SparkContext(new SparkConf() .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) sc.addSparkListener(listener) - val status = new File(args(1)) + val status = new File(args(0)) var result = "failure" try { val data = sc.parallelize(1 to 4, 4).collect().toSet @@ -253,7 +318,48 @@ private object YarnClusterDriver extends Logging with Matchers { result = "success" } finally { sc.stop() - Files.write(result, status, Charsets.UTF_8) + Files.write(result, status, UTF_8) + } + + // verify log urls are present + listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + } + } + +} + +private object YarnClasspathTest { + + def main(args: Array[String]): Unit = { + if (args.length != 2) { + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: YarnClasspathTest [driver result file] [executor result file] + """.stripMargin) + System.exit(1) + } + + readResource(args(0)) + val sc = new SparkContext(new SparkConf()) + try { + sc.parallelize(Seq(1)).foreach { x => readResource(args(1)) } + } finally { + sc.stop() + } + } + + private def readResource(resultPath: String): Unit = { + var result = "failure" + try { + val ccl = Thread.currentThread().getContextClassLoader() + val resource = ccl.getResourceAsStream("test.resource") + val bytes = ByteStreams.toByteArray(resource) + result = new String(bytes, 0, bytes.length, UTF_8) + } finally { + Files.write(result, new File(resultPath), UTF_8) } } From a95ed52157473fb0e42e910ee15270e7f0edf943 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 9 Feb 2015 21:18:48 -0800 Subject: [PATCH 030/817] [SPARK-5703] AllJobsPage throws empty.max exception If you have a `SparkListenerJobEnd` event without the corresponding `SparkListenerJobStart` event, then `JobProgressListener` will create an empty `JobUIData` with an empty `stageIds` list. However, later in `AllJobsPage` we call `stageIds.max`. If this is empty, it will throw an exception. This crashed my history server. Author: Andrew Or Closes #4490 from andrewor14/jobs-page-max and squashes the following commits: 21797d3 [Andrew Or] Check nonEmpty before calling max --- .../src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 045c69da06feb..bd923d78a86ce 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -42,7 +42,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def makeRow(job: JobUIData): Seq[Node] = { - val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageInfo = Option(job.stageIds) + .filter(_.nonEmpty) + .flatMap { ids => listener.stageIdToInfo.get(ids.max) } val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } From a2d33d0b01af87e931d9d883638a52d7a86f6248 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 9 Feb 2015 21:22:09 -0800 Subject: [PATCH 031/817] [SPARK-5701] Only set ShuffleReadMetrics when task has shuffle deps The updateShuffleReadMetrics method in TaskMetrics (called by the executor heartbeater) will currently always add a ShuffleReadMetrics to TaskMetrics (with values set to 0), even when the task didn't read any shuffle data. ShuffleReadMetrics should only be added if the task reads shuffle data. Author: Kay Ousterhout Closes #4488 from kayousterhout/SPARK-5701 and squashes the following commits: 673ed58 [Kay Ousterhout] SPARK-5701: Only set ShuffleReadMetrics when task has shuffle deps --- .../apache/spark/executor/TaskMetrics.scala | 22 ++++++++------- .../spark/executor/TaskMetricsSuite.scala | 28 +++++++++++++++++++ 2 files changed, 40 insertions(+), 10 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index d05659193b334..bf3f1e4fc7832 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -177,8 +177,8 @@ class TaskMetrics extends Serializable { * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, * we can store all the different inputMetrics (one per readMethod). */ - private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): - InputMetrics =synchronized { + private[spark] def getInputMetricsForReadMethod( + readMethod: DataReadMethod): InputMetrics = synchronized { _inputMetrics match { case None => val metrics = new InputMetrics(readMethod) @@ -195,15 +195,17 @@ class TaskMetrics extends Serializable { * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. */ private[spark] def updateShuffleReadMetrics(): Unit = synchronized { - val merged = new ShuffleReadMetrics() - for (depMetrics <- depsShuffleReadMetrics) { - merged.incFetchWaitTime(depMetrics.fetchWaitTime) - merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) - merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) - merged.incRemoteBytesRead(depMetrics.remoteBytesRead) - merged.incRecordsRead(depMetrics.recordsRead) + if (!depsShuffleReadMetrics.isEmpty) { + val merged = new ShuffleReadMetrics() + for (depMetrics <- depsShuffleReadMetrics) { + merged.incFetchWaitTime(depMetrics.fetchWaitTime) + merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) + merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) + merged.incRemoteBytesRead(depMetrics.remoteBytesRead) + merged.incRecordsRead(depMetrics.recordsRead) + } + _shuffleReadMetrics = Some(merged) } - _shuffleReadMetrics = Some(merged) } private[spark] def updateInputMetrics(): Unit = synchronized { diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala new file mode 100644 index 0000000000000..326e203afe136 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.scalatest.FunSuite + +class TaskMetricsSuite extends FunSuite { + test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") { + val taskMetrics = new TaskMetrics() + taskMetrics.updateShuffleReadMetrics() + assert(taskMetrics.shuffleReadMetrics.isEmpty) + } +} From bd0b5ea708aa5b84adb67c039ec52408289718bb Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 9 Feb 2015 21:33:34 -0800 Subject: [PATCH 032/817] [SQL] Remove the duplicated code Author: Cheng Hao Closes #4494 from chenghao-intel/tiny_code_change and squashes the following commits: 450dfe7 [Cheng Hao] remove the duplicated code --- .../org/apache/spark/sql/execution/SparkStrategies.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 81bcf5a6f32dd..edf8a5be64ff1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -342,11 +342,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ExecutedCommand( RunnableDescribeCommand(resultPlan, resultPlan.output, isExtended)) :: Nil - case LogicalDescribeCommand(table, isExtended) => - val resultPlan = self.sqlContext.executePlan(table).executedPlan - ExecutedCommand( - RunnableDescribeCommand(resultPlan, resultPlan.output, isExtended)) :: Nil - case _ => Nil } } From ef2f55b97f58fa06acb30e9e0172fb66fba383bc Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 9 Feb 2015 22:09:07 -0800 Subject: [PATCH 033/817] [SPARK-5597][MLLIB] save/load for decision trees and emsembles This is based on #4444 from jkbradley with the following changes: 1. Node schema updated to ~~~ treeId: int nodeId: Int predict/ |- predict: Double |- prob: Double impurity: Double isLeaf: Boolean split/ |- feature: Int |- threshold: Double |- featureType: Int |- categories: Array[Double] leftNodeId: Integer rightNodeId: Integer infoGain: Double ~~~ 2. Some refactor of the implementation. Closes #4444. Author: Joseph K. Bradley Author: Xiangrui Meng Closes #4493 from mengxr/SPARK-5597 and squashes the following commits: 75e3bb6 [Xiangrui Meng] fix style 2b0033d [Xiangrui Meng] update tree export schema and refactor the implementation 45873a2 [Joseph K. Bradley] org imports 1d4c264 [Joseph K. Bradley] Added save/load for tree ensembles dcdbf85 [Joseph K. Bradley] added save/load for decision tree but need to generalize it to ensembles --- .../mllib/tree/model/DecisionTreeModel.scala | 197 +++++++++++++++++- .../tree/model/InformationGainStats.scala | 4 +- .../apache/spark/mllib/tree/model/Node.scala | 5 + .../spark/mllib/tree/model/Predict.scala | 7 + .../mllib/tree/model/treeEnsembleModels.scala | 157 +++++++++++++- .../spark/mllib/tree/DecisionTreeSuite.scala | 120 ++++++++++- .../tree/GradientBoostedTreesSuite.scala | 81 ++++--- .../spark/mllib/tree/RandomForestSuite.scala | 28 ++- 8 files changed, 561 insertions(+), 38 deletions(-) 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 a25e625a4017a..89ecf3773dd77 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -17,11 +17,17 @@ package org.apache.spark.mllib.tree.model +import scala.collection.mutable + +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLContext} /** * :: Experimental :: @@ -31,7 +37,7 @@ import org.apache.spark.rdd.RDD * @param algo algorithm type -- classification or regression */ @Experimental -class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable { +class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable with Saveable { /** * Predict values for a single data point using the model trained. @@ -98,4 +104,193 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable header + topNode.subtreeToString(2) } + override def save(sc: SparkContext, path: String): Unit = { + DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) + } + + override protected def formatVersion: String = "1.0" +} + +object DecisionTreeModel extends Loader[DecisionTreeModel] { + + private[tree] object SaveLoadV1_0 { + + def thisFormatVersion = "1.0" + + // Hard-code class name string in case it changes in the future + def thisClassName = "org.apache.spark.mllib.tree.DecisionTreeModel" + + case class PredictData(predict: Double, prob: Double) { + def toPredict: Predict = new Predict(predict, prob) + } + + object PredictData { + def apply(p: Predict): PredictData = PredictData(p.predict, p.prob) + + def apply(r: Row): PredictData = PredictData(r.getDouble(0), r.getDouble(1)) + } + + case class SplitData( + feature: Int, + threshold: Double, + featureType: Int, + categories: Seq[Double]) { // TODO: Change to List once SPARK-3365 is fixed + def toSplit: Split = { + new Split(feature, threshold, FeatureType(featureType), categories.toList) + } + } + + object SplitData { + def apply(s: Split): SplitData = { + SplitData(s.feature, s.threshold, s.featureType.id, s.categories) + } + + def apply(r: Row): SplitData = { + SplitData(r.getInt(0), r.getDouble(1), r.getInt(2), r.getAs[Seq[Double]](3)) + } + } + + /** Model data for model import/export */ + case class NodeData( + treeId: Int, + nodeId: Int, + predict: PredictData, + impurity: Double, + isLeaf: Boolean, + split: Option[SplitData], + leftNodeId: Option[Int], + rightNodeId: Option[Int], + infoGain: Option[Double]) + + object NodeData { + def apply(treeId: Int, n: Node): NodeData = { + NodeData(treeId, n.id, PredictData(n.predict), n.impurity, n.isLeaf, + n.split.map(SplitData.apply), n.leftNode.map(_.id), n.rightNode.map(_.id), + n.stats.map(_.gain)) + } + + def apply(r: Row): NodeData = { + val split = if (r.isNullAt(5)) None else Some(SplitData(r.getStruct(5))) + val leftNodeId = if (r.isNullAt(6)) None else Some(r.getInt(6)) + val rightNodeId = if (r.isNullAt(7)) None else Some(r.getInt(7)) + val infoGain = if (r.isNullAt(8)) None else Some(r.getDouble(8)) + NodeData(r.getInt(0), r.getInt(1), PredictData(r.getStruct(2)), r.getDouble(3), + r.getBoolean(4), split, leftNodeId, rightNodeId, infoGain) + } + } + + def save(sc: SparkContext, path: String, model: DecisionTreeModel): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadataRDD = sc.parallelize( + Seq((thisClassName, thisFormatVersion, model.algo.toString, model.numNodes)), 1) + .toDataFrame("class", "version", "algo", "numNodes") + metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + + // Create Parquet data. + val nodes = model.topNode.subtreeIterator.toSeq + val dataRDD: DataFrame = sc.parallelize(nodes) + .map(NodeData.apply(0, _)) + .toDataFrame + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String, algo: String, numNodes: Int): DecisionTreeModel = { + val datapath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + // Load Parquet data. + val dataRDD = sqlContext.parquetFile(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) + // Build node data into a tree. + val trees = constructTrees(nodes) + assert(trees.size == 1, + "Decision tree should contain exactly one tree but got ${trees.size} trees.") + val model = new DecisionTreeModel(trees(0), Algo.fromString(algo)) + assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $datapath." + + s" Expected $numNodes nodes but found ${model.numNodes}") + model + } + + def constructTrees(nodes: RDD[NodeData]): Array[Node] = { + val trees = nodes + .groupBy(_.treeId) + .mapValues(_.toArray) + .collect() + .map { case (treeId, data) => + (treeId, constructTree(data)) + }.sortBy(_._1) + val numTrees = trees.size + val treeIndices = trees.map(_._1).toSeq + assert(treeIndices == (0 until numTrees), + s"Tree indices must start from 0 and increment by 1, but we found $treeIndices.") + trees.map(_._2) + } + + /** + * Given a list of nodes from a tree, construct the tree. + * @param data array of all node data in a tree. + */ + def constructTree(data: Array[NodeData]): Node = { + val dataMap: Map[Int, NodeData] = data.map(n => n.nodeId -> n).toMap + assert(dataMap.contains(1), + s"DecisionTree missing root node (id = 1).") + constructNode(1, dataMap, mutable.Map.empty) + } + + /** + * Builds a node from the node data map and adds new nodes to the input nodes map. + */ + private def constructNode( + id: Int, + dataMap: Map[Int, NodeData], + nodes: mutable.Map[Int, Node]): Node = { + if (nodes.contains(id)) { + return nodes(id) + } + val data = dataMap(id) + val node = + if (data.isLeaf) { + Node(data.nodeId, data.predict.toPredict, data.impurity, data.isLeaf) + } else { + val leftNode = constructNode(data.leftNodeId.get, dataMap, nodes) + val rightNode = constructNode(data.rightNodeId.get, dataMap, nodes) + val stats = new InformationGainStats(data.infoGain.get, data.impurity, leftNode.impurity, + rightNode.impurity, leftNode.predict, rightNode.predict) + new Node(data.nodeId, data.predict.toPredict, data.impurity, data.isLeaf, + data.split.map(_.toSplit), Some(leftNode), Some(rightNode), Some(stats)) + } + nodes += node.id -> node + node + } + } + + override def load(sc: SparkContext, path: String): DecisionTreeModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + val (algo: String, numNodes: Int) = try { + val algo_numNodes = metadata.select("algo", "numNodes").collect() + assert(algo_numNodes.length == 1) + algo_numNodes(0) match { + case Row(a: String, n: Int) => (a, n) + } + } catch { + // Catch both Error and Exception since the checks above can throw either. + case e: Throwable => + throw new Exception( + s"Unable to load DecisionTreeModel metadata from: ${Loader.metadataPath(path)}." + + s" Error message: ${e.getMessage}") + } + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, version) match { + case (className, "1.0") if className == classNameV1_0 => + SaveLoadV1_0.load(sc, path, algo, numNodes) + case _ => throw new Exception( + s"DecisionTreeModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 9a50ecb550c38..80990aa9a603f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -49,7 +49,9 @@ class InformationGainStats( gain == other.gain && impurity == other.impurity && leftImpurity == other.leftImpurity && - rightImpurity == other.rightImpurity + rightImpurity == other.rightImpurity && + leftPredict == other.leftPredict && + rightPredict == other.rightPredict } case _ => false } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 2179da8dbe03e..d961081d185e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -166,6 +166,11 @@ class Node ( } } + /** Returns an iterator that traverses (DFS, left to right) the subtree of this node. */ + private[tree] def subtreeIterator: Iterator[Node] = { + Iterator.single(this) ++ leftNode.map(_.subtreeIterator).getOrElse(Iterator.empty) ++ + rightNode.map(_.subtreeIterator).getOrElse(Iterator.empty) + } } private[tree] object Node { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 004838ee5ba0e..ad4c0dbbfb3e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -32,4 +32,11 @@ class Predict( override def toString = { "predict = %f, prob = %f".format(predict, prob) } + + override def equals(other: Any): Boolean = { + other match { + case p: Predict => predict == p.predict && prob == p.prob + case _ => false + } + } } 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 22997110de8dd..23bd46baabf65 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 @@ -21,12 +21,17 @@ import scala.collection.mutable import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.Algo import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ +import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} + /** * :: Experimental :: @@ -38,9 +43,42 @@ import org.apache.spark.rdd.RDD @Experimental class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) extends TreeEnsembleModel(algo, trees, Array.fill(trees.size)(1.0), - combiningStrategy = if (algo == Classification) Vote else Average) { + combiningStrategy = if (algo == Classification) Vote else Average) + with Saveable { require(trees.forall(_.algo == algo)) + + override def save(sc: SparkContext, path: String): Unit = { + TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, + RandomForestModel.SaveLoadV1_0.thisClassName) + } + + override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion +} + +object RandomForestModel extends Loader[RandomForestModel] { + + override def load(sc: SparkContext, path: String): RandomForestModel = { + val (loadedClassName, version, metadataRDD) = Loader.loadMetadata(sc, path) + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, version) match { + case (className, "1.0") if className == classNameV1_0 => + val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(metadataRDD, path) + assert(metadata.treeWeights.forall(_ == 1.0)) + val trees = + TreeEnsembleModel.SaveLoadV1_0.loadTrees(sc, path, metadata.treeAlgo) + new RandomForestModel(Algo.fromString(metadata.algo), trees) + case _ => throw new Exception(s"RandomForestModel.load did not recognize model" + + s" with (className, format version): ($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } + + private object SaveLoadV1_0 { + // Hard-code class name string in case it changes in the future + def thisClassName = "org.apache.spark.mllib.tree.model.RandomForestModel" + } + } /** @@ -56,9 +94,42 @@ class GradientBoostedTreesModel( override val algo: Algo, override val trees: Array[DecisionTreeModel], override val treeWeights: Array[Double]) - extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) { + extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) + with Saveable { require(trees.size == treeWeights.size) + + override def save(sc: SparkContext, path: String): Unit = { + TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, + GradientBoostedTreesModel.SaveLoadV1_0.thisClassName) + } + + override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion +} + +object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { + + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { + val (loadedClassName, version, metadataRDD) = Loader.loadMetadata(sc, path) + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, version) match { + case (className, "1.0") if className == classNameV1_0 => + val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(metadataRDD, path) + assert(metadata.combiningStrategy == Sum.toString) + val trees = + TreeEnsembleModel.SaveLoadV1_0.loadTrees(sc, path, metadata.treeAlgo) + new GradientBoostedTreesModel(Algo.fromString(metadata.algo), trees, metadata.treeWeights) + case _ => throw new Exception(s"GradientBoostedTreesModel.load did not recognize model" + + s" with (className, format version): ($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } + + private object SaveLoadV1_0 { + // Hard-code class name string in case it changes in the future + def thisClassName = "org.apache.spark.mllib.tree.model.GradientBoostedTreesModel" + } + } /** @@ -176,3 +247,85 @@ private[tree] sealed class TreeEnsembleModel( */ def totalNumNodes: Int = trees.map(_.numNodes).sum } + +private[tree] object TreeEnsembleModel { + + object SaveLoadV1_0 { + + import DecisionTreeModel.SaveLoadV1_0.{NodeData, constructTrees} + + def thisFormatVersion = "1.0" + + case class Metadata( + algo: String, + treeAlgo: String, + combiningStrategy: String, + treeWeights: Array[Double]) + + /** + * Model data for model import/export. + * We have to duplicate NodeData here since Spark SQL does not yet support extracting subfields + * of nested fields; once that is possible, we can use something like: + * case class EnsembleNodeData(treeId: Int, node: NodeData), + * where NodeData is from DecisionTreeModel. + */ + case class EnsembleNodeData(treeId: Int, node: NodeData) + + def save(sc: SparkContext, path: String, model: TreeEnsembleModel, className: String): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadata = Metadata(model.algo.toString, model.trees(0).algo.toString, + model.combiningStrategy.toString, model.treeWeights) + val metadataRDD = sc.parallelize(Seq((className, thisFormatVersion, metadata)), 1) + .toDataFrame("class", "version", "metadata") + metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + + // Create Parquet data. + val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => + tree.topNode.subtreeIterator.toSeq.map(node => NodeData(treeId, node)) + }.toDataFrame + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + /** + * Read metadata from the loaded metadata DataFrame. + * @param path Path for loading data, used for debug messages. + */ + def readMetadata(metadata: DataFrame, path: String): Metadata = { + try { + // We rely on the try-catch for schema checking rather than creating a schema just for this. + val metadataArray = metadata.select("metadata.algo", "metadata.treeAlgo", + "metadata.combiningStrategy", "metadata.treeWeights").collect() + assert(metadataArray.size == 1) + Metadata(metadataArray(0).getString(0), metadataArray(0).getString(1), + metadataArray(0).getString(2), metadataArray(0).getAs[Seq[Double]](3).toArray) + } catch { + // Catch both Error and Exception since the checks above can throw either. + case e: Throwable => + throw new Exception( + s"Unable to load TreeEnsembleModel metadata from: ${Loader.metadataPath(path)}." + + s" Error message: ${e.getMessage}") + } + } + + /** + * Load trees for an ensemble, and return them in order. + * @param path path to load the model from + * @param treeAlgo Algorithm for individual trees (which may differ from the ensemble's + * algorithm). + */ + def loadTrees( + sc: SparkContext, + path: String, + treeAlgo: String): Array[DecisionTreeModel] = { + val datapath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val nodes = sqlContext.parquetFile(datapath).map(NodeData.apply) + val trees = constructTrees(nodes) + trees.map(new DecisionTreeModel(_, Algo.fromString(treeAlgo))) + } + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 9347eaf9221a8..7b1aed5ffeb3e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -29,8 +29,10 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, Strategy} import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} +import org.apache.spark.mllib.tree.model._ import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.Utils + class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { @@ -857,9 +859,32 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(topNode.leftNode.get.impurity === 0.0) assert(topNode.rightNode.get.impurity === 0.0) } + + test("Node.subtreeIterator") { + val model = DecisionTreeSuite.createModel(Classification) + val nodeIds = model.topNode.subtreeIterator.map(_.id).toArray.sorted + assert(nodeIds === DecisionTreeSuite.createdModelNodeIds) + } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + Array(Classification, Regression).foreach { algo => + val model = DecisionTreeSuite.createModel(algo) + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = DecisionTreeModel.load(sc, path) + DecisionTreeSuite.checkEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } } -object DecisionTreeSuite { +object DecisionTreeSuite extends FunSuite { def validateClassifier( model: DecisionTreeModel, @@ -979,4 +1004,95 @@ object DecisionTreeSuite { arr } + /** Create a leaf node with the given node ID */ + private def createLeafNode(id: Int): Node = { + Node(nodeIndex = id, new Predict(0.0, 1.0), impurity = 0.5, isLeaf = true) + } + + /** + * Create an internal node with the given node ID and feature type. + * Note: This does NOT set the child nodes. + */ + private def createInternalNode(id: Int, featureType: FeatureType): Node = { + val node = Node(nodeIndex = id, new Predict(0.0, 1.0), impurity = 0.5, isLeaf = false) + featureType match { + case Continuous => + node.split = Some(new Split(feature = 0, threshold = 0.5, Continuous, + categories = List.empty[Double])) + case Categorical => + node.split = Some(new Split(feature = 1, threshold = 0.0, Categorical, + categories = List(0.0, 1.0))) + } + // TODO: The information gain stats should be consistent with the same info stored in children. + node.stats = Some(new InformationGainStats(gain = 0.1, impurity = 0.2, + leftImpurity = 0.3, rightImpurity = 0.4, new Predict(1.0, 0.4), new Predict(0.0, 0.6))) + node + } + + /** + * Create a tree model. This is deterministic and contains a variety of node and feature types. + */ + private[tree] def createModel(algo: Algo): DecisionTreeModel = { + val topNode = createInternalNode(id = 1, Continuous) + val (node2, node3) = (createLeafNode(id = 2), createInternalNode(id = 3, Categorical)) + val (node6, node7) = (createLeafNode(id = 6), createLeafNode(id = 7)) + topNode.leftNode = Some(node2) + topNode.rightNode = Some(node3) + node3.leftNode = Some(node6) + node3.rightNode = Some(node7) + new DecisionTreeModel(topNode, algo) + } + + /** Sorted Node IDs matching the model returned by [[createModel()]] */ + private val createdModelNodeIds = Array(1, 2, 3, 6, 7) + + /** + * Check if the two trees are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + * If the trees are not equal, this prints the two trees and throws an exception. + */ + private[tree] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + try { + assert(a.algo === b.algo) + checkEqual(a.topNode, b.topNode) + } catch { + case ex: Exception => + throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + "TREE A:\n" + a.toDebugString + "\n" + + "TREE B:\n" + b.toDebugString + "\n", ex) + } + } + + /** + * Return true iff the two nodes and their descendents are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + */ + private def checkEqual(a: Node, b: Node): Unit = { + assert(a.id === b.id) + assert(a.predict === b.predict) + assert(a.impurity === b.impurity) + assert(a.isLeaf === b.isLeaf) + assert(a.split === b.split) + (a.stats, b.stats) match { + // TODO: Check other fields besides the infomation gain. + case (Some(aStats), Some(bStats)) => assert(aStats.gain === bStats.gain) + case (None, None) => + case _ => throw new AssertionError( + s"Only one instance has stats defined. (a.stats: ${a.stats}, b.stats: ${b.stats})") + } + (a.leftNode, b.leftNode) match { + case (Some(aNode), Some(bNode)) => checkEqual(aNode, bNode) + case (None, None) => + case _ => throw new AssertionError("Only one instance has leftNode defined. " + + s"(a.leftNode: ${a.leftNode}, b.leftNode: ${b.leftNode})") + } + (a.rightNode, b.rightNode) match { + case (Some(aNode: Node), Some(bNode: Node)) => checkEqual(aNode, bNode) + case (None, None) => + case _ => throw new AssertionError("Only one instance has rightNode defined. " + + s"(a.rightNode: ${a.rightNode}, b.rightNode: ${b.rightNode})") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index e8341a5d0d104..bde47606eb001 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -24,8 +24,10 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity.Variance import org.apache.spark.mllib.tree.loss.{AbsoluteError, SquaredError, LogLoss} - +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.Utils + /** * Test suite for [[GradientBoostedTrees]]. @@ -35,32 +37,30 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { test("Regression with continuous features: SquaredError") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - GradientBoostedTreesSuite.randomSeeds.foreach { randomSeed => - val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) - - val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, - categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) - val boostingStrategy = - new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) - - val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) - - assert(gbt.trees.size === numIterations) - try { - EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06) - } catch { - case e: java.lang.AssertionError => - println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + - s" subsamplingRate=$subsamplingRate") - throw e - } - - val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) - val dt = DecisionTree.train(remappedInput, treeStrategy) - - // Make sure trees are the same. - assert(gbt.trees.head.toString == dt.toString) + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) + + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) + val boostingStrategy = + new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) + + val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) + + assert(gbt.trees.size === numIterations) + try { + EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06) + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e } + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val dt = DecisionTree.train(remappedInput, treeStrategy) + + // Make sure trees are the same. + assert(gbt.trees.head.toString == dt.toString) } } @@ -133,14 +133,37 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { BoostingStrategy.defaultParams(algo) } } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val trees = Range(0, 3).map(_ => DecisionTreeSuite.createModel(Regression)).toArray + val treeWeights = Array(0.1, 0.3, 1.1) + + Array(Classification, Regression).foreach { algo => + val model = new GradientBoostedTreesModel(algo, trees, treeWeights) + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = GradientBoostedTreesModel.load(sc, path) + assert(model.algo == sameModel.algo) + model.trees.zip(sameModel.trees).foreach { case (treeA, treeB) => + DecisionTreeSuite.checkEqual(treeA, treeB) + } + assert(model.treeWeights === sameModel.treeWeights) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } } -object GradientBoostedTreesSuite { +private object GradientBoostedTreesSuite { // Combinations for estimators, learning rates and subsamplingRate val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) - val randomSeeds = Array(681283, 4398) - val data = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 55e963977b54f..ee3bc98486862 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -27,8 +27,10 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.DecisionTreeMetadata import org.apache.spark.mllib.tree.impurity.{Gini, Variance} -import org.apache.spark.mllib.tree.model.Node +import org.apache.spark.mllib.tree.model.{Node, RandomForestModel} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.Utils + /** * Test suite for [[RandomForest]]. @@ -212,6 +214,26 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { assert(rf1.toDebugString != rf2.toDebugString) } -} - + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + Array(Classification, Regression).foreach { algo => + val trees = Range(0, 3).map(_ => DecisionTreeSuite.createModel(algo)).toArray + val model = new RandomForestModel(algo, trees) + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = RandomForestModel.load(sc, path) + assert(model.algo == sameModel.algo) + model.trees.zip(sameModel.trees).foreach { case (treeA, treeB) => + DecisionTreeSuite.checkEqual(treeA, treeB) + } + } finally { + Utils.deleteRecursively(tempDir) + } + } + } +} From c15134632e74e3dee05eda20c6ef79915e15d02e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 9 Feb 2015 22:45:48 -0800 Subject: [PATCH 034/817] [SPARK-4964][Streaming][Kafka] More updates to Exactly-once Kafka stream Changes - Added example - Added a critical unit test that verifies that offset ranges can be recovered through checkpoints Might add more changes. Author: Tathagata Das Closes #4384 from tdas/new-kafka-fixes and squashes the following commits: 7c931c3 [Tathagata Das] Small update 3ed9284 [Tathagata Das] updated scala doc 83d0402 [Tathagata Das] Added JavaDirectKafkaWordCount example. 26df23c [Tathagata Das] Updates based on PR comments from Cody e4abf69 [Tathagata Das] Scala doc improvements and stuff. bb65232 [Tathagata Das] Fixed test bug and refactored KafkaStreamSuite 50f2b56 [Tathagata Das] Added Java API and added more Scala and Java unit tests. Also updated docs. e73589c [Tathagata Das] Minor changes. 4986784 [Tathagata Das] Added unit test to kafka offset recovery 6a91cab [Tathagata Das] Added example --- .../streaming/JavaDirectKafkaWordCount.java | 113 ++++++ .../streaming/DirectKafkaWordCount.scala | 71 ++++ .../kafka/DirectKafkaInputDStream.scala | 5 +- .../spark/streaming/kafka/KafkaCluster.scala | 3 + .../spark/streaming/kafka/KafkaRDD.scala | 12 +- .../streaming/kafka/KafkaRDDPartition.scala | 23 +- .../spark/streaming/kafka/KafkaUtils.scala | 353 ++++++++++++++---- .../apache/spark/streaming/kafka/Leader.scala | 21 +- .../spark/streaming/kafka/OffsetRange.scala | 53 ++- .../kafka/JavaDirectKafkaStreamSuite.java | 159 ++++++++ .../streaming/kafka/JavaKafkaStreamSuite.java | 5 +- .../kafka/DirectKafkaStreamSuite.scala | 302 +++++++++++++++ .../streaming/kafka/KafkaClusterSuite.scala | 24 +- .../kafka/KafkaDirectStreamSuite.scala | 92 ----- .../spark/streaming/kafka/KafkaRDDSuite.scala | 8 +- .../streaming/kafka/KafkaStreamSuite.scala | 62 +-- .../kafka/ReliableKafkaStreamSuite.scala | 4 +- 17 files changed, 1048 insertions(+), 262 deletions(-) create mode 100644 examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java create mode 100644 examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala create mode 100644 external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala delete mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala diff --git a/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java b/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java new file mode 100644 index 0000000000000..bab9f2478e779 --- /dev/null +++ b/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Arrays; +import java.util.regex.Pattern; + +import scala.Tuple2; + +import com.google.common.collect.Lists; +import kafka.serializer.StringDecoder; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.*; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.streaming.kafka.KafkaUtils; +import org.apache.spark.streaming.Durations; + +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: DirectKafkaWordCount + * is a list of one or more Kafka brokers + * is a list of one or more kafka topics to consume from + * + * Example: + * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 + */ + +public final class JavaDirectKafkaWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: DirectKafkaWordCount \n" + + " is a list of one or more Kafka brokers\n" + + " is a list of one or more kafka topics to consume from\n\n"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + String brokers = args[0]; + String topics = args[1]; + + // Create context with 2 second batch interval + SparkConf sparkConf = new SparkConf().setAppName("JavaDirectKafkaWordCount"); + JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, Durations.seconds(2)); + + HashSet topicsSet = new HashSet(Arrays.asList(topics.split(","))); + HashMap kafkaParams = new HashMap(); + kafkaParams.put("metadata.broker.list", brokers); + + // Create direct kafka stream with brokers and topics + JavaPairInputDStream messages = KafkaUtils.createDirectStream( + jssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topicsSet + ); + + // Get the lines, split them into words, count the words and print + JavaDStream lines = messages.map(new Function, String>() { + @Override + public String call(Tuple2 tuple2) { + return tuple2._2(); + } + }); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey( + new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + wordCounts.print(); + + // Start the computation + jssc.start(); + jssc.awaitTermination(); + } +} diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala new file mode 100644 index 0000000000000..deb08fd57b8c7 --- /dev/null +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import kafka.serializer.StringDecoder + +import org.apache.spark.streaming._ +import org.apache.spark.streaming.kafka._ +import org.apache.spark.SparkConf + +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: DirectKafkaWordCount + * is a list of one or more Kafka brokers + * is a list of one or more kafka topics to consume from + * + * Example: + * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 + */ +object DirectKafkaWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println(s""" + |Usage: DirectKafkaWordCount + | is a list of one or more Kafka brokers + | is a list of one or more kafka topics to consume from + | + """".stripMargin) + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(brokers, topics) = args + + // Create context with 2 second batch interval + val sparkConf = new SparkConf().setAppName("DirectKafkaWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) + + // Create direct kafka stream with brokers and topics + val topicsSet = topics.split(",").toSet + val kafkaParams = Map[String, String]("metadata.broker.list" -> brokers) + val messages = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topicsSet) + + // Get the lines, split them into words, count the words and print + val lines = messages.map(_._2) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1L)).reduceByKey(_ + _) + wordCounts.print() + + // Start the computation + ssc.start() + ssc.awaitTermination() + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index c7bca43eb889d..04e65cb3d708c 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -50,14 +50,13 @@ import org.apache.spark.streaming.dstream._ * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) * starting point of the stream * @param messageHandler function for translating each message into the desired type - * @param maxRetries maximum number of times in a row to retry getting leaders' offsets */ private[streaming] class DirectKafkaInputDStream[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, + U <: Decoder[K]: ClassTag, + T <: Decoder[V]: ClassTag, R: ClassTag]( @transient ssc_ : StreamingContext, val kafkaParams: Map[String, String], diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index ccc62bfe8f057..2f7e0ab39fefd 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -332,6 +332,9 @@ object KafkaCluster { extends ConsumerConfig(originalProps) { val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => val hpa = hp.split(":") + if (hpa.size == 1) { + throw new SparkException(s"Broker not the in correct format of : [$brokers]") + } (hpa(0), hpa(1).toInt) } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 50bf7cbdb8dbf..d56cc01be9514 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -36,14 +36,12 @@ import kafka.utils.VerifiableProperties * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param batch Each KafkaRDDPartition in the batch corresponds to a - * range of offsets for a given Kafka topic/partition + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD * @param messageHandler function for translating each message into the desired type */ -private[spark] +private[kafka] class KafkaRDD[ K: ClassTag, V: ClassTag, @@ -183,7 +181,7 @@ class KafkaRDD[ } } -private[spark] +private[kafka] object KafkaRDD { import KafkaCluster.LeaderOffset diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala index 36372e08f65f6..a842a6f17766f 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -26,7 +26,7 @@ import org.apache.spark.Partition * @param host preferred kafka host, i.e. the leader at the time the rdd was created * @param port preferred kafka host's port */ -private[spark] +private[kafka] class KafkaRDDPartition( val index: Int, val topic: String, @@ -36,24 +36,3 @@ class KafkaRDDPartition( val host: String, val port: Int ) extends Partition - -private[spark] -object KafkaRDDPartition { - def apply( - index: Int, - topic: String, - partition: Int, - fromOffset: Long, - untilOffset: Long, - host: String, - port: Int - ): KafkaRDDPartition = new KafkaRDDPartition( - index, - topic, - partition, - fromOffset, - untilOffset, - host, - port - ) -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index f8aa6c5c6263c..7a2c3abdcc24b 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -18,7 +18,9 @@ package org.apache.spark.streaming.kafka import java.lang.{Integer => JInt} +import java.lang.{Long => JLong} import java.util.{Map => JMap} +import java.util.{Set => JSet} import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -27,18 +29,19 @@ import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.{Decoder, StringDecoder} - +import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.api.java.{JavaPairInputDStream, JavaInputDStream, JavaPairReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} +import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} object KafkaUtils { /** - * Create an input stream that pulls messages from a Kafka Broker. + * Create an input stream that pulls messages from Kafka Brokers. * @param ssc StreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) * @param groupId The group id for this consumer @@ -62,7 +65,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages from a Kafka Broker. + * Create an input stream that pulls messages from Kafka Brokers. * @param ssc StreamingContext object * @param kafkaParams Map of kafka configuration parameters, * see http://kafka.apache.org/08/configuration.html @@ -81,7 +84,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages from a Kafka Broker. + * Create an input stream that pulls messages from Kafka Brokers. * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) @@ -99,7 +102,7 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages from a Kafka Broker. + * Create an input stream that pulls messages from Kafka Brokers. * @param jssc JavaStreamingContext object * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. @@ -119,10 +122,10 @@ object KafkaUtils { } /** - * Create an input stream that pulls messages from a Kafka Broker. + * Create an input stream that pulls messages from Kafka Brokers. * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD + * @param keyTypeClass Key type of DStream + * @param valueTypeClass value type of Dstream * @param keyDecoderClass Type of kafka key decoder * @param valueDecoderClass Type of kafka value decoder * @param kafkaParams Map of kafka configuration parameters, @@ -151,14 +154,14 @@ object KafkaUtils { jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } - /** A batch-oriented interface for consuming from Kafka. - * Starting and ending offsets are specified in advance, - * so that you can control exactly-once semantics. + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. + * * @param sc SparkContext object * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition */ @@ -166,12 +169,12 @@ object KafkaUtils { def createRDD[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag] ( + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag]( sc: SparkContext, kafkaParams: Map[String, String], offsetRanges: Array[OffsetRange] - ): RDD[(K, V)] = { + ): RDD[(K, V)] = { val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) val kc = new KafkaCluster(kafkaParams) val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet @@ -179,121 +182,196 @@ object KafkaUtils { errs => throw new SparkException(errs.mkString("\n")), ok => ok ) - new KafkaRDD[K, V, U, T, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) + new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) } - /** A batch-oriented interface for consuming from Kafka. - * Starting and ending offsets are specified in advance, - * so that you can control exactly-once semantics. + /** + * :: Experimental :: + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * * @param sc SparkContext object * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition * @param leaders Kafka leaders for each offset range in batch - * @param messageHandler function for translating each message into the desired type + * @param messageHandler Function for translating each message and metadata into the desired type */ @Experimental def createRDD[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, - R: ClassTag] ( + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag]( sc: SparkContext, kafkaParams: Map[String, String], offsetRanges: Array[OffsetRange], leaders: Array[Leader], messageHandler: MessageAndMetadata[K, V] => R - ): RDD[R] = { - + ): RDD[R] = { val leaderMap = leaders .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port)) .toMap - new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler) + new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler) } + /** - * This stream can guarantee that each message from Kafka is included in transformations - * (as opposed to output actions) exactly once, even in most failure situations. + * Create a RDD from Kafka using offset ranges for each topic and partition. * - * Points to note: - * - * Failure Recovery - You must checkpoint this stream, or save offsets yourself and provide them - * as the fromOffsets parameter on restart. - * Kafka must have sufficient log retention to obtain messages after failure. - * - * Getting offsets from the stream - see programming guide + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + */ + @Experimental + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange] + ): JavaPairRDD[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + new JavaPairRDD(createRDD[K, V, KD, VD]( + jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges)) + } + + /** + * :: Experimental :: + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. * -. * Zookeeper - This does not use Zookeeper to store offsets. For interop with Kafka monitors - * that depend on Zookeeper, you must store offsets in ZK yourself. + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param leaders Kafka leaders for each offset range in batch + * @param messageHandler Function for translating each message and metadata into the desired type + */ + @Experimental + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + leaders: Array[Leader], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaRDD[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + createRDD[K, V, KD, VD, R]( + jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaders, messageHandler.call _) + } + + /** + * :: Experimental :: + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). * - * End-to-end semantics - This does not guarantee that any output operation will push each record - * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and - * outputting exactly once), you have to either ensure that the output operation is - * idempotent, or transactionally store offsets with the output. See the programming guide for - * more details. + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. * * @param ssc StreamingContext object * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param messageHandler function for translating each message into the desired type - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type */ @Experimental def createDirectStream[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, R: ClassTag] ( ssc: StreamingContext, kafkaParams: Map[String, String], fromOffsets: Map[TopicAndPartition, Long], messageHandler: MessageAndMetadata[K, V] => R ): InputDStream[R] = { - new DirectKafkaInputDStream[K, V, U, T, R]( + new DirectKafkaInputDStream[K, V, KD, VD, R]( ssc, kafkaParams, fromOffsets, messageHandler) } /** - * This stream can guarantee that each message from Kafka is included in transformations - * (as opposed to output actions) exactly once, even in most failure situations. + * :: Experimental :: + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). * * Points to note: - * - * Failure Recovery - You must checkpoint this stream. - * Kafka must have sufficient log retention to obtain messages after failure. - * - * Getting offsets from the stream - see programming guide - * -. * Zookeeper - This does not use Zookeeper to store offsets. For interop with Kafka monitors - * that depend on Zookeeper, you must store offsets in ZK yourself. - * - * End-to-end semantics - This does not guarantee that any output operation will push each record - * exactly once. To ensure end-to-end exactly-once semantics (that is, receiving exactly once and - * outputting exactly once), you have to ensure that the output operation is idempotent. + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. * * @param ssc StreamingContext object * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * If starting without a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" * to determine where the stream starts (defaults to "largest") - * @param topics names of the topics to consume + * @param topics Names of the topics to consume */ @Experimental def createDirectStream[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag] ( + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag] ( ssc: StreamingContext, kafkaParams: Map[String, String], topics: Set[String] @@ -313,11 +391,128 @@ object KafkaUtils { val fromOffsets = leaderOffsets.map { case (tp, lo) => (tp, lo.offset) } - new DirectKafkaInputDStream[K, V, U, T, (K, V)]( + new DirectKafkaInputDStream[K, V, KD, VD, (K, V)]( ssc, kafkaParams, fromOffsets, messageHandler) }).fold( errs => throw new SparkException(errs.mkString("\n")), ok => ok ) } + + /** + * :: Experimental :: + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class of the value decoder + * @param recordClass Class of the records in DStream + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + */ + @Experimental + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaInputDStream[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + createDirectStream[K, V, KD, VD, R]( + jssc.ssc, + Map(kafkaParams.toSeq: _*), + Map(fromOffsets.mapValues { _.longValue() }.toSeq: _*), + messageHandler.call _ + ) + } + + /** + * :: Experimental :: + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class type of the value decoder + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + */ + @Experimental + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + createDirectStream[K, V, KD, VD]( + jssc.ssc, + Map(kafkaParams.toSeq: _*), + Set(topics.toSeq: _*) + ) + } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala index 3454d92e72b47..c129a26836c0d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala @@ -19,17 +19,28 @@ package org.apache.spark.streaming.kafka import kafka.common.TopicAndPartition -/** Host info for the leader of a Kafka TopicAndPartition */ +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Represent the host info for the leader of a Kafka partition. + */ +@Experimental final class Leader private( - /** kafka topic name */ + /** Kafka topic name */ val topic: String, - /** kafka partition id */ + /** Kafka partition id */ val partition: Int, - /** kafka hostname */ + /** Leader's hostname */ val host: String, - /** kafka host's port */ + /** Leader's port */ val port: Int) extends Serializable +/** + * :: Experimental :: + * Companion object the provides methods to create instances of [[Leader]]. + */ +@Experimental object Leader { def create(topic: String, partition: Int, host: String, port: Int): Leader = new Leader(topic, partition, host, port) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 334c12e4627b4..9c3dfeb8f5928 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -19,16 +19,35 @@ package org.apache.spark.streaming.kafka import kafka.common.TopicAndPartition -/** Something that has a collection of OffsetRanges */ +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Represents any object that has a collection of [[OffsetRange]]s. This can be used access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * [[KafkaUtils.createDirectStream()]]). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +@Experimental trait HasOffsetRanges { def offsetRanges: Array[OffsetRange] } -/** Represents a range of offsets from a single Kafka TopicAndPartition */ +/** + * :: Experimental :: + * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class + * can be created with `OffsetRange.create()`. + */ +@Experimental final class OffsetRange private( - /** kafka topic name */ + /** Kafka topic name */ val topic: String, - /** kafka partition id */ + /** Kafka partition id */ val partition: Int, /** inclusive starting offset */ val fromOffset: Long, @@ -36,11 +55,33 @@ final class OffsetRange private( val untilOffset: Long) extends Serializable { import OffsetRange.OffsetRangeTuple + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset]" + } + /** this is to avoid ClassNotFoundException during checkpoint restore */ private[streaming] def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) } +/** + * :: Experimental :: + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +@Experimental object OffsetRange { def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = new OffsetRange(topic, partition, fromOffset, untilOffset) @@ -61,10 +102,10 @@ object OffsetRange { new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[spark] + private[kafka] type OffsetRangeTuple = (String, Int, Long, Long) - private[streaming] + private[kafka] def apply(t: OffsetRangeTuple) = new OffsetRange(t._1, t._2, t._3, t._4) } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java new file mode 100644 index 0000000000000..1334cc8fd1b57 --- /dev/null +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Random; +import java.util.Arrays; + +import org.apache.spark.SparkConf; + +import scala.Tuple2; + +import junit.framework.Assert; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +import org.junit.Test; +import org.junit.After; +import org.junit.Before; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient Random random = new Random(); + private transient KafkaStreamSuiteBase suiteBase = null; + + @Before + public void setUp() { + suiteBase = new KafkaStreamSuiteBase() { }; + suiteBase.setupKafka(); + System.clearProperty("spark.driver.port"); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + } + + @After + public void tearDown() { + ssc.stop(); + ssc = null; + System.clearProperty("spark.driver.port"); + suiteBase.tearDownKafka(); + } + + @Test + public void testKafkaStream() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + HashSet sent = new HashSet(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaDStream stream1 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topicToSet(topic1) + ).map( + new Function, String>() { + @Override + public String call(scala.Tuple2 kv) throws Exception { + return kv._2(); + } + } + ); + + JavaDStream stream2 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + topicOffsetToMap(topic2, (long) 0), + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) throws Exception { + return msgAndMd.message(); + } + } + ); + JavaDStream unifiedStream = stream1.union(stream2); + + final HashSet result = new HashSet(); + unifiedStream.foreachRDD( + new Function, Void>() { + @Override + public Void call(org.apache.spark.api.java.JavaRDD rdd) throws Exception { + result.addAll(rdd.collect()); + return null; + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private HashSet topicToSet(String topic) { + HashSet topicSet = new HashSet(); + topicSet.add(topic); + return topicSet; + } + + private HashMap topicOffsetToMap(String topic, Long offsetToStart) { + HashMap topicMap = new HashMap(); + topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); + return topicMap; + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + suiteBase.createTopic(topic); + suiteBase.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 6e1abf3f385ee..208cc51b29876 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -79,9 +79,10 @@ public void testKafkaStream() throws InterruptedException { suiteBase.createTopic(topic); HashMap tmp = new HashMap(sent); - suiteBase.produceAndSendMessage(topic, + suiteBase.sendMessages(topic, JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms())); + Predef.>conforms()) + ); HashMap kafkaParams = new HashMap(); kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala new file mode 100644 index 0000000000000..b25c2120d54f7 --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps + +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.{Eventually, Timeouts} + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.dstream.{DStream, InputDStream} +import org.apache.spark.util.Utils +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata + +class DirectKafkaStreamSuite extends KafkaStreamSuiteBase + with BeforeAndAfter with BeforeAndAfterAll with Eventually { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + var sc: SparkContext = _ + var ssc: StreamingContext = _ + var testDir: File = _ + + override def beforeAll { + setupKafka() + } + + override def afterAll { + tearDownKafka() + } + + after { + if (ssc != null) { + ssc.stop() + sc = null + } + if (sc != null) { + sc.stop() + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + + test("basic stream receiving with multiple topics and smallest starting offset") { + val topics = Set("basic1", "basic2", "basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + createTopic(t) + sendMessages(t, data) + } + val kafkaParams = Map( + "metadata.broker.list" -> s"$brokerAddress", + "auto.offset.reset" -> "smallest" + ) + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics) + } + var total = 0L + + stream.foreachRDD { rdd => + // Get the offset ranges in the RDD + val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsets(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + total += collected.size // Add up all the collected items + } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(total === data.values.sum * topics.size, "didn't get all messages") + } + ssc.stop() + } + + test("receiving from largest starting offset") { + val topic = "largest" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> s"$brokerAddress", + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new mutable.ArrayBuffer[String]() + stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } + ssc.start() + val newData = Map("b" -> 10) + sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + + test("creating stream by offset") { + val topic = "offset" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> s"$brokerAddress", + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String]( + ssc, kafkaParams, Map(topicPartition -> 11L), + (m: MessageAndMetadata[String, String]) => m.message()) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new mutable.ArrayBuffer[String]() + stream.foreachRDD { rdd => collectedData ++= rdd.collect() } + ssc.start() + val newData = Map("b" -> 10) + sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "recovery" + createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = Map( + "metadata.broker.list" -> s"$brokerAddress", + "auto.offset.reset" -> "smallest" + ) + + // Send data to Kafka and wait for it to be received + def sendDataAndWaitForReceive(data: Seq[Int]) { + val strings = data.map { _.toString} + sendMessages(topic, strings.map { _ -> 1}.toMap) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) + } + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + val kafkaStream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + DirectKafkaStreamSuite.collectedData.appendAll(data) + } + + // This is ensure all the data is eventually receiving only once + stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => + rdd.collect().headOption.foreach { x => DirectKafkaStreamSuite.total = x._2 } + } + ssc.start() + + // Send some data and wait for them to be received + for (i <- (1 to 10).grouped(4)) { + sendDataAndWaitForReceive(i) + } + + // Verify that offset ranges were generated + val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) + assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") + assert( + offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) + ssc.stop() + logInfo("====== RESTARTING ========") + + // Recover context from checkpoints + ssc = new StreamingContext(testDir.getAbsolutePath) + val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] + + // Verify offset ranges have been recovered + val recoveredOffsetRanges = getOffsetRanges(recoveredStream) + assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") + val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } + assert( + recoveredOffsetRanges.forall { or => + earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) + }, + "Recovered ranges are not the same as the ones generated" + ) + + // Restart context, give more data and verify the total at the end + // If the total is write that means each records has been received only once + ssc.start() + sendDataAndWaitForReceive(11 to 20) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total === (1 to 20).sum) + } + ssc.stop() + } + + /** Get the generated offset ranges from the DirectKafkaStream */ + private def getOffsetRanges[K, V]( + kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream.generatedRDDs.mapValues { rdd => + rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges + }.toSeq.sortBy { _._1 } + } +} + +object DirectKafkaStreamSuite { + val collectedData = new mutable.ArrayBuffer[String]() + var total = -1L +} diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index e57c8f6987fdc..fc9275b7207be 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -19,33 +19,29 @@ package org.apache.spark.streaming.kafka import scala.util.Random -import org.scalatest.BeforeAndAfter import kafka.common.TopicAndPartition +import org.scalatest.BeforeAndAfterAll -class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfter { - val brokerHost = "localhost" - - val kafkaParams = Map("metadata.broker.list" -> s"$brokerHost:$brokerPort") - - val kc = new KafkaCluster(kafkaParams) - +class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { val topic = "kcsuitetopic" + Random.nextInt(10000) - val topicAndPartition = TopicAndPartition(topic, 0) + var kc: KafkaCluster = null - before { + override def beforeAll() { setupKafka() createTopic(topic) - produceAndSendMessage(topic, Map("a" -> 1)) + sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> s"$brokerAddress")) } - after { + override def afterAll() { tearDownKafka() } test("metadata apis") { - val leader = kc.findLeaders(Set(topicAndPartition)).right.get - assert(leader(topicAndPartition) === (brokerHost, brokerPort), "didn't get leader") + val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) + val leaderAddress = s"${leader._1}:${leader._2}" + assert(leaderAddress === brokerAddress, "didn't get leader") val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala deleted file mode 100644 index 0891ce344f16a..0000000000000 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.util.Random -import scala.concurrent.duration._ - -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Eventually - -import kafka.serializer.StringDecoder - -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} - -class KafkaDirectStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { - val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - - val brokerHost = "localhost" - - val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerHost:$brokerPort", - "auto.offset.reset" -> "smallest" - ) - - var ssc: StreamingContext = _ - - before { - setupKafka() - - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - } - - after { - if (ssc != null) { - ssc.stop() - } - tearDownKafka() - } - - test("multi topic stream") { - val topics = Set("newA", "newB") - val data = Map("a" -> 7, "b" -> 9) - topics.foreach { t => - createTopic(t) - produceAndSendMessage(t, data) - } - val stream = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics) - var total = 0L; - - stream.foreachRDD { rdd => - val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - val collected = rdd.mapPartitionsWithIndex { (i, iter) => - val off = offsets(i) - val all = iter.toSeq - val partSize = all.size - val rangeSize = off.untilOffset - off.fromOffset - all.map { _ => - (partSize, rangeSize) - }.toIterator - }.collect - collected.foreach { case (partSize, rangeSize) => - assert(partSize === rangeSize, "offset ranges are wrong") - } - total += collected.size - } - ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(total === data.values.sum * topics.size, "didn't get all messages") - } - ssc.stop() - } -} diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 9b9e3f5fce8bd..6774db854a0d0 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -46,9 +46,9 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter { val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) createTopic(topic) - produceAndSendMessage(topic, sent) + sendMessages(topic, sent) - val kafkaParams = Map("metadata.broker.list" -> s"localhost:$brokerPort", + val kafkaParams = Map("metadata.broker.list" -> brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val kc = new KafkaCluster(kafkaParams) @@ -65,14 +65,14 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter { val rdd2 = getRdd(kc, Set(topic)) val sent2 = Map("d" -> 1) - produceAndSendMessage(topic, sent2) + sendMessages(topic, sent2) // this is the "0 messages" case // make sure we dont get anything, since messages were sent after rdd was defined assert(rdd2.isDefined) assert(rdd2.get.count === 0) val rdd3 = getRdd(kc, Set(topic)) - produceAndSendMessage(topic, Map("extra" -> 22)) + sendMessages(topic, Map("extra" -> 22)) // this is the "exactly 1 message" case // make sure we get exactly one message, despite there being lots more available assert(rdd3.isDefined) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index f207dc6d4fa04..e4966eebb9b34 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -48,30 +48,41 @@ import org.apache.spark.util.Utils */ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging { - var zkAddress: String = _ - var zkClient: ZkClient = _ - private val zkHost = "localhost" + private var zkPort: Int = 0 private val zkConnectionTimeout = 6000 private val zkSessionTimeout = 6000 private var zookeeper: EmbeddedZookeeper = _ - private var zkPort: Int = 0 - protected var brokerPort = 9092 + private val brokerHost = "localhost" + private var brokerPort = 9092 private var brokerConf: KafkaConfig = _ private var server: KafkaServer = _ private var producer: Producer[String, String] = _ + private var zkReady = false + private var brokerReady = false + + protected var zkClient: ZkClient = _ + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } def setupKafka() { // Zookeeper server startup zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") // Get the actual zookeeper binding port zkPort = zookeeper.actualPort - zkAddress = s"$zkHost:$zkPort" - logInfo("==================== 0 ====================") + zkReady = true + logInfo("==================== Zookeeper Started ====================") - zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, - ZKStringSerializer) - logInfo("==================== 1 ====================") + zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + logInfo("==================== Zookeeper Client Created ====================") // Kafka broker startup var bindSuccess: Boolean = false @@ -80,9 +91,8 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin val brokerProps = getBrokerConfig() brokerConf = new KafkaConfig(brokerProps) server = new KafkaServer(brokerConf) - logInfo("==================== 2 ====================") server.startup() - logInfo("==================== 3 ====================") + logInfo("==================== Kafka Broker Started ====================") bindSuccess = true } catch { case e: KafkaException => @@ -94,10 +104,13 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin } Thread.sleep(2000) - logInfo("==================== 4 ====================") + logInfo("==================== Kafka + Zookeeper Ready ====================") + brokerReady = true } def tearDownKafka() { + brokerReady = false + zkReady = false if (producer != null) { producer.close() producer = null @@ -121,26 +134,23 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin } } - private def createTestMessage(topic: String, sent: Map[String, Int]) - : Seq[KeyedMessage[String, String]] = { - val messages = for ((s, freq) <- sent; i <- 0 until freq) yield { - new KeyedMessage[String, String](topic, s) - } - messages.toSeq - } - def createTopic(topic: String) { AdminUtils.createTopic(zkClient, topic, 1, 1) - logInfo("==================== 5 ====================") // wait until metadata is propagated waitUntilMetadataIsPropagated(topic, 0) + logInfo(s"==================== Topic $topic Created ====================") } - def produceAndSendMessage(topic: String, sent: Map[String, Int]) { + def sendMessages(topic: String, messageToFreq: Map[String, Int]) { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + def sendMessages(topic: String, messages: Array[String]) { producer = new Producer[String, String](new ProducerConfig(getProducerConfig())) - producer.send(createTestMessage(topic, sent): _*) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) producer.close() - logInfo("==================== 6 ====================") + logInfo(s"==================== Sent Messages: ${messages.mkString(", ")} ====================") } private def getBrokerConfig(): Properties = { @@ -218,7 +228,7 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) createTopic(topic) - produceAndSendMessage(topic, sent) + sendMessages(topic, sent) val kafkaParams = Map("zookeeper.connect" -> zkAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}", diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 64ccc92c81fa9..fc53c23abda85 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -79,7 +79,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter test("Reliable Kafka input stream with single topic") { var topic = "test-topic" createTopic(topic) - produceAndSendMessage(topic, data) + sendMessages(topic, data) // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === None) @@ -111,7 +111,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => createTopic(t) - produceAndSendMessage(t, data) + sendMessages(t, data) } // Before started, verify all the group/topic/partition offsets are 0. From 2d1e916730492f5d61b97da6c483d3223ca44315 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 10 Feb 2015 09:19:01 +0000 Subject: [PATCH 035/817] SPARK-5239 [CORE] JdbcRDD throws "java.lang.AbstractMethodError: oracle.jdbc.driver.xxxxxx.isClosed()Z" This is a completion of https://github.com/apache/spark/pull/4033 which was withdrawn for some reason. Author: Sean Owen Closes #4470 from srowen/SPARK-5239.2 and squashes the following commits: 2398bde [Sean Owen] Avoid use of JDBC4-only isClosed() --- core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala | 6 +++--- .../src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 642a12c1edf6c..4fe7622bda00f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -99,21 +99,21 @@ class JdbcRDD[T: ClassTag]( override def close() { try { - if (null != rs && ! rs.isClosed()) { + if (null != rs) { rs.close() } } catch { case e: Exception => logWarning("Exception closing resultset", e) } try { - if (null != stmt && ! stmt.isClosed()) { + if (null != stmt) { stmt.close() } } catch { case e: Exception => logWarning("Exception closing statement", e) } try { - if (null != conn && ! conn.isClosed()) { + if (null != conn) { conn.close() } logInfo("closed connection") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 0bec32cca1325..87304ce2496b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -370,21 +370,21 @@ private[sql] class JDBCRDD( def close() { if (closed) return try { - if (null != rs && ! rs.isClosed()) { + if (null != rs) { rs.close() } } catch { case e: Exception => logWarning("Exception closing resultset", e) } try { - if (null != stmt && ! stmt.isClosed()) { + if (null != stmt) { stmt.close() } } catch { case e: Exception => logWarning("Exception closing statement", e) } try { - if (null != conn && ! conn.isClosed()) { + if (null != conn) { conn.close() } logInfo("closed connection") From ba667935f8670293f10b8bbe1e317b28d00f9875 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 10 Feb 2015 02:28:47 -0800 Subject: [PATCH 036/817] [SPARK-5700] [SQL] [Build] Bumps jets3t to 0.9.3 for hadoop-2.3 and hadoop-2.4 profiles This is a follow-up PR for #4454 and #4484. JetS3t 0.9.2 contains a log4j.properties file inside the artifact and breaks our tests (see SPARK-5696). This is fixed in 0.9.3. This PR also reverts hotfix changes introduced in #4484. The reason is that asking users to configure HiveThriftServer2 logging configurations in hive-log4j.properties can be unintuitive. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4499) Author: Cheng Lian Closes #4499 from liancheng/spark-5700 and squashes the following commits: 4f020c7 [Cheng Lian] Bumps jets3t to 0.9.3 for hadoop-2.3 and hadoop-2.4 profiles --- pom.xml | 4 ++-- .../spark/sql/hive/thriftserver/HiveThriftServer2.scala | 3 --- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index a9e968af25453..56e37d42265c0 100644 --- a/pom.xml +++ b/pom.xml @@ -1578,7 +1578,7 @@ 2.3.0 2.5.0 - 0.9.2 + 0.9.3 0.98.7-hadoop2 3.1.1 hadoop2 @@ -1591,7 +1591,7 @@ 2.4.0 2.5.0 - 0.9.2 + 0.9.3 0.98.7-hadoop2 3.1.1 hadoop2 diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 525777aa454c4..6e07df18b0e15 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.common.LogUtils import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} @@ -55,8 +54,6 @@ object HiveThriftServer2 extends Logging { System.exit(-1) } - LogUtils.initHiveLog4j() - logInfo("Starting SparkContext") SparkSQLEnv.init() From 50820f15277187e8522520a3ecae412abbdb4397 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 10 Feb 2015 15:45:38 +0000 Subject: [PATCH 037/817] [SPARK-1805] [EC2] Validate instance types Addresses [SPARK-1805](https://issues.apache.org/jira/browse/SPARK-1805), though doesn't resolve it completely. Error out quickly if the user asks for the master and slaves to have different AMI virtualization types, since we don't currently support that. In addition to that, we print warnings if the inputted instance types are not recognized, though I would prefer if we errored out. Elsewhere in the script it seems [we allow unrecognized instance types](https://github.com/apache/spark/blob/5de14cc2763a8211f77eeb55940dec025822eb78/ec2/spark_ec2.py#L331), though I think we should remove that. It's messy, but it should serve us until we enhance spark-ec2 to support clusters with mixed virtualization types. Author: Nicholas Chammas Closes #4455 from nchammas/ec2-master-slave-different-virtualization and squashes the following commits: ce28609 [Nicholas Chammas] fix style b0adba0 [Nicholas Chammas] validate input instance types --- ec2/spark_ec2.py | 132 +++++++++++++++++++++++++++++------------------ 1 file changed, 81 insertions(+), 51 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 3e4c49c0e1db6..fe510f12bcec6 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -112,6 +112,7 @@ def parse_args(): version="%prog {v}".format(v=SPARK_EC2_VERSION), usage="%prog [options] \n\n" + " can be: launch, destroy, login, stop, start, get-master, reboot-slaves") + parser.add_option( "-s", "--slaves", type="int", default=1, help="Number of slaves to launch (default: %default)") @@ -139,7 +140,9 @@ def parse_args(): help="Availability zone to launch instances in, or 'all' to spread " + "slaves across multiple (an additional $0.01/Gb for bandwidth" + "between zones applies) (default: a single zone chosen at random)") - parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") + parser.add_option( + "-a", "--ami", + help="Amazon Machine Image ID to use") parser.add_option( "-v", "--spark-version", default=DEFAULT_SPARK_VERSION, help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") @@ -179,10 +182,11 @@ def parse_args(): "Only possible on EBS-backed AMIs. " + "EBS volumes are only attached if --ebs-vol-size > 0." + "Only support up to 8 EBS volumes.") - parser.add_option("--placement-group", type="string", default=None, - help="Which placement group to try and launch " + - "instances into. Assumes placement group is already " + - "created.") + parser.add_option( + "--placement-group", type="string", default=None, + help="Which placement group to try and launch " + + "instances into. Assumes placement group is already " + + "created.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, help="Swap space to set up per node, in MB (default: %default)") @@ -226,9 +230,11 @@ def parse_args(): "--copy-aws-credentials", action="store_true", default=False, help="Add AWS credentials to hadoop configuration to allow Spark to access S3") parser.add_option( - "--subnet-id", default=None, help="VPC subnet to launch instances in") + "--subnet-id", default=None, + help="VPC subnet to launch instances in") parser.add_option( - "--vpc-id", default=None, help="VPC to launch instances in") + "--vpc-id", default=None, + help="VPC to launch instances in") (opts, args) = parser.parse_args() if len(args) != 2: @@ -290,52 +296,54 @@ def is_active(instance): return (instance.state in ['pending', 'running', 'stopping', 'stopped']) -# Attempt to resolve an appropriate AMI given the architecture and region of the request. # Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ # Last Updated: 2014-06-20 # For easy maintainability, please keep this manually-inputted dictionary sorted by key. +EC2_INSTANCE_TYPES = { + "c1.medium": "pvm", + "c1.xlarge": "pvm", + "c3.2xlarge": "pvm", + "c3.4xlarge": "pvm", + "c3.8xlarge": "pvm", + "c3.large": "pvm", + "c3.xlarge": "pvm", + "cc1.4xlarge": "hvm", + "cc2.8xlarge": "hvm", + "cg1.4xlarge": "hvm", + "cr1.8xlarge": "hvm", + "hi1.4xlarge": "pvm", + "hs1.8xlarge": "pvm", + "i2.2xlarge": "hvm", + "i2.4xlarge": "hvm", + "i2.8xlarge": "hvm", + "i2.xlarge": "hvm", + "m1.large": "pvm", + "m1.medium": "pvm", + "m1.small": "pvm", + "m1.xlarge": "pvm", + "m2.2xlarge": "pvm", + "m2.4xlarge": "pvm", + "m2.xlarge": "pvm", + "m3.2xlarge": "hvm", + "m3.large": "hvm", + "m3.medium": "hvm", + "m3.xlarge": "hvm", + "r3.2xlarge": "hvm", + "r3.4xlarge": "hvm", + "r3.8xlarge": "hvm", + "r3.large": "hvm", + "r3.xlarge": "hvm", + "t1.micro": "pvm", + "t2.medium": "hvm", + "t2.micro": "hvm", + "t2.small": "hvm", +} + + +# Attempt to resolve an appropriate AMI given the architecture and region of the request. def get_spark_ami(opts): - instance_types = { - "c1.medium": "pvm", - "c1.xlarge": "pvm", - "c3.2xlarge": "pvm", - "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm", - "c3.large": "pvm", - "c3.xlarge": "pvm", - "cc1.4xlarge": "hvm", - "cc2.8xlarge": "hvm", - "cg1.4xlarge": "hvm", - "cr1.8xlarge": "hvm", - "hi1.4xlarge": "pvm", - "hs1.8xlarge": "pvm", - "i2.2xlarge": "hvm", - "i2.4xlarge": "hvm", - "i2.8xlarge": "hvm", - "i2.xlarge": "hvm", - "m1.large": "pvm", - "m1.medium": "pvm", - "m1.small": "pvm", - "m1.xlarge": "pvm", - "m2.2xlarge": "pvm", - "m2.4xlarge": "pvm", - "m2.xlarge": "pvm", - "m3.2xlarge": "hvm", - "m3.large": "hvm", - "m3.medium": "hvm", - "m3.xlarge": "hvm", - "r3.2xlarge": "hvm", - "r3.4xlarge": "hvm", - "r3.8xlarge": "hvm", - "r3.large": "hvm", - "r3.xlarge": "hvm", - "t1.micro": "pvm", - "t2.medium": "hvm", - "t2.micro": "hvm", - "t2.small": "hvm", - } - if opts.instance_type in instance_types: - instance_type = instance_types[opts.instance_type] + if opts.instance_type in EC2_INSTANCE_TYPES: + instance_type = EC2_INSTANCE_TYPES[opts.instance_type] else: instance_type = "pvm" print >> stderr,\ @@ -605,8 +613,6 @@ def launch_cluster(conn, opts, cluster_name): # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves - - def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." reservations = conn.get_all_reservations() @@ -1050,6 +1056,30 @@ def real_main(): print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file) sys.exit(1) + if opts.instance_type not in EC2_INSTANCE_TYPES: + print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format( + t=opts.instance_type) + + if opts.master_instance_type != "": + if opts.master_instance_type not in EC2_INSTANCE_TYPES: + print >> stderr, \ + "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( + t=opts.master_instance_type) + # Since we try instance types even if we can't resolve them, we check if they resolve first + # and, if they do, see if they resolve to the same virtualization type. + if opts.instance_type in EC2_INSTANCE_TYPES and \ + opts.master_instance_type in EC2_INSTANCE_TYPES: + if EC2_INSTANCE_TYPES[opts.instance_type] != \ + EC2_INSTANCE_TYPES[opts.master_instance_type]: + print >> stderr, \ + "Error: spark-ec2 currently does not support having a master and slaves with " + \ + "different AMI virtualization types." + print >> stderr, "master instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.master_instance_type]) + print >> stderr, "slave instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.instance_type]) + sys.exit(1) + if opts.ebs_vol_num > 8: print >> stderr, "ebs-vol-num cannot be greater than 8" sys.exit(1) From 6cc96cf0c3ea87ab65d42a59725959d94701577b Mon Sep 17 00:00:00 2001 From: JqueryFan Date: Tue, 10 Feb 2015 17:37:32 +0000 Subject: [PATCH 038/817] [Spark-5717] [MLlib] add stop and reorganize import Trivial. add sc stop and reorganize import https://issues.apache.org/jira/browse/SPARK-5717 Author: JqueryFan Author: Yuhao Yang Closes #4503 from hhbyyh/scstop and squashes the following commits: 7837a2c [JqueryFan] revert import change 2e85cc1 [Yuhao Yang] add stop and reorganize import --- .../java/org/apache/spark/examples/mllib/JavaLDAExample.java | 1 + .../main/scala/org/apache/spark/examples/mllib/LDAExample.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java index f394ff2084463..36207ae38d9a9 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java @@ -71,5 +71,6 @@ public Tuple2 call(Tuple2 doc_id) { } System.out.println(); } + sc.stop(); } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 0e1b27a8bd2ee..11399a7633638 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -159,7 +159,7 @@ object LDAExample { } println() } - + sc.stop() } /** From c7ad80ae4256c88e380e7488d48cf6eb14a92d76 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 10 Feb 2015 11:08:21 -0800 Subject: [PATCH 039/817] [SPARK-5716] [SQL] Support TOK_CHARSETLITERAL in HiveQl Author: Daoyuan Wang Closes #4502 from adrian-wang/utf8 and squashes the following commits: 4d7b0ee [Daoyuan Wang] remove useless import 606f981 [Daoyuan Wang] support TOK_CHARSETLITERAL in HiveQl --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 1 + .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 4 ++++ .../golden/inputddl5-0-ebbf2aec5f76af7225c2efaf870b8ba7 | 0 .../golden/inputddl5-1-2691407ccdc5c848a4ba2aecb6dbad75 | 0 .../golden/inputddl5-2-ca2faacf63dc4785f8bfd2ecc397e69b | 1 + .../golden/inputddl5-3-4f28c7412a05cff89c0bd86b65aa7ce | 1 + .../golden/inputddl5-4-bd7e25cff73f470d2e2336876342b783 | 1 + 7 files changed, 8 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/inputddl5-0-ebbf2aec5f76af7225c2efaf870b8ba7 create mode 100644 sql/hive/src/test/resources/golden/inputddl5-1-2691407ccdc5c848a4ba2aecb6dbad75 create mode 100644 sql/hive/src/test/resources/golden/inputddl5-2-ca2faacf63dc4785f8bfd2ecc397e69b create mode 100644 sql/hive/src/test/resources/golden/inputddl5-3-4f28c7412a05cff89c0bd86b65aa7ce create mode 100644 sql/hive/src/test/resources/golden/inputddl5-4-bd7e25cff73f470d2e2336876342b783 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index a6266f611c219..e443e5bd5f54d 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -518,6 +518,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl2", "inputddl3", "inputddl4", + "inputddl5", "inputddl6", "inputddl7", "inputddl8", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index f51af62d3340b..969868aef2917 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.sql.Date + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf @@ -1237,6 +1238,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case ast: ASTNode if ast.getType == HiveParser.TOK_DATELITERAL => Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1))) + case ast: ASTNode if ast.getType == HiveParser.TOK_CHARSETLITERAL => + Literal(BaseSemanticAnalyzer.charSetString(ast.getChild(0).getText, ast.getChild(1).getText)) + case a: ASTNode => throw new NotImplementedError( s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : diff --git a/sql/hive/src/test/resources/golden/inputddl5-0-ebbf2aec5f76af7225c2efaf870b8ba7 b/sql/hive/src/test/resources/golden/inputddl5-0-ebbf2aec5f76af7225c2efaf870b8ba7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl5-1-2691407ccdc5c848a4ba2aecb6dbad75 b/sql/hive/src/test/resources/golden/inputddl5-1-2691407ccdc5c848a4ba2aecb6dbad75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl5-2-ca2faacf63dc4785f8bfd2ecc397e69b b/sql/hive/src/test/resources/golden/inputddl5-2-ca2faacf63dc4785f8bfd2ecc397e69b new file mode 100644 index 0000000000000..518a70918b2c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl5-2-ca2faacf63dc4785f8bfd2ecc397e69b @@ -0,0 +1 @@ +name string diff --git a/sql/hive/src/test/resources/golden/inputddl5-3-4f28c7412a05cff89c0bd86b65aa7ce b/sql/hive/src/test/resources/golden/inputddl5-3-4f28c7412a05cff89c0bd86b65aa7ce new file mode 100644 index 0000000000000..33398360345d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl5-3-4f28c7412a05cff89c0bd86b65aa7ce @@ -0,0 +1 @@ +邵铮 diff --git a/sql/hive/src/test/resources/golden/inputddl5-4-bd7e25cff73f470d2e2336876342b783 b/sql/hive/src/test/resources/golden/inputddl5-4-bd7e25cff73f470d2e2336876342b783 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl5-4-bd7e25cff73f470d2e2336876342b783 @@ -0,0 +1 @@ +1 From 69bc3bb6cffe82aee5ecd0b09410a847ba486b15 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 10 Feb 2015 11:07:25 -0800 Subject: [PATCH 040/817] SPARK-4136. Under dynamic allocation, cancel outstanding executor requests when no longer needed This takes advantage of the changes made in SPARK-4337 to cancel pending requests to YARN when they are no longer needed. Each time the timer in `ExecutorAllocationManager` strikes, we compute `maxNumNeededExecutors`, the maximum number of executors we could fill with the current load. This is calculated as the total number of running and pending tasks divided by the number of cores per executor. If `maxNumNeededExecutors` is below the total number of running and pending executors, we call `requestTotalExecutors(maxNumNeededExecutors)` to let the cluster manager know that it should cancel any pending requests above this amount. If not, `maxNumNeededExecutors` is just used as a bound in alongside the configured `maxExecutors` to limit the number of new requests. The patch modifies the API exposed by `ExecutorAllocationClient` for requesting additional executors by moving from `requestExecutors` to `requestTotalExecutors`. This makes the communication between the `ExecutorAllocationManager` and the `YarnAllocator` easier to reason about and removes some state that needed to be kept in the `CoarseGrainedSchedulerBackend`. I think an argument can be made that this makes for a less attractive user-facing API in `SparkContext`, but I'm having trouble envisioning situations where a user would want to use either of these APIs. This will likely break some tests, but I wanted to get feedback on the approach before adding tests and polishing. Author: Sandy Ryza Closes #4168 from sryza/sandy-spark-4136 and squashes the following commits: 37ce77d [Sandy Ryza] Warn on negative number cd3b2ff [Sandy Ryza] SPARK-4136 --- .../spark/ExecutorAllocationClient.scala | 8 + .../spark/ExecutorAllocationManager.scala | 149 ++++++++++++------ .../scala/org/apache/spark/SparkContext.scala | 21 ++- .../CoarseGrainedSchedulerBackend.scala | 20 ++- .../ExecutorAllocationManagerSuite.scala | 36 ++++- 5 files changed, 184 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index a46a81eabd965..079055e00c6c3 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -19,9 +19,17 @@ package org.apache.spark /** * A client that communicates with the cluster manager to request or kill executors. + * This is currently supported only in YARN mode. */ private[spark] trait ExecutorAllocationClient { + /** + * Express a preference to the cluster manager for a given total number of executors. + * This can result in canceling pending requests or filing additional requests. + * Return whether the request is acknowledged by the cluster manager. + */ + private[spark] def requestTotalExecutors(numExecutors: Int): Boolean + /** * Request an additional number of executors from the cluster manager. * Return whether the request is acknowledged by the cluster manager. diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 02d54bf3b53cc..998695b6ac8ab 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -201,18 +201,34 @@ private[spark] class ExecutorAllocationManager( } /** - * If the add time has expired, request new executors and refresh the add time. - * If the remove time for an existing executor has expired, kill the executor. + * The number of executors we would have if the cluster manager were to fulfill all our existing + * requests. + */ + private def targetNumExecutors(): Int = + numExecutorsPending + executorIds.size - executorsPendingToRemove.size + + /** + * The maximum number of executors we would need under the current load to satisfy all running + * and pending tasks, rounded up. + */ + private def maxNumExecutorsNeeded(): Int = { + val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks + (numRunningOrPendingTasks + tasksPerExecutor - 1) / tasksPerExecutor + } + + /** + * This is called at a fixed interval to regulate the number of pending executor requests + * and number of executors running. + * + * First, adjust our requested executors based on the add time and our current needs. + * Then, if the remove time for an existing executor has expired, kill the executor. + * * This is factored out into its own method for testing. */ private def schedule(): Unit = synchronized { val now = clock.getTimeMillis - if (addTime != NOT_SET && now >= addTime) { - addExecutors() - logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 - } + + addOrCancelExecutorRequests(now) removeTimes.retain { case (executorId, expireTime) => val expired = now >= expireTime @@ -223,59 +239,89 @@ private[spark] class ExecutorAllocationManager( } } + /** + * Check to see whether our existing allocation and the requests we've made previously exceed our + * current needs. If so, let the cluster manager know so that it can cancel pending requests that + * are unneeded. + * + * If not, and the add time has expired, see if we can request new executors and refresh the add + * time. + * + * @return the delta in the target number of executors. + */ + private def addOrCancelExecutorRequests(now: Long): Int = synchronized { + val currentTarget = targetNumExecutors + val maxNeeded = maxNumExecutorsNeeded + + if (maxNeeded < currentTarget) { + // The target number exceeds the number we actually need, so stop adding new + // executors and inform the cluster manager to cancel the extra pending requests. + val newTotalExecutors = math.max(maxNeeded, minNumExecutors) + client.requestTotalExecutors(newTotalExecutors) + numExecutorsToAdd = 1 + updateNumExecutorsPending(newTotalExecutors) + } else if (addTime != NOT_SET && now >= addTime) { + val delta = addExecutors(maxNeeded) + logDebug(s"Starting timer to add more executors (to " + + s"expire in $sustainedSchedulerBacklogTimeout seconds)") + addTime += sustainedSchedulerBacklogTimeout * 1000 + delta + } else { + 0 + } + } + /** * Request a number of executors from the cluster manager. * If the cap on the number of executors is reached, give up and reset the * number of executors to add next round instead of continuing to double it. - * Return the number actually requested. + * + * @param maxNumExecutorsNeeded the maximum number of executors all currently running or pending + * tasks could fill + * @return the number of additional executors actually requested. */ - private def addExecutors(): Int = synchronized { - // Do not request more executors if we have already reached the upper bound - val numExistingExecutors = executorIds.size + numExecutorsPending - if (numExistingExecutors >= maxNumExecutors) { + private def addExecutors(maxNumExecutorsNeeded: Int): Int = { + // Do not request more executors if it would put our target over the upper bound + val currentTarget = targetNumExecutors + if (currentTarget >= maxNumExecutors) { logDebug(s"Not adding executors because there are already ${executorIds.size} " + s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)") numExecutorsToAdd = 1 return 0 } - // The number of executors needed to satisfy all pending tasks is the number of tasks pending - // divided by the number of tasks each executor can fit, rounded up. - val maxNumExecutorsPending = - (listener.totalPendingTasks() + tasksPerExecutor - 1) / tasksPerExecutor - if (numExecutorsPending >= maxNumExecutorsPending) { - logDebug(s"Not adding executors because there are already $numExecutorsPending " + - s"pending and pending tasks could only fill $maxNumExecutorsPending") - numExecutorsToAdd = 1 - return 0 - } - - // It's never useful to request more executors than could satisfy all the pending tasks, so - // cap request at that amount. - // Also cap request with respect to the configured upper bound. - val maxNumExecutorsToAdd = math.min( - maxNumExecutorsPending - numExecutorsPending, - maxNumExecutors - numExistingExecutors) - assert(maxNumExecutorsToAdd > 0) - - val actualNumExecutorsToAdd = math.min(numExecutorsToAdd, maxNumExecutorsToAdd) - - val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd - val addRequestAcknowledged = testing || client.requestExecutors(actualNumExecutorsToAdd) + val actualMaxNumExecutors = math.min(maxNumExecutors, maxNumExecutorsNeeded) + val newTotalExecutors = math.min(currentTarget + numExecutorsToAdd, actualMaxNumExecutors) + val addRequestAcknowledged = testing || client.requestTotalExecutors(newTotalExecutors) if (addRequestAcknowledged) { - logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " + - s"tasks are backlogged (new desired total will be $newTotalExecutors)") - numExecutorsToAdd = - if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1 - numExecutorsPending += actualNumExecutorsToAdd - actualNumExecutorsToAdd + val delta = updateNumExecutorsPending(newTotalExecutors) + logInfo(s"Requesting $delta new executor(s) because tasks are backlogged" + + s" (new desired total will be $newTotalExecutors)") + numExecutorsToAdd = if (delta == numExecutorsToAdd) { + numExecutorsToAdd * 2 + } else { + 1 + } + delta } else { - logWarning(s"Unable to reach the cluster manager " + - s"to request $actualNumExecutorsToAdd executors!") + logWarning( + s"Unable to reach the cluster manager to request $newTotalExecutors total executors!") 0 } } + /** + * Given the new target number of executors, update the number of pending executor requests, + * and return the delta from the old number of pending requests. + */ + private def updateNumExecutorsPending(newTotalExecutors: Int): Int = { + val newNumExecutorsPending = + newTotalExecutors - executorIds.size + executorsPendingToRemove.size + val delta = newNumExecutorsPending - numExecutorsPending + numExecutorsPending = newNumExecutorsPending + delta + } + /** * Request the cluster manager to remove the given executor. * Return whether the request is received. @@ -415,6 +461,8 @@ private[spark] class ExecutorAllocationManager( private val stageIdToNumTasks = new mutable.HashMap[Int, Int] private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] + // Number of tasks currently running on the cluster. Should be 0 when no stages are active. + private var numRunningTasks: Int = _ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { val stageId = stageSubmitted.stageInfo.stageId @@ -435,6 +483,10 @@ private[spark] class ExecutorAllocationManager( // This is needed in case the stage is aborted for any reason if (stageIdToNumTasks.isEmpty) { allocationManager.onSchedulerQueueEmpty() + if (numRunningTasks != 0) { + logWarning("No stages are running, but numRunningTasks != 0") + numRunningTasks = 0 + } } } } @@ -446,6 +498,7 @@ private[spark] class ExecutorAllocationManager( val executorId = taskStart.taskInfo.executorId allocationManager.synchronized { + numRunningTasks += 1 // This guards against the race condition in which the `SparkListenerTaskStart` // event is posted before the `SparkListenerBlockManagerAdded` event, which is // possible because these events are posted in different threads. (see SPARK-4951) @@ -475,7 +528,8 @@ private[spark] class ExecutorAllocationManager( val executorId = taskEnd.taskInfo.executorId val taskId = taskEnd.taskInfo.taskId allocationManager.synchronized { - // If the executor is no longer running scheduled any tasks, mark it as idle + numRunningTasks -= 1 + // If the executor is no longer running any scheduled tasks, mark it as idle if (executorIdToTaskIds.contains(executorId)) { executorIdToTaskIds(executorId) -= taskId if (executorIdToTaskIds(executorId).isEmpty) { @@ -514,6 +568,11 @@ private[spark] class ExecutorAllocationManager( }.sum } + /** + * The number of tasks currently running across all stages. + */ + def totalRunningTasks(): Int = numRunningTasks + /** * Return true if an executor is not currently running a task, and false otherwise. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8d3c3d000adf3..04ca5d1019e4b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1103,10 +1103,27 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli listenerBus.addListener(listener) } + /** + * Express a preference to the cluster manager for a given total number of executors. + * This can result in canceling pending requests or filing additional requests. + * This is currently only supported in YARN mode. Return whether the request is received. + */ + private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Requesting executors is currently only supported in YARN mode") + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => + b.requestTotalExecutors(numExecutors) + case _ => + logWarning("Requesting executors is only supported in coarse-grained mode") + false + } + } + /** * :: DeveloperApi :: * Request an additional number of executors from the cluster manager. - * This is currently only supported in Yarn mode. Return whether the request is received. + * This is currently only supported in YARN mode. Return whether the request is received. */ @DeveloperApi override def requestExecutors(numAdditionalExecutors: Int): Boolean = { @@ -1124,7 +1141,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** * :: DeveloperApi :: * Request that the cluster manager kill the specified executors. - * This is currently only supported in Yarn mode. Return whether the request is received. + * This is currently only supported in YARN mode. Return whether the request is received. */ @DeveloperApi override def killExecutors(executorIds: Seq[String]): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index f9ca93432bf41..99986c32b0fde 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -311,7 +311,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste /** * Request an additional number of executors from the cluster manager. - * Return whether the request is acknowledged. + * @return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { if (numAdditionalExecutors < 0) { @@ -327,6 +327,22 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste doRequestTotalExecutors(newTotal) } + /** + * Express a preference to the cluster manager for a given total number of executors. This can + * result in canceling pending requests or filing additional requests. + * @return whether the request is acknowledged. + */ + final override def requestTotalExecutors(numExecutors: Int): Boolean = synchronized { + if (numAdditionalExecutors < 0) { + throw new IllegalArgumentException( + "Attempted to request a negative number of executor(s) " + + s"$numExecutors from the cluster manager. Please specify a positive number!") + } + numPendingExecutors = + math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) + doRequestTotalExecutors(numExecutors) + } + /** * Request executors from the cluster manager by specifying the total number desired, * including existing pending and running executors. @@ -337,7 +353,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * insufficient resources to satisfy the first request. We make the assumption here that the * cluster manager will eventually fulfill all requests when resources free up. * - * Return whether the request is acknowledged. + * @return whether the request is acknowledged. */ protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 9eb87f016068d..5d96eabd34eee 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -175,6 +175,33 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(numExecutorsPending(manager) === 9) } + test("cancel pending executors when no longer needed") { + sc = createSparkContext(1, 10) + val manager = sc.executorAllocationManager.get + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 5))) + + assert(numExecutorsPending(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 1) + assert(numExecutorsPending(manager) === 1) + assert(numExecutorsToAdd(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 3) + + val task1Info = createTaskInfo(0, 0, "executor-1") + sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task1Info)) + + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 2) + + val task2Info = createTaskInfo(1, 0, "executor-1") + sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info)) + sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task1Info, null)) + sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, null, task2Info, null)) + + assert(adjustRequestedExecutors(manager) === -1) + } + test("remove executors") { sc = createSparkContext(5, 10) val manager = sc.executorAllocationManager.get @@ -679,6 +706,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending) + private val _maxNumExecutorsNeeded = PrivateMethod[Int]('maxNumExecutorsNeeded) private val _executorsPendingToRemove = PrivateMethod[collection.Set[String]]('executorsPendingToRemove) private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds) @@ -686,6 +714,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) private val _schedule = PrivateMethod[Unit]('schedule) private val _addExecutors = PrivateMethod[Int]('addExecutors) + private val _addOrCancelExecutorRequests = PrivateMethod[Int]('addOrCancelExecutorRequests) private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) @@ -724,7 +753,12 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { } private def addExecutors(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _addExecutors() + val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded() + manager invokePrivate _addExecutors(maxNumExecutorsNeeded) + } + + private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _addOrCancelExecutorRequests(0L) } private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { From b640c841fca92bb0bca77267db2965ff8f79586f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 10 Feb 2015 11:18:01 -0800 Subject: [PATCH 041/817] [HOTFIX][SPARK-4136] Fix compilation and tests --- .../org/apache/spark/ExecutorAllocationClient.scala | 8 ++++---- .../cluster/CoarseGrainedSchedulerBackend.scala | 2 +- .../apache/spark/ExecutorAllocationManagerSuite.scala | 10 ++++------ 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index 079055e00c6c3..443830f8d03b6 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -26,25 +26,25 @@ private[spark] trait ExecutorAllocationClient { /** * Express a preference to the cluster manager for a given total number of executors. * This can result in canceling pending requests or filing additional requests. - * Return whether the request is acknowledged by the cluster manager. + * @return whether the request is acknowledged by the cluster manager. */ private[spark] def requestTotalExecutors(numExecutors: Int): Boolean /** * Request an additional number of executors from the cluster manager. - * Return whether the request is acknowledged by the cluster manager. + * @return whether the request is acknowledged by the cluster manager. */ def requestExecutors(numAdditionalExecutors: Int): Boolean /** * Request that the cluster manager kill the specified executors. - * Return whether the request is acknowledged by the cluster manager. + * @return whether the request is acknowledged by the cluster manager. */ def killExecutors(executorIds: Seq[String]): Boolean /** * Request that the cluster manager kill the specified executor. - * Return whether the request is acknowledged by the cluster manager. + * @return whether the request is acknowledged by the cluster manager. */ def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 99986c32b0fde..6f77fa32ce37b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -333,7 +333,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * @return whether the request is acknowledged. */ final override def requestTotalExecutors(numExecutors: Int): Boolean = synchronized { - if (numAdditionalExecutors < 0) { + if (numExecutors < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of executor(s) " + s"$numExecutors from the cluster manager. Please specify a positive number!") diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 5d96eabd34eee..d3123e854016b 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -297,15 +297,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeExecutor(manager, "5")) assert(removeExecutor(manager, "6")) assert(executorIds(manager).size === 10) - assert(addExecutors(manager) === 0) // still at upper limit + assert(addExecutors(manager) === 1) onExecutorRemoved(manager, "3") onExecutorRemoved(manager, "4") assert(executorIds(manager).size === 8) // Add succeeds again, now that we are no longer at the upper limit // Number of executors added restarts at 1 - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 1) // upper limit reached assert(addExecutors(manager) === 0) assert(executorIds(manager).size === 8) onExecutorRemoved(manager, "5") @@ -313,9 +313,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onExecutorAdded(manager, "13") onExecutorAdded(manager, "14") assert(executorIds(manager).size === 8) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) // upper limit reached again - assert(addExecutors(manager) === 0) + assert(addExecutors(manager) === 0) // still at upper limit onExecutorAdded(manager, "15") onExecutorAdded(manager, "16") assert(executorIds(manager).size === 10) From 59272dad77eb95c5ae8e0652e00d02a2675cda53 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 10 Feb 2015 11:54:30 -0800 Subject: [PATCH 042/817] [SPARK-5592][SQL] java.net.URISyntaxException when insert data to a partitioned table flowing sql get URISyntaxException: ``` create table sc as select * from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) union all select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) union all select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s; create table sc_part (key string) partitioned by (ts string) stored as rcfile; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; insert overwrite table sc_part partition(ts) select * from sc; ``` java.net.URISyntaxException: Relative path in absolute URI: ts=2011-01-11+15:18:26 at org.apache.hadoop.fs.Path.initialize(Path.java:206) at org.apache.hadoop.fs.Path.(Path.java:172) at org.apache.hadoop.fs.Path.(Path.java:94) at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.org$apache$spark$sql$hive$SparkHiveDynamicPartitionWriterContainer$$newWriter$1(hiveWriterContainers.scala:230) at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer$$anonfun$getLocalFileWriter$1.apply(hiveWriterContainers.scala:243) at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer$$anonfun$getLocalFileWriter$1.apply(hiveWriterContainers.scala:243) at scala.collection.mutable.MapLike$class.getOrElseUpdate(MapLike.scala:189) at scala.collection.mutable.AbstractMap.getOrElseUpdate(Map.scala:91) at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.getLocalFileWriter(hiveWriterContainers.scala:243) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1$1.apply(InsertIntoHiveTable.scala:113) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1$1.apply(InsertIntoHiveTable.scala:105) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1(InsertIntoHiveTable.scala:105) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:87) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:87) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61) at org.apache.spark.scheduler.Task.run(Task.scala:64) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:194) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) at java.lang.Thread.run(Thread.java:722) Caused by: java.net.URISyntaxException: Relative path in absolute URI: ts=2011-01-11+15:18:26 at java.net.URI.checkPath(URI.java:1804) at java.net.URI.(URI.java:752) at org.apache.hadoop.fs.Path.initialize(Path.java:203) Author: wangfei Author: Fei Wang Closes #4368 from scwf/SPARK-5592 and squashes the following commits: aa55ef4 [Fei Wang] comments addressed f8f8bb1 [wangfei] added test case f24624f [wangfei] Merge branch 'master' of https://github.com/apache/spark into SPARK-5592 9998177 [wangfei] added test case ea81daf [wangfei] fix URISyntaxException --- .../spark/sql/hive/hiveWriterContainers.scala | 12 +++++++++--- .../sql/hive/execution/HiveQuerySuite.scala | 16 ++++++++++++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index aae175e426ade..f136e43acc8f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.hadoop.hive.common.FileUtils import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row @@ -212,9 +213,14 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( .zip(row.toSeq.takeRight(dynamicPartColNames.length)) .map { case (col, rawVal) => val string = if (rawVal == null) null else String.valueOf(rawVal) - s"/$col=${if (string == null || string.isEmpty) defaultPartName else string}" - } - .mkString + val colString = + if (string == null || string.isEmpty) { + defaultPartName + } else { + FileUtils.escapePathName(string) + } + s"/$col=$colString" + }.mkString def newWriter = { val newFileSinkDesc = new FileSinkDesc( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 27047ce4b1b0b..405b200d05412 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -859,6 +859,22 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + test("SPARK-5592: get java.net.URISyntaxException when dynamic partitioning") { + sql(""" + |create table sc as select * + |from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) + |union all + |select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) + |union all + |select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s + """.stripMargin) + sql("create table sc_part (key string) partitioned by (ts string) stored as rcfile") + sql("set hive.exec.dynamic.partition=true") + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("insert overwrite table sc_part partition(ts) select * from sc") + sql("drop table sc_part") + } + test("Partition spec validation") { sql("DROP TABLE IF EXISTS dp_test") sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)") From c49a4049845c91b225e70fd630cdf6ddc055faf8 Mon Sep 17 00:00:00 2001 From: Miguel Peralvo Date: Tue, 10 Feb 2015 19:54:52 +0000 Subject: [PATCH 043/817] [SPARK-5668] Display region in spark_ec2.py get_existing_cluster() Show the region for the different messages displayed by get_existing_cluster(): The search, found and error messages. Author: Miguel Peralvo Closes #4457 from MiguelPeralvo/patch-2 and squashes the following commits: a5514c8 [Miguel Peralvo] Update spark_ec2.py 0a837b0 [Miguel Peralvo] Update spark_ec2.py 3923f36 [Miguel Peralvo] Update spark_ec2.py 4ecd9f9 [Miguel Peralvo] [SPARK-5668] Display region in spark_ec2.py get_existing_cluster() --- ec2/spark_ec2.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index fe510f12bcec6..0ea7365d75b83 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -134,7 +134,7 @@ def parse_args(): help="Master instance type (leave empty for same as instance-type)") parser.add_option( "-r", "--region", default="us-east-1", - help="EC2 region zone to launch instances in") + help="EC2 region used to launch instances in, or to find them in") parser.add_option( "-z", "--zone", default="", help="Availability zone to launch instances in, or 'all' to spread " + @@ -614,7 +614,8 @@ def launch_cluster(conn, opts, cluster_name): # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): - print "Searching for existing cluster " + cluster_name + "..." + print "Searching for existing cluster " + cluster_name + " in region " \ + + opts.region + "..." reservations = conn.get_all_reservations() master_nodes = [] slave_nodes = [] @@ -632,9 +633,11 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name \ + + "-master" + " in region " + opts.region else: - print >> sys.stderr, "ERROR: Could not find any existing cluster" + print >> sys.stderr, "ERROR: Could not find any existing cluster" \ + + " in region " + opts.region sys.exit(1) From de80b1ba4d3c4b1b3316d482d62e4668b996f6ac Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 10 Feb 2015 13:14:01 -0800 Subject: [PATCH 044/817] [SQL] Add toString to DataFrame/Column Author: Michael Armbrust Closes #4436 from marmbrus/dfToString and squashes the following commits: 8a3c35f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into dfToString b72a81b [Michael Armbrust] add toString --- python/pyspark/sql/dataframe.py | 2 +- .../sql/catalyst/expressions/Expression.scala | 12 ++++++++ .../expressions/namedExpressions.scala | 20 +++++++++++++ .../org/apache/spark/sql/DataFrame.scala | 8 +++++ .../org/apache/spark/sql/DataFrameImpl.scala | 10 +++---- .../apache/spark/sql/IncomputableColumn.scala | 2 ++ .../spark/sql/execution/debug/package.scala | 11 ++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 29 +++++++++++++++++++ 8 files changed, 86 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index cda704eea75f5..04be65fe241c4 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -447,7 +447,7 @@ def selectExpr(self, *expr): `select` that accepts SQL expressions. >>> df.selectExpr("age * 2", "abs(age)").collect() - [Row(('age * 2)=4, Abs('age)=2), Row(('age * 2)=10, Abs('age)=5)] + [Row((age * 2)=4, Abs(age)=2), Row((age * 2)=10, Abs(age)=5)] """ jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index cf14992ef835c..c32a4b886eb82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode @@ -66,6 +67,17 @@ abstract class Expression extends TreeNode[Expression] { */ def childrenResolved = !children.exists(!_.resolved) + /** + * Returns a string representation of this expression that does not have developer centric + * debugging information like the expression id. + */ + def prettyString: String = { + transform { + case a: AttributeReference => PrettyAttribute(a.name) + case u: UnresolvedAttribute => PrettyAttribute(u.name) + }.toString + } + /** * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type * and do any casting necessary of child evaluation. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index e6ab1fd8d7939..7f122e9d55734 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -190,6 +190,26 @@ case class AttributeReference( override def toString: String = s"$name#${exprId.id}$typeSuffix" } +/** + * A place holder used when printing expressions without debugging information such as the + * expression id or the unresolved indicator. + */ +case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { + type EvaluatedType = Any + + override def toString = name + + override def withNullability(newNullability: Boolean): Attribute = ??? + override def newInstance(): Attribute = ??? + override def withQualifiers(newQualifiers: Seq[String]): Attribute = ??? + override def withName(newName: String): Attribute = ??? + override def qualifiers: Seq[String] = ??? + override def exprId: ExprId = ??? + override def eval(input: Row): EvaluatedType = ??? + override def nullable: Boolean = ??? + override def dataType: DataType = ??? +} + object VirtualColumn { val groupingIdName = "grouping__id" def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)() 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 6abfb7853cf1c..04e0d09947492 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 @@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils +import scala.util.control.NonFatal + private[sql] object DataFrame { def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = { @@ -92,6 +94,12 @@ trait DataFrame extends RDDApi[Row] { */ def toDataFrame: DataFrame = this + override def toString = + try schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]") catch { + case NonFatal(e) => + s"Invalid tree; ${e.getMessage}:\n$queryExecution" + } + /** * Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion * from a RDD of tuples into a [[DataFrame]] with meaningful names. For example: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 73393295ab0a5..1ee16ad5161c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -201,13 +201,11 @@ private[sql] class DataFrameImpl protected[sql]( override def as(alias: Symbol): DataFrame = Subquery(alias.name, logicalPlan) override def select(cols: Column*): DataFrame = { - val exprs = cols.zipWithIndex.map { - case (Column(expr: NamedExpression), _) => - expr - case (Column(expr: Expression), _) => - Alias(expr, expr.toString)() + val namedExpressions = cols.map { + case Column(expr: NamedExpression) => expr + case Column(expr: Expression) => Alias(expr, expr.prettyString)() } - Project(exprs.toSeq, logicalPlan) + Project(namedExpressions.toSeq, logicalPlan) } override def select(col: String, cols: String*): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 0600dcc226b4d..ce0557b88196f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -40,6 +40,8 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten throw new UnsupportedOperationException("Cannot run this method on an UncomputableColumn") } + override def toString = expr.prettyString + override def isComputable: Boolean = false override val sqlContext: SQLContext = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 5cc67cdd13944..acef49aabfe70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.HashSet import org.apache.spark.{AccumulatorParam, Accumulator, SparkContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext._ -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{SQLConf, SQLContext, DataFrame, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.types._ @@ -37,6 +37,15 @@ import org.apache.spark.sql.types._ */ package object debug { + /** + * Augments [[SQLContext]] with debug methods. + */ + implicit class DebugSQLContext(sqlContext: SQLContext) { + def debug() = { + sqlContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "false") + } + } + /** * :: DeveloperApi :: * Augments [[DataFrame]]s with debug methods. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 5aa3db720c886..02623f73c7f76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.TestData._ + import scala.language.postfixOps import org.apache.spark.sql.Dsl._ @@ -53,6 +55,33 @@ class DataFrameSuite extends QueryTest { TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } + test("dataframe toString") { + assert(testData.toString === "[key: int, value: string]") + assert(testData("key").toString === "[key: int]") + } + + test("incomputable toString") { + assert($"test".toString === "test") + } + + test("invalid plan toString, debug mode") { + val oldSetting = TestSQLContext.conf.dataFrameEagerAnalysis + TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "true") + + // Turn on debug mode so we can see invalid query plans. + import org.apache.spark.sql.execution.debug._ + TestSQLContext.debug() + + val badPlan = testData.select('badColumn) + + assert(badPlan.toString contains badPlan.queryExecution.toString, + "toString on bad query plans should include the query execution but was:\n" + + badPlan.toString) + + // Set the flag back to original value before this test. + TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) + } + test("table scan") { checkAnswer( testData, From f98707c043f1be9569ec774796edb783132773a8 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Tue, 10 Feb 2015 13:20:15 -0800 Subject: [PATCH 045/817] [SPARK-5686][SQL] Add show current roles command in HiveQl show current roles Author: OopsOutOfMemory Closes #4471 from OopsOutOfMemory/show_current_role and squashes the following commits: 1c6b210 [OopsOutOfMemory] add show current roles --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 969868aef2917..8618301ba84d6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -77,6 +77,7 @@ private[hive] object HiveQl { "TOK_REVOKE", "TOK_SHOW_GRANT", "TOK_SHOW_ROLE_GRANT", + "TOK_SHOW_SET_ROLE", "TOK_CREATEFUNCTION", "TOK_DROPFUNCTION", From fd2c032f95bbee342ca539df9e44927482981659 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 10 Feb 2015 14:05:55 -0800 Subject: [PATCH 046/817] [SPARK-5021] [MLlib] Gaussian Mixture now supports Sparse Input Following discussion in the Jira. Author: MechCoder Closes #4459 from MechCoder/sparse_gmm and squashes the following commits: 1b18dab [MechCoder] Rewrite syr for sparse matrices e579041 [MechCoder] Add test for covariance matrix 5cb370b [MechCoder] Separate tests for sparse data 5e096bd [MechCoder] Alphabetize and correct error message e180f4c [MechCoder] [SPARK-5021] Gaussian Mixture now supports Sparse Input --- .../mllib/clustering/GaussianMixture.scala | 31 ++++----- .../org/apache/spark/mllib/linalg/BLAS.scala | 36 +++++++++- .../distribution/MultivariateGaussian.scala | 10 +-- .../clustering/GaussianMixtureSuite.scala | 66 +++++++++++++++++-- .../apache/spark/mllib/linalg/BLASSuite.scala | 8 +++ 5 files changed, 125 insertions(+), 26 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 0be3014de862e..80584ef5e5979 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -19,10 +19,12 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.IndexedSeq -import breeze.linalg.{DenseMatrix => BreezeMatrix, DenseVector => BreezeVector, Transpose, diag} +import breeze.linalg.{diag, DenseMatrix => BreezeMatrix, DenseVector => BDV, SparseVector => BSV, + Transpose, Vector => BV} import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, Matrices, Vector, Vectors} +import org.apache.spark.mllib.linalg.{BLAS, DenseVector, DenseMatrix, Matrices, + SparseVector, Vector, Vectors} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD @@ -130,7 +132,7 @@ class GaussianMixture private ( val sc = data.sparkContext // we will operate on the data as breeze data - val breezeData = data.map(u => u.toBreeze.toDenseVector).cache() + val breezeData = data.map(_.toBreeze).cache() // Get length of the input vectors val d = breezeData.first().length @@ -148,7 +150,7 @@ class GaussianMixture private ( (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => val slice = samples.view(i * nSamples, (i + 1) * nSamples) new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) - }) + }) } } @@ -169,7 +171,7 @@ class GaussianMixture private ( var i = 0 while (i < k) { val mu = sums.means(i) / sums.weights(i) - BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu).asInstanceOf[DenseVector], + BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu), Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix]) weights(i) = sums.weights(i) / sumWeights gaussians(i) = new MultivariateGaussian(mu, sums.sigmas(i) / sums.weights(i)) @@ -185,8 +187,8 @@ class GaussianMixture private ( } /** Average of dense breeze vectors */ - private def vectorMean(x: IndexedSeq[BreezeVector[Double]]): BreezeVector[Double] = { - val v = BreezeVector.zeros[Double](x(0).length) + private def vectorMean(x: IndexedSeq[BV[Double]]): BDV[Double] = { + val v = BDV.zeros[Double](x(0).length) x.foreach(xi => v += xi) v / x.length.toDouble } @@ -195,10 +197,10 @@ class GaussianMixture private ( * Construct matrix where diagonal entries are element-wise * variance of input vectors (computes biased variance) */ - private def initCovariance(x: IndexedSeq[BreezeVector[Double]]): BreezeMatrix[Double] = { + private def initCovariance(x: IndexedSeq[BV[Double]]): BreezeMatrix[Double] = { val mu = vectorMean(x) - val ss = BreezeVector.zeros[Double](x(0).length) - x.map(xi => (xi - mu) :^ 2.0).foreach(u => ss += u) + val ss = BDV.zeros[Double](x(0).length) + x.foreach(xi => ss += (xi - mu) :^ 2.0) diag(ss / x.length.toDouble) } } @@ -207,7 +209,7 @@ class GaussianMixture private ( private object ExpectationSum { def zero(k: Int, d: Int): ExpectationSum = { new ExpectationSum(0.0, Array.fill(k)(0.0), - Array.fill(k)(BreezeVector.zeros(d)), Array.fill(k)(BreezeMatrix.zeros(d,d))) + Array.fill(k)(BDV.zeros(d)), Array.fill(k)(BreezeMatrix.zeros(d,d))) } // compute cluster contributions for each input point @@ -215,19 +217,18 @@ private object ExpectationSum { def add( weights: Array[Double], dists: Array[MultivariateGaussian]) - (sums: ExpectationSum, x: BreezeVector[Double]): ExpectationSum = { + (sums: ExpectationSum, x: BV[Double]): ExpectationSum = { val p = weights.zip(dists).map { case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(x) } val pSum = p.sum sums.logLikelihood += math.log(pSum) - val xxt = x * new Transpose(x) var i = 0 while (i < sums.k) { p(i) /= pSum sums.weights(i) += p(i) sums.means(i) += x * p(i) - BLAS.syr(p(i), Vectors.fromBreeze(x).asInstanceOf[DenseVector], + BLAS.syr(p(i), Vectors.fromBreeze(x), Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix]) i = i + 1 } @@ -239,7 +240,7 @@ private object ExpectationSum { private class ExpectationSum( var logLikelihood: Double, val weights: Array[Double], - val means: Array[BreezeVector[Double]], + val means: Array[BDV[Double]], val sigmas: Array[BreezeMatrix[Double]]) extends Serializable { val k = weights.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 079f7ca564a92..87052e1ba8539 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -235,12 +235,24 @@ private[spark] object BLAS extends Serializable with Logging { * @param x the vector x that contains the n elements. * @param A the symmetric matrix A. Size of n x n. */ - def syr(alpha: Double, x: DenseVector, A: DenseMatrix) { + def syr(alpha: Double, x: Vector, A: DenseMatrix) { val mA = A.numRows val nA = A.numCols - require(mA == nA, s"A is not a symmetric matrix. A: $mA x $nA") + require(mA == nA, s"A is not a square matrix (and hence is not symmetric). A: $mA x $nA") require(mA == x.size, s"The size of x doesn't match the rank of A. A: $mA x $nA, x: ${x.size}") + x match { + case dv: DenseVector => syr(alpha, dv, A) + case sv: SparseVector => syr(alpha, sv, A) + case _ => + throw new IllegalArgumentException(s"syr doesn't support vector type ${x.getClass}.") + } + } + + private def syr(alpha: Double, x: DenseVector, A: DenseMatrix) { + val nA = A.numRows + val mA = A.numCols + nativeBLAS.dsyr("U", x.size, alpha, x.values, 1, A.values, nA) // Fill lower triangular part of A @@ -255,6 +267,26 @@ private[spark] object BLAS extends Serializable with Logging { } } + private def syr(alpha: Double, x: SparseVector, A: DenseMatrix) { + val mA = A.numCols + val xIndices = x.indices + val xValues = x.values + val nnz = xValues.length + val Avalues = A.values + + var i = 0 + while (i < nnz) { + val multiplier = alpha * xValues(i) + val offset = xIndices(i) * mA + var j = 0 + while (j < nnz) { + Avalues(xIndices(j) + offset) += multiplier * xValues(j) + j += 1 + } + i += 1 + } + } + /** * C := alpha * A * B + beta * C * @param alpha a scalar to scale the multiplication A * B. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index fd186b5ee6f72..cd6add9d60b0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.stat.distribution -import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym} +import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym, Vector => BV} import org.apache.spark.annotation.DeveloperApi; import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} @@ -62,21 +62,21 @@ class MultivariateGaussian ( /** Returns density of this multivariate Gaussian at given point, x */ def pdf(x: Vector): Double = { - pdf(x.toBreeze.toDenseVector) + pdf(x.toBreeze) } /** Returns the log-density of this multivariate Gaussian at given point, x */ def logpdf(x: Vector): Double = { - logpdf(x.toBreeze.toDenseVector) + logpdf(x.toBreeze) } /** Returns density of this multivariate Gaussian at given point, x */ - private[mllib] def pdf(x: DBV[Double]): Double = { + private[mllib] def pdf(x: BV[Double]): Double = { math.exp(logpdf(x)) } /** Returns the log-density of this multivariate Gaussian at given point, x */ - private[mllib] def logpdf(x: DBV[Double]): Double = { + private[mllib] def logpdf(x: BV[Double]): Double = { val delta = x - breezeMu val v = rootSigmaInv * delta u + v.t * v * -0.5 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index c2cd56ea40adc..1b46a4012d731 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -31,7 +31,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { Vectors.dense(5.0, 10.0), Vectors.dense(4.0, 11.0) )) - + // expectations val Ew = 1.0 val Emu = Vectors.dense(5.0, 10.0) @@ -44,6 +44,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { assert(gmm.gaussians(0).mu ~== Emu absTol 1E-5) assert(gmm.gaussians(0).sigma ~== Esigma absTol 1E-5) } + } test("two clusters") { @@ -54,7 +55,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) )) - + // we set an initial gaussian to induce expected results val initialGmm = new GaussianMixtureModel( Array(0.5, 0.5), @@ -63,7 +64,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { new MultivariateGaussian(Vectors.dense(1.0), Matrices.dense(1, 1, Array(1.0))) ) ) - + val Ew = Array(1.0 / 3.0, 2.0 / 3.0) val Emu = Array(Vectors.dense(-4.3673), Vectors.dense(5.1604)) val Esigma = Array(Matrices.dense(1, 1, Array(1.1098)), Matrices.dense(1, 1, Array(0.86644))) @@ -72,7 +73,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { .setK(2) .setInitialModel(initialGmm) .run(data) - + assert(gmm.weights(0) ~== Ew(0) absTol 1E-3) assert(gmm.weights(1) ~== Ew(1) absTol 1E-3) assert(gmm.gaussians(0).mu ~== Emu(0) absTol 1E-3) @@ -80,4 +81,61 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { assert(gmm.gaussians(0).sigma ~== Esigma(0) absTol 1E-3) assert(gmm.gaussians(1).sigma ~== Esigma(1) absTol 1E-3) } + + test("single cluster with sparse data") { + val data = sc.parallelize(Array( + Vectors.sparse(3, Array(0, 2), Array(4.0, 2.0)), + Vectors.sparse(3, Array(0, 2), Array(2.0, 4.0)), + Vectors.sparse(3, Array(1), Array(6.0)) + )) + + val Ew = 1.0 + val Emu = Vectors.dense(2.0, 2.0, 2.0) + val Esigma = Matrices.dense(3, 3, + Array(8.0 / 3.0, -4.0, 4.0 / 3.0, -4.0, 8.0, -4.0, 4.0 / 3.0, -4.0, 8.0 / 3.0) + ) + + val seeds = Array(42, 1994, 27, 11, 0) + seeds.foreach { seed => + val gmm = new GaussianMixture().setK(1).setSeed(seed).run(data) + assert(gmm.weights(0) ~== Ew absTol 1E-5) + assert(gmm.gaussians(0).mu ~== Emu absTol 1E-5) + assert(gmm.gaussians(0).sigma ~== Esigma absTol 1E-5) + } + } + + test("two clusters with sparse data") { + val data = sc.parallelize(Array( + Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), + Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), + Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), + Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), + Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) + )) + + val sparseData = data.map(point => Vectors.sparse(1, Array(0), point.toArray)) + // we set an initial gaussian to induce expected results + val initialGmm = new GaussianMixtureModel( + Array(0.5, 0.5), + Array( + new MultivariateGaussian(Vectors.dense(-1.0), Matrices.dense(1, 1, Array(1.0))), + new MultivariateGaussian(Vectors.dense(1.0), Matrices.dense(1, 1, Array(1.0))) + ) + ) + val Ew = Array(1.0 / 3.0, 2.0 / 3.0) + val Emu = Array(Vectors.dense(-4.3673), Vectors.dense(5.1604)) + val Esigma = Array(Matrices.dense(1, 1, Array(1.1098)), Matrices.dense(1, 1, Array(0.86644))) + + val sparseGMM = new GaussianMixture() + .setK(2) + .setInitialModel(initialGmm) + .run(data) + + assert(sparseGMM.weights(0) ~== Ew(0) absTol 1E-3) + assert(sparseGMM.weights(1) ~== Ew(1) absTol 1E-3) + assert(sparseGMM.gaussians(0).mu ~== Emu(0) absTol 1E-3) + assert(sparseGMM.gaussians(1).mu ~== Emu(1) absTol 1E-3) + assert(sparseGMM.gaussians(0).sigma ~== Esigma(0) absTol 1E-3) + assert(sparseGMM.gaussians(1).sigma ~== Esigma(1) absTol 1E-3) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index b0b78acd6df16..002cb253862b5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -166,6 +166,14 @@ class BLASSuite extends FunSuite { syr(alpha, y, dA) } } + + val xSparse = new SparseVector(4, Array(0, 2, 3), Array(1.0, 3.0, 4.0)) + val dD = new DenseMatrix(4, 4, + Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8)) + syr(0.1, xSparse, dD) + val expectedSparse = new DenseMatrix(4, 4, + Array(0.1, 1.2, 2.5, 3.5, 1.2, 3.2, 5.3, 4.6, 2.5, 5.3, 2.7, 4.2, 3.5, 4.6, 4.2, 2.4)) + assert(dD ~== expectedSparse absTol 1e-15) } test("gemm") { From 5820961289eb98e45eb467efa316c7592b8d619c Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 10 Feb 2015 14:57:00 -0800 Subject: [PATCH 047/817] [SPARK-5343][GraphX]: ShortestPaths traverses backwards Corrected the logic with ShortestPaths so that the calculation will run forward rather than backwards. Output before looked like: ```scala import org.apache.spark.graphx._ val g = Graph(sc.makeRDD(Array((1L,""), (2L,""), (3L,""))), sc.makeRDD(Array(Edge(1L,2L,""), Edge(2L,3L,"")))) lib.ShortestPaths.run(g,Array(3)).vertices.collect // res0: Array[(org.apache.spark.graphx.VertexId, org.apache.spark.graphx.lib.ShortestPaths.SPMap)] = Array((1,Map()), (3,Map(3 -> 0)), (2,Map())) lib.ShortestPaths.run(g,Array(1)).vertices.collect // res1: Array[(org.apache.spark.graphx.VertexId, org.apache.spark.graphx.lib.ShortestPaths.SPMap)] = Array((1,Map(1 -> 0)), (3,Map(1 -> 2)), (2,Map(1 -> 1))) ``` And new output after the changes looks like: ```scala import org.apache.spark.graphx._ val g = Graph(sc.makeRDD(Array((1L,""), (2L,""), (3L,""))), sc.makeRDD(Array(Edge(1L,2L,""), Edge(2L,3L,"")))) lib.ShortestPaths.run(g,Array(3)).vertices.collect // res0: Array[(org.apache.spark.graphx.VertexId, org.apache.spark.graphx.lib.ShortestPaths.SPMap)] = Array((1,Map(3 -> 2)), (2,Map(3 -> 1)), (3,Map(3 -> 0))) lib.ShortestPaths.run(g,Array(1)).vertices.collect // res1: Array[(org.apache.spark.graphx.VertexId, org.apache.spark.graphx.lib.ShortestPaths.SPMap)] = Array((1,Map(1 -> 0)), (2,Map()), (3,Map())) ``` Author: Brennon York Closes #4478 from brennonyork/SPARK-5343 and squashes the following commits: aa57f83 [Brennon York] updated to set ShortestPaths to run 'forward' rather than 'backward' --- .../scala/org/apache/spark/graphx/lib/ShortestPaths.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala index 590f0474957dd..179f2843818e0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala @@ -61,8 +61,8 @@ object ShortestPaths { } def sendMessage(edge: EdgeTriplet[SPMap, _]): Iterator[(VertexId, SPMap)] = { - val newAttr = incrementMap(edge.srcAttr) - if (edge.dstAttr != addMaps(newAttr, edge.dstAttr)) Iterator((edge.dstId, newAttr)) + val newAttr = incrementMap(edge.dstAttr) + if (edge.srcAttr != addMaps(newAttr, edge.srcAttr)) Iterator((edge.srcId, newAttr)) else Iterator.empty } From 52983d7f4f1a155433b6df3687cf5dc71804cfd5 Mon Sep 17 00:00:00 2001 From: Sephiroth-Lin Date: Tue, 10 Feb 2015 23:23:35 +0000 Subject: [PATCH 048/817] [SPARK-5644] [Core]Delete tmp dir when sc is stop When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly. Author: Sephiroth-Lin Closes #4412 from Sephiroth-Lin/bug-fix-master-01 and squashes the following commits: fbbc785 [Sephiroth-Lin] using an interpolated string b968e14 [Sephiroth-Lin] using an interpolated string 4edf394 [Sephiroth-Lin] rename the variable and update comment 1339c96 [Sephiroth-Lin] add a member to store the reference of tmp dir b2018a5 [Sephiroth-Lin] check sparkFilesDir before delete f48a3c6 [Sephiroth-Lin] don't check sparkFilesDir, check executorId dd9686e [Sephiroth-Lin] format code b38e0f0 [Sephiroth-Lin] add dir check before delete d7ccc64 [Sephiroth-Lin] Change log level 1d70926 [Sephiroth-Lin] update comment e2a2b1b [Sephiroth-Lin] update comment aeac518 [Sephiroth-Lin] Delete tmp dir when sc is stop c0d5b28 [Sephiroth-Lin] Delete tmp dir when sc is stop --- .../org/apache/spark/HttpFileServer.scala | 9 ++++++ .../scala/org/apache/spark/SparkEnv.scala | 29 ++++++++++++++++++- 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 3f33332a81eaf..7e706bcc42f04 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -50,6 +50,15 @@ private[spark] class HttpFileServer( def stop() { httpServer.stop() + + // If we only stop sc, but the driver process still run as a services then we need to delete + // the tmp dir, if not, it will create too many tmp dirs + try { + Utils.deleteRecursively(baseDir) + } catch { + case e: Exception => + logWarning(s"Exception while deleting Spark temp dir: ${baseDir.getAbsolutePath}", e) + } } def addFile(file: File) : String = { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index f25db7f8de565..b63bea5b102b6 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -76,6 +76,8 @@ class SparkEnv ( // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() + private var driverTmpDirToDelete: Option[String] = None + private[spark] def stop() { isStopped = true pythonWorkers.foreach { case(key, worker) => worker.stop() } @@ -93,6 +95,22 @@ class SparkEnv ( // actorSystem.awaitTermination() // Note that blockTransferService is stopped by BlockManager since it is started by it. + + // If we only stop sc, but the driver process still run as a services then we need to delete + // the tmp dir, if not, it will create too many tmp dirs. + // We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the + // current working dir in executor which we do not need to delete. + driverTmpDirToDelete match { + case Some(path) => { + try { + Utils.deleteRecursively(new File(path)) + } catch { + case e: Exception => + logWarning(s"Exception while deleting Spark temp dir: $path", e) + } + } + case None => // We just need to delete tmp dir created by driver, so do nothing on executor + } } private[spark] @@ -350,7 +368,7 @@ object SparkEnv extends Logging { "levels using the RDD.persist() method instead.") } - new SparkEnv( + val envInstance = new SparkEnv( executorId, actorSystem, serializer, @@ -367,6 +385,15 @@ object SparkEnv extends Logging { metricsSystem, shuffleMemoryManager, conf) + + // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is + // called, and we only need to do it for driver. Because driver may run as a service, and if we + // don't delete this tmp dir when sc is stopped, then will create too many tmp dirs. + if (isDriver) { + envInstance.driverTmpDirToDelete = Some(sparkFilesDir) + } + + envInstance } /** From 91e3512544d9ab684799ac9a9c341ab465e1b427 Mon Sep 17 00:00:00 2001 From: "Sheng, Li" Date: Wed, 11 Feb 2015 00:59:46 +0000 Subject: [PATCH 049/817] [SQL][Minor] correct some comments Author: Sheng, Li Author: OopsOutOfMemory Closes #4508 from OopsOutOfMemory/cmt and squashes the following commits: d8a68c6 [Sheng, Li] Update ddl.scala f24aeaf [OopsOutOfMemory] correct style --- sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index a692ef51b31ed..bf2ad14763e9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -141,7 +141,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { /* * describe [extended] table avroTable - * This will display all columns of table `avroTable` includes column_name,column_type,nullable + * This will display all columns of table `avroTable` includes column_name,column_type,comment */ protected lazy val describeTable: Parser[LogicalPlan] = (DESCRIBE ~> opt(EXTENDED)) ~ (ident <~ ".").? ~ ident ^^ { From 2d50a010ff57a861b13c2088ac048662d535f5e7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 10 Feb 2015 17:02:44 -0800 Subject: [PATCH 050/817] [SPARK-5725] [SQL] Fixes ParquetRelation2.equals [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4513) Author: Cheng Lian Closes #4513 from liancheng/spark-5725 and squashes the following commits: bf6a087 [Cheng Lian] Fixes ParquetRelation2.equals --- .../src/main/scala/org/apache/spark/sql/parquet/newParquet.scala | 1 + 1 file changed, 1 insertion(+) 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 49d46334b6525..04804f78f5c34 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 @@ -152,6 +152,7 @@ case class ParquetRelation2 paths.toSet == relation.paths.toSet && maybeMetastoreSchema == relation.maybeMetastoreSchema && (shouldMergeSchemas == relation.shouldMergeSchemas || schema == relation.schema) + case _ => false } private[sql] def sparkContext = sqlContext.sparkContext From e28b6bdbb5c5e4fd62ec0b547b77719c3f7e476e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 10 Feb 2015 17:06:12 -0800 Subject: [PATCH 051/817] [SQL] Make Options in the data source API CREATE TABLE statements optional. Users will not need to put `Options()` in a CREATE TABLE statement when there is not option provided. Author: Yin Huai Closes #4515 from yhuai/makeOptionsOptional and squashes the following commits: 1a898d3 [Yin Huai] Make options optional. --- .../src/main/scala/org/apache/spark/sql/sources/ddl.scala | 7 ++++--- .../apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 4 +--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index bf2ad14763e9f..9f64f761002c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -106,13 +106,14 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected lazy val createTable: Parser[LogicalPlan] = ( (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident - ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options) ~ (AS ~> restInput).? ^^ { + ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ { case temp ~ allowExisting ~ tableName ~ columns ~ provider ~ opts ~ query => if (temp.isDefined && allowExisting.isDefined) { throw new DDLException( "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.") } + val options = opts.getOrElse(Map.empty[String, String]) if (query.isDefined) { if (columns.isDefined) { throw new DDLException( @@ -121,7 +122,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { CreateTableUsingAsSelect(tableName, provider, temp.isDefined, - opts, + options, allowExisting.isDefined, query.get) } else { @@ -131,7 +132,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { userSpecifiedSchema, provider, temp.isDefined, - opts, + options, allowExisting.isDefined) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 036efa84d7c85..9ce058909f429 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -361,9 +361,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { s""" |CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource - |OPTIONS ( - | - |) AS + |AS |SELECT * FROM jsonTable """.stripMargin) From ed167e70c6d355f39b366ea0d3b92dd26d826a0b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 10 Feb 2015 17:19:10 -0800 Subject: [PATCH 052/817] [SPARK-5493] [core] Add option to impersonate user. Hadoop has a feature that allows users to impersonate other users when submitting applications or talking to HDFS, for example. These impersonated users are referred generally as "proxy users". Services such as Oozie or Hive use this feature to run applications as the requesting user. This change makes SparkSubmit accept a new command line option to run the application as a proxy user. It also fixes the plumbing of the user name through the UI (and a couple of other places) to refer to the correct user running the application, which can be different than `sys.props("user.name")` even without proxies (e.g. when using kerberos). Author: Marcelo Vanzin Closes #4405 from vanzin/SPARK-5493 and squashes the following commits: df82427 [Marcelo Vanzin] Clarify the reason for the special exception handling. 05bfc08 [Marcelo Vanzin] Remove unneeded annotation. 4840de9 [Marcelo Vanzin] Review feedback. 8af06ff [Marcelo Vanzin] Fix usage string. 2e4fa8f [Marcelo Vanzin] Merge branch 'master' into SPARK-5493 b6c947d [Marcelo Vanzin] Merge branch 'master' into SPARK-5493 0540d38 [Marcelo Vanzin] [SPARK-5493] [core] Add option to impersonate user. --- bin/utils.sh | 3 +- bin/windows-utils.cmd | 1 + .../org/apache/spark/SecurityManager.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 16 ++---- .../apache/spark/deploy/SparkHadoopUtil.scala | 19 +++---- .../org/apache/spark/deploy/SparkSubmit.scala | 56 ++++++++++++++++--- .../spark/deploy/SparkSubmitArguments.scala | 7 +++ .../scala/org/apache/spark/util/Utils.scala | 11 ++++ 8 files changed, 82 insertions(+), 34 deletions(-) diff --git a/bin/utils.sh b/bin/utils.sh index 2241200082018..748dbe345a74c 100755 --- a/bin/utils.sh +++ b/bin/utils.sh @@ -35,7 +35,8 @@ function gatherSparkSubmitOpts() { --master | --deploy-mode | --class | --name | --jars | --packages | --py-files | --files | \ --conf | --repositories | --properties-file | --driver-memory | --driver-java-options | \ --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ - --total-executor-cores | --executor-cores | --queue | --num-executors | --archives) + --total-executor-cores | --executor-cores | --queue | --num-executors | --archives | \ + --proxy-user) if [[ $# -lt 2 ]]; then "$SUBMIT_USAGE_FUNCTION" exit 1; diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd index 567b8733f7f77..0cf9e87ca554b 100644 --- a/bin/windows-utils.cmd +++ b/bin/windows-utils.cmd @@ -33,6 +33,7 @@ SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<-- SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>" SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>" SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\> \<--packages\> \<--repositories\>" +SET opts="%opts:~1,-1% \<--proxy-user\>" echo %1 | findstr %opts% >nul if %ERRORLEVEL% equ 0 ( diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 88d35a4bacc6e..3653f724ba192 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.network.sasl.SecretKeyHolder +import org.apache.spark.util.Utils /** * Spark class responsible for security. @@ -203,7 +204,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), - Option(System.getenv("SPARK_USER")).getOrElse("")).filter(!_.isEmpty) + Utils.getCurrentUserName()) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 04ca5d1019e4b..53fce6b0defdf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -191,7 +191,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - + private[spark] val conf = config.clone() conf.validateSettings() @@ -335,11 +335,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. - val sparkUser = Option { - Option(System.getenv("SPARK_USER")).getOrElse(System.getProperty("user.name")) - }.getOrElse { - SparkContext.SPARK_UNKNOWN_USER - } + val sparkUser = Utils.getCurrentUserName() executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler @@ -826,7 +822,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli vClass: Class[V], conf: Configuration = hadoopConfiguration): RDD[(K, V)] = { assertNotStopped() - // The call to new NewHadoopJob automatically adds security credentials to conf, + // The call to new NewHadoopJob automatically adds security credentials to conf, // so we don't need to explicitly add them ourselves val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) @@ -1626,8 +1622,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli @deprecated("use defaultMinPartitions", "1.0.0") def defaultMinSplits: Int = math.min(defaultParallelism, 2) - /** - * Default min number of partitions for Hadoop RDDs when not given by user + /** + * Default min number of partitions for Hadoop RDDs when not given by user * Notice that we use math.min so the "defaultMinPartitions" cannot be higher than 2. * The reasons for this are discussed in https://github.com/mesos/spark/pull/718 */ @@ -1844,8 +1840,6 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" - private[spark] val SPARK_UNKNOWN_USER = "" - private[spark] val DRIVER_IDENTIFIER = "" // The following deprecated objects have already been copied to `object AccumulatorParam` to diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 03238e9fa0088..e0a32fb65cd51 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -52,18 +52,13 @@ class SparkHadoopUtil extends Logging { * do a FileSystem.closeAllForUGI in order to avoid leaking Filesystems */ def runAsSparkUser(func: () => Unit) { - val user = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) - if (user != SparkContext.SPARK_UNKNOWN_USER) { - logDebug("running as user: " + user) - val ugi = UserGroupInformation.createRemoteUser(user) - transferCredentials(UserGroupInformation.getCurrentUser(), ugi) - ugi.doAs(new PrivilegedExceptionAction[Unit] { - def run: Unit = func() - }) - } else { - logDebug("running as SPARK_UNKNOWN_USER") - func() - } + val user = Utils.getCurrentUserName() + logDebug("running as user: " + user) + val ugi = UserGroupInformation.createRemoteUser(user) + transferCredentials(UserGroupInformation.getCurrentUser(), ugi) + ugi.doAs(new PrivilegedExceptionAction[Unit] { + def run: Unit = func() + }) } def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c4bc5054d61a1..80cc0587286b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,12 +18,14 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} -import java.lang.reflect.{InvocationTargetException, Modifier} +import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL +import java.security.PrivilegedExceptionAction import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.hadoop.fs.Path +import org.apache.hadoop.security.UserGroupInformation import org.apache.ivy.Ivy import org.apache.ivy.core.LogOptions import org.apache.ivy.core.module.descriptor._ @@ -79,7 +81,7 @@ object SparkSubmit { private val CLASS_NOT_FOUND_EXIT_STATUS = 101 // Exposed for testing - private[spark] var exitFn: () => Unit = () => System.exit(-1) + private[spark] var exitFn: () => Unit = () => System.exit(1) private[spark] var printStream: PrintStream = System.err private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) private[spark] def printErrorAndExit(str: String) = { @@ -126,6 +128,34 @@ object SparkSubmit { */ private[spark] def submit(args: SparkSubmitArguments): Unit = { val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args) + + def doRunMain(): Unit = { + if (args.proxyUser != null) { + val proxyUser = UserGroupInformation.createProxyUser(args.proxyUser, + UserGroupInformation.getCurrentUser()) + try { + proxyUser.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + runMain(childArgs, childClasspath, sysProps, childMainClass, args.verbose) + } + }) + } catch { + case e: Exception => + // Hadoop's AuthorizationException suppresses the exception's stack trace, which + // makes the message printed to the output by the JVM not very helpful. Instead, + // detect exceptions with empty stack traces here, and treat them differently. + if (e.getStackTrace().length == 0) { + printStream.println(s"ERROR: ${e.getClass().getName()}: ${e.getMessage()}") + exitFn() + } else { + throw e + } + } + } else { + runMain(childArgs, childClasspath, sysProps, childMainClass, args.verbose) + } + } + // In standalone cluster mode, there are two submission gateways: // (1) The traditional Akka gateway using o.a.s.deploy.Client as a wrapper // (2) The new REST-based gateway introduced in Spark 1.3 @@ -134,7 +164,7 @@ object SparkSubmit { if (args.isStandaloneCluster && args.useRest) { try { printStream.println("Running Spark using the REST application submission protocol.") - runMain(childArgs, childClasspath, sysProps, childMainClass) + doRunMain() } catch { // Fail over to use the legacy submission gateway case e: SubmitRestConnectionException => @@ -145,7 +175,7 @@ object SparkSubmit { } // In all other modes, just run the main class as prepared } else { - runMain(childArgs, childClasspath, sysProps, childMainClass) + doRunMain() } } @@ -457,7 +487,7 @@ object SparkSubmit { childClasspath: Seq[String], sysProps: Map[String, String], childMainClass: String, - verbose: Boolean = false) { + verbose: Boolean): Unit = { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") @@ -507,13 +537,21 @@ object SparkSubmit { if (!Modifier.isStatic(mainMethod.getModifiers)) { throw new IllegalStateException("The main method in the given main class must be static") } + + def findCause(t: Throwable): Throwable = t match { + case e: UndeclaredThrowableException => + if (e.getCause() != null) findCause(e.getCause()) else e + case e: InvocationTargetException => + if (e.getCause() != null) findCause(e.getCause()) else e + case e: Throwable => + e + } + try { mainMethod.invoke(null, childArgs.toArray) } catch { - case e: InvocationTargetException => e.getCause match { - case cause: Throwable => throw cause - case null => throw e - } + case t: Throwable => + throw findCause(t) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index bd0ae26fd8210..fa38070c6fcfe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -57,6 +57,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St var pyFiles: String = null var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() + var proxyUser: String = null // Standalone cluster mode only var supervise: Boolean = false @@ -405,6 +406,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } parse(tail) + case ("--proxy-user") :: value :: tail => + proxyUser = value + parse(tail) + case ("--help" | "-h") :: tail => printUsageAndExit(0) @@ -476,6 +481,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | + | --proxy-user NAME User to impersonate when submitting the application. + | | --help, -h Show this help message and exit | --verbose, -v Print additional debug output | diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 61d287ca9c3ac..6af8dd555f2aa 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -38,6 +38,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ @@ -1986,6 +1987,16 @@ private[spark] object Utils extends Logging { throw new SparkException("Invalid master URL: " + sparkUrl, e) } } + + /** + * Returns the current user name. This is the currently logged in user, unless that's been + * overridden by the `SPARK_USER` environment variable. + */ + def getCurrentUserName(): String = { + Option(System.getenv("SPARK_USER")) + .getOrElse(UserGroupInformation.getCurrentUser().getUserName()) + } + } /** From aaf50d05c7616e4f8f16654b642500ae06cdd774 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 10 Feb 2015 17:29:52 -0800 Subject: [PATCH 053/817] [SPARK-5658][SQL] Finalize DDL and write support APIs https://issues.apache.org/jira/browse/SPARK-5658 Author: Yin Huai This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #4446 from yhuai/writeSupportFollowup and squashes the following commits: f3a96f7 [Yin Huai] davies's comments. 225ff71 [Yin Huai] Use Scala TestHiveContext to initialize the Python HiveContext in Python tests. 2306f93 [Yin Huai] Style. 2091fcd [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup 537e28f [Yin Huai] Correctly clean up temp data. ae4649e [Yin Huai] Fix Python test. 609129c [Yin Huai] Doc format. 92b6659 [Yin Huai] Python doc and other minor updates. cbc717f [Yin Huai] Rename dataSourceName to source. d1c12d3 [Yin Huai] No need to delete the duplicate rule since it has been removed in master. 22cfa70 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup d91ecb8 [Yin Huai] Fix test. 4c76d78 [Yin Huai] Simplify APIs. 3abc215 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup 0832ce4 [Yin Huai] Fix test. 98e7cdb [Yin Huai] Python style. 2bf44ef [Yin Huai] Python APIs. c204967 [Yin Huai] Format a10223d [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup 9ff97d8 [Yin Huai] Add SaveMode to saveAsTable. 9b6e570 [Yin Huai] Update doc. c2be775 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup 99950a2 [Yin Huai] Use Java enum for SaveMode. 4679665 [Yin Huai] Remove duplicate rule. 77d89dc [Yin Huai] Update doc. e04d908 [Yin Huai] Move import and add (Scala-specific) to scala APIs. cf5703d [Yin Huai] Add checkAnswer to Java tests. 7db95ff [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup 6dfd386 [Yin Huai] Add java test. f2f33ef [Yin Huai] Fix test. e702386 [Yin Huai] Apache header. b1e9b1b [Yin Huai] Format. ed4e1b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup af9e9b3 [Yin Huai] DDL and write support API followup. 2a6213a [Yin Huai] Update API names. e6a0b77 [Yin Huai] Update test. 43bae01 [Yin Huai] Remove createTable from HiveContext. 5ffc372 [Yin Huai] Add more load APIs to SQLContext. 5390743 [Yin Huai] Add more save APIs to DataFrame. --- python/pyspark/sql/context.py | 68 ++++++++ python/pyspark/sql/dataframe.py | 72 +++++++- python/pyspark/sql/tests.py | 107 +++++++++++- .../apache/spark/sql/sources/SaveMode.java | 45 +++++ .../org/apache/spark/sql/DataFrame.scala | 160 ++++++++++++++--- .../org/apache/spark/sql/DataFrameImpl.scala | 61 ++----- .../apache/spark/sql/IncomputableColumn.scala | 27 +-- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 164 +++++++++++++++++- .../spark/sql/execution/SparkStrategies.scala | 14 +- .../apache/spark/sql/json/JSONRelation.scala | 30 +++- .../apache/spark/sql/parquet/newParquet.scala | 45 ++++- .../org/apache/spark/sql/sources/ddl.scala | 40 ++++- .../apache/spark/sql/sources/interfaces.scala | 19 ++ .../spark/sql/sources/JavaSaveLoadSuite.java | 97 +++++++++++ .../org/apache/spark/sql/QueryTest.scala | 92 ++++++---- .../sources/CreateTableAsSelectSuite.scala | 29 +++- .../spark/sql/sources/SaveLoadSuite.scala | 59 +++++-- .../apache/spark/sql/hive/HiveContext.scala | 76 -------- .../spark/sql/hive/HiveStrategies.scala | 13 +- .../spark/sql/hive/execution/commands.scala | 105 ++++++++--- .../spark/sql/hive/{ => test}/TestHive.scala | 20 +-- .../hive/JavaMetastoreDataSourcesSuite.java | 147 ++++++++++++++++ .../org/apache/spark/sql/QueryTest.scala | 64 +++++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 33 ++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 118 +++++++++++-- 26 files changed, 1357 insertions(+), 350 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java create mode 100644 sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java rename sql/hive/src/main/scala/org/apache/spark/sql/hive/{ => test}/TestHive.scala (99%) create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 49f016a9cf2e9..882c0f98ea40b 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -21,6 +21,7 @@ from itertools import imap from py4j.protocol import Py4JError +from py4j.java_collections import MapConverter from pyspark.rdd import _prepare_for_python_RDD from pyspark.serializers import AutoBatchedSerializer, PickleSerializer @@ -87,6 +88,18 @@ def _ssql_ctx(self): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext + def setConf(self, key, value): + """Sets the given Spark SQL configuration property. + """ + self._ssql_ctx.setConf(key, value) + + def getConf(self, key, defaultValue): + """Returns the value of Spark SQL configuration property for the given key. + + If the key is not set, returns defaultValue. + """ + return self._ssql_ctx.getConf(key, defaultValue) + def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -455,6 +468,61 @@ def func(iterator): df = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) return DataFrame(df, self) + def load(self, path=None, source=None, schema=None, **options): + """Returns the dataset in a data source as a DataFrame. + + The data source is specified by the `source` and a set of `options`. + If `source` is not specified, the default data source configured by + spark.sql.sources.default will be used. + + Optionally, a schema can be provided as the schema of the returned DataFrame. + """ + if path is not None: + options["path"] = path + if source is None: + source = self.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + joptions = MapConverter().convert(options, + self._sc._gateway._gateway_client) + if schema is None: + df = self._ssql_ctx.load(source, joptions) + else: + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) + df = self._ssql_ctx.load(source, scala_datatype, joptions) + return DataFrame(df, self) + + def createExternalTable(self, tableName, path=None, source=None, + schema=None, **options): + """Creates an external table based on the dataset in a data source. + + It returns the DataFrame associated with the external table. + + The data source is specified by the `source` and a set of `options`. + If `source` is not specified, the default data source configured by + spark.sql.sources.default will be used. + + Optionally, a schema can be provided as the schema of the returned DataFrame and + created external table. + """ + if path is not None: + options["path"] = path + if source is None: + source = self.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + joptions = MapConverter().convert(options, + self._sc._gateway._gateway_client) + if schema is None: + df = self._ssql_ctx.createExternalTable(tableName, source, joptions) + else: + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) + df = self._ssql_ctx.createExternalTable(tableName, source, scala_datatype, + joptions) + return DataFrame(df, self) + def sql(self, sqlQuery): """Return a L{DataFrame} representing the result of the given query. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 04be65fe241c4..3eef0cc376a2d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -146,9 +146,75 @@ def insertInto(self, tableName, overwrite=False): """ self._jdf.insertInto(tableName, overwrite) - def saveAsTable(self, tableName): - """Creates a new table with the contents of this DataFrame.""" - self._jdf.saveAsTable(tableName) + def _java_save_mode(self, mode): + """Returns the Java save mode based on the Python save mode represented by a string. + """ + jSaveMode = self._sc._jvm.org.apache.spark.sql.sources.SaveMode + jmode = jSaveMode.ErrorIfExists + mode = mode.lower() + if mode == "append": + jmode = jSaveMode.Append + elif mode == "overwrite": + jmode = jSaveMode.Overwrite + elif mode == "ignore": + jmode = jSaveMode.Ignore + elif mode == "error": + pass + else: + raise ValueError( + "Only 'append', 'overwrite', 'ignore', and 'error' are acceptable save mode.") + return jmode + + def saveAsTable(self, tableName, source=None, mode="append", **options): + """Saves the contents of the DataFrame to a data source as a table. + + The data source is specified by the `source` and a set of `options`. + If `source` is not specified, the default data source configured by + spark.sql.sources.default will be used. + + Additionally, mode is used to specify the behavior of the saveAsTable operation when + table already exists in the data source. There are four modes: + + * append: Contents of this DataFrame are expected to be appended to existing table. + * overwrite: Data in the existing table is expected to be overwritten by the contents of \ + this DataFrame. + * error: An exception is expected to be thrown. + * ignore: The save operation is expected to not save the contents of the DataFrame and \ + to not change the existing table. + """ + if source is None: + source = self.sql_ctx.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + jmode = self._java_save_mode(mode) + joptions = MapConverter().convert(options, + self.sql_ctx._sc._gateway._gateway_client) + self._jdf.saveAsTable(tableName, source, jmode, joptions) + + def save(self, path=None, source=None, mode="append", **options): + """Saves the contents of the DataFrame to a data source. + + The data source is specified by the `source` and a set of `options`. + If `source` is not specified, the default data source configured by + spark.sql.sources.default will be used. + + Additionally, mode is used to specify the behavior of the save operation when + data already exists in the data source. There are four modes: + + * append: Contents of this DataFrame are expected to be appended to existing data. + * overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. + * error: An exception is expected to be thrown. + * ignore: The save operation is expected to not save the contents of the DataFrame and \ + to not change the existing data. + """ + if path is not None: + options["path"] = path + if source is None: + source = self.sql_ctx.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + jmode = self._java_save_mode(mode) + joptions = MapConverter().convert(options, + self._sc._gateway._gateway_client) + self._jdf.save(source, jmode, joptions) def schema(self): """Returns the schema of this DataFrame (represented by diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index d25c6365ed067..bc945091f7042 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -34,10 +34,9 @@ else: import unittest - -from pyspark.sql import SQLContext, Column +from pyspark.sql import SQLContext, HiveContext, Column from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, StructField, \ - UserDefinedType, DoubleType, LongType + UserDefinedType, DoubleType, LongType, StringType from pyspark.tests import ReusedPySparkTestCase @@ -286,6 +285,37 @@ def test_aggregator(self): self.assertTrue(95 < g.agg(Dsl.approxCountDistinct(df.key)).first()[0]) self.assertEqual(100, g.agg(Dsl.countDistinct(df.value)).first()[0]) + def test_save_and_load(self): + df = self.df + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + df.save(tmpPath, "org.apache.spark.sql.json", "error") + actual = self.sqlCtx.load(tmpPath, "org.apache.spark.sql.json") + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + + schema = StructType([StructField("value", StringType(), True)]) + actual = self.sqlCtx.load(tmpPath, "org.apache.spark.sql.json", schema) + self.assertTrue(sorted(df.select("value").collect()) == sorted(actual.collect())) + + df.save(tmpPath, "org.apache.spark.sql.json", "overwrite") + actual = self.sqlCtx.load(tmpPath, "org.apache.spark.sql.json") + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + + df.save(source="org.apache.spark.sql.json", mode="overwrite", path=tmpPath, + noUse="this options will not be used in save.") + actual = self.sqlCtx.load(source="org.apache.spark.sql.json", path=tmpPath, + noUse="this options will not be used in load.") + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + + defaultDataSourceName = self.sqlCtx.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + self.sqlCtx.sql("SET spark.sql.sources.default=org.apache.spark.sql.json") + actual = self.sqlCtx.load(path=tmpPath) + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + self.sqlCtx.sql("SET spark.sql.sources.default=" + defaultDataSourceName) + + shutil.rmtree(tmpPath) + def test_help_command(self): # Regression test for SPARK-5464 rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) @@ -296,5 +326,76 @@ def test_help_command(self): pydoc.render_doc(df.take(1)) +class HiveContextSQLTests(ReusedPySparkTestCase): + + @classmethod + def setUpClass(cls): + ReusedPySparkTestCase.setUpClass() + cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(cls.tempdir.name) + print "type", type(cls.sc) + print "type", type(cls.sc._jsc) + _scala_HiveContext =\ + cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) + cls.sqlCtx = HiveContext(cls.sc, _scala_HiveContext) + cls.testData = [Row(key=i, value=str(i)) for i in range(100)] + rdd = cls.sc.parallelize(cls.testData) + cls.df = cls.sqlCtx.inferSchema(rdd) + + @classmethod + def tearDownClass(cls): + ReusedPySparkTestCase.tearDownClass() + shutil.rmtree(cls.tempdir.name, ignore_errors=True) + + def test_save_and_load_table(self): + df = self.df + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + df.saveAsTable("savedJsonTable", "org.apache.spark.sql.json", "append", path=tmpPath) + actual = self.sqlCtx.createExternalTable("externalJsonTable", tmpPath, + "org.apache.spark.sql.json") + self.assertTrue( + sorted(df.collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM savedJsonTable").collect())) + self.assertTrue( + sorted(df.collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM externalJsonTable").collect())) + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + self.sqlCtx.sql("DROP TABLE externalJsonTable") + + df.saveAsTable("savedJsonTable", "org.apache.spark.sql.json", "overwrite", path=tmpPath) + schema = StructType([StructField("value", StringType(), True)]) + actual = self.sqlCtx.createExternalTable("externalJsonTable", + source="org.apache.spark.sql.json", + schema=schema, path=tmpPath, + noUse="this options will not be used") + self.assertTrue( + sorted(df.collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM savedJsonTable").collect())) + self.assertTrue( + sorted(df.select("value").collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM externalJsonTable").collect())) + self.assertTrue(sorted(df.select("value").collect()) == sorted(actual.collect())) + self.sqlCtx.sql("DROP TABLE savedJsonTable") + self.sqlCtx.sql("DROP TABLE externalJsonTable") + + defaultDataSourceName = self.sqlCtx.getConf("spark.sql.sources.default", + "org.apache.spark.sql.parquet") + self.sqlCtx.sql("SET spark.sql.sources.default=org.apache.spark.sql.json") + df.saveAsTable("savedJsonTable", path=tmpPath, mode="overwrite") + actual = self.sqlCtx.createExternalTable("externalJsonTable", path=tmpPath) + self.assertTrue( + sorted(df.collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM savedJsonTable").collect())) + self.assertTrue( + sorted(df.collect()) == + sorted(self.sqlCtx.sql("SELECT * FROM externalJsonTable").collect())) + self.assertTrue(sorted(df.collect()) == sorted(actual.collect())) + self.sqlCtx.sql("DROP TABLE savedJsonTable") + self.sqlCtx.sql("DROP TABLE externalJsonTable") + self.sqlCtx.sql("SET spark.sql.sources.default=" + defaultDataSourceName) + + shutil.rmtree(tmpPath) + if __name__ == "__main__": unittest.main() diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java new file mode 100644 index 0000000000000..3109f5716da2c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources; + +/** + * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. + */ +public enum SaveMode { + /** + * Append mode means that when saving a DataFrame to a data source, if data/table already exists, + * contents of the DataFrame are expected to be appended to existing data. + */ + Append, + /** + * Overwrite mode means that when saving a DataFrame to a data source, + * if data/table already exists, existing data is expected to be overwritten by the contents of + * the DataFrame. + */ + Overwrite, + /** + * ErrorIfExists mode means that when saving a DataFrame to a data source, if data already exists, + * an exception is expected to be thrown. + */ + ErrorIfExists, + /** + * Ignore mode means that when saving a DataFrame to a data source, if data already exists, + * the save operation is expected to not save the contents of the DataFrame and to not + * change the existing data. + */ + Ignore +} 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 04e0d09947492..ca8d552c5febf 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 @@ -17,19 +17,19 @@ package org.apache.spark.sql +import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.util.control.NonFatal import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.sources.SaveMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -import scala.util.control.NonFatal - - private[sql] object DataFrame { def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = { new DataFrameImpl(sqlContext, logicalPlan) @@ -574,8 +574,64 @@ trait DataFrame extends RDDApi[Row] { /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame. This will fail if the table already - * exists. + * Creates a table from the the contents of this DataFrame. + * It will use the default data source configured by spark.sql.sources.default. + * This will fail if the table already exists. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + */ + @Experimental + def saveAsTable(tableName: String): Unit = { + saveAsTable(tableName, SaveMode.ErrorIfExists) + } + + /** + * :: Experimental :: + * Creates a table from the the contents of this DataFrame, using the default data source + * configured by spark.sql.sources.default and [[SaveMode.ErrorIfExists]] as the save mode. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + */ + @Experimental + def saveAsTable(tableName: String, mode: SaveMode): Unit = { + if (sqlContext.catalog.tableExists(Seq(tableName)) && mode == SaveMode.Append) { + // If table already exists and the save mode is Append, + // we will just call insertInto to append the contents of this DataFrame. + insertInto(tableName, overwrite = false) + } else { + val dataSourceName = sqlContext.conf.defaultDataSourceName + saveAsTable(tableName, dataSourceName, mode) + } + } + + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source and a set of options, + * using [[SaveMode.ErrorIfExists]] as the save mode. + * + * Note that this currently only works with DataFrames that are created from a HiveContext as + * there is no notion of a persisted catalog in a standard SQL context. Instead you can write + * an RDD out to a parquet file, and then register that file as a table. This "table" can then + * be the target of an `insertInto`. + */ + @Experimental + def saveAsTable( + tableName: String, + source: String): Unit = { + saveAsTable(tableName, source, SaveMode.ErrorIfExists) + } + + /** + * :: Experimental :: + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, and a set of options. * * Note that this currently only works with DataFrames that are created from a HiveContext as * there is no notion of a persisted catalog in a standard SQL context. Instead you can write @@ -583,12 +639,17 @@ trait DataFrame extends RDDApi[Row] { * be the target of an `insertInto`. */ @Experimental - def saveAsTable(tableName: String): Unit + def saveAsTable( + tableName: String, + source: String, + mode: SaveMode): Unit = { + saveAsTable(tableName, source, mode, Map.empty[String, String]) + } /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame based on a given data source and - * a set of options. This will fail if the table already exists. + * Creates a table at the given path from the the contents of this DataFrame + * based on a given data source, [[SaveMode]] specified by mode, and a set of options. * * Note that this currently only works with DataFrames that are created from a HiveContext as * there is no notion of a persisted catalog in a standard SQL context. Instead you can write @@ -598,14 +659,17 @@ trait DataFrame extends RDDApi[Row] { @Experimental def saveAsTable( tableName: String, - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit + source: String, + mode: SaveMode, + options: java.util.Map[String, String]): Unit = { + saveAsTable(tableName, source, mode, options.toMap) + } /** * :: Experimental :: - * Creates a table from the the contents of this DataFrame based on a given data source and - * a set of options. This will fail if the table already exists. + * (Scala-specific) + * Creates a table from the the contents of this DataFrame based on a given data source, + * [[SaveMode]] specified by mode, and a set of options. * * Note that this currently only works with DataFrames that are created from a HiveContext as * there is no notion of a persisted catalog in a standard SQL context. Instead you can write @@ -615,22 +679,76 @@ trait DataFrame extends RDDApi[Row] { @Experimental def saveAsTable( tableName: String, - dataSourceName: String, - options: java.util.Map[String, String]): Unit + source: String, + mode: SaveMode, + options: Map[String, String]): Unit + + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path, + * using the default data source configured by spark.sql.sources.default and + * [[SaveMode.ErrorIfExists]] as the save mode. + */ + @Experimental + def save(path: String): Unit = { + save(path, SaveMode.ErrorIfExists) + } + + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path and [[SaveMode]] specified by mode, + * using the default data source configured by spark.sql.sources.default. + */ + @Experimental + def save(path: String, mode: SaveMode): Unit = { + val dataSourceName = sqlContext.conf.defaultDataSourceName + save(path, dataSourceName, mode) + } + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source, + * using [[SaveMode.ErrorIfExists]] as the save mode. + */ + @Experimental + def save(path: String, source: String): Unit = { + save(source, SaveMode.ErrorIfExists, Map("path" -> path)) + } + + /** + * :: Experimental :: + * Saves the contents of this DataFrame to the given path based on the given data source and + * [[SaveMode]] specified by mode. + */ @Experimental - def save(path: String): Unit + def save(path: String, source: String, mode: SaveMode): Unit = { + save(source, mode, Map("path" -> path)) + } + /** + * :: Experimental :: + * Saves the contents of this DataFrame based on the given data source, + * [[SaveMode]] specified by mode, and a set of options. + */ @Experimental def save( - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit + source: String, + mode: SaveMode, + options: java.util.Map[String, String]): Unit = { + save(source, mode, options.toMap) + } + /** + * :: Experimental :: + * (Scala-specific) + * Saves the contents of this DataFrame based on the given data source, + * [[SaveMode]] specified by mode, and a set of options + */ @Experimental def save( - dataSourceName: String, - options: java.util.Map[String, String]): Unit + source: String, + mode: SaveMode, + options: Map[String, String]): Unit /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 1ee16ad5161c8..11f9334556981 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -28,13 +28,14 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.{SqlParser, ScalaReflection} -import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, ResolvedStar, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsLogicalPlan} +import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{NumericType, StructType} @@ -341,68 +342,34 @@ private[sql] class DataFrameImpl protected[sql]( override def saveAsParquetFile(path: String): Unit = { if (sqlContext.conf.parquetUseDataSourceApi) { - save("org.apache.spark.sql.parquet", "path" -> path) + save("org.apache.spark.sql.parquet", SaveMode.ErrorIfExists, Map("path" -> path)) } else { sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd } } - override def saveAsTable(tableName: String): Unit = { - val dataSourceName = sqlContext.conf.defaultDataSourceName - val cmd = - CreateTableUsingAsLogicalPlan( - tableName, - dataSourceName, - temporary = false, - Map.empty, - allowExisting = false, - logicalPlan) - - sqlContext.executePlan(cmd).toRdd - } - override def saveAsTable( tableName: String, - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit = { + source: String, + mode: SaveMode, + options: Map[String, String]): Unit = { val cmd = CreateTableUsingAsLogicalPlan( tableName, - dataSourceName, + source, temporary = false, - (option +: options).toMap, - allowExisting = false, + mode, + options, logicalPlan) sqlContext.executePlan(cmd).toRdd } - override def saveAsTable( - tableName: String, - dataSourceName: String, - options: java.util.Map[String, String]): Unit = { - val opts = options.toSeq - saveAsTable(tableName, dataSourceName, opts.head, opts.tail:_*) - } - - override def save(path: String): Unit = { - val dataSourceName = sqlContext.conf.defaultDataSourceName - save(dataSourceName, "path" -> path) - } - - override def save( - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit = { - ResolvedDataSource(sqlContext, dataSourceName, (option +: options).toMap, this) - } - override def save( - dataSourceName: String, - options: java.util.Map[String, String]): Unit = { - val opts = options.toSeq - save(dataSourceName, opts.head, opts.tail:_*) + source: String, + mode: SaveMode, + options: Map[String, String]): Unit = { + ResolvedDataSource(sqlContext, source, mode, options, this) } override def insertInto(tableName: String, overwrite: Boolean): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index ce0557b88196f..494e49c1317b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedSt import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.sources.SaveMode import org.apache.spark.sql.types.StructType - private[sql] class IncomputableColumn(protected[sql] val expr: Expression) extends Column { def this(name: String) = this(name match { @@ -156,29 +156,16 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def saveAsParquetFile(path: String): Unit = err() - override def saveAsTable(tableName: String): Unit = err() - - override def saveAsTable( - tableName: String, - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit = err() - override def saveAsTable( tableName: String, - dataSourceName: String, - options: java.util.Map[String, String]): Unit = err() - - override def save(path: String): Unit = err() - - override def save( - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit = err() + source: String, + mode: SaveMode, + options: Map[String, String]): Unit = err() override def save( - dataSourceName: String, - options: java.util.Map[String, String]): Unit = err() + source: String, + mode: SaveMode, + options: Map[String, String]): Unit = err() override def insertInto(tableName: String, overwrite: Boolean): Unit = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 180f5e765fb91..39f6c2f4bc8b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -50,7 +50,7 @@ private[spark] object SQLConf { val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" // This is used to set the default data source - val DEFAULT_DATA_SOURCE_NAME = "spark.sql.default.datasource" + val DEFAULT_DATA_SOURCE_NAME = "spark.sql.sources.default" // Whether to perform eager analysis on a DataFrame. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.dataframe.eagerAnalysis" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 97e3777f933e4..801505bceb956 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -401,27 +401,173 @@ class SQLContext(@transient val sparkContext: SparkContext) jsonRDD(json.rdd, samplingRatio); } + /** + * :: Experimental :: + * Returns the dataset stored at path as a DataFrame, + * using the default data source configured by spark.sql.sources.default. + */ @Experimental def load(path: String): DataFrame = { val dataSourceName = conf.defaultDataSourceName - load(dataSourceName, ("path", path)) + load(path, dataSourceName) } + /** + * :: Experimental :: + * Returns the dataset stored at path as a DataFrame, + * using the given data source. + */ @Experimental - def load( - dataSourceName: String, - option: (String, String), - options: (String, String)*): DataFrame = { - val resolved = ResolvedDataSource(this, None, dataSourceName, (option +: options).toMap) + def load(path: String, source: String): DataFrame = { + load(source, Map("path" -> path)) + } + + /** + * :: Experimental :: + * Returns the dataset specified by the given data source and a set of options as a DataFrame. + */ + @Experimental + def load(source: String, options: java.util.Map[String, String]): DataFrame = { + load(source, options.toMap) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Returns the dataset specified by the given data source and a set of options as a DataFrame. + */ + @Experimental + def load(source: String, options: Map[String, String]): DataFrame = { + val resolved = ResolvedDataSource(this, None, source, options) DataFrame(this, LogicalRelation(resolved.relation)) } + /** + * :: Experimental :: + * Returns the dataset specified by the given data source and a set of options as a DataFrame, + * using the given schema as the schema of the DataFrame. + */ @Experimental def load( - dataSourceName: String, + source: String, + schema: StructType, options: java.util.Map[String, String]): DataFrame = { - val opts = options.toSeq - load(dataSourceName, opts.head, opts.tail:_*) + load(source, schema, options.toMap) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Returns the dataset specified by the given data source and a set of options as a DataFrame, + * using the given schema as the schema of the DataFrame. + */ + @Experimental + def load( + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + val resolved = ResolvedDataSource(this, Some(schema), source, options) + DataFrame(this, LogicalRelation(resolved.relation)) + } + + /** + * :: Experimental :: + * Creates an external table from the given path and returns the corresponding DataFrame. + * It will use the default data source configured by spark.sql.sources.default. + */ + @Experimental + def createExternalTable(tableName: String, path: String): DataFrame = { + val dataSourceName = conf.defaultDataSourceName + createExternalTable(tableName, path, dataSourceName) + } + + /** + * :: Experimental :: + * Creates an external table from the given path based on a data source + * and returns the corresponding DataFrame. + */ + @Experimental + def createExternalTable( + tableName: String, + path: String, + source: String): DataFrame = { + createExternalTable(tableName, source, Map("path" -> path)) + } + + /** + * :: Experimental :: + * Creates an external table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + */ + @Experimental + def createExternalTable( + tableName: String, + source: String, + options: java.util.Map[String, String]): DataFrame = { + createExternalTable(tableName, source, options.toMap) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Creates an external table from the given path based on a data source and a set of options. + * Then, returns the corresponding DataFrame. + */ + @Experimental + def createExternalTable( + tableName: String, + source: String, + options: Map[String, String]): DataFrame = { + val cmd = + CreateTableUsing( + tableName, + userSpecifiedSchema = None, + source, + temporary = false, + options, + allowExisting = false, + managedIfNoPath = false) + executePlan(cmd).toRdd + table(tableName) + } + + /** + * :: Experimental :: + * Create an external table from the given path based on a data source, a schema and + * a set of options. Then, returns the corresponding DataFrame. + */ + @Experimental + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + createExternalTable(tableName, source, schema, options.toMap) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Create an external table from the given path based on a data source, a schema and + * a set of options. Then, returns the corresponding DataFrame. + */ + @Experimental + def createExternalTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + val cmd = + CreateTableUsing( + tableName, + userSpecifiedSchema = Some(schema), + source, + temporary = false, + options, + allowExisting = false, + managedIfNoPath = false) + executePlan(cmd).toRdd + table(tableName) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index edf8a5be64ff1..e915e0e6a0ec1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -309,7 +309,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object DDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case CreateTableUsing(tableName, userSpecifiedSchema, provider, true, opts, false) => + case CreateTableUsing(tableName, userSpecifiedSchema, provider, true, opts, false, _) => ExecutedCommand( CreateTempTableUsing( tableName, userSpecifiedSchema, provider, opts)) :: Nil @@ -318,24 +318,20 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case c: CreateTableUsing if c.temporary && c.allowExisting => sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsSelect(tableName, provider, true, opts, false, query) => + case CreateTableUsingAsSelect(tableName, provider, true, mode, opts, query) => val logicalPlan = sqlContext.parseSql(query) val cmd = - CreateTempTableUsingAsSelect(tableName, provider, opts, logicalPlan) + CreateTempTableUsingAsSelect(tableName, provider, mode, opts, logicalPlan) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") - case c: CreateTableUsingAsSelect if c.temporary && c.allowExisting => - sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsLogicalPlan(tableName, provider, true, opts, false, query) => + case CreateTableUsingAsLogicalPlan(tableName, provider, true, mode, opts, query) => val cmd = - CreateTempTableUsingAsSelect(tableName, provider, opts, query) + CreateTempTableUsingAsSelect(tableName, provider, mode, opts, query) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsLogicalPlan if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") - case c: CreateTableUsingAsLogicalPlan if c.temporary && c.allowExisting => - sys.error("allowExisting should be set to false when creating a temporary table.") case LogicalDescribeCommand(table, isExtended) => val resultPlan = self.sqlContext.executePlan(table).executedPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index c4e14c6c92908..f828bcdd65c9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -29,6 +28,10 @@ import org.apache.spark.sql.types.StructType private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { + private def checkPath(parameters: Map[String, String]): String = { + parameters.getOrElse("path", sys.error("'path' must be specified for json data.")) + } + /** Returns a new base relation with the parameters. */ override def createRelation( sqlContext: SQLContext, @@ -52,15 +55,30 @@ private[sql] class DefaultSource override def createRelation( sqlContext: SQLContext, + mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { - val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val path = checkPath(parameters) val filesystemPath = new Path(path) val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - if (fs.exists(filesystemPath)) { - sys.error(s"path $path already exists.") + val doSave = if (fs.exists(filesystemPath)) { + mode match { + case SaveMode.Append => + sys.error(s"Append mode is not supported by ${this.getClass.getCanonicalName}") + case SaveMode.Overwrite => + fs.delete(filesystemPath, true) + true + case SaveMode.ErrorIfExists => + sys.error(s"path $path already exists.") + case SaveMode.Ignore => false + } + } else { + true + } + if (doSave) { + // Only save data when the save mode is not ignore. + data.toJSON.saveAsTextFile(path) } - data.toJSON.saveAsTextFile(path) createRelation(sqlContext, parameters, data.schema) } 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 04804f78f5c34..aef9c10fbcd01 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 @@ -80,18 +80,45 @@ class DefaultSource override def createRelation( sqlContext: SQLContext, + mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { val path = checkPath(parameters) - ParquetRelation.createEmpty( - path, - data.schema.toAttributes, - false, - sqlContext.sparkContext.hadoopConfiguration, - sqlContext) - - val relation = createRelation(sqlContext, parameters, data.schema) - relation.asInstanceOf[ParquetRelation2].insert(data, true) + val filesystemPath = new Path(path) + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + val doSave = if (fs.exists(filesystemPath)) { + mode match { + case SaveMode.Append => + sys.error(s"Append mode is not supported by ${this.getClass.getCanonicalName}") + case SaveMode.Overwrite => + fs.delete(filesystemPath, true) + true + case SaveMode.ErrorIfExists => + sys.error(s"path $path already exists.") + case SaveMode.Ignore => false + } + } else { + true + } + + val relation = if (doSave) { + // Only save data when the save mode is not ignore. + ParquetRelation.createEmpty( + path, + data.schema.toAttributes, + false, + sqlContext.sparkContext.hadoopConfiguration, + sqlContext) + + val createdRelation = createRelation(sqlContext, parameters, data.schema) + createdRelation.asInstanceOf[ParquetRelation2].insert(data, true) + + createdRelation + } else { + // If the save mode is Ignore, we will just create the relation based on existing data. + createRelation(sqlContext, parameters) + } + relation } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 9f64f761002c9..6487c14b1eb8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -119,11 +119,20 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { throw new DDLException( "a CREATE TABLE AS SELECT statement does not allow column definitions.") } + // When IF NOT EXISTS clause appears in the query, the save mode will be ignore. + val mode = if (allowExisting.isDefined) { + SaveMode.Ignore + } else if (temp.isDefined) { + SaveMode.Overwrite + } else { + SaveMode.ErrorIfExists + } + CreateTableUsingAsSelect(tableName, provider, temp.isDefined, + mode, options, - allowExisting.isDefined, query.get) } else { val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) @@ -133,7 +142,8 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { provider, temp.isDefined, options, - allowExisting.isDefined) + allowExisting.isDefined, + managedIfNoPath = false) } } ) @@ -264,6 +274,7 @@ object ResolvedDataSource { def apply( sqlContext: SQLContext, provider: String, + mode: SaveMode, options: Map[String, String], data: DataFrame): ResolvedDataSource = { val loader = Utils.getContextOrSparkClassLoader @@ -277,7 +288,7 @@ object ResolvedDataSource { val relation = clazz.newInstance match { case dataSource: CreatableRelationProvider => - dataSource.createRelation(sqlContext, options, data) + dataSource.createRelation(sqlContext, mode, options, data) case _ => sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") } @@ -307,28 +318,40 @@ private[sql] case class DescribeCommand( new MetadataBuilder().putString("comment", "comment of the column").build())()) } +/** + * Used to represent the operation of create table using a data source. + * @param tableName + * @param userSpecifiedSchema + * @param provider + * @param temporary + * @param options + * @param allowExisting If it is true, we will do nothing when the table already exists. + * If it is false, an exception will be thrown + * @param managedIfNoPath + */ private[sql] case class CreateTableUsing( tableName: String, userSpecifiedSchema: Option[StructType], provider: String, temporary: Boolean, options: Map[String, String], - allowExisting: Boolean) extends Command + allowExisting: Boolean, + managedIfNoPath: Boolean) extends Command private[sql] case class CreateTableUsingAsSelect( tableName: String, provider: String, temporary: Boolean, + mode: SaveMode, options: Map[String, String], - allowExisting: Boolean, query: String) extends Command private[sql] case class CreateTableUsingAsLogicalPlan( tableName: String, provider: String, temporary: Boolean, + mode: SaveMode, options: Map[String, String], - allowExisting: Boolean, query: LogicalPlan) extends Command private [sql] case class CreateTempTableUsing( @@ -348,12 +371,13 @@ private [sql] case class CreateTempTableUsing( private [sql] case class CreateTempTableUsingAsSelect( tableName: String, provider: String, + mode: SaveMode, options: Map[String, String], query: LogicalPlan) extends RunnableCommand { def run(sqlContext: SQLContext) = { val df = DataFrame(sqlContext, query) - val resolved = ResolvedDataSource(sqlContext, provider, options, df) + val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) sqlContext.registerRDDAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) @@ -364,7 +388,7 @@ private [sql] case class CreateTempTableUsingAsSelect( /** * Builds a map in which keys are case insensitive */ -protected class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String] +protected[sql] class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String] with Serializable { val baseMap = map.map(kv => kv.copy(_1 = kv._1.toLowerCase)) 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 5eecc303ef72b..37fda7ba6e5d0 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 @@ -79,8 +79,27 @@ trait SchemaRelationProvider { @DeveloperApi trait CreatableRelationProvider { + /** + * Creates a relation with the given parameters based on the contents of the given + * DataFrame. The mode specifies the expected behavior of createRelation when + * data already exists. + * Right now, there are three modes, Append, Overwrite, and ErrorIfExists. + * Append mode means that when saving a DataFrame to a data source, if data already exists, + * contents of the DataFrame are expected to be appended to existing data. + * Overwrite mode means that when saving a DataFrame to a data source, if data already exists, + * existing data is expected to be overwritten by the contents of the DataFrame. + * ErrorIfExists mode means that when saving a DataFrame to a data source, + * if data already exists, an exception is expected to be thrown. + * + * @param sqlContext + * @param mode + * @param parameters + * @param data + * @return + */ def createRelation( sqlContext: SQLContext, + mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation } diff --git a/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java new file mode 100644 index 0000000000000..852baf0e09245 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -0,0 +1,97 @@ +/* + * 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.io.File; +import java.io.IOException; +import java.util.*; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.test.TestSQLContext$; +import org.apache.spark.sql.*; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.Utils; + +public class JavaSaveLoadSuite { + + private transient JavaSparkContext sc; + private transient SQLContext sqlContext; + + String originalDefaultSource; + File path; + DataFrame df; + + private void checkAnswer(DataFrame actual, List expected) { + String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); + if (errorMessage != null) { + Assert.fail(errorMessage); + } + } + + @Before + public void setUp() throws IOException { + sqlContext = TestSQLContext$.MODULE$; + sc = new JavaSparkContext(sqlContext.sparkContext()); + + originalDefaultSource = sqlContext.conf().defaultDataSourceName(); + path = + Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource").getCanonicalFile(); + if (path.exists()) { + path.delete(); + } + + List jsonObjects = new ArrayList(10); + for (int i = 0; i < 10; i++) { + jsonObjects.add("{\"a\":" + i + ", \"b\":\"str" + i + "\"}"); + } + JavaRDD rdd = sc.parallelize(jsonObjects); + df = sqlContext.jsonRDD(rdd); + df.registerTempTable("jsonTable"); + } + + @Test + public void saveAndLoad() { + Map options = new HashMap(); + options.put("path", path.toString()); + df.save("org.apache.spark.sql.json", SaveMode.ErrorIfExists, options); + + DataFrame loadedDF = sqlContext.load("org.apache.spark.sql.json", options); + + checkAnswer(loadedDF, df.collectAsList()); + } + + @Test + public void saveAndLoadWithSchema() { + Map options = new HashMap(); + options.put("path", path.toString()); + df.save("org.apache.spark.sql.json", SaveMode.ErrorIfExists, options); + + List fields = new ArrayList<>(); + fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); + StructType schema = DataTypes.createStructType(fields); + DataFrame loadedDF = sqlContext.load("org.apache.spark.sql.json", schema, options); + + checkAnswer(loadedDF, sqlContext.sql("SELECT b FROM jsonTable").collectAsList()); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index f9ddd2ca5c567..dfb6858957fb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.util.{Locale, TimeZone} +import scala.collection.JavaConversions._ + import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.columnar.InMemoryRelation @@ -52,9 +54,51 @@ class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { + QueryTest.checkAnswer(rdd, expectedAnswer) match { + case Some(errorMessage) => fail(errorMessage) + case None => + } + } + + protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { + checkAnswer(rdd, Seq(expectedAnswer)) + } + + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { + test(sqlString) { + checkAnswer(sqlContext.sql(sqlString), expectedAnswer) + } + } + + /** + * Asserts that a given [[DataFrame]] will be executed using the given number of cached results. + */ + def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } +} + +object QueryTest { + /** + * Runs the plan and makes sure the answer matches the expected result. + * If there was exception during the execution or the contents of the DataFrame does not + * match the expected result, an error message will be returned. Otherwise, a [[None]] will + * be returned. + * @param rdd the [[DataFrame]] to be executed + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + */ + def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty def prepareAnswer(answer: Seq[Row]): Seq[Row] = { // Converts data to types that we can do equality comparison using Scala collections. @@ -70,18 +114,20 @@ class QueryTest extends PlanTest { } val sparkAnswer = try rdd.collect().toSeq catch { case e: Exception => - fail( + val errorMessage = s""" |Exception thrown while executing query: |${rdd.queryExecution} |== Exception == |$e |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} - """.stripMargin) + """.stripMargin + return Some(errorMessage) } if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - fail(s""" + val errorMessage = + s""" |Results do not match for query: |${rdd.logicalPlan} |== Analyzed Plan == @@ -90,37 +136,21 @@ class QueryTest extends PlanTest { |${rdd.queryExecution.executedPlan} |== Results == |${sideBySide( - s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} - """.stripMargin) + s"== Correct Answer - ${expectedAnswer.size} ==" +: + prepareAnswer(expectedAnswer).map(_.toString), + s"== Spark Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} + """.stripMargin + return Some(errorMessage) } - } - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { - checkAnswer(rdd, Seq(expectedAnswer)) - } - - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { - test(sqlString) { - checkAnswer(sqlContext.sql(sqlString), expectedAnswer) - } + return None } - /** - * Asserts that a given [[DataFrame]] will be executed using the given number of cached results. - */ - def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = { - val planWithCaching = query.queryExecution.withCachedData - val cachedData = planWithCaching collect { - case cached: InMemoryRelation => cached + def checkAnswer(rdd: DataFrame, expectedAnswer: java.util.List[Row]): String = { + checkAnswer(rdd, expectedAnswer.toSeq) match { + case Some(errorMessage) => errorMessage + case None => null } - - assert( - cachedData.size == numCachedTables, - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + - planWithCaching) } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index b02389978b625..29caed9337ff6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -77,12 +77,10 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { sql("SELECT a, b FROM jsonTable"), sql("SELECT a, b FROM jt").collect()) - dropTempTable("jsonTable") - - val message = intercept[RuntimeException]{ + val message = intercept[DDLException]{ sql( s""" - |CREATE TEMPORARY TABLE jsonTable + |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( | path '${path.toString}' @@ -91,10 +89,25 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { """.stripMargin) }.getMessage assert( - message.contains(s"path ${path.toString} already exists."), + message.contains(s"a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") - // Explicitly delete it. + // Overwrite the temporary table. + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a * 4 FROM jt + """.stripMargin) + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT a * 4 FROM jt").collect()) + + dropTempTable("jsonTable") + // Explicitly delete the data. if (path.exists()) Utils.deleteRecursively(path) sql( @@ -104,12 +117,12 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { |OPTIONS ( | path '${path.toString}' |) AS - |SELECT a * 4 FROM jt + |SELECT b FROM jt """.stripMargin) checkAnswer( sql("SELECT * FROM jsonTable"), - sql("SELECT a * 4 FROM jt").collect()) + sql("SELECT b FROM jt").collect()) dropTempTable("jsonTable") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index fe2f76cc397f5..a51004567175c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -21,10 +21,10 @@ import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.DataFrame -import org.apache.spark.util.Utils - import org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.{SQLConf, DataFrame} +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { @@ -38,42 +38,60 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { override def beforeAll(): Unit = { originalDefaultSource = conf.defaultDataSourceName - conf.setConf("spark.sql.default.datasource", "org.apache.spark.sql.json") path = util.getTempFilePath("datasource").getCanonicalFile val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) df = jsonRDD(rdd) + df.registerTempTable("jsonTable") } override def afterAll(): Unit = { - conf.setConf("spark.sql.default.datasource", originalDefaultSource) + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) } after { + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) if (path.exists()) Utils.deleteRecursively(path) } def checkLoad(): Unit = { + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") checkAnswer(load(path.toString), df.collect()) - checkAnswer(load("org.apache.spark.sql.json", ("path", path.toString)), df.collect()) + + // Test if we can pick up the data source name passed in load. + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + checkAnswer(load(path.toString, "org.apache.spark.sql.json"), df.collect()) + checkAnswer(load("org.apache.spark.sql.json", Map("path" -> path.toString)), df.collect()) + val schema = StructType(StructField("b", StringType, true) :: Nil) + checkAnswer( + load("org.apache.spark.sql.json", schema, Map("path" -> path.toString)), + sql("SELECT b FROM jsonTable").collect()) } - test("save with overwrite and load") { + test("save with path and load") { + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") df.save(path.toString) - checkLoad + checkLoad() + } + + test("save with path and datasource, and load") { + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + df.save(path.toString, "org.apache.spark.sql.json") + checkLoad() } test("save with data source and options, and load") { - df.save("org.apache.spark.sql.json", ("path", path.toString)) - checkLoad + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + df.save("org.apache.spark.sql.json", SaveMode.ErrorIfExists, Map("path" -> path.toString)) + checkLoad() } test("save and save again") { - df.save(path.toString) + df.save(path.toString, "org.apache.spark.sql.json") - val message = intercept[RuntimeException] { - df.save(path.toString) + var message = intercept[RuntimeException] { + df.save(path.toString, "org.apache.spark.sql.json") }.getMessage assert( @@ -82,7 +100,18 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { if (path.exists()) Utils.deleteRecursively(path) - df.save(path.toString) - checkLoad + df.save(path.toString, "org.apache.spark.sql.json") + checkLoad() + + df.save("org.apache.spark.sql.json", SaveMode.Overwrite, Map("path" -> path.toString)) + checkLoad() + + message = intercept[RuntimeException] { + df.save("org.apache.spark.sql.json", SaveMode.Append, Map("path" -> path.toString)) + }.getMessage + + assert( + message.contains("Append mode is not supported"), + "We should complain that 'Append mode is not supported' for JSON source.") } } \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 2c00659496972..7ae6ed6f841bf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -79,18 +79,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - /** - * Creates a table using the schema of the given class. - * - * @param tableName The name of the table to create. - * @param allowExisting When false, an exception will be thrown if the table already exists. - * @tparam A A case class that is used to describe the schema of the table to be created. - */ - @Deprecated - def createTable[A <: Product : TypeTag](tableName: String, allowExisting: Boolean = true) { - catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) - } - /** * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, * Spark SQL or the external data source library it uses might cache certain metadata about a @@ -107,70 +95,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.invalidateTable("default", tableName) } - @Experimental - def createTable(tableName: String, path: String, allowExisting: Boolean): Unit = { - val dataSourceName = conf.defaultDataSourceName - createTable(tableName, dataSourceName, allowExisting, ("path", path)) - } - - @Experimental - def createTable( - tableName: String, - dataSourceName: String, - allowExisting: Boolean, - option: (String, String), - options: (String, String)*): Unit = { - val cmd = - CreateTableUsing( - tableName, - userSpecifiedSchema = None, - dataSourceName, - temporary = false, - (option +: options).toMap, - allowExisting) - executePlan(cmd).toRdd - } - - @Experimental - def createTable( - tableName: String, - dataSourceName: String, - schema: StructType, - allowExisting: Boolean, - option: (String, String), - options: (String, String)*): Unit = { - val cmd = - CreateTableUsing( - tableName, - userSpecifiedSchema = Some(schema), - dataSourceName, - temporary = false, - (option +: options).toMap, - allowExisting) - executePlan(cmd).toRdd - } - - @Experimental - def createTable( - tableName: String, - dataSourceName: String, - allowExisting: Boolean, - options: java.util.Map[String, String]): Unit = { - val opts = options.toSeq - createTable(tableName, dataSourceName, allowExisting, opts.head, opts.tail:_*) - } - - @Experimental - def createTable( - tableName: String, - dataSourceName: String, - schema: StructType, - allowExisting: Boolean, - options: java.util.Map[String, String]): Unit = { - val opts = options.toSeq - createTable(tableName, dataSourceName, schema, allowExisting, opts.head, opts.tail:_*) - } - /** * Analyzes the given table in the current database to generate statistics, which will be * used in query optimizations. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 95abc363ae767..cb138be90e2e1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -216,20 +216,21 @@ private[hive] trait HiveStrategies { object HiveDDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, opts, allowExisting) => + case CreateTableUsing( + tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => ExecutedCommand( CreateMetastoreDataSource( - tableName, userSpecifiedSchema, provider, opts, allowExisting)) :: Nil + tableName, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath)) :: Nil - case CreateTableUsingAsSelect(tableName, provider, false, opts, allowExisting, query) => + case CreateTableUsingAsSelect(tableName, provider, false, mode, opts, query) => val logicalPlan = hiveContext.parseSql(query) val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, opts, allowExisting, logicalPlan) + CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, logicalPlan) ExecutedCommand(cmd) :: Nil - case CreateTableUsingAsLogicalPlan(tableName, provider, false, opts, allowExisting, query) => + case CreateTableUsingAsLogicalPlan(tableName, provider, false, mode, opts, query) => val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, opts, allowExisting, query) + CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, query) ExecutedCommand(cmd) :: Nil case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 95dcaccefdc54..f6bea1c6a6fe1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.sources.ResolvedDataSource +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.sources._ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -105,7 +107,8 @@ case class CreateMetastoreDataSource( userSpecifiedSchema: Option[StructType], provider: String, options: Map[String, String], - allowExisting: Boolean) extends RunnableCommand { + allowExisting: Boolean, + managedIfNoPath: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] @@ -120,7 +123,7 @@ case class CreateMetastoreDataSource( var isExternal = true val optionsWithPath = - if (!options.contains("path")) { + if (!options.contains("path") && managedIfNoPath) { isExternal = false options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableName)) } else { @@ -141,22 +144,13 @@ case class CreateMetastoreDataSource( case class CreateMetastoreDataSourceAsSelect( tableName: String, provider: String, + mode: SaveMode, options: Map[String, String], - allowExisting: Boolean, query: LogicalPlan) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] - - if (hiveContext.catalog.tableExists(tableName :: Nil)) { - if (allowExisting) { - return Seq.empty[Row] - } else { - sys.error(s"Table $tableName already exists.") - } - } - - val df = DataFrame(hiveContext, query) + var createMetastoreTable = false var isExternal = true val optionsWithPath = if (!options.contains("path")) { @@ -166,15 +160,82 @@ case class CreateMetastoreDataSourceAsSelect( options } - // Create the relation based on the data of df. - ResolvedDataSource(sqlContext, provider, optionsWithPath, df) + if (sqlContext.catalog.tableExists(Seq(tableName))) { + // Check if we need to throw an exception or just return. + mode match { + case SaveMode.ErrorIfExists => + sys.error(s"Table $tableName already exists. " + + s"If you want to append into it, please set mode to SaveMode.Append. " + + s"Or, if you want to overwrite it, please set mode to SaveMode.Overwrite.") + case SaveMode.Ignore => + // Since the table already exists and the save mode is Ignore, we will just return. + return Seq.empty[Row] + case SaveMode.Append => + // Check if the specified data source match the data source of the existing table. + val resolved = + ResolvedDataSource(sqlContext, Some(query.schema), provider, optionsWithPath) + val createdRelation = LogicalRelation(resolved.relation) + EliminateAnalysisOperators(sqlContext.table(tableName).logicalPlan) match { + case l @ LogicalRelation(i: InsertableRelation) => + if (l.schema != createdRelation.schema) { + val errorDescription = + s"Cannot append to table $tableName because the schema of this " + + s"DataFrame does not match the schema of table $tableName." + val errorMessage = + s""" + |$errorDescription + |== Schemas == + |${sideBySide( + s"== Expected Schema ==" +: + l.schema.treeString.split("\\\n"), + s"== Actual Schema ==" +: + createdRelation.schema.treeString.split("\\\n")).mkString("\n")} + """.stripMargin + sys.error(errorMessage) + } else if (i != createdRelation.relation) { + val errorDescription = + s"Cannot append to table $tableName because the resolved relation does not " + + s"match the existing relation of $tableName. " + + s"You can use insertInto($tableName, false) to append this DataFrame to the " + + s"table $tableName and using its data source and options." + val errorMessage = + s""" + |$errorDescription + |== Relations == + |${sideBySide( + s"== Expected Relation ==" :: + l.toString :: Nil, + s"== Actual Relation ==" :: + createdRelation.toString :: Nil).mkString("\n")} + """.stripMargin + sys.error(errorMessage) + } + case o => + sys.error(s"Saving data in ${o.toString} is not supported.") + } + case SaveMode.Overwrite => + hiveContext.sql(s"DROP TABLE IF EXISTS $tableName") + // Need to create the table again. + createMetastoreTable = true + } + } else { + // The table does not exist. We need to create it in metastore. + createMetastoreTable = true + } - hiveContext.catalog.createDataSourceTable( - tableName, - None, - provider, - optionsWithPath, - isExternal) + val df = DataFrame(hiveContext, query) + + // Create the relation based on the data of df. + ResolvedDataSource(sqlContext, provider, mode, optionsWithPath, df) + + if (createMetastoreTable) { + hiveContext.catalog.createDataSourceTable( + tableName, + Some(df.schema), + provider, + optionsWithPath, + isExternal) + } Seq.empty[Row] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 7c1d1133c3425..840fbc197259a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -20,9 +20,6 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} -import scala.collection.mutable -import scala.language.implicitConversions - import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table @@ -30,16 +27,18 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.util.Utils +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.CacheTableCommand import org.apache.spark.sql.hive._ -import org.apache.spark.sql.SQLConf import org.apache.spark.sql.hive.execution.HiveNativeCommand +import org.apache.spark.util.Utils +import org.apache.spark.{SparkConf, SparkContext} + +import scala.collection.mutable +import scala.language.implicitConversions /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -224,11 +223,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } }), TestTable("src_thrift", () => { - import org.apache.thrift.protocol.TBinaryProtocol - import org.apache.hadoop.hive.serde2.thrift.test.Complex import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer - import org.apache.hadoop.mapred.SequenceFileInputFormat - import org.apache.hadoop.mapred.SequenceFileOutputFormat + import org.apache.hadoop.hive.serde2.thrift.test.Complex + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol val srcThrift = new Table("default", "src_thrift") srcThrift.setFields(Nil) diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java new file mode 100644 index 0000000000000..9744a2aa3f59c --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.sources.SaveMode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.QueryTest$; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.hive.test.TestHive$; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.Utils; + +public class JavaMetastoreDataSourcesSuite { + private transient JavaSparkContext sc; + private transient HiveContext sqlContext; + + String originalDefaultSource; + File path; + Path hiveManagedPath; + FileSystem fs; + DataFrame df; + + private void checkAnswer(DataFrame actual, List expected) { + String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); + if (errorMessage != null) { + Assert.fail(errorMessage); + } + } + + @Before + public void setUp() throws IOException { + sqlContext = TestHive$.MODULE$; + sc = new JavaSparkContext(sqlContext.sparkContext()); + + originalDefaultSource = sqlContext.conf().defaultDataSourceName(); + path = + Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource").getCanonicalFile(); + if (path.exists()) { + path.delete(); + } + hiveManagedPath = new Path(sqlContext.catalog().hiveDefaultTableFilePath("javaSavedTable")); + fs = hiveManagedPath.getFileSystem(sc.hadoopConfiguration()); + if (fs.exists(hiveManagedPath)){ + fs.delete(hiveManagedPath, true); + } + + List jsonObjects = new ArrayList(10); + for (int i = 0; i < 10; i++) { + jsonObjects.add("{\"a\":" + i + ", \"b\":\"str" + i + "\"}"); + } + JavaRDD rdd = sc.parallelize(jsonObjects); + df = sqlContext.jsonRDD(rdd); + df.registerTempTable("jsonTable"); + } + + @After + public void tearDown() throws IOException { + // Clean up tables. + sqlContext.sql("DROP TABLE IF EXISTS javaSavedTable"); + sqlContext.sql("DROP TABLE IF EXISTS externalTable"); + } + + @Test + public void saveExternalTableAndQueryIt() { + Map options = new HashMap(); + options.put("path", path.toString()); + df.saveAsTable("javaSavedTable", "org.apache.spark.sql.json", SaveMode.Append, options); + + checkAnswer( + sqlContext.sql("SELECT * FROM javaSavedTable"), + df.collectAsList()); + + DataFrame loadedDF = + sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", options); + + checkAnswer(loadedDF, df.collectAsList()); + checkAnswer( + sqlContext.sql("SELECT * FROM externalTable"), + df.collectAsList()); + } + + @Test + public void saveExternalTableWithSchemaAndQueryIt() { + Map options = new HashMap(); + options.put("path", path.toString()); + df.saveAsTable("javaSavedTable", "org.apache.spark.sql.json", SaveMode.Append, options); + + checkAnswer( + sqlContext.sql("SELECT * FROM javaSavedTable"), + df.collectAsList()); + + List fields = new ArrayList<>(); + fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); + StructType schema = DataTypes.createStructType(fields); + DataFrame loadedDF = + sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", schema, options); + + checkAnswer( + loadedDF, + sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList()); + checkAnswer( + sqlContext.sql("SELECT * FROM externalTable"), + sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList()); + } + + @Test + public void saveTableAndQueryIt() { + Map options = new HashMap(); + df.saveAsTable("javaSavedTable", "org.apache.spark.sql.json", SaveMode.Append, options); + + checkAnswer( + sqlContext.sql("SELECT * FROM javaSavedTable"), + df.collectAsList()); + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index ba391293884bd..0270e63557963 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -17,10 +17,8 @@ package org.apache.spark.sql -import org.scalatest.FunSuite +import scala.collection.JavaConversions._ -import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ @@ -55,9 +53,36 @@ class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { + QueryTest.checkAnswer(rdd, expectedAnswer) match { + case Some(errorMessage) => fail(errorMessage) + case None => + } + } + + protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { + checkAnswer(rdd, Seq(expectedAnswer)) + } + + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { + test(sqlString) { + checkAnswer(sqlContext.sql(sqlString), expectedAnswer) + } + } +} + +object QueryTest { + /** + * Runs the plan and makes sure the answer matches the expected result. + * If there was exception during the execution or the contents of the DataFrame does not + * match the expected result, an error message will be returned. Otherwise, a [[None]] will + * be returned. + * @param rdd the [[DataFrame]] to be executed + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + */ + def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty def prepareAnswer(answer: Seq[Row]): Seq[Row] = { // Converts data to types that we can do equality comparison using Scala collections. @@ -73,18 +98,20 @@ class QueryTest extends PlanTest { } val sparkAnswer = try rdd.collect().toSeq catch { case e: Exception => - fail( + val errorMessage = s""" |Exception thrown while executing query: |${rdd.queryExecution} |== Exception == |$e |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} - """.stripMargin) + """.stripMargin + return Some(errorMessage) } if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - fail(s""" + val errorMessage = + s""" |Results do not match for query: |${rdd.logicalPlan} |== Analyzed Plan == @@ -93,22 +120,21 @@ class QueryTest extends PlanTest { |${rdd.queryExecution.executedPlan} |== Results == |${sideBySide( - s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} - """.stripMargin) + s"== Correct Answer - ${expectedAnswer.size} ==" +: + prepareAnswer(expectedAnswer).map(_.toString), + s"== Spark Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} + """.stripMargin + return Some(errorMessage) } - } - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { - checkAnswer(rdd, Seq(expectedAnswer)) + return None } - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { - test(sqlString) { - checkAnswer(sqlContext.sql(sqlString), expectedAnswer) + def checkAnswer(rdd: DataFrame, expectedAnswer: java.util.List[Row]): String = { + checkAnswer(rdd, expectedAnswer.toSeq) match { + case Some(errorMessage) => errorMessage + case None => null } } - } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 869d01eb398c5..43da7519ac8db 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -19,7 +19,11 @@ package org.apache.spark.sql.hive import java.io.File +import org.scalatest.BeforeAndAfter + import com.google.common.io.Files + +import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.types._ @@ -29,15 +33,22 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) -class InsertIntoHiveTableSuite extends QueryTest { +class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ val testData = TestHive.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))) - testData.registerTempTable("testData") + + before { + // Since every we are doing tests for DDL statements, + // it is better to reset before every test. + TestHive.reset() + // Register the testData, which will be used in every test. + testData.registerTempTable("testData") + } test("insertInto() HiveTable") { - createTable[TestData]("createAndInsertTest") + sql("CREATE TABLE createAndInsertTest (key int, value string)") // Add some data. testData.insertInto("createAndInsertTest") @@ -68,16 +79,18 @@ class InsertIntoHiveTableSuite extends QueryTest { } test("Double create fails when allowExisting = false") { - createTable[TestData]("doubleCreateAndInsertTest") + sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") - intercept[org.apache.hadoop.hive.ql.metadata.HiveException] { - createTable[TestData]("doubleCreateAndInsertTest", allowExisting = false) - } + val message = intercept[QueryExecutionException] { + sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") + }.getMessage + + println("message!!!!" + message) } test("Double create does not fail when allowExisting = true") { - createTable[TestData]("createAndInsertTest") - createTable[TestData]("createAndInsertTest") + sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") + sql("CREATE TABLE IF NOT EXISTS doubleCreateAndInsertTest (key int, value string)") } test("SPARK-4052: scala.collection.Map as value type of MapType") { @@ -98,7 +111,7 @@ class InsertIntoHiveTableSuite extends QueryTest { } test("SPARK-4203:random partition directory order") { - createTable[TestData]("tmp_table") + sql("CREATE TABLE tmp_table (key int, value string)") val tmpDir = Files.createTempDir() sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 9ce058909f429..f94aabd29ad23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.hive import java.io.File +import org.apache.spark.sql.sources.SaveMode import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql.catalyst.util import org.apache.spark.sql._ @@ -41,11 +43,11 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { override def afterEach(): Unit = { reset() - if (ctasPath.exists()) Utils.deleteRecursively(ctasPath) + if (tempPath.exists()) Utils.deleteRecursively(tempPath) } val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile - var ctasPath: File = util.getTempFilePath("jsonCTAS").getCanonicalFile + var tempPath: File = util.getTempFilePath("jsonCTAS").getCanonicalFile test ("persistent JSON table") { sql( @@ -270,7 +272,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '${ctasPath}' + | path '${tempPath}' |) AS |SELECT * FROM jsonTable """.stripMargin) @@ -297,7 +299,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '${ctasPath}' + | path '${tempPath}' |) AS |SELECT * FROM jsonTable """.stripMargin) @@ -309,7 +311,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '${ctasPath}' + | path '${tempPath}' |) AS |SELECT * FROM jsonTable """.stripMargin) @@ -325,7 +327,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |CREATE TABLE IF NOT EXISTS ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '${ctasPath}' + | path '${tempPath}' |) AS |SELECT a FROM jsonTable """.stripMargin) @@ -400,38 +402,122 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("DROP TABLE jsonTable").collect().foreach(println) } - test("save and load table") { + test("save table") { val originalDefaultSource = conf.defaultDataSourceName - conf.setConf("spark.sql.default.datasource", "org.apache.spark.sql.json") val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) val df = jsonRDD(rdd) + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + // Save the df as a managed table (by not specifiying the path). df.saveAsTable("savedJsonTable") checkAnswer( sql("SELECT * FROM savedJsonTable"), df.collect()) - createTable("createdJsonTable", catalog.hiveDefaultTableFilePath("savedJsonTable"), false) + // Right now, we cannot append to an existing JSON table. + intercept[RuntimeException] { + df.saveAsTable("savedJsonTable", SaveMode.Append) + } + + // We can overwrite it. + df.saveAsTable("savedJsonTable", SaveMode.Overwrite) + checkAnswer( + sql("SELECT * FROM savedJsonTable"), + df.collect()) + + // When the save mode is Ignore, we will do nothing when the table already exists. + df.select("b").saveAsTable("savedJsonTable", SaveMode.Ignore) + assert(df.schema === table("savedJsonTable").schema) + checkAnswer( + sql("SELECT * FROM savedJsonTable"), + df.collect()) + + // Drop table will also delete the data. + sql("DROP TABLE savedJsonTable") + intercept[InvalidInputException] { + jsonFile(catalog.hiveDefaultTableFilePath("savedJsonTable")) + } + + // Create an external table by specifying the path. + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + df.saveAsTable( + "savedJsonTable", + "org.apache.spark.sql.json", + SaveMode.Append, + Map("path" -> tempPath.toString)) + checkAnswer( + sql("SELECT * FROM savedJsonTable"), + df.collect()) + + // Data should not be deleted after we drop the table. + sql("DROP TABLE savedJsonTable") + checkAnswer( + jsonFile(tempPath.toString), + df.collect()) + + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) + } + + test("create external table") { + val originalDefaultSource = conf.defaultDataSourceName + + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + val df = jsonRDD(rdd) + + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + df.saveAsTable( + "savedJsonTable", + "org.apache.spark.sql.json", + SaveMode.Append, + Map("path" -> tempPath.toString)) + + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + createExternalTable("createdJsonTable", tempPath.toString) assert(table("createdJsonTable").schema === df.schema) checkAnswer( sql("SELECT * FROM createdJsonTable"), df.collect()) - val message = intercept[RuntimeException] { - createTable("createdJsonTable", filePath.toString, false) + var message = intercept[RuntimeException] { + createExternalTable("createdJsonTable", filePath.toString) }.getMessage assert(message.contains("Table createdJsonTable already exists."), "We should complain that ctasJsonTable already exists") - createTable("createdJsonTable", filePath.toString, true) - // createdJsonTable should be not changed. - assert(table("createdJsonTable").schema === df.schema) + // Data should not be deleted. + sql("DROP TABLE createdJsonTable") checkAnswer( - sql("SELECT * FROM createdJsonTable"), + jsonFile(tempPath.toString), df.collect()) - conf.setConf("spark.sql.default.datasource", originalDefaultSource) + // Try to specify the schema. + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + val schema = StructType(StructField("b", StringType, true) :: Nil) + createExternalTable( + "createdJsonTable", + "org.apache.spark.sql.json", + schema, + Map("path" -> tempPath.toString)) + checkAnswer( + sql("SELECT * FROM createdJsonTable"), + sql("SELECT b FROM savedJsonTable").collect()) + + sql("DROP TABLE createdJsonTable") + + message = intercept[RuntimeException] { + createExternalTable( + "createdJsonTable", + "org.apache.spark.sql.json", + schema, + Map.empty[String, String]) + }.getMessage + assert( + message.contains("Option 'path' not specified"), + "We should complain that path is not specified.") + + sql("DROP TABLE savedJsonTable") + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) } } From 6195e2473b98253ccc9edc3d624ba2bf59ffc398 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 10 Feb 2015 17:32:42 -0800 Subject: [PATCH 054/817] [SQL] Add an exception for analysis errors. Also start from the bottom so we show the first error instead of the top error. Author: Michael Armbrust Closes #4439 from marmbrus/analysisException and squashes the following commits: 45862a0 [Michael Armbrust] fix hive test a773bba [Michael Armbrust] Merge remote-tracking branch 'origin/master' into analysisException f88079f [Michael Armbrust] update more cases fede90a [Michael Armbrust] newline fbf4bc3 [Michael Armbrust] move to sql 6235db4 [Michael Armbrust] [SQL] Add an exception for analysis errors. --- .../apache/spark/sql/AnalysisException.scala | 23 +++++++++++++++++++ .../sql/catalyst/analysis/Analyzer.scala | 21 ++++++++++------- .../sql/catalyst/analysis/AnalysisSuite.scala | 14 +++++------ .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../hive/execution/HiveResolutionSuite.scala | 3 ++- 5 files changed, 46 insertions(+), 17 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala new file mode 100644 index 0000000000000..871d560b9d54f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +/** + * Thrown when a query fails to analyze, usually because the query itself is invalid. + */ +class AnalysisException(message: String) extends Exception(message) with Serializable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fb2ff014cef07..3f0d77ad6322a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.util.collection.OpenHashSet +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -80,16 +81,18 @@ class Analyzer(catalog: Catalog, */ object CheckResolution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { - plan.transform { + plan.transformUp { case p if p.expressions.exists(!_.resolved) => - throw new TreeNodeException(p, - s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}") + val missing = p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",") + val from = p.inputSet.map(_.name).mkString("{", ", ", "}") + + throw new AnalysisException(s"Cannot resolve '$missing' given input columns $from") case p if !p.resolved && p.childrenResolved => - throw new TreeNodeException(p, "Unresolved plan found") + throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}") } match { // As a backstop, use the root node to check that the entire plan tree is resolved. case p if !p.resolved => - throw new TreeNodeException(p, "Unresolved plan in tree") + throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}") case p => p } } @@ -314,10 +317,11 @@ class Analyzer(catalog: Catalog, val checkField = (f: StructField) => resolver(f.name, fieldName) val ordinal = fields.indexWhere(checkField) if (ordinal == -1) { - sys.error( + throw new AnalysisException( s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - sys.error(s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") } else { ordinal } @@ -329,7 +333,8 @@ class Analyzer(catalog: Catalog, case ArrayType(StructType(fields), containsNull) => val ordinal = findField(fields) ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => sys.error(s"GetField is not valid on fields of type $otherType") + case otherType => + throw new AnalysisException(s"GetField is not valid on fields of type $otherType") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 60060bf02913b..f011a5ff15ea9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ @@ -69,12 +69,12 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) - val e = intercept[TreeNodeException[_]] { + val e = intercept[AnalysisException] { caseSensitiveAnalyze( Project(Seq(UnresolvedAttribute("tBl.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) } - assert(e.getMessage().toLowerCase.contains("unresolved")) + assert(e.getMessage().toLowerCase.contains("cannot resolve")) assert( caseInsensitiveAnalyze( @@ -109,10 +109,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { } test("throw errors for unresolved attributes during analysis") { - val e = intercept[TreeNodeException[_]] { + val e = intercept[AnalysisException] { caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) } - assert(e.getMessage().toLowerCase.contains("unresolved attribute")) + assert(e.getMessage().toLowerCase.contains("cannot resolve")) } test("throw errors for unresolved plans during analysis") { @@ -120,10 +120,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { override lazy val resolved = false override def output = Nil } - val e = intercept[TreeNodeException[_]] { + val e = intercept[AnalysisException] { caseSensitiveAnalyze(UnresolvedTestPlan()) } - assert(e.getMessage().toLowerCase.contains("unresolved plan")) + assert(e.getMessage().toLowerCase.contains("unresolved")) } test("divide should be casted into fractional types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 11502edf972e9..55fd0b0892fa1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -589,7 +589,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Row(_))) // Column type mismatches where a coercion is not possible, in this case between integer // and array types, trigger a TreeNodeException. - intercept[TreeNodeException[_]] { + intercept[AnalysisException] { sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect() } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index ff8130ae5f6bc..ab5f9cdddf508 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.test.TestHive.{sparkContext, jsonRDD, sql} import org.apache.spark.sql.hive.test.TestHive.implicits._ @@ -40,7 +41,7 @@ class HiveResolutionSuite extends HiveComparisonTest { """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested") // there are 2 filed matching field name "b", we should report Ambiguous reference error - val exception = intercept[RuntimeException] { + val exception = intercept[AnalysisException] { sql("SELECT a[0].b from nested").queryExecution.analyzed } assert(exception.getMessage.contains("Ambiguous reference to fields")) From a60aea86b4d4b716b5ec3bff776b509fe0831342 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 10 Feb 2015 18:19:56 -0800 Subject: [PATCH 055/817] [SPARK-5683] [SQL] Avoid multiple json generator created Author: Cheng Hao Closes #4468 from chenghao-intel/json and squashes the following commits: aeb7801 [Cheng Hao] avoid multiple json generator created --- .../org/apache/spark/sql/DataFrameImpl.scala | 24 +++++++++++++++++-- .../org/apache/spark/sql/json/JsonRDD.scala | 13 +++------- .../org/apache/spark/sql/json/JsonSuite.scala | 8 +++---- 3 files changed, 29 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 11f9334556981..0134b038f3c5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.io.CharArrayWriter + import scala.language.implicitConversions import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -380,8 +382,26 @@ private[sql] class DataFrameImpl protected[sql]( override def toJSON: RDD[String] = { val rowSchema = this.schema this.mapPartitions { iter => - val jsonFactory = new JsonFactory() - iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory)) + val writer = new CharArrayWriter() + // create the Generator without separator inserted between 2 records + val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) + + new Iterator[String] { + override def hasNext() = iter.hasNext + override def next(): String = { + JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) + gen.flush() + + val json = writer.toString + if (hasNext) { + writer.reset() + } else { + gen.close() + } + + json + } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 33ce71b51b213..1043eefcfc6a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -23,8 +23,7 @@ import java.sql.{Date, Timestamp} import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} -import com.fasterxml.jackson.core.JsonProcessingException -import com.fasterxml.jackson.core.JsonFactory +import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException, JsonFactory} import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD @@ -430,14 +429,11 @@ private[sql] object JsonRDD extends Logging { /** Transforms a single Row to JSON using Jackson * - * @param jsonFactory a JsonFactory object to construct a JsonGenerator * @param rowSchema the schema object used for conversion + * @param gen a JsonGenerator object * @param row The row to convert */ - private[sql] def rowToJSON(rowSchema: StructType, jsonFactory: JsonFactory)(row: Row): String = { - val writer = new StringWriter() - val gen = jsonFactory.createGenerator(writer) - + private[sql] def rowToJSON(rowSchema: StructType, gen: JsonGenerator)(row: Row) = { def valWriter: (DataType, Any) => Unit = { case (_, null) | (NullType, _) => gen.writeNull() case (StringType, v: String) => gen.writeString(v) @@ -479,8 +475,5 @@ private[sql] object JsonRDD extends Logging { } valWriter(rowSchema, row) - gen.close() - writer.toString } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 7870cf9b0a868..4fc92e3e3b8c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -824,8 +824,8 @@ class JsonSuite extends QueryTest { df1.registerTempTable("applySchema1") val df2 = df1.toDataFrame val result = df2.toJSON.collect() - assert(result(0) == "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") - assert(result(3) == "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") + assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") + assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") val schema2 = StructType( StructField("f1", StructType( @@ -846,8 +846,8 @@ class JsonSuite extends QueryTest { val df4 = df3.toDataFrame val result2 = df4.toJSON.collect() - assert(result2(1) == "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}") - assert(result2(3) == "{\"f1\":{\"f11\":4,\"f12\":true},\"f2\":{\"D4\":2147483644}}") + assert(result2(1) === "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}") + assert(result2(3) === "{\"f1\":{\"f11\":4,\"f12\":true},\"f2\":{\"D4\":2147483644}}") val jsonDF = jsonRDD(primitiveFieldAndType) val primTable = jsonRDD(jsonDF.toJSON) From ea60284095cad43aa7ac98256576375d0e91a52a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 10 Feb 2015 19:40:12 -0800 Subject: [PATCH 056/817] [SPARK-5704] [SQL] [PySpark] createDataFrame from RDD with columns Deprecate inferSchema() and applySchema(), use createDataFrame() instead, which could take an optional `schema` to create an DataFrame from an RDD. The `schema` could be StructType or list of names of columns. Author: Davies Liu Closes #4498 from davies/create and squashes the following commits: 08469c1 [Davies Liu] remove Scala/Java API for now c80a7a9 [Davies Liu] fix hive test d1bd8f2 [Davies Liu] cleanup applySchema 9526e97 [Davies Liu] createDataFrame from RDD with columns --- docs/ml-guide.md | 12 +-- docs/sql-programming-guide.md | 16 ++-- .../ml/JavaCrossValidatorExample.java | 4 +- .../examples/ml/JavaDeveloperApiExample.java | 4 +- .../examples/ml/JavaSimpleParamsExample.java | 4 +- .../JavaSimpleTextClassificationPipeline.java | 4 +- .../spark/examples/sql/JavaSparkSQL.java | 2 +- examples/src/main/python/sql.py | 4 +- .../spark/ml/tuning/CrossValidator.scala | 4 +- .../apache/spark/ml/JavaPipelineSuite.java | 2 +- .../JavaLogisticRegressionSuite.java | 2 +- .../regression/JavaLinearRegressionSuite.java | 2 +- .../ml/tuning/JavaCrossValidatorSuite.java | 2 +- python/pyspark/sql/context.py | 87 +++++++++++++---- python/pyspark/sql/tests.py | 26 ++--- .../org/apache/spark/sql/SQLContext.scala | 95 +++++++++++++++++-- .../spark/sql/ColumnExpressionSuite.scala | 4 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 9 +- .../spark/sql/execution/PlannerSuite.scala | 2 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 18 ++-- .../org/apache/spark/sql/json/JsonSuite.scala | 4 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 8 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- 23 files changed, 222 insertions(+), 97 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index be178d7689fdd..4bf14fba34eec 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -260,7 +260,7 @@ List localTraining = Lists.newArrayList( new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); -JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); +JavaSchemaRDD training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -300,7 +300,7 @@ List localTest = Lists.newArrayList( new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); -JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); +JavaSchemaRDD test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. @@ -443,7 +443,7 @@ List localTraining = Lists.newArrayList( new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); JavaSchemaRDD training = - jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -469,7 +469,7 @@ List localTest = Lists.newArrayList( new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); JavaSchemaRDD test = - jsql.applySchema(jsc.parallelize(localTest), Document.class); + jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. model.transform(test).registerAsTable("prediction"); @@ -626,7 +626,7 @@ List localTraining = Lists.newArrayList( new LabeledDocument(10L, "spark compile", 1.0), new LabeledDocument(11L, "hadoop software", 0.0)); JavaSchemaRDD training = - jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -669,7 +669,7 @@ List localTest = Lists.newArrayList( new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); -JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); +JavaSchemaRDD test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test).registerAsTable("prediction"); diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 38f617d0c836c..b2b007509c735 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -225,7 +225,7 @@ public static class Person implements Serializable { {% endhighlight %} -A schema can be applied to an existing RDD by calling `applySchema` and providing the Class object +A schema can be applied to an existing RDD by calling `createDataFrame` and providing the Class object for the JavaBean. {% highlight java %} @@ -247,7 +247,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m }); // Apply a schema to an RDD of JavaBeans and register it as a table. -JavaSchemaRDD schemaPeople = sqlContext.applySchema(people, Person.class); +JavaSchemaRDD schemaPeople = sqlContext.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. @@ -315,7 +315,7 @@ a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of `Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided by `SQLContext`. For example: @@ -341,7 +341,7 @@ val schema = val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. -val peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema) +val peopleSchemaRDD = sqlContext.createDataFrame(rowRDD, schema) // Register the SchemaRDD as a table. peopleSchemaRDD.registerTempTable("people") @@ -367,7 +367,7 @@ a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of `Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided by `JavaSQLContext`. For example: @@ -406,7 +406,7 @@ JavaRDD rowRDD = people.map( }); // Apply the schema to the RDD. -JavaSchemaRDD peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema); +JavaSchemaRDD peopleSchemaRDD = sqlContext.createDataFrame(rowRDD, schema); // Register the SchemaRDD as a table. peopleSchemaRDD.registerTempTable("people"); @@ -436,7 +436,7 @@ a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `applySchema` method provided by `SQLContext`. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SQLContext`. For example: {% highlight python %} @@ -458,7 +458,7 @@ fields = [StructField(field_name, StringType(), True) for field_name in schemaSt schema = StructType(fields) # Apply the schema to the RDD. -schemaPeople = sqlContext.applySchema(people, schema) +schemaPeople = sqlContext.createDataFrame(people, schema) # Register the SchemaRDD as a table. schemaPeople.registerTempTable("people") diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 5041e0b6d34b0..5d8c5d0a92daa 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -71,7 +71,7 @@ public static void main(String[] args) { new LabeledDocument(9L, "a e c l", 0.0), new LabeledDocument(10L, "spark compile", 1.0), new LabeledDocument(11L, "hadoop software", 0.0)); - DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -112,7 +112,7 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test).registerTempTable("prediction"); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 4d9dad9f23038..19d0eb216848e 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -62,7 +62,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); @@ -80,7 +80,7 @@ public static void main(String[] args) throws Exception { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). DataFrame results = model.transform(test); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index cc69e6315fdda..4c4d532388781 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -54,7 +54,7 @@ public static void main(String[] args) { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -94,7 +94,7 @@ public static void main(String[] args) { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index d929f1ad2014a..fdcfc888c235f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -54,7 +54,7 @@ public static void main(String[] args) { new LabeledDocument(1L, "b d", 0.0), new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); - DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -79,7 +79,7 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - DataFrame test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. model.transform(test).registerTempTable("prediction"); diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 8defb769ffaaf..dee794840a3e1 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -74,7 +74,7 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - DataFrame schemaPeople = sqlCtx.applySchema(people, Person.class); + DataFrame schemaPeople = sqlCtx.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 7f5c68e3d0fe2..47202fde7510b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -31,7 +31,7 @@ Row(name="Smith", age=23), Row(name="Sarah", age=18)]) # Infer schema from the first row, create a DataFrame and print the schema - some_df = sqlContext.inferSchema(some_rdd) + some_df = sqlContext.createDataFrame(some_rdd) some_df.printSchema() # Another RDD is created from a list of tuples @@ -40,7 +40,7 @@ schema = StructType([StructField("person_name", StringType(), False), StructField("person_age", IntegerType(), False)]) # Create a DataFrame by applying the schema to the RDD and print the schema - another_df = sqlContext.applySchema(another_rdd, schema) + another_df = sqlContext.createDataFrame(another_rdd, schema) another_df.printSchema() # root # |-- age: integer (nullable = true) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 5d51c51346665..324b1ba784387 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -76,8 +76,8 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP val metrics = new Array[Double](epm.size) val splits = MLUtils.kFold(dataset.rdd, map(numFolds), 0) splits.zipWithIndex.foreach { case ((training, validation), splitIndex) => - val trainingDataset = sqlCtx.applySchema(training, schema).cache() - val validationDataset = sqlCtx.applySchema(validation, schema).cache() + val trainingDataset = sqlCtx.createDataFrame(training, schema).cache() + val validationDataset = sqlCtx.createDataFrame(validation, schema).cache() // multi-model training logDebug(s"Train split $splitIndex with multiple sets of parameters.") val models = est.fit(trainingDataset, epm).asInstanceOf[Seq[Model[_]]] diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java index 50995ffef9ad5..0a8c9e5954676 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -45,7 +45,7 @@ public void setUp() { jsql = new SQLContext(jsc); JavaRDD points = jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2); - dataset = jsql.applySchema(points, LabeledPoint.class); + dataset = jsql.createDataFrame(points, LabeledPoint.class); } @After diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index d4b664479255d..3f8e59de0f05c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -50,7 +50,7 @@ public void setUp() { jsql = new SQLContext(jsc); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); + dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); } diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java index 40d5a92bb32af..0cc36c8d56d70 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -46,7 +46,7 @@ public void setUp() { jsql = new SQLContext(jsc); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); + dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); dataset.registerTempTable("dataset"); } diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java index 074b58c07df7a..0bb6b489f2757 100644 --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java @@ -45,7 +45,7 @@ public void setUp() { jsc = new JavaSparkContext("local", "JavaCrossValidatorSuite"); jsql = new SQLContext(jsc); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); - dataset = jsql.applySchema(jsc.parallelize(points, 2), LabeledPoint.class); + dataset = jsql.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); } @After diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 882c0f98ea40b..9d29ef4839a43 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -25,7 +25,7 @@ from pyspark.rdd import _prepare_for_python_RDD from pyspark.serializers import AutoBatchedSerializer, PickleSerializer -from pyspark.sql.types import StringType, StructType, _verify_type, \ +from pyspark.sql.types import StringType, StructType, _infer_type, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter from pyspark.sql.dataframe import DataFrame @@ -47,23 +47,11 @@ def __init__(self, sparkContext, sqlContext=None): :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new SQLContext in the JVM, instead we make all calls to this object. - >>> df = sqlCtx.inferSchema(rdd) - >>> sqlCtx.inferSchema(df) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - TypeError:... - - >>> bad_rdd = sc.parallelize([1,2,3]) - >>> sqlCtx.inferSchema(bad_rdd) # doctest: +IGNORE_EXCEPTION_DETAIL - Traceback (most recent call last): - ... - ValueError:... - >>> from datetime import datetime >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) - >>> df = sqlCtx.inferSchema(allTypes) + >>> df = sqlCtx.createDataFrame(allTypes) >>> df.registerTempTable("allTypes") >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() @@ -131,6 +119,9 @@ def registerFunction(self, name, f, returnType=StringType()): def inferSchema(self, rdd, samplingRatio=None): """Infer and apply a schema to an RDD of L{Row}. + ::note: + Deprecated in 1.3, use :func:`createDataFrame` instead + When samplingRatio is specified, the schema is inferred by looking at the types of each row in the sampled dataset. Otherwise, the first 100 rows of the RDD are inspected. Nested collections are @@ -199,7 +190,7 @@ def inferSchema(self, rdd, samplingRatio=None): warnings.warn("Some of types cannot be determined by the " "first 100 rows, please try again with sampling") else: - if samplingRatio > 0.99: + if samplingRatio < 0.99: rdd = rdd.sample(False, float(samplingRatio)) schema = rdd.map(_infer_schema).reduce(_merge_type) @@ -211,6 +202,9 @@ def applySchema(self, rdd, schema): """ Applies the given schema to the given RDD of L{tuple} or L{list}. + ::note: + Deprecated in 1.3, use :func:`createDataFrame` instead + These tuples or lists can contain complex nested structures like lists, maps or nested rows. @@ -300,13 +294,68 @@ def applySchema(self, rdd, schema): df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) return DataFrame(df, self) + def createDataFrame(self, rdd, schema=None, samplingRatio=None): + """ + Create a DataFrame from an RDD of tuple/list and an optional `schema`. + + `schema` could be :class:`StructType` or a list of column names. + + When `schema` is a list of column names, the type of each column + will be inferred from `rdd`. + + When `schema` is None, it will try to infer the column name and type + from `rdd`, which should be an RDD of :class:`Row`, or namedtuple, + or dict. + + If referring needed, `samplingRatio` is used to determined how many + rows will be used to do referring. The first row will be used if + `samplingRatio` is None. + + :param rdd: an RDD of Row or tuple or list or dict + :param schema: a StructType or list of names of columns + :param samplingRatio: the sample ratio of rows used for inferring + :return: a DataFrame + + >>> rdd = sc.parallelize([('Alice', 1)]) + >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) + >>> df.collect() + [Row(name=u'Alice', age=1)] + + >>> from pyspark.sql import Row + >>> Person = Row('name', 'age') + >>> person = rdd.map(lambda r: Person(*r)) + >>> df2 = sqlCtx.createDataFrame(person) + >>> df2.collect() + [Row(name=u'Alice', age=1)] + + >>> from pyspark.sql.types import * + >>> schema = StructType([ + ... StructField("name", StringType(), True), + ... StructField("age", IntegerType(), True)]) + >>> df3 = sqlCtx.createDataFrame(rdd, schema) + >>> df3.collect() + [Row(name=u'Alice', age=1)] + """ + if isinstance(rdd, DataFrame): + raise TypeError("rdd is already a DataFrame") + + if isinstance(schema, StructType): + return self.applySchema(rdd, schema) + else: + if isinstance(schema, (list, tuple)): + first = rdd.first() + if not isinstance(first, (list, tuple)): + raise ValueError("each row in `rdd` should be list or tuple") + row_cls = Row(*schema) + rdd = rdd.map(lambda r: row_cls(*r)) + return self.inferSchema(rdd, samplingRatio) + def registerRDDAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. Temporary tables exist only during the lifetime of this instance of SQLContext. - >>> df = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(df, "table1") """ if (rdd.__class__ is DataFrame): @@ -321,7 +370,6 @@ def parquetFile(self, *paths): >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) - >>> df = sqlCtx.inferSchema(rdd) >>> df.saveAsParquetFile(parquetFile) >>> df2 = sqlCtx.parquetFile(parquetFile) >>> sorted(df.collect()) == sorted(df2.collect()) @@ -526,7 +574,6 @@ def createExternalTable(self, tableName, path=None, source=None, def sql(self, sqlQuery): """Return a L{DataFrame} representing the result of the given query. - >>> df = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(df, "table1") >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() @@ -537,7 +584,6 @@ def sql(self, sqlQuery): def table(self, tableName): """Returns the specified table as a L{DataFrame}. - >>> df = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(df, "table1") >>> df2 = sqlCtx.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) @@ -685,11 +731,12 @@ def _test(): sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc globs['sqlCtx'] = sqlCtx = SQLContext(sc) - globs['rdd'] = sc.parallelize( + globs['rdd'] = rdd = sc.parallelize( [Row(field1=1, field2="row1"), Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) + globs['df'] = sqlCtx.createDataFrame(rdd) jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index bc945091f7042..5e41e36897b5d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -96,7 +96,7 @@ def setUpClass(cls): cls.sqlCtx = SQLContext(cls.sc) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] rdd = cls.sc.parallelize(cls.testData) - cls.df = cls.sqlCtx.inferSchema(rdd) + cls.df = cls.sqlCtx.createDataFrame(rdd) @classmethod def tearDownClass(cls): @@ -110,14 +110,14 @@ def test_udf(self): def test_udf2(self): self.sqlCtx.registerFunction("strlen", lambda string: len(string), IntegerType()) - self.sqlCtx.inferSchema(self.sc.parallelize([Row(a="test")])).registerTempTable("test") + self.sqlCtx.createDataFrame(self.sc.parallelize([Row(a="test")])).registerTempTable("test") [res] = self.sqlCtx.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() self.assertEqual(4, res[0]) def test_udf_with_array_type(self): d = [Row(l=range(3), d={"key": range(5)})] rdd = self.sc.parallelize(d) - self.sqlCtx.inferSchema(rdd).registerTempTable("test") + self.sqlCtx.createDataFrame(rdd).registerTempTable("test") self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() @@ -155,17 +155,17 @@ def test_basic_functions(self): def test_apply_schema_to_row(self): df = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""])) - df2 = self.sqlCtx.applySchema(df.map(lambda x: x), df.schema()) + df2 = self.sqlCtx.createDataFrame(df.map(lambda x: x), df.schema()) self.assertEqual(df.collect(), df2.collect()) rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) - df3 = self.sqlCtx.applySchema(rdd, df.schema()) + df3 = self.sqlCtx.createDataFrame(rdd, df.schema()) self.assertEqual(10, df3.count()) def test_serialize_nested_array_and_map(self): d = [Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})] rdd = self.sc.parallelize(d) - df = self.sqlCtx.inferSchema(rdd) + df = self.sqlCtx.createDataFrame(rdd) row = df.head() self.assertEqual(1, len(row.l)) self.assertEqual(1, row.l[0].a) @@ -187,14 +187,14 @@ def test_infer_schema(self): d = [Row(l=[], d={}), Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")] rdd = self.sc.parallelize(d) - df = self.sqlCtx.inferSchema(rdd) + df = self.sqlCtx.createDataFrame(rdd) self.assertEqual([], df.map(lambda r: r.l).first()) self.assertEqual([None, ""], df.map(lambda r: r.s).collect()) df.registerTempTable("test") result = self.sqlCtx.sql("SELECT l[0].a from test where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) - df2 = self.sqlCtx.inferSchema(rdd, 1.0) + df2 = self.sqlCtx.createDataFrame(rdd, 1.0) self.assertEqual(df.schema(), df2.schema()) self.assertEqual({}, df2.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.map(lambda r: r.s).collect()) @@ -205,7 +205,7 @@ def test_infer_schema(self): def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] rdd = self.sc.parallelize(d) - df = self.sqlCtx.inferSchema(rdd) + df = self.sqlCtx.createDataFrame(rdd) k, v = df.head().m.items()[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -214,7 +214,7 @@ def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) rdd = self.sc.parallelize([row]) - df = self.sqlCtx.inferSchema(rdd) + df = self.sqlCtx.createDataFrame(rdd) df.registerTempTable("test") row = self.sqlCtx.sql("select l, d from test").head() self.assertEqual(1, row.asDict()["l"][0].a) @@ -224,7 +224,7 @@ def test_infer_schema_with_udt(self): from pyspark.sql.tests import ExamplePoint, ExamplePointUDT row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) rdd = self.sc.parallelize([row]) - df = self.sqlCtx.inferSchema(rdd) + df = self.sqlCtx.createDataFrame(rdd) schema = df.schema() field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), ExamplePointUDT) @@ -238,7 +238,7 @@ def test_apply_schema_with_udt(self): rdd = self.sc.parallelize([row]) schema = StructType([StructField("label", DoubleType(), False), StructField("point", ExamplePointUDT(), False)]) - df = self.sqlCtx.applySchema(rdd, schema) + df = self.sqlCtx.createDataFrame(rdd, schema) point = df.head().point self.assertEquals(point, ExamplePoint(1.0, 2.0)) @@ -246,7 +246,7 @@ def test_parquet_with_udt(self): from pyspark.sql.tests import ExamplePoint row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) rdd = self.sc.parallelize([row]) - df0 = self.sqlCtx.inferSchema(rdd) + df0 = self.sqlCtx.createDataFrame(rdd) output_dir = os.path.join(self.tempdir.name, "labeled_point") df0.saveAsParquetFile(output_dir) df1 = self.sqlCtx.parquetFile(output_dir) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 801505bceb956..523911d108029 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -243,7 +243,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * val people = * sc.textFile("examples/src/main/resources/people.txt").map( * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) - * val dataFrame = sqlContext. applySchema(people, schema) + * val dataFrame = sqlContext.createDataFrame(people, schema) * dataFrame.printSchema * // root * // |-- name: string (nullable = false) @@ -252,11 +252,9 @@ class SQLContext(@transient val sparkContext: SparkContext) * dataFrame.registerTempTable("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} - * - * @group userf */ @DeveloperApi - def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { + def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) @@ -264,8 +262,21 @@ class SQLContext(@transient val sparkContext: SparkContext) } @DeveloperApi - def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { - applySchema(rowRDD.rdd, schema); + def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD.rdd, schema) + } + + /** + * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s by applying + * a seq of names of columns to this RDD, the data type for each column will + * be inferred by the first row. + * + * @param rowRDD an JavaRDD of Row + * @param columns names for each column + * @return DataFrame + */ + def createDataFrame(rowRDD: JavaRDD[Row], columns: java.util.List[String]): DataFrame = { + createDataFrame(rowRDD.rdd, columns.toSeq) } /** @@ -274,7 +285,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. */ - def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = { val attributeSeq = getSchema(beanClass) val className = beanClass.getName val rowRdd = rdd.mapPartitions { iter => @@ -301,8 +312,72 @@ class SQLContext(@transient val sparkContext: SparkContext) * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. */ + def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd.rdd, beanClass) + } + + /** + * :: DeveloperApi :: + * Creates a [[DataFrame]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD. + * It is important to make sure that the structure of every [[Row]] of the provided RDD matches + * the provided schema. Otherwise, there will be runtime exception. + * Example: + * {{{ + * import org.apache.spark.sql._ + * val sqlContext = new org.apache.spark.sql.SQLContext(sc) + * + * val schema = + * StructType( + * StructField("name", StringType, false) :: + * StructField("age", IntegerType, true) :: Nil) + * + * val people = + * sc.textFile("examples/src/main/resources/people.txt").map( + * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) + * val dataFrame = sqlContext. applySchema(people, schema) + * dataFrame.printSchema + * // root + * // |-- name: string (nullable = false) + * // |-- age: integer (nullable = true) + * + * dataFrame.registerTempTable("people") + * sqlContext.sql("select name from people").collect.foreach(println) + * }}} + * + * @group userf + */ + @DeveloperApi + @deprecated("use createDataFrame", "1.3.0") + def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + @DeveloperApi + @deprecated("use createDataFrame", "1.3.0") + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, + * SELECT * queries will return the columns in an undefined order. + */ + @deprecated("use createDataFrame", "1.3.0") + def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, + * SELECT * queries will return the columns in an undefined order. + */ + @deprecated("use createDataFrame", "1.3.0") def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { - applySchema(rdd.rdd, beanClass) + createDataFrame(rdd, beanClass) } /** @@ -375,7 +450,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - applySchema(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema) } @Experimental @@ -393,7 +468,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - applySchema(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema) } @Experimental diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index fa4cdecbcb340..1d71039872434 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -180,7 +180,7 @@ class ColumnExpressionSuite extends QueryTest { } test("!==") { - val nullData = TestSQLContext.applySchema(TestSQLContext.sparkContext.parallelize( + val nullData = TestSQLContext.createDataFrame(TestSQLContext.sparkContext.parallelize( Row(1, 1) :: Row(1, 2) :: Row(1, null) :: @@ -240,7 +240,7 @@ class ColumnExpressionSuite extends QueryTest { testData2.collect().toSeq.filter(r => r.getInt(0) <= r.getInt(1))) } - val booleanData = TestSQLContext.applySchema(TestSQLContext.sparkContext.parallelize( + val booleanData = TestSQLContext.createDataFrame(TestSQLContext.sparkContext.parallelize( Row(false, false) :: Row(false, true) :: Row(true, false) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 55fd0b0892fa1..bba8899651259 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -34,6 +34,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { TestData import org.apache.spark.sql.test.TestSQLContext.implicits._ + val sqlCtx = TestSQLContext test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( @@ -669,7 +670,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(values(0).toInt, values(1), values(2).toBoolean, v4) } - val df1 = applySchema(rowRDD1, schema1) + val df1 = sqlCtx.createDataFrame(rowRDD1, schema1) df1.registerTempTable("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), @@ -699,7 +700,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) } - val df2 = applySchema(rowRDD2, schema2) + val df2 = sqlCtx.createDataFrame(rowRDD2, schema2) df2.registerTempTable("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), @@ -724,7 +725,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(Row(values(0).toInt, values(2).toBoolean), scala.collection.mutable.Map(values(1) -> v4)) } - val df3 = applySchema(rowRDD3, schema2) + val df3 = sqlCtx.createDataFrame(rowRDD3, schema2) df3.registerTempTable("applySchema3") checkAnswer( @@ -769,7 +770,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { .build() val schemaWithMeta = new StructType(Array( schema("id"), schema("name").copy(metadata = metadata), schema("age"))) - val personWithMeta = applySchema(person.rdd, schemaWithMeta) + val personWithMeta = sqlCtx.createDataFrame(person.rdd, schemaWithMeta) def validateMetadata(rdd: DataFrame): Unit = { assert(rdd.schema("name").metadata.getString(docKey) == docValue) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index df108a9d262bb..c3210733f1d42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -71,7 +71,7 @@ class PlannerSuite extends FunSuite { val schema = StructType(fields) val row = Row.fromSeq(Seq.fill(fields.size)(null)) val rowRDD = org.apache.spark.sql.test.TestSQLContext.sparkContext.parallelize(row :: Nil) - applySchema(rowRDD, schema).registerTempTable("testLimit") + createDataFrame(rowRDD, schema).registerTempTable("testLimit") val planned = sql( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index e581ac9b50c2b..21e70936102fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -54,7 +54,7 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { StructField("seq", IntegerType) :: Nil) test("Basic CREATE") { - val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) srdd.createJDBCTable(url, "TEST.BASICCREATETEST", false) assert(2 == TestSQLContext.jdbcRDD(url, "TEST.BASICCREATETEST").count) @@ -62,8 +62,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { } test("CREATE with overwrite") { - val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) - val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) + val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) srdd.createJDBCTable(url, "TEST.DROPTEST", false) assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) @@ -75,8 +75,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { } test("CREATE then INSERT to append") { - val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) srdd.createJDBCTable(url, "TEST.APPENDTEST", false) srdd2.insertIntoJDBC(url, "TEST.APPENDTEST", false) @@ -85,8 +85,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { } test("CREATE then INSERT to truncate") { - val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) srdd.createJDBCTable(url, "TEST.TRUNCATETEST", false) srdd2.insertIntoJDBC(url, "TEST.TRUNCATETEST", true) @@ -95,8 +95,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { } test("Incompatible INSERT to append") { - val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) + val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) srdd.createJDBCTable(url, "TEST.INCOMPATIBLETEST", false) intercept[org.apache.spark.SparkException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 4fc92e3e3b8c0..fde4b47438c56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -820,7 +820,7 @@ class JsonSuite extends QueryTest { Row(values(0).toInt, values(1), values(2).toBoolean, r.split(",").toList, v5) } - val df1 = applySchema(rowRDD1, schema1) + val df1 = createDataFrame(rowRDD1, schema1) df1.registerTempTable("applySchema1") val df2 = df1.toDataFrame val result = df2.toJSON.collect() @@ -841,7 +841,7 @@ class JsonSuite extends QueryTest { Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) } - val df3 = applySchema(rowRDD2, schema2) + val df3 = createDataFrame(rowRDD2, schema2) df3.registerTempTable("applySchema2") val df4 = df3.toDataFrame val result2 = df4.toJSON.collect() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 43da7519ac8db..89b18c3439cf6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -97,7 +97,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) val rowRDD = TestHive.sparkContext.parallelize( (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) - val df = applySchema(rowRDD, schema) + val df = TestHive.createDataFrame(rowRDD, schema) df.registerTempTable("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m MAP )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -142,7 +142,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { val schema = StructType(Seq( StructField("a", ArrayType(StringType, containsNull = false)))) val rowRDD = TestHive.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) - val df = applySchema(rowRDD, schema) + val df = TestHive.createDataFrame(rowRDD, schema) df.registerTempTable("tableWithArrayValue") sql("CREATE TABLE hiveTableWithArrayValue(a Array )") sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") @@ -159,7 +159,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { StructField("m", MapType(StringType, StringType, valueContainsNull = false)))) val rowRDD = TestHive.sparkContext.parallelize( (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) - val df = applySchema(rowRDD, schema) + val df = TestHive.createDataFrame(rowRDD, schema) df.registerTempTable("tableWithMapValue") sql("CREATE TABLE hiveTableWithMapValue(m Map )") sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") @@ -176,7 +176,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { StructField("s", StructType(Seq(StructField("f", StringType, nullable = false)))))) val rowRDD = TestHive.sparkContext.parallelize( (1 to 100).map(i => Row(Row(s"value$i")))) - val df = applySchema(rowRDD, schema) + val df = TestHive.createDataFrame(rowRDD, schema) df.registerTempTable("tableWithStructValue") sql("CREATE TABLE hiveTableWithStructValue(s Struct )") sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 49fe79d989259..9a6e8650a0ec4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLConf} @@ -34,6 +35,7 @@ case class Nested3(f3: Int) class SQLQuerySuite extends QueryTest { import org.apache.spark.sql.hive.test.TestHive.implicits._ + val sqlCtx = TestHive test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { checkAnswer( @@ -277,7 +279,7 @@ class SQLQuerySuite extends QueryTest { val rowRdd = sparkContext.parallelize(row :: Nil) - applySchema(rowRdd, schema).registerTempTable("testTable") + sqlCtx.createDataFrame(rowRdd, schema).registerTempTable("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes From 45df77b8418873a00d770e435358bf603765595f Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 10 Feb 2015 19:40:51 -0800 Subject: [PATCH 057/817] [SPARK-5709] [SQL] Add EXPLAIN support in DataFrame API for debugging purpose Author: Cheng Hao Closes #4496 from chenghao-intel/df_explain and squashes the following commits: 552aa58 [Cheng Hao] Add explain support for DF --- .../main/scala/org/apache/spark/sql/Column.scala | 8 ++++++++ .../main/scala/org/apache/spark/sql/DataFrame.scala | 6 ++++++ .../scala/org/apache/spark/sql/DataFrameImpl.scala | 13 ++++++++++--- .../org/apache/spark/sql/execution/commands.scala | 7 +++++-- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 8 +++----- 5 files changed, 32 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 1011bf0bb5ef4..b0e95908ee71a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -600,6 +600,14 @@ trait Column extends DataFrame { def desc: Column = exprToColumn(SortOrder(expr, Descending), computable = false) def asc: Column = exprToColumn(SortOrder(expr, Ascending), computable = false) + + override def explain(extended: Boolean): Unit = { + if (extended) { + println(expr) + } else { + println(expr.prettyString) + } + } } 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 ca8d552c5febf..17900c5ee3892 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 @@ -124,6 +124,12 @@ trait DataFrame extends RDDApi[Row] { /** Prints the schema to the console in a nice tree format. */ def printSchema(): Unit + /** Prints the plans (logical and physical) to the console for debugging purpose. */ + def explain(extended: Boolean): Unit + + /** Only prints the physical plan to the console for debugging purpose. */ + def explain(): Unit = explain(false) + /** * Returns true if the `collect` and `take` methods can be run locally * (without any Spark executors). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 0134b038f3c5a..9638ce0865db0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -30,12 +30,11 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.{SqlParser, ScalaReflection} -import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, ResolvedStar, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.sideBySide -import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} +import org.apache.spark.sql.execution.{ExplainCommand, LogicalRDD, EvaluatePython} import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{NumericType, StructType} @@ -115,6 +114,14 @@ private[sql] class DataFrameImpl protected[sql]( override def printSchema(): Unit = println(schema.treeString) + override def explain(extended: Boolean): Unit = { + ExplainCommand( + logicalPlan, + extended = extended).queryExecution.executedPlan.executeCollect().map { + r => println(r.getString(0)) + } + } + override def isLocal: Boolean = { logicalPlan.isInstanceOf[LocalRelation] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 335757087deef..2b1726ad4e89f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StringType import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import scala.collection.mutable.ArrayBuffer @@ -116,7 +117,9 @@ case class SetCommand( @DeveloperApi case class ExplainCommand( logicalPlan: LogicalPlan, - override val output: Seq[Attribute], extended: Boolean = false) extends RunnableCommand { + override val output: Seq[Attribute] = + Seq(AttributeReference("plan", StringType, nullable = false)()), + extended: Boolean = false) extends RunnableCommand { // Run through the optimizer to generate the physical plan. override def run(sqlContext: SQLContext) = try { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 8618301ba84d6..f3c9e63652a8e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -466,23 +466,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => - ExplainCommand(NoRelation, Seq(AttributeReference("plan", StringType, nullable = false)())) + ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) if "TOK_CREATETABLE" == explainArgs.head.getText => val Some(crtTbl) :: _ :: extended :: Nil = getClauses(Seq("TOK_CREATETABLE", "FORMATTED", "EXTENDED"), explainArgs) ExplainCommand( nodeToPlan(crtTbl), - Seq(AttributeReference("plan", StringType,nullable = false)()), - extended != None) + extended = extended.isDefined) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) ExplainCommand( nodeToPlan(query), - Seq(AttributeReference("plan", StringType, nullable = false)()), - extended != None) + extended = extended.isDefined) case Token("TOK_DESCTABLE", describeArgs) => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL From 7e24249af1e2f896328ef0402fa47db78cb6f9ec Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 10 Feb 2015 19:50:44 -0800 Subject: [PATCH 058/817] [SQL][DataFrame] Fix column computability bug. Do not recursively strip out projects. Only strip the first level project. ```scala df("colA") + df("colB").as("colC") ``` Previously, the above would construct an invalid plan. Author: Reynold Xin Closes #4519 from rxin/computability and squashes the following commits: 87ff763 [Reynold Xin] Code review feedback. 015c4fc [Reynold Xin] [SQL][DataFrame] Fix column computability. --- .../MatrixFactorizationModel.scala | 2 +- .../scala/org/apache/spark/sql/Column.scala | 35 ++++++++++++++----- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../spark/sql/ColumnExpressionSuite.scala | 13 +++++-- 4 files changed, 39 insertions(+), 15 deletions(-) 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 9ff06ac362a31..16979c9ed43ca 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 @@ -180,7 +180,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { def save(model: MatrixFactorizationModel, path: String): Unit = { val sc = model.userFeatures.sparkContext val sqlContext = new SQLContext(sc) - import sqlContext.implicits.createDataFrame + import sqlContext.implicits._ val metadata = (thisClassName, thisFormatVersion, model.rank) val metadataRDD = sc.parallelize(Seq(metadata), 1).toDataFrame("class", "version", "rank") metadataRDD.toJSON.saveAsTextFile(metadataPath(path)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index b0e95908ee71a..9d5d6e78bd487 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -66,27 +66,44 @@ trait Column extends DataFrame { */ def isComputable: Boolean + /** Removes the top project so we can get to the underlying plan. */ + private def stripProject(p: LogicalPlan): LogicalPlan = p match { + case Project(_, child) => child + case p => sys.error("Unexpected logical plan (expected Project): " + p) + } + private def computableCol(baseCol: ComputableColumn, expr: Expression) = { - val plan = Project(Seq(expr match { + val namedExpr = expr match { case named: NamedExpression => named case unnamed: Expression => Alias(unnamed, "col")() - }), baseCol.plan) + } + val plan = Project(Seq(namedExpr), stripProject(baseCol.plan)) Column(baseCol.sqlContext, plan, expr) } + /** + * Construct a new column based on the expression and the other column value. + * + * There are two cases that can happen here: + * If otherValue is a constant, it is first turned into a Column. + * If otherValue is a Column, then: + * - If this column and otherValue are both computable and come from the same logical plan, + * then we can construct a ComputableColumn by applying a Project on top of the base plan. + * - If this column is not computable, but otherValue is computable, then we can construct + * a ComputableColumn based on otherValue's base plan. + * - If this column is computable, but otherValue is not, then we can construct a + * ComputableColumn based on this column's base plan. + * - If neither columns are computable, then we create an IncomputableColumn. + */ private def constructColumn(otherValue: Any)(newExpr: Column => Expression): Column = { - // Removes all the top level projection and subquery so we can get to the underlying plan. - @tailrec def stripProject(p: LogicalPlan): LogicalPlan = p match { - case Project(_, child) => stripProject(child) - case Subquery(_, child) => stripProject(child) - case _ => p - } - + // lit(otherValue) returns a Column always. (this, lit(otherValue)) match { case (left: ComputableColumn, right: ComputableColumn) => if (stripProject(left.plan).sameResult(stripProject(right.plan))) { computableCol(right, newExpr(right)) } else { + // We don't want to throw an exception here because "df1("a") === df2("b")" can be + // a valid expression for join conditions, even though standalone they are not valid. Column(newExpr(right)) } case (left: ComputableColumn, right) => computableCol(left, newExpr(right)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 523911d108029..05ac1623d78ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -183,14 +183,14 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - implicit def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { + implicit def rddToDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { self.createDataFrame(rdd) } /** * Creates a DataFrame from a local Seq of Product. */ - implicit def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { + implicit def localSeqToDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { self.createDataFrame(data) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 1d71039872434..e3e6f652ed3ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types.{BooleanType, IntegerType, StructField, StructType} @@ -44,10 +45,10 @@ class ColumnExpressionSuite extends QueryTest { shouldBeComputable(-testData2("a")) shouldBeComputable(!testData2("a")) - shouldBeComputable(testData2.select(($"a" + 1).as("c"))("c") + testData2("b")) - shouldBeComputable( + shouldNotBeComputable(testData2.select(($"a" + 1).as("c"))("c") + testData2("b")) + shouldNotBeComputable( testData2.select(($"a" + 1).as("c"))("c") + testData2.select(($"b" / 2).as("d"))("d")) - shouldBeComputable( + shouldNotBeComputable( testData2.select(($"a" + 1).as("c")).select(($"c" + 2).as("d"))("d") + testData2("b")) // Literals and unresolved columns should not be computable. @@ -66,6 +67,12 @@ class ColumnExpressionSuite extends QueryTest { shouldNotBeComputable(sum(testData2("a"))) } + test("collect on column produced by a binary operator") { + val df = Seq((1, 2, 3)).toDataFrame("a", "b", "c") + checkAnswer(df("a") + df("b"), Seq(Row(3))) + checkAnswer(df("a") + df("b").as("c"), Seq(Row(3))) + } + test("star") { checkAnswer(testData.select($"*"), testData.collect().toSeq) } From 1cb37700753437045b15c457b983532cd5a27fa5 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 10 Feb 2015 20:12:18 -0800 Subject: [PATCH 059/817] [SPARK-4879] Use driver to coordinate Hadoop output committing for speculative tasks MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, SparkHadoopWriter always committed its tasks without question. The problem is that when speculation is enabled sometimes this can result in multiple tasks committing their output to the same file. Even though an HDFS-writing task may be re-launched due to speculation, the original task is not killed and may eventually commit as well. This can cause strange race conditions where multiple tasks that commit interfere with each other, with the result being that some partition files are actually lost entirely. For more context on these kinds of scenarios, see SPARK-4879. In Hadoop MapReduce jobs, the application master is a central coordinator that authorizes whether or not any given task can commit. Before a task commits its output, it queries the application master as to whether or not such a commit is safe, and the application master does bookkeeping as tasks are requesting commits. Duplicate tasks that would write to files that were already written to from other tasks are prohibited from committing. This patch emulates that functionality - the crucial missing component was a central arbitrator, which is now a module called the OutputCommitCoordinator. The coordinator lives on the driver and the executors can obtain a reference to this actor and request its permission to commit. As tasks commit and are reported as completed successfully or unsuccessfully by the DAGScheduler, the commit coordinator is informed of the task completion events as well to update its internal state. Future work includes more rigorous unit testing and extra optimizations should this patch cause a performance regression. It is unclear what the overall cost of communicating back to the driver on every hadoop-committing task will be. It's also important for those hitting this issue to backport this onto previous version of Spark because the bug has serious consequences, that is, data is lost. Currently, the OutputCommitCoordinator is only used when `spark.speculation` is true. It can be disabled by setting `spark.hadoop.outputCommitCoordination.enabled=false` in SparkConf. This patch is an updated version of #4155 (by mccheah), which in turn was an updated version of this PR. Closes #4155. Author: mcheah Author: Josh Rosen Closes #4066 from JoshRosen/SPARK-4879-sparkhadoopwriter-fix and squashes the following commits: 658116b [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix ed783b2 [Josh Rosen] Address Andrew’s feedback. e7be65a [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix 14861ea [Josh Rosen] splitID -> partitionID in a few places ed8b554 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix 48d5c1c [Josh Rosen] Roll back copiesRunning change in TaskSetManager 3969f5f [Josh Rosen] Re-enable guarding of commit coordination with spark.speculation setting. ede7590 [Josh Rosen] Add test to ensure that a job that denies all commits cannot complete successfully. 97da5fe [Josh Rosen] Use actor only for RPC; call methods directly in DAGScheduler. f582574 [Josh Rosen] Some cleanup in OutputCommitCoordinatorSuite a7c0e29 [Josh Rosen] Create fake TaskInfo using dummy fields instead of Mockito. 997b41b [Josh Rosen] Roll back unnecessary DAGSchedulerSingleThreadedProcessLoop refactoring: 459310a [Josh Rosen] Roll back TaskSetManager changes that broke other tests. dd00b7c [Josh Rosen] Move CommitDeniedException to executors package; remove `@DeveloperAPI` annotation. c79df98 [Josh Rosen] Some misc. code style + doc changes: f7d69c5 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4879-sparkhadoopwriter-fix 92e6dc9 [Josh Rosen] Bug fix: use task ID instead of StageID to index into authorizedCommitters. b344bad [Josh Rosen] (Temporarily) re-enable “always coordinate” for testing purposes. 0aec91e [Josh Rosen] Only coordinate when speculation is enabled; add configuration option to bypass new coordination. 594e41a [mcheah] Fixing a scalastyle error 60a47f4 [mcheah] Writing proper unit test for OutputCommitCoordinator and fixing bugs. d63f63f [mcheah] Fixing compiler error 9fe6495 [mcheah] Fixing scalastyle 1df2a91 [mcheah] Throwing exception if SparkHadoopWriter commit denied d431144 [mcheah] Using more concurrency to process OutputCommitCoordinator requests. c334255 [mcheah] Properly handling messages that could be sent after actor shutdown. 8d5a091 [mcheah] Was mistakenly serializing the accumulator in test suite. 9c6a4fa [mcheah] More OutputCommitCoordinator cleanup on stop() 78eb1b5 [mcheah] Better OutputCommitCoordinatorActor stopping; simpler canCommit 83de900 [mcheah] Making the OutputCommitCoordinatorMessage serializable abc7db4 [mcheah] TaskInfo can't be null in DAGSchedulerSuite f135a8e [mcheah] Moving the output commit coordinator from class into method. 1c2b219 [mcheah] Renaming oudated names for test function classes 66a71cd [mcheah] Removing whitespace modifications 6b543ba [mcheah] Removing redundant accumulator in unit test c9decc6 [mcheah] Scalastyle fixes bc80770 [mcheah] Unit tests for OutputCommitCoordinator 6e6f748 [mcheah] [SPARK-4879] Use the Spark driver to authorize Hadoop commits. --- .../scala/org/apache/spark/SparkContext.scala | 11 +- .../scala/org/apache/spark/SparkEnv.scala | 22 +- .../org/apache/spark/SparkHadoopWriter.scala | 43 +++- .../org/apache/spark/TaskEndReason.scala | 14 ++ .../executor/CommitDeniedException.scala | 35 +++ .../org/apache/spark/executor/Executor.scala | 5 + .../apache/spark/scheduler/DAGScheduler.scala | 15 +- .../scheduler/OutputCommitCoordinator.scala | 172 ++++++++++++++ .../spark/scheduler/TaskSchedulerImpl.scala | 9 +- .../spark/scheduler/TaskSetManager.scala | 8 +- .../spark/scheduler/DAGSchedulerSuite.scala | 25 +- .../OutputCommitCoordinatorSuite.scala | 213 ++++++++++++++++++ 12 files changed, 549 insertions(+), 23 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 53fce6b0defdf..24a316e40e673 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -249,7 +249,16 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) // Create the Spark execution environment (cache, map output tracker, etc) - private[spark] val env = SparkEnv.createDriverEnv(conf, isLocal, listenerBus) + + // This function allows components created by SparkEnv to be mocked in unit tests: + private[spark] def createSparkEnv( + conf: SparkConf, + isLocal: Boolean, + listenerBus: LiveListenerBus): SparkEnv = { + SparkEnv.createDriverEnv(conf, isLocal, listenerBus) + } + + private[spark] val env = createSparkEnv(conf, isLocal, listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index b63bea5b102b6..2a0c7e756dd3a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -34,7 +34,8 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.network.nio.NioBlockTransferService -import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus} +import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ @@ -67,6 +68,7 @@ class SparkEnv ( val sparkFilesDir: String, val metricsSystem: MetricsSystem, val shuffleMemoryManager: ShuffleMemoryManager, + val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { private[spark] var isStopped = false @@ -88,6 +90,7 @@ class SparkEnv ( blockManager.stop() blockManager.master.stop() metricsSystem.stop() + outputCommitCoordinator.stop() actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release @@ -169,7 +172,8 @@ object SparkEnv extends Logging { private[spark] def createDriverEnv( conf: SparkConf, isLocal: Boolean, - listenerBus: LiveListenerBus): SparkEnv = { + listenerBus: LiveListenerBus, + mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { assert(conf.contains("spark.driver.host"), "spark.driver.host is not set on the driver!") assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") val hostname = conf.get("spark.driver.host") @@ -181,7 +185,8 @@ object SparkEnv extends Logging { port, isDriver = true, isLocal = isLocal, - listenerBus = listenerBus + listenerBus = listenerBus, + mockOutputCommitCoordinator = mockOutputCommitCoordinator ) } @@ -220,7 +225,8 @@ object SparkEnv extends Logging { isDriver: Boolean, isLocal: Boolean, listenerBus: LiveListenerBus = null, - numUsableCores: Int = 0): SparkEnv = { + numUsableCores: Int = 0, + mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { // Listener bus is only used on the driver if (isDriver) { @@ -368,6 +374,13 @@ object SparkEnv extends Logging { "levels using the RDD.persist() method instead.") } + val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { + new OutputCommitCoordinator(conf) + } + val outputCommitCoordinatorActor = registerOrLookup("OutputCommitCoordinator", + new OutputCommitCoordinatorActor(outputCommitCoordinator)) + outputCommitCoordinator.coordinatorActor = Some(outputCommitCoordinatorActor) + val envInstance = new SparkEnv( executorId, actorSystem, @@ -384,6 +397,7 @@ object SparkEnv extends Logging { sparkFilesDir, metricsSystem, shuffleMemoryManager, + outputCommitCoordinator, conf) // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 40237596570de..6eb4537d10477 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path +import org.apache.spark.executor.CommitDeniedException import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD @@ -105,24 +106,56 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def commit() { val taCtxt = getTaskContext() val cmtr = getOutputCommitter() - if (cmtr.needsTaskCommit(taCtxt)) { + + // Called after we have decided to commit + def performCommit(): Unit = { try { cmtr.commitTask(taCtxt) - logInfo (taID + ": Committed") + logInfo (s"$taID: Committed") } catch { - case e: IOException => { + case e: IOException => logError("Error committing the output of task: " + taID.value, e) cmtr.abortTask(taCtxt) throw e + } + } + + // First, check whether the task's output has already been committed by some other attempt + if (cmtr.needsTaskCommit(taCtxt)) { + // The task output needs to be committed, but we don't know whether some other task attempt + // might be racing to commit the same output partition. Therefore, coordinate with the driver + // in order to determine whether this attempt can commit (see SPARK-4879). + val shouldCoordinateWithDriver: Boolean = { + val sparkConf = SparkEnv.get.conf + // We only need to coordinate with the driver if there are multiple concurrent task + // attempts, which should only occur if speculation is enabled + val speculationEnabled = sparkConf.getBoolean("spark.speculation", false) + // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs + sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) + } + if (shouldCoordinateWithDriver) { + val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator + val canCommit = outputCommitCoordinator.canCommit(jobID, splitID, attemptID) + if (canCommit) { + performCommit() + } else { + val msg = s"$taID: Not committed because the driver did not authorize commit" + logInfo(msg) + // We need to abort the task so that the driver can reschedule new attempts, if necessary + cmtr.abortTask(taCtxt) + throw new CommitDeniedException(msg, jobID, splitID, attemptID) } + } else { + // Speculation is disabled or a user has chosen to manually bypass the commit coordination + performCommit() } } else { - logInfo ("No need to commit output of task: " + taID.value) + // Some other attempt committed the output, so we do nothing and signal success + logInfo(s"No need to commit output of task because needsTaskCommit=false: ${taID.value}") } } def commitJob() { - // always ? Or if cmtr.needsTaskCommit ? val cmtr = getOutputCommitter() cmtr.commitJob(getJobContext()) } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index af5fd8e0ac00c..29a5cd5fdac76 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -146,6 +146,20 @@ case object TaskKilled extends TaskFailedReason { override def toErrorString: String = "TaskKilled (killed intentionally)" } +/** + * :: DeveloperApi :: + * Task requested the driver to commit, but was denied. + */ +@DeveloperApi +case class TaskCommitDenied( + jobID: Int, + partitionID: Int, + attemptID: Int) + extends TaskFailedReason { + override def toErrorString: String = s"TaskCommitDenied (Driver denied task commit)" + + s" for job: $jobID, partition: $partitionID, attempt: $attemptID" +} + /** * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala new file mode 100644 index 0000000000000..f7604a321f007 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.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.executor + +import org.apache.spark.{TaskCommitDenied, TaskEndReason} + +/** + * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver. + */ +class CommitDeniedException( + msg: String, + jobID: Int, + splitID: Int, + attemptID: Int) + extends Exception(msg) { + + def toTaskEndReason: TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID) + +} + diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6b22dcd6f5cbf..b684fb704956b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -253,6 +253,11 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } + case cDE: CommitDeniedException => { + val reason = cDE.toTaskEndReason + execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + } + case t: Throwable => { // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 1cfe98673773a..79035571adb05 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -38,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.{CallSite, EventLoop, SystemClock, Clock, Utils} +import org.apache.spark.util._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -63,7 +63,7 @@ class DAGScheduler( mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv, - clock: Clock = SystemClock) + clock: org.apache.spark.util.Clock = SystemClock) extends Logging { def this(sc: SparkContext, taskScheduler: TaskScheduler) = { @@ -126,6 +126,8 @@ class DAGScheduler( private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) + private val outputCommitCoordinator = env.outputCommitCoordinator + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessLoop.post(BeginEvent(task, taskInfo)) @@ -808,6 +810,7 @@ class DAGScheduler( // will be posted, which should always come after a corresponding SparkListenerStageSubmitted // event. stage.latestInfo = StageInfo.fromStage(stage, Some(partitionsToCompute.size)) + outputCommitCoordinator.stageStart(stage.id) listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. @@ -865,6 +868,7 @@ class DAGScheduler( } else { // Because we posted SparkListenerStageSubmitted earlier, we should post // SparkListenerStageCompleted here in case there are no tasks to run. + outputCommitCoordinator.stageEnd(stage.id) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -909,6 +913,9 @@ class DAGScheduler( val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) + outputCommitCoordinator.taskCompleted(stageId, task.partitionId, + event.taskInfo.attempt, event.reason) + // The success case is dealt with separately below, since we need to compute accumulator // updates before posting. if (event.reason != Success) { @@ -921,6 +928,7 @@ class DAGScheduler( // Skip all the actions if the stage has been cancelled. return } + val stage = stageIdToStage(task.stageId) def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { @@ -1073,6 +1081,9 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) } + case commitDenied: TaskCommitDenied => + // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits + case ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics) => // Do nothing here, left up to the TaskScheduler to decide how to handle user failures diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala new file mode 100644 index 0000000000000..759df023a6dcf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable + +import akka.actor.{ActorRef, Actor} + +import org.apache.spark._ +import org.apache.spark.util.{AkkaUtils, ActorLogReceive} + +private sealed trait OutputCommitCoordinationMessage extends Serializable + +private case object StopCoordinator extends OutputCommitCoordinationMessage +private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttempt: Long) + +/** + * Authority that decides whether tasks can commit output to HDFS. Uses a "first committer wins" + * policy. + * + * OutputCommitCoordinator is instantiated in both the drivers and executors. On executors, it is + * configured with a reference to the driver's OutputCommitCoordinatorActor, so requests to commit + * output will be forwarded to the driver's OutputCommitCoordinator. + * + * This class was introduced in SPARK-4879; see that JIRA issue (and the associated pull requests) + * for an extensive design discussion. + */ +private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { + + // Initialized by SparkEnv + var coordinatorActor: Option[ActorRef] = None + private val timeout = AkkaUtils.askTimeout(conf) + private val maxAttempts = AkkaUtils.numRetries(conf) + private val retryInterval = AkkaUtils.retryWaitMs(conf) + + private type StageId = Int + private type PartitionId = Long + private type TaskAttemptId = Long + + /** + * Map from active stages's id => partition id => task attempt with exclusive lock on committing + * output for that partition. + * + * Entries are added to the top-level map when stages start and are removed they finish + * (either successfully or unsuccessfully). + * + * Access to this map should be guarded by synchronizing on the OutputCommitCoordinator instance. + */ + private val authorizedCommittersByStage: CommittersByStageMap = mutable.Map() + private type CommittersByStageMap = mutable.Map[StageId, mutable.Map[PartitionId, TaskAttemptId]] + + /** + * Called by tasks to ask whether they can commit their output to HDFS. + * + * If a task attempt has been authorized to commit, then all other attempts to commit the same + * task will be denied. If the authorized task attempt fails (e.g. due to its executor being + * lost), then a subsequent task attempt may be authorized to commit its output. + * + * @param stage the stage number + * @param partition the partition number + * @param attempt a unique identifier for this task attempt + * @return true if this task is authorized to commit, false otherwise + */ + def canCommit( + stage: StageId, + partition: PartitionId, + attempt: TaskAttemptId): Boolean = { + val msg = AskPermissionToCommitOutput(stage, partition, attempt) + coordinatorActor match { + case Some(actor) => + AkkaUtils.askWithReply[Boolean](msg, actor, maxAttempts, retryInterval, timeout) + case None => + logError( + "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?") + false + } + } + + // Called by DAGScheduler + private[scheduler] def stageStart(stage: StageId): Unit = synchronized { + authorizedCommittersByStage(stage) = mutable.HashMap[PartitionId, TaskAttemptId]() + } + + // Called by DAGScheduler + private[scheduler] def stageEnd(stage: StageId): Unit = synchronized { + authorizedCommittersByStage.remove(stage) + } + + // Called by DAGScheduler + private[scheduler] def taskCompleted( + stage: StageId, + partition: PartitionId, + attempt: TaskAttemptId, + reason: TaskEndReason): Unit = synchronized { + val authorizedCommitters = authorizedCommittersByStage.getOrElse(stage, { + logDebug(s"Ignoring task completion for completed stage") + return + }) + reason match { + case Success => + // The task output has been committed successfully + case denied: TaskCommitDenied => + logInfo( + s"Task was denied committing, stage: $stage, partition: $partition, attempt: $attempt") + case otherReason => + logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" + + s" clearing lock") + authorizedCommitters.remove(partition) + } + } + + def stop(): Unit = synchronized { + coordinatorActor.foreach(_ ! StopCoordinator) + coordinatorActor = None + authorizedCommittersByStage.clear() + } + + // Marked private[scheduler] instead of private so this can be mocked in tests + private[scheduler] def handleAskPermissionToCommit( + stage: StageId, + partition: PartitionId, + attempt: TaskAttemptId): Boolean = synchronized { + authorizedCommittersByStage.get(stage) match { + case Some(authorizedCommitters) => + authorizedCommitters.get(partition) match { + case Some(existingCommitter) => + logDebug(s"Denying $attempt to commit for stage=$stage, partition=$partition; " + + s"existingCommitter = $existingCommitter") + false + case None => + logDebug(s"Authorizing $attempt to commit for stage=$stage, partition=$partition") + authorizedCommitters(partition) = attempt + true + } + case None => + logDebug(s"Stage $stage has completed, so not allowing task attempt $attempt to commit") + false + } + } +} + +private[spark] object OutputCommitCoordinator { + + // This actor is used only for RPC + class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator) + extends Actor with ActorLogReceive with Logging { + + override def receiveWithLogging = { + case AskPermissionToCommitOutput(stage, partition, taskAttempt) => + sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt) + case StopCoordinator => + logInfo("OutputCommitCoordinator stopped!") + context.stop(self) + sender ! true + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 79f84e70df9d5..54f8fcfc416d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -158,7 +158,7 @@ private[spark] class TaskSchedulerImpl( val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") this.synchronized { - val manager = new TaskSetManager(this, taskSet, maxTaskFailures) + val manager = createTaskSetManager(taskSet, maxTaskFailures) activeTaskSets(taskSet.id) = manager schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) @@ -180,6 +180,13 @@ private[spark] class TaskSchedulerImpl( backend.reviveOffers() } + // Label as private[scheduler] to allow tests to swap in different task set managers if necessary + private[scheduler] def createTaskSetManager( + taskSet: TaskSet, + maxTaskFailures: Int): TaskSetManager = { + new TaskSetManager(this, taskSet, maxTaskFailures) + } + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { logInfo("Cancelling stage " + stageId) activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 55024ecd55e61..99a5f7117790d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -292,7 +292,8 @@ private[spark] class TaskSetManager( * an attempt running on this host, in case the host is slow. In addition, the task should meet * the given locality constraint. */ - private def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) + // Labeled as protected to allow tests to override providing speculative tasks if necessary + protected def dequeueSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) : Option[(Int, TaskLocality.Value)] = { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set @@ -708,7 +709,10 @@ private[spark] class TaskSetManager( put(info.executorId, clock.getTime()) sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics) addPendingTask(index) - if (!isZombie && state != TaskState.KILLED) { + if (!isZombie && state != TaskState.KILLED && !reason.isInstanceOf[TaskCommitDenied]) { + // If a task failed because its attempt to commit was denied, do not count this failure + // towards failing the stage. This is intended to prevent spurious stage failures in cases + // where many speculative tasks are launched and denied to commit. assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index eb116213f69fc..9d0c1273695f6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -208,7 +208,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) } } } @@ -219,7 +219,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, - Map[Long, Any]((accumId, 1)), null, null)) + Map[Long, Any]((accumId, 1)), createFakeTaskInfo(), null)) } } } @@ -476,7 +476,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null, Map[Long, Any](), - null, + createFakeTaskInfo(), null)) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(1)) @@ -487,7 +487,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), null, Map[Long, Any](), - null, + createFakeTaskInfo(), null)) // The SparkListener should not receive redundant failure events. assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -507,14 +507,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(newEpoch > oldEpoch) val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, null, null)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -766,5 +766,14 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) } + + // Nothing in this test should break if the task info's fields are null, but + // OutputCommitCoordinator requires the task info itself to not be null. + private def createFakeTaskInfo(): TaskInfo = { + val info = new TaskInfo(0, 0, 0, 0L, "", "", TaskLocality.ANY, false) + info.finishTime = 1 // to prevent spurious errors in JobProgressListener + info + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala new file mode 100644 index 0000000000000..3cc860caa1d9b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.io.File +import java.util.concurrent.TimeoutException + +import org.mockito.Matchers +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.hadoop.mapred.{TaskAttemptID, JobConf, TaskAttemptContext, OutputCommitter} + +import org.apache.spark._ +import org.apache.spark.rdd.{RDD, FakeOutputCommitter} +import org.apache.spark.util.Utils + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.postfixOps + +/** + * Unit tests for the output commit coordination functionality. + * + * The unit test makes both the original task and the speculated task + * attempt to commit, where committing is emulated by creating a + * directory. If both tasks create directories then the end result is + * a failure. + * + * Note that there are some aspects of this test that are less than ideal. + * In particular, the test mocks the speculation-dequeuing logic to always + * dequeue a task and consider it as speculated. Immediately after initially + * submitting the tasks and calling reviveOffers(), reviveOffers() is invoked + * again to pick up the speculated task. This may be hacking the original + * behavior in too much of an unrealistic fashion. + * + * Also, the validation is done by checking the number of files in a directory. + * Ideally, an accumulator would be used for this, where we could increment + * the accumulator in the output committer's commitTask() call. If the call to + * commitTask() was called twice erroneously then the test would ideally fail because + * the accumulator would be incremented twice. + * + * The problem with this test implementation is that when both a speculated task and + * its original counterpart complete, only one of the accumulator's increments is + * captured. This results in a paradox where if the OutputCommitCoordinator logic + * was not in SparkHadoopWriter, the tests would still pass because only one of the + * increments would be captured even though the commit in both tasks was executed + * erroneously. + */ +class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { + + var outputCommitCoordinator: OutputCommitCoordinator = null + var tempDir: File = null + var sc: SparkContext = null + + before { + tempDir = Utils.createTempDir() + val conf = new SparkConf() + .setMaster("local[4]") + .setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName) + .set("spark.speculation", "true") + sc = new SparkContext(conf) { + override private[spark] def createSparkEnv( + conf: SparkConf, + isLocal: Boolean, + listenerBus: LiveListenerBus): SparkEnv = { + outputCommitCoordinator = spy(new OutputCommitCoordinator(conf)) + // Use Mockito.spy() to maintain the default infrastructure everywhere else. + // This mocking allows us to control the coordinator responses in test cases. + SparkEnv.createDriverEnv(conf, isLocal, listenerBus, Some(outputCommitCoordinator)) + } + } + // Use Mockito.spy() to maintain the default infrastructure everywhere else + val mockTaskScheduler = spy(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl]) + + doAnswer(new Answer[Unit]() { + override def answer(invoke: InvocationOnMock): Unit = { + // Submit the tasks, then force the task scheduler to dequeue the + // speculated task + invoke.callRealMethod() + mockTaskScheduler.backend.reviveOffers() + } + }).when(mockTaskScheduler).submitTasks(Matchers.any()) + + doAnswer(new Answer[TaskSetManager]() { + override def answer(invoke: InvocationOnMock): TaskSetManager = { + val taskSet = invoke.getArguments()(0).asInstanceOf[TaskSet] + new TaskSetManager(mockTaskScheduler, taskSet, 4) { + var hasDequeuedSpeculatedTask = false + override def dequeueSpeculativeTask( + execId: String, + host: String, + locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { + if (!hasDequeuedSpeculatedTask) { + hasDequeuedSpeculatedTask = true + Some(0, TaskLocality.PROCESS_LOCAL) + } else { + None + } + } + } + } + }).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any()) + + sc.taskScheduler = mockTaskScheduler + val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler) + sc.taskScheduler.setDAGScheduler(dagSchedulerWithMockTaskScheduler) + sc.dagScheduler = dagSchedulerWithMockTaskScheduler + } + + after { + sc.stop() + tempDir.delete() + outputCommitCoordinator = null + } + + test("Only one of two duplicate commit tasks should commit") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully _, + 0 until rdd.partitions.size, allowLocal = false) + assert(tempDir.list().size === 1) + } + + test("If commit fails, if task is retried it should not be locked, and will succeed.") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).failFirstCommitAttempt _, + 0 until rdd.partitions.size, allowLocal = false) + assert(tempDir.list().size === 1) + } + + test("Job should not complete if all commits are denied") { + // Create a mock OutputCommitCoordinator that denies all attempts to commit + doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit( + Matchers.any(), Matchers.any(), Matchers.any()) + val rdd: RDD[Int] = sc.parallelize(Seq(1), 1) + def resultHandler(x: Int, y: Unit): Unit = {} + val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, + OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully, + 0 until rdd.partitions.size, resultHandler, 0) + // It's an error if the job completes successfully even though no committer was authorized, + // so throw an exception if the job was allowed to complete. + intercept[TimeoutException] { + Await.result(futureAction, 5 seconds) + } + assert(tempDir.list().size === 0) + } +} + +/** + * Class with methods that can be passed to runJob to test commits with a mock committer. + */ +private case class OutputCommitFunctions(tempDirPath: String) { + + // Mock output committer that simulates a successful commit (after commit is authorized) + private def successfulOutputCommitter = new FakeOutputCommitter { + override def commitTask(context: TaskAttemptContext): Unit = { + Utils.createDirectory(tempDirPath) + } + } + + // Mock output committer that simulates a failed commit (after commit is authorized) + private def failingOutputCommitter = new FakeOutputCommitter { + override def commitTask(taskAttemptContext: TaskAttemptContext) { + throw new RuntimeException + } + } + + def commitSuccessfully(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + runCommitWithProvidedCommitter(ctx, iter, successfulOutputCommitter) + } + + def failFirstCommitAttempt(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + runCommitWithProvidedCommitter(ctx, iter, + if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter) + } + + private def runCommitWithProvidedCommitter( + ctx: TaskContext, + iter: Iterator[Int], + outputCommitter: OutputCommitter): Unit = { + def jobConf = new JobConf { + override def getOutputCommitter(): OutputCommitter = outputCommitter + } + val sparkHadoopWriter = new SparkHadoopWriter(jobConf) { + override def newTaskAttemptContext( + conf: JobConf, + attemptId: TaskAttemptID): TaskAttemptContext = { + mock(classOf[TaskAttemptContext]) + } + } + sparkHadoopWriter.setup(ctx.stageId, ctx.partitionId, ctx.attemptNumber) + sparkHadoopWriter.commit() + } +} From b969182659aa7ea94c38329b86d98a31b23efce8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 10 Feb 2015 20:19:14 -0800 Subject: [PATCH 060/817] [SPARK-5729] Potential NPE in standalone REST API If the user specifies a bad REST URL, the server will throw an NPE instead of propagating the error back. This is because the default `ErrorServlet` has the wrong prefix. This is a one line fix. I am will add more comprehensive tests in a separate patch. Author: Andrew Or Closes #4518 from andrewor14/rest-npe and squashes the following commits: 16b15bc [Andrew Or] Correct ErrorServlet context prefix --- .../org/apache/spark/deploy/rest/StandaloneRestServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 6e4486e20fcba..acd3a2b5abe6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -77,7 +77,7 @@ private[spark] class StandaloneRestServer( new SubmitRequestServlet(masterActor, masterUrl, masterConf) -> s"$baseContext/create/*", new KillRequestServlet(masterActor, masterConf) -> s"$baseContext/kill/*", new StatusRequestServlet(masterActor, masterConf) -> s"$baseContext/status/*", - new ErrorServlet -> "/" // default handler + new ErrorServlet -> "/*" // default handler ) /** Start the server and return the bound port. */ From b8f88d32723eaea4807c10b5b79d0c76f30b0510 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 10 Feb 2015 20:40:21 -0800 Subject: [PATCH 061/817] [SPARK-5702][SQL] Allow short names for built-in data sources. Also took the chance to fixed up some style ... Author: Reynold Xin Closes #4489 from rxin/SPARK-5702 and squashes the following commits: 74f42e3 [Reynold Xin] [SPARK-5702][SQL] Allow short names for built-in data sources. --- .../apache/spark/sql/jdbc/JDBCRelation.scala | 26 +++---- .../apache/spark/sql/json/JSONRelation.scala | 1 + .../org/apache/spark/sql/sources/ddl.scala | 77 ++++++++++--------- .../sql/sources/ResolvedDataSourceSuite.scala | 34 ++++++++ 4 files changed, 90 insertions(+), 48 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 66ad38eb7c45b..beb76f2c553c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -48,11 +48,6 @@ private[sql] object JDBCRelation { * exactly once. The parameters minValue and maxValue are advisory in that * incorrect values may cause the partitioning to be poor, but no data * will fail to be represented. - * - * @param column - Column name. Must refer to a column of integral type. - * @param numPartitions - Number of partitions - * @param minValue - Smallest value of column. Advisory. - * @param maxValue - Largest value of column. Advisory. */ def columnPartition(partitioning: JDBCPartitioningInfo): Array[Partition] = { if (partitioning == null) return Array[Partition](JDBCPartition(null, 0)) @@ -68,12 +63,17 @@ private[sql] object JDBCRelation { var currentValue: Long = partitioning.lowerBound var ans = new ArrayBuffer[Partition]() while (i < numPartitions) { - val lowerBound = (if (i != 0) s"$column >= $currentValue" else null) + val lowerBound = if (i != 0) s"$column >= $currentValue" else null currentValue += stride - val upperBound = (if (i != numPartitions - 1) s"$column < $currentValue" else null) - val whereClause = (if (upperBound == null) lowerBound - else if (lowerBound == null) upperBound - else s"$lowerBound AND $upperBound") + val upperBound = if (i != numPartitions - 1) s"$column < $currentValue" else null + val whereClause = + if (upperBound == null) { + lowerBound + } else if (lowerBound == null) { + upperBound + } else { + s"$lowerBound AND $upperBound" + } ans += JDBCPartition(whereClause, i) i = i + 1 } @@ -96,8 +96,7 @@ private[sql] class DefaultSource extends RelationProvider { if (driver != null) Class.forName(driver) - if ( - partitionColumn != null + if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { sys.error("Partitioning incompletely specified") } @@ -119,7 +118,8 @@ private[sql] class DefaultSource extends RelationProvider { private[sql] case class JDBCRelation( url: String, table: String, - parts: Array[Partition])(@transient val sqlContext: SQLContext) extends PrunedFilteredScan { + parts: Array[Partition])(@transient val sqlContext: SQLContext) + extends PrunedFilteredScan { override val schema = JDBCRDD.resolveTable(url, table) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f828bcdd65c9e..51ff2443f3717 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path + import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 6487c14b1eb8f..d3d72089c3303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -234,65 +234,73 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { primitiveType } -object ResolvedDataSource { - def apply( - sqlContext: SQLContext, - userSpecifiedSchema: Option[StructType], - provider: String, - options: Map[String, String]): ResolvedDataSource = { +private[sql] object ResolvedDataSource { + + private val builtinSources = Map( + "jdbc" -> classOf[org.apache.spark.sql.jdbc.DefaultSource], + "json" -> classOf[org.apache.spark.sql.json.DefaultSource], + "parquet" -> classOf[org.apache.spark.sql.parquet.DefaultSource] + ) + + /** Given a provider name, look up the data source class definition. */ + def lookupDataSource(provider: String): Class[_] = { + if (builtinSources.contains(provider)) { + return builtinSources(provider) + } + val loader = Utils.getContextOrSparkClassLoader - val clazz: Class[_] = try loader.loadClass(provider) catch { + try { + loader.loadClass(provider) + } catch { case cnf: java.lang.ClassNotFoundException => - try loader.loadClass(provider + ".DefaultSource") catch { + try { + loader.loadClass(provider + ".DefaultSource") + } catch { case cnf: java.lang.ClassNotFoundException => sys.error(s"Failed to load class for data source: $provider") } } + } + /** Create a [[ResolvedDataSource]] for reading data in. */ + def apply( + sqlContext: SQLContext, + userSpecifiedSchema: Option[StructType], + provider: String, + options: Map[String, String]): ResolvedDataSource = { + val clazz: Class[_] = lookupDataSource(provider) val relation = userSpecifiedSchema match { - case Some(schema: StructType) => { - clazz.newInstance match { - case dataSource: SchemaRelationProvider => - dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) - case dataSource: org.apache.spark.sql.sources.RelationProvider => - sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") - } + case Some(schema: StructType) => clazz.newInstance() match { + case dataSource: SchemaRelationProvider => + dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) + case dataSource: org.apache.spark.sql.sources.RelationProvider => + sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") } - case None => { - clazz.newInstance match { - case dataSource: RelationProvider => - dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) - case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => - sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") - } + + case None => clazz.newInstance() match { + case dataSource: RelationProvider => + dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) + case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => + sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") } } - new ResolvedDataSource(clazz, relation) } + /** Create a [[ResolvedDataSource]] for saving the content of the given [[DataFrame]]. */ def apply( sqlContext: SQLContext, provider: String, mode: SaveMode, options: Map[String, String], data: DataFrame): ResolvedDataSource = { - val loader = Utils.getContextOrSparkClassLoader - val clazz: Class[_] = try loader.loadClass(provider) catch { - case cnf: java.lang.ClassNotFoundException => - try loader.loadClass(provider + ".DefaultSource") catch { - case cnf: java.lang.ClassNotFoundException => - sys.error(s"Failed to load class for data source: $provider") - } - } - - val relation = clazz.newInstance match { + val clazz: Class[_] = lookupDataSource(provider) + val relation = clazz.newInstance() match { case dataSource: CreatableRelationProvider => dataSource.createRelation(sqlContext, mode, options, data) case _ => sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") } - new ResolvedDataSource(clazz, relation) } } @@ -405,6 +413,5 @@ protected[sql] class CaseInsensitiveMap(map: Map[String, String]) extends Map[St /** * The exception thrown from the DDL parser. - * @param message */ protected[sql] class DDLException(message: String) extends Exception(message) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala new file mode 100644 index 0000000000000..8331a14c9295c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -0,0 +1,34 @@ +/* +* 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 org.scalatest.FunSuite + +class ResolvedDataSourceSuite extends FunSuite { + + test("builtin sources") { + assert(ResolvedDataSource.lookupDataSource("jdbc") === + classOf[org.apache.spark.sql.jdbc.DefaultSource]) + + assert(ResolvedDataSource.lookupDataSource("json") === + classOf[org.apache.spark.sql.json.DefaultSource]) + + assert(ResolvedDataSource.lookupDataSource("parquet") === + classOf[org.apache.spark.sql.parquet.DefaultSource]) + } +} From f86a89a2e081ee4593ce03398c2283fd77daac6e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Feb 2015 21:51:15 -0800 Subject: [PATCH 062/817] [SPARK-5714][Mllib] Refactor initial step of LDA to remove redundant operations The `initialState` of LDA performs several RDD operations that looks redundant. This pr tries to simplify these operations. Author: Liang-Chi Hsieh Closes #4501 from viirya/sim_lda and squashes the following commits: 4870fe4 [Liang-Chi Hsieh] For comments. 9af1487 [Liang-Chi Hsieh] Refactor initial step of LDA to remove redundant operations. --- .../apache/spark/mllib/clustering/LDA.scala | 37 +++++++------------ 1 file changed, 13 insertions(+), 24 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index a1d3df03a1140..5e17c8da61134 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -450,34 +450,23 @@ private[clustering] object LDA { // Create vertices. // Initially, we use random soft assignments of tokens to topics (random gamma). - val edgesWithGamma: RDD[(Edge[TokenCount], TopicCounts)] = - edges.mapPartitionsWithIndex { case (partIndex, partEdges) => - val random = new Random(partIndex + randomSeed) - partEdges.map { edge => - // Create a random gamma_{wjk} - (edge, normalize(BDV.fill[Double](k)(random.nextDouble()), 1.0)) + def createVertices(): RDD[(VertexId, TopicCounts)] = { + val verticesTMP: RDD[(VertexId, TopicCounts)] = + edges.mapPartitionsWithIndex { case (partIndex, partEdges) => + val random = new Random(partIndex + randomSeed) + partEdges.flatMap { edge => + val gamma = normalize(BDV.fill[Double](k)(random.nextDouble()), 1.0) + val sum = gamma * edge.attr + Seq((edge.srcId, sum), (edge.dstId, sum)) + } } - } - def createVertices(sendToWhere: Edge[TokenCount] => VertexId): RDD[(VertexId, TopicCounts)] = { - val verticesTMP: RDD[(VertexId, (TokenCount, TopicCounts))] = - edgesWithGamma.map { case (edge, gamma: TopicCounts) => - (sendToWhere(edge), (edge.attr, gamma)) - } - verticesTMP.aggregateByKey(BDV.zeros[Double](k))( - (sum, t) => { - brzAxpy(t._1, t._2, sum) - sum - }, - (sum0, sum1) => { - sum0 += sum1 - } - ) + verticesTMP.reduceByKey(_ + _) } - val docVertices = createVertices(_.srcId) - val termVertices = createVertices(_.dstId) + + val docTermVertices = createVertices() // Partition such that edges are grouped by document - val graph = Graph(docVertices ++ termVertices, edges) + val graph = Graph(docTermVertices, edges) .partitionBy(PartitionStrategy.EdgePartition1D) new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointInterval) From 7e2f8821e08fddae661fcb484bf462210ad879e6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Feb 2015 22:43:32 -0800 Subject: [PATCH 063/817] HOTFIX: Java 6 compilation error in Spark SQL --- .../java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java | 2 +- .../apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index 852baf0e09245..311f1bdd07510 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -87,7 +87,7 @@ public void saveAndLoadWithSchema() { options.put("path", path.toString()); df.save("org.apache.spark.sql.json", SaveMode.ErrorIfExists, options); - List fields = new ArrayList<>(); + List fields = new ArrayList(); fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); StructType schema = DataTypes.createStructType(fields); DataFrame loadedDF = sqlContext.load("org.apache.spark.sql.json", schema, options); diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 9744a2aa3f59c..313e84756b6bb 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -121,7 +121,7 @@ public void saveExternalTableWithSchemaAndQueryIt() { sqlContext.sql("SELECT * FROM javaSavedTable"), df.collectAsList()); - List fields = new ArrayList<>(); + List fields = new ArrayList(); fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); StructType schema = DataTypes.createStructType(fields); DataFrame loadedDF = From c2131c0cdc57a4871ea23cd71e27e066d3c9a42c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Feb 2015 23:39:21 -0800 Subject: [PATCH 064/817] HOTFIX: Adding Junit to Hive tests for Maven build --- sql/hive/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 58b0722464be8..72c474d66055c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -84,6 +84,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + From 658687b25491047f30ee8558733d11e5a0572070 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 11 Feb 2015 00:13:27 -0800 Subject: [PATCH 065/817] [SPARK-4964] [Streaming] refactor createRDD to take leaders via map instead of array Author: cody koeninger Closes #4511 from koeninger/kafkaRdd-leader-to-broker and squashes the following commits: f7151d4 [cody koeninger] [SPARK-4964] test refactoring 6f8680b [cody koeninger] [SPARK-4964] add test of the scala api for KafkaUtils.createRDD f81e016 [cody koeninger] [SPARK-4964] leave KafkaStreamSuite host and port as private 5173f3f [cody koeninger] [SPARK-4964] test the Java variations of createRDD e9cece4 [cody koeninger] [SPARK-4964] pass leaders as a map to ensure 1 leader per TopicPartition --- .../kafka/{Leader.scala => Broker.scala} | 57 ++++--- .../spark/streaming/kafka/KafkaUtils.scala | 44 +++-- .../streaming/kafka/JavaKafkaRDDSuite.java | 156 ++++++++++++++++++ .../spark/streaming/kafka/KafkaRDDSuite.scala | 96 +++++++---- 4 files changed, 287 insertions(+), 66 deletions(-) rename external/kafka/src/main/scala/org/apache/spark/streaming/kafka/{Leader.scala => Broker.scala} (50%) create mode 100644 external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala similarity index 50% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala rename to external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala index c129a26836c0d..5a74febb4bd46 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala @@ -17,41 +17,52 @@ package org.apache.spark.streaming.kafka -import kafka.common.TopicAndPartition - import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * Represent the host info for the leader of a Kafka partition. + * Represent the host and port info for a Kafka broker. + * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID */ @Experimental -final class Leader private( - /** Kafka topic name */ - val topic: String, - /** Kafka partition id */ - val partition: Int, - /** Leader's hostname */ +final class Broker private( + /** Broker's hostname */ val host: String, - /** Leader's port */ - val port: Int) extends Serializable + /** Broker's port */ + val port: Int) extends Serializable { + override def equals(obj: Any): Boolean = obj match { + case that: Broker => + this.host == that.host && + this.port == that.port + case _ => false + } + + override def hashCode: Int = { + 41 * (41 + host.hashCode) + port + } + + override def toString(): String = { + s"Broker($host, $port)" + } +} /** * :: Experimental :: - * Companion object the provides methods to create instances of [[Leader]]. + * Companion object that provides methods to create instances of [[Broker]]. */ @Experimental -object Leader { - def create(topic: String, partition: Int, host: String, port: Int): Leader = - new Leader(topic, partition, host, port) - - def create(topicAndPartition: TopicAndPartition, host: String, port: Int): Leader = - new Leader(topicAndPartition.topic, topicAndPartition.partition, host, port) - - def apply(topic: String, partition: Int, host: String, port: Int): Leader = - new Leader(topic, partition, host, port) +object Broker { + def create(host: String, port: Int): Broker = + new Broker(host, port) - def apply(topicAndPartition: TopicAndPartition, host: String, port: Int): Leader = - new Leader(topicAndPartition.topic, topicAndPartition.partition, host, port) + def apply(host: String, port: Int): Broker = + new Broker(host, port) + def unapply(broker: Broker): Option[(String, Int)] = { + if (broker == null) { + None + } else { + Some((broker.host, broker.port)) + } + } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 7a2c3abdcc24b..af04bc6576148 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -154,6 +154,19 @@ object KafkaUtils { jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } + /** get leaders for the given offset ranges, or throw an exception */ + private def leadersForRanges( + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { + val kc = new KafkaCluster(kafkaParams) + val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet + val leaders = kc.findLeaders(topics).fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + leaders + } + /** * Create a RDD from Kafka using offset ranges for each topic and partition. * @@ -176,12 +189,7 @@ object KafkaUtils { offsetRanges: Array[OffsetRange] ): RDD[(K, V)] = { val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet - val leaders = kc.findLeaders(topics).fold( - errs => throw new SparkException(errs.mkString("\n")), - ok => ok - ) + val leaders = leadersForRanges(kafkaParams, offsetRanges) new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) } @@ -198,7 +206,8 @@ object KafkaUtils { * host1:port1,host2:port2 form. * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition - * @param leaders Kafka leaders for each offset range in batch + * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. * @param messageHandler Function for translating each message and metadata into the desired type */ @Experimental @@ -211,12 +220,17 @@ object KafkaUtils { sc: SparkContext, kafkaParams: Map[String, String], offsetRanges: Array[OffsetRange], - leaders: Array[Leader], + leaders: Map[TopicAndPartition, Broker], messageHandler: MessageAndMetadata[K, V] => R ): RDD[R] = { - val leaderMap = leaders - .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port)) - .toMap + val leaderMap = if (leaders.isEmpty) { + leadersForRanges(kafkaParams, offsetRanges) + } else { + // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker + leaders.map { + case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port)) + }.toMap + } new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler) } @@ -263,7 +277,8 @@ object KafkaUtils { * host1:port1,host2:port2 form. * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition - * @param leaders Kafka leaders for each offset range in batch + * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. * @param messageHandler Function for translating each message and metadata into the desired type */ @Experimental @@ -276,7 +291,7 @@ object KafkaUtils { recordClass: Class[R], kafkaParams: JMap[String, String], offsetRanges: Array[OffsetRange], - leaders: Array[Leader], + leaders: JMap[TopicAndPartition, Broker], messageHandler: JFunction[MessageAndMetadata[K, V], R] ): JavaRDD[R] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) @@ -284,8 +299,9 @@ object KafkaUtils { implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val leaderMap = Map(leaders.toSeq: _*) createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaders, messageHandler.call _) + jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaderMap, messageHandler.call _) } /** diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java new file mode 100644 index 0000000000000..9d2e1705c6c73 --- /dev/null +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Arrays; + +import org.apache.spark.SparkConf; + +import scala.Tuple2; + +import junit.framework.Assert; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +import org.junit.Test; +import org.junit.After; +import org.junit.Before; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaStreamSuiteBase suiteBase = null; + + @Before + public void setUp() { + suiteBase = new KafkaStreamSuiteBase() { }; + suiteBase.setupKafka(); + System.clearProperty("spark.driver.port"); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + suiteBase.tearDownKafka(); + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + HashMap emptyLeaders = new HashMap(); + HashMap leaders = new HashMap(); + String[] hostAndPort = suiteBase.brokerAddress().split(":"); + Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); + leaders.put(new TopicAndPartition(topic1, 0), broker); + leaders.put(new TopicAndPartition(topic2, 0), broker); + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + offsetRanges + ).map( + new Function, String>() { + @Override + public String call(scala.Tuple2 kv) throws Exception { + return kv._2(); + } + } + ); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + emptyLeaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) throws Exception { + return msgAndMd.message(); + } + } + ); + + JavaRDD rdd3 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + leaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) throws Exception { + return msgAndMd.message(); + } + } + ); + + // just making sure the java user apis work; the scala tests handle logic corner cases + long count1 = rdd1.count(); + long count2 = rdd2.count(); + long count3 = rdd3.count(); + Assert.assertTrue(count1 > 0); + Assert.assertEquals(count1, count2); + Assert.assertEquals(count1, count3); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + suiteBase.createTopic(topic); + suiteBase.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 6774db854a0d0..a223da70b043f 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -21,18 +21,22 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfter +import kafka.message.MessageAndMetadata +import org.scalatest.BeforeAndAfterAll import org.apache.spark._ import org.apache.spark.SparkContext._ -class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter { +class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) var sc: SparkContext = _ - before { + override def beforeAll { + sc = new SparkContext(sparkConf) + setupKafka() } - after { + override def afterAll { if (sc != null) { sc.stop sc = null @@ -40,60 +44,94 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter { tearDownKafka() } - test("Kafka RDD") { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - sc = new SparkContext(sparkConf) + test("basic usage") { + val topic = "topicbasic" + createTopic(topic) + val messages = Set("the", "quick", "brown", "fox") + sendMessages(topic, messages.toArray) + + + val kafkaParams = Map("metadata.broker.list" -> brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt(10000)}") + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, offsetRanges) + + val received = rdd.map(_._2).collect.toSet + assert(received === messages) + } + + test("iterator boundary conditions") { + // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) createTopic(topic) - sendMessages(topic, sent) val kafkaParams = Map("metadata.broker.list" -> brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val kc = new KafkaCluster(kafkaParams) - val rdd = getRdd(kc, Set(topic)) // this is the "lots of messages" case - // make sure we get all of them + sendMessages(topic, sent) + // rdd defined from leaders after sending messages, should get the number sent + val rdd = getRdd(kc, Set(topic)) + assert(rdd.isDefined) - assert(rdd.get.count === sent.values.sum) + assert(rdd.get.count === sent.values.sum, "didn't get all sent messages") - kc.setConsumerOffsets( - kafkaParams("group.id"), - rdd.get.offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap) + val ranges = rdd.get.asInstanceOf[HasOffsetRanges] + .offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap + + kc.setConsumerOffsets(kafkaParams("group.id"), ranges) - val rdd2 = getRdd(kc, Set(topic)) - val sent2 = Map("d" -> 1) - sendMessages(topic, sent2) // this is the "0 messages" case - // make sure we dont get anything, since messages were sent after rdd was defined + val rdd2 = getRdd(kc, Set(topic)) + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + sendMessages(topic, sentOnlyOne) assert(rdd2.isDefined) - assert(rdd2.get.count === 0) + assert(rdd2.get.count === 0, "got messages when there shouldn't be any") + // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above val rdd3 = getRdd(kc, Set(topic)) + // send lots of messages after rdd was defined, they shouldn't show up sendMessages(topic, Map("extra" -> 22)) - // this is the "exactly 1 message" case - // make sure we get exactly one message, despite there being lots more available + assert(rdd3.isDefined) - assert(rdd3.get.count === sent2.values.sum) + assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") } // get an rdd from the committed consumer offsets until the latest leader offsets, private def getRdd(kc: KafkaCluster, topics: Set[String]) = { val groupId = kc.kafkaParams("group.id") - for { - topicPartitions <- kc.getPartitions(topics).right.toOption - from <- kc.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( + def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = { + kc.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs => offs.map(kv => kv._1 -> kv._2.offset) } ) - until <- kc.getLatestLeaderOffsets(topicPartitions).right.toOption - } yield { - KafkaRDD[String, String, StringDecoder, StringDecoder, String]( - sc, kc.kafkaParams, from, until, mmd => s"${mmd.offset} ${mmd.message}") + } + kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => + consumerOffsets(topicPartitions).flatMap { from => + kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until => + val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset) + }.toArray + + val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) => + tp -> Broker(lo.host, lo.port) + }.toMap + + KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String]( + sc, kc.kafkaParams, offsetRanges, leaders, + (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}") + } + } } } } From da89720bf4023392436e75b6ed5e10ed8588a132 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Feb 2015 08:13:51 +0000 Subject: [PATCH 066/817] SPARK-5728 [STREAMING] MQTTStreamSuite leaves behind ActiveMQ database files Use temp dir for ActiveMQ database Author: Sean Owen Closes #4517 from srowen/SPARK-5728 and squashes the following commits: 1d3aeb8 [Sean Owen] Use temp dir for ActiveMQ database --- .../scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index e84adc088a680..19c9271af77be 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -93,6 +93,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { private def setupMQTT() { broker = new BrokerService() + broker.setDataDirectoryFile(Utils.createTempDir()) connector = new TransportConnector() connector.setName("mqtt") connector.setUri(new URI("mqtt:" + brokerUri)) From bd0d6e0cc3a329c4a1c08451a6d8a9281a422958 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Feb 2015 08:30:16 +0000 Subject: [PATCH 067/817] SPARK-5727 [BUILD] Deprecate Debian packaging This just adds a deprecation message. It's intended for backporting to branch 1.3 but can go in master too, to be followed by another PR that removes it for 1.4. Author: Sean Owen Closes #4516 from srowen/SPARK-5727.1 and squashes the following commits: d48989f [Sean Owen] Refer to Spark 1.4 6c1c8b3 [Sean Owen] Deprecate Debian packaging --- assembly/pom.xml | 18 ++++++++++++++++++ docs/building-spark.md | 2 ++ 2 files changed, 20 insertions(+) diff --git a/assembly/pom.xml b/assembly/pom.xml index 301ff69c2ae3b..c1bcdbb664dd0 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -221,6 +221,24 @@ deb + + maven-antrun-plugin + + + prepare-package + + run + + + + + NOTE: Debian packaging is deprecated and is scheduled to be removed in Spark 1.4. + + + + + + org.codehaus.mojo buildnumber-maven-plugin diff --git a/docs/building-spark.md b/docs/building-spark.md index db69905813817..d3824fb61eef9 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -161,6 +161,8 @@ For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troub # Building Spark Debian Packages +_NOTE: Debian packaging is deprecated and is scheduled to be removed in Spark 1.4._ + The Maven build includes support for building a Debian package containing the assembly 'fat-jar', PySpark, and the necessary scripts and configuration files. This can be created by specifying the following: mvn -Pdeb -DskipTests clean package From 1ac099e3e00ddb01af8e6e3a84c70f8363f04b5c Mon Sep 17 00:00:00 2001 From: guliangliang Date: Wed, 11 Feb 2015 15:55:49 +0000 Subject: [PATCH 068/817] [SPARK-5733] Error Link in Pagination of HistroyPage when showing Incomplete Applications The links in pagination of HistroyPage is wrong when showing Incomplete Applications. If "2" is click on the following page "http://history-server:18080/?page=1&showIncomplete=true", it will go to "http://history-server:18080/?page=2" instead of "http://history-server:18080/?page=2&showIncomplete=true". Author: guliangliang Closes #4523 from marsishandsome/Spark5733 and squashes the following commits: 9d7b593 [guliangliang] [SPARK-5733] Error Link in Pagination of HistroyPage when showing Incomplete Applications --- .../org/apache/spark/deploy/history/HistoryPage.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index e4e7bc2216014..26ebc75971c66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -61,9 +61,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { // page, `...` will be displayed. if (allApps.size > 0) { val leftSideIndices = - rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _) + rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _, requestedIncomplete) val rightSideIndices = - rangeIndices(actualPage + 1 to actualPage + plusOrMinus, _ < pageCount) + rangeIndices(actualPage + 1 to actualPage + plusOrMinus, _ < pageCount, + requestedIncomplete)

Showing {actualFirst + 1}-{last + 1} of {allApps.size} @@ -122,8 +123,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Spark User", "Last Updated") - private def rangeIndices(range: Seq[Int], condition: Int => Boolean): Seq[Node] = { - range.filter(condition).map(nextPage => {nextPage} ) + private def rangeIndices(range: Seq[Int], condition: Int => Boolean, showIncomplete: Boolean): + Seq[Node] = { + range.filter(condition).map(nextPage => + {nextPage} ) } private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { From b694eb9c2fefeaa33891d3e61f9bea369bc09984 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 11 Feb 2015 12:13:16 -0800 Subject: [PATCH 069/817] [SPARK-5677] [SPARK-5734] [SQL] [PySpark] Python DataFrame API remaining tasks 1. DataFrame.renameColumn 2. DataFrame.show() and _repr_ 3. Use simpleString() rather than jsonValue in DataFrame.dtypes 4. createDataFrame from local Python data, including pandas.DataFrame Author: Davies Liu Closes #4528 from davies/df3 and squashes the following commits: 014acea [Davies Liu] fix typo 6ba526e [Davies Liu] fix tests 46f5f95 [Davies Liu] address comments 6cbc154 [Davies Liu] dataframe.show() and improve dtypes 6f94f25 [Davies Liu] create DataFrame from local Python data --- python/pyspark/sql/context.py | 114 +++++++++++------- python/pyspark/sql/dataframe.py | 42 ++++++- python/pyspark/sql/tests.py | 2 +- python/pyspark/sql/types.py | 32 +++++ .../org/apache/spark/sql/DataFrameImpl.scala | 15 ++- 5 files changed, 155 insertions(+), 50 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 9d29ef4839a43..db4bcbece2c1b 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -23,12 +23,18 @@ from py4j.protocol import Py4JError from py4j.java_collections import MapConverter -from pyspark.rdd import _prepare_for_python_RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD from pyspark.serializers import AutoBatchedSerializer, PickleSerializer -from pyspark.sql.types import StringType, StructType, _infer_type, _verify_type, \ +from pyspark.sql.types import StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter from pyspark.sql.dataframe import DataFrame +try: + import pandas + has_pandas = True +except ImportError: + has_pandas = False + __all__ = ["SQLContext", "HiveContext"] @@ -116,6 +122,31 @@ def registerFunction(self, name, f, returnType=StringType()): self._sc._javaAccumulator, returnType.json()) + def _inferSchema(self, rdd, samplingRatio=None): + first = rdd.first() + if not first: + raise ValueError("The first row in RDD is empty, " + "can not infer schema") + if type(first) is dict: + warnings.warn("Using RDD of dict to inferSchema is deprecated," + "please use pyspark.sql.Row instead") + + if samplingRatio is None: + schema = _infer_schema(first) + if _has_nulltype(schema): + for row in rdd.take(100)[1:]: + schema = _merge_type(schema, _infer_schema(row)) + if not _has_nulltype(schema): + break + else: + raise ValueError("Some of types cannot be determined by the " + "first 100 rows, please try again with sampling") + else: + if samplingRatio < 0.99: + rdd = rdd.sample(False, float(samplingRatio)) + schema = rdd.map(_infer_schema).reduce(_merge_type) + return schema + def inferSchema(self, rdd, samplingRatio=None): """Infer and apply a schema to an RDD of L{Row}. @@ -171,29 +202,7 @@ def inferSchema(self, rdd, samplingRatio=None): if isinstance(rdd, DataFrame): raise TypeError("Cannot apply schema to DataFrame") - first = rdd.first() - if not first: - raise ValueError("The first row in RDD is empty, " - "can not infer schema") - if type(first) is dict: - warnings.warn("Using RDD of dict to inferSchema is deprecated," - "please use pyspark.sql.Row instead") - - if samplingRatio is None: - schema = _infer_schema(first) - if _has_nulltype(schema): - for row in rdd.take(100)[1:]: - schema = _merge_type(schema, _infer_schema(row)) - if not _has_nulltype(schema): - break - else: - warnings.warn("Some of types cannot be determined by the " - "first 100 rows, please try again with sampling") - else: - if samplingRatio < 0.99: - rdd = rdd.sample(False, float(samplingRatio)) - schema = rdd.map(_infer_schema).reduce(_merge_type) - + schema = self._inferSchema(rdd, samplingRatio) converter = _create_converter(schema) rdd = rdd.map(converter) return self.applySchema(rdd, schema) @@ -274,7 +283,7 @@ def applySchema(self, rdd, schema): raise TypeError("Cannot apply schema to DataFrame") if not isinstance(schema, StructType): - raise TypeError("schema should be StructType") + raise TypeError("schema should be StructType, but got %s" % schema) # take the first few rows to verify schema rows = rdd.take(10) @@ -294,9 +303,9 @@ def applySchema(self, rdd, schema): df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) return DataFrame(df, self) - def createDataFrame(self, rdd, schema=None, samplingRatio=None): + def createDataFrame(self, data, schema=None, samplingRatio=None): """ - Create a DataFrame from an RDD of tuple/list and an optional `schema`. + Create a DataFrame from an RDD of tuple/list, list or pandas.DataFrame. `schema` could be :class:`StructType` or a list of column names. @@ -311,12 +320,20 @@ def createDataFrame(self, rdd, schema=None, samplingRatio=None): rows will be used to do referring. The first row will be used if `samplingRatio` is None. - :param rdd: an RDD of Row or tuple or list or dict + :param data: an RDD of Row/tuple/list/dict, list, or pandas.DataFrame :param schema: a StructType or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring :return: a DataFrame - >>> rdd = sc.parallelize([('Alice', 1)]) + >>> l = [('Alice', 1)] + >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect() + [Row(name=u'Alice', age=1)] + + >>> d = [{'name': 'Alice', 'age': 1}] + >>> sqlCtx.createDataFrame(d).collect() + [Row(age=1, name=u'Alice')] + + >>> rdd = sc.parallelize(l) >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name=u'Alice', age=1)] @@ -336,19 +353,32 @@ def createDataFrame(self, rdd, schema=None, samplingRatio=None): >>> df3.collect() [Row(name=u'Alice', age=1)] """ - if isinstance(rdd, DataFrame): - raise TypeError("rdd is already a DataFrame") + if isinstance(data, DataFrame): + raise TypeError("data is already a DataFrame") - if isinstance(schema, StructType): - return self.applySchema(rdd, schema) - else: - if isinstance(schema, (list, tuple)): - first = rdd.first() - if not isinstance(first, (list, tuple)): - raise ValueError("each row in `rdd` should be list or tuple") - row_cls = Row(*schema) - rdd = rdd.map(lambda r: row_cls(*r)) - return self.inferSchema(rdd, samplingRatio) + if has_pandas and isinstance(data, pandas.DataFrame): + data = self._sc.parallelize(data.to_records(index=False)) + if schema is None: + schema = list(data.columns) + + if not isinstance(data, RDD): + try: + # data could be list, tuple, generator ... + data = self._sc.parallelize(data) + except Exception: + raise ValueError("cannot create an RDD from type: %s" % type(data)) + + if schema is None: + return self.inferSchema(data, samplingRatio) + + if isinstance(schema, (list, tuple)): + first = data.first() + if not isinstance(first, (list, tuple)): + raise ValueError("each row in `rdd` should be list or tuple") + row_cls = Row(*schema) + schema = self._inferSchema(data.map(lambda r: row_cls(*r)), samplingRatio) + + return self.applySchema(data, schema) def registerRDDAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 3eef0cc376a2d..3eb56ed74cc6f 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -236,6 +236,24 @@ def printSchema(self): """ print (self._jdf.schema().treeString()) + def show(self): + """ + Print the first 20 rows. + + >>> df.show() + age name + 2 Alice + 5 Bob + >>> df + age name + 2 Alice + 5 Bob + """ + print (self) + + def __repr__(self): + return self._jdf.showString() + def count(self): """Return the number of elements in this RDD. @@ -380,9 +398,9 @@ def dtypes(self): """Return all column names and their data types as a list. >>> df.dtypes - [('age', 'integer'), ('name', 'string')] + [('age', 'int'), ('name', 'string')] """ - return [(str(f.name), f.dataType.jsonValue()) for f in self.schema().fields] + return [(str(f.name), f.dataType.simpleString()) for f in self.schema().fields] @property def columns(self): @@ -606,6 +624,17 @@ def addColumn(self, colName, col): """ return self.select('*', col.alias(colName)) + def renameColumn(self, existing, new): + """ Rename an existing column to a new name + + >>> df.renameColumn('age', 'age2').collect() + [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] + """ + cols = [Column(_to_java_column(c), self.sql_ctx).alias(new) + if c == existing else c + for c in self.columns] + return self.select(*cols) + def to_pandas(self): """ Collect all the rows and return a `pandas.DataFrame`. @@ -885,6 +914,12 @@ def cast(self, dataType): jc = self._jc.cast(jdt) return Column(jc, self.sql_ctx) + def __repr__(self): + if self._jdf.isComputable(): + return self._jdf.samples() + else: + return 'Column<%s>' % self._jdf.toString() + def to_pandas(self): """ Return a pandas.Series from the column @@ -1030,7 +1065,8 @@ def _test(): globs['df'] = sqlCtx.inferSchema(rdd2) globs['df2'] = sqlCtx.inferSchema(rdd3) (failure_count, test_count) = doctest.testmod( - pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS) + pyspark.sql.dataframe, globs=globs, + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 5e41e36897b5d..43e5c3a1b00fa 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -194,7 +194,7 @@ def test_infer_schema(self): result = self.sqlCtx.sql("SELECT l[0].a from test where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) - df2 = self.sqlCtx.createDataFrame(rdd, 1.0) + df2 = self.sqlCtx.createDataFrame(rdd, samplingRatio=1.0) self.assertEqual(df.schema(), df2.schema()) self.assertEqual({}, df2.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.map(lambda r: r.s).collect()) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 41afefe48ee5e..40bd7e54a9d7b 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -52,6 +52,9 @@ def __ne__(self, other): def typeName(cls): return cls.__name__[:-4].lower() + def simpleString(self): + return self.typeName() + def jsonValue(self): return self.typeName() @@ -145,6 +148,12 @@ def __init__(self, precision=None, scale=None): self.scale = scale self.hasPrecisionInfo = precision is not None + def simpleString(self): + if self.hasPrecisionInfo: + return "decimal(%d,%d)" % (self.precision, self.scale) + else: + return "decimal(10,0)" + def jsonValue(self): if self.hasPrecisionInfo: return "decimal(%d,%d)" % (self.precision, self.scale) @@ -180,6 +189,8 @@ class ByteType(PrimitiveType): The data type representing int values with 1 singed byte. """ + def simpleString(self): + return 'tinyint' class IntegerType(PrimitiveType): @@ -188,6 +199,8 @@ class IntegerType(PrimitiveType): The data type representing int values. """ + def simpleString(self): + return 'int' class LongType(PrimitiveType): @@ -198,6 +211,8 @@ class LongType(PrimitiveType): beyond the range of [-9223372036854775808, 9223372036854775807], please use DecimalType. """ + def simpleString(self): + return 'bigint' class ShortType(PrimitiveType): @@ -206,6 +221,8 @@ class ShortType(PrimitiveType): The data type representing int values with 2 signed bytes. """ + def simpleString(self): + return 'smallint' class ArrayType(DataType): @@ -233,6 +250,9 @@ def __init__(self, elementType, containsNull=True): self.elementType = elementType self.containsNull = containsNull + def simpleString(self): + return 'array<%s>' % self.elementType.simpleString() + def __repr__(self): return "ArrayType(%s,%s)" % (self.elementType, str(self.containsNull).lower()) @@ -283,6 +303,9 @@ def __init__(self, keyType, valueType, valueContainsNull=True): self.valueType = valueType self.valueContainsNull = valueContainsNull + def simpleString(self): + return 'map<%s,%s>' % (self.keyType.simpleString(), self.valueType.simpleString()) + def __repr__(self): return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, str(self.valueContainsNull).lower()) @@ -337,6 +360,9 @@ def __init__(self, name, dataType, nullable=True, metadata=None): self.nullable = nullable self.metadata = metadata or {} + def simpleString(self): + return '%s:%s' % (self.name, self.dataType.simpleString()) + def __repr__(self): return "StructField(%s,%s,%s)" % (self.name, self.dataType, str(self.nullable).lower()) @@ -379,6 +405,9 @@ def __init__(self, fields): """ self.fields = fields + def simpleString(self): + return 'struct<%s>' % (','.join(f.simpleString() for f in self.fields)) + def __repr__(self): return ("StructType(List(%s))" % ",".join(str(field) for field in self.fields)) @@ -435,6 +464,9 @@ def deserialize(self, datum): """ raise NotImplementedError("UDT must implement deserialize().") + def simpleString(self): + return 'null' + def json(self): return json.dumps(self.jsonValue(), separators=(',', ':'), sort_keys=True) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 9638ce0865db0..41da4424ae459 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -126,7 +126,10 @@ private[sql] class DataFrameImpl protected[sql]( logicalPlan.isInstanceOf[LocalRelation] } - override def show(): Unit = { + /** + * Internal API for Python + */ + private[sql] def showString(): String = { val data = take(20) val numCols = schema.fieldNames.length @@ -146,12 +149,16 @@ private[sql] class DataFrameImpl protected[sql]( } } - // Pad the cells and print them - println(rows.map { row => + // Pad the cells + rows.map { row => row.zipWithIndex.map { case (cell, i) => String.format(s"%-${colWidths(i)}s", cell) }.mkString(" ") - }.mkString("\n")) + }.mkString("\n") + } + + override def show(): Unit = { + println(showString) } override def join(right: DataFrame): DataFrame = { From 03bf704bf442ac7dd960795295b51957ce972491 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Wed, 11 Feb 2015 20:24:17 +0000 Subject: [PATCH 070/817] Remove outdated remark about take(n). Looking at the code, I believe this remark about `take(n)` computing partitions on the driver is no longer correct. Apologies if I'm wrong. This came up in http://stackoverflow.com/q/28436559/3318517. Author: Daniel Darabos Closes #4533 from darabos/patch-2 and squashes the following commits: cc80f3a [Daniel Darabos] Remove outdated remark about take(n). --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 6b365e83fb56d..118701549a759 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -975,7 +975,7 @@ for details. take(n) - Return an array with the first n elements of the dataset. Note that this is currently not executed in parallel. Instead, the driver program computes all the elements. + Return an array with the first n elements of the dataset. takeSample(withReplacement, num, [seed]) From a60d2b70adff3a8fb3bdfac226b1d86fdb443da4 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 11 Feb 2015 12:31:56 -0800 Subject: [PATCH 071/817] [SPARK-5454] More robust handling of self joins Also I fix a bunch of bad output in test cases. Author: Michael Armbrust Closes #4520 from marmbrus/selfJoin and squashes the following commits: 4f4a85c [Michael Armbrust] comments 49c8e26 [Michael Armbrust] fix tests 6fc38de [Michael Armbrust] fix style 55d64b3 [Michael Armbrust] fix dataframe selfjoins --- .../sql/catalyst/analysis/Analyzer.scala | 26 ++++++++++++++++--- .../analysis/MultiInstanceRelation.scala | 21 +-------------- .../spark/sql/catalyst/plans/PlanTest.scala | 4 +-- .../org/apache/spark/sql/SQLContext.scala | 2 ++ sql/core/src/test/resources/log4j.properties | 3 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 10 +++++++ .../spark/sql/catalyst/plans/PlanTest.scala | 4 +-- 7 files changed, 40 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3f0d77ad6322a..2d1fa106a2aa9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -53,14 +53,11 @@ class Analyzer(catalog: Catalog, val extendedRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( - Batch("MultiInstanceRelations", Once, - NewRelationInstances), Batch("Resolution", fixedPoint, - ResolveReferences :: ResolveRelations :: + ResolveReferences :: ResolveGroupingAnalytics :: ResolveSortReferences :: - NewRelationInstances :: ImplicitGenerate :: ResolveFunctions :: GlobalAggregates :: @@ -285,6 +282,27 @@ class Analyzer(catalog: Catalog, } ) + // Special handling for cases when self-join introduce duplicate expression ids. + case j @ Join(left, right, _, _) if left.outputSet.intersect(right.outputSet).nonEmpty => + val conflictingAttributes = left.outputSet.intersect(right.outputSet) + + val (oldRelation, newRelation, attributeRewrites) = right.collect { + case oldVersion: MultiInstanceRelation + if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => + val newVersion = oldVersion.newInstance() + val newAttributes = AttributeMap(oldVersion.output.zip(newVersion.output)) + (oldVersion, newVersion, newAttributes) + }.head // Only handle first case found, others will be fixed on the next pass. + + val newRight = right transformUp { + case r if r == oldRelation => newRelation + case other => other transformExpressions { + case a: Attribute => attributeRewrites.get(a).getOrElse(a) + } + } + + j.copy(right = newRight) + case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressionsUp { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 4c5fb3f45bf49..894c3500cf533 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -26,28 +26,9 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * produced by distinct operators in a query tree as this breaks the guarantee that expression * ids, which are used to differentiate attributes, are unique. * - * Before analysis, all operators that include this trait will be asked to produce a new version + * During analysis, operators that include this trait may be asked to produce a new version * of itself with globally unique expression ids. */ trait MultiInstanceRelation { def newInstance(): this.type } - -/** - * If any MultiInstanceRelation appears more than once in the query plan then the plan is updated so - * that each instance has unique expression ids for the attributes produced. - */ -object NewRelationInstances extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - val localRelations = plan collect { case l: MultiInstanceRelation => l} - val multiAppearance = localRelations - .groupBy(identity[MultiInstanceRelation]) - .filter { case (_, ls) => ls.size > 1 } - .map(_._1) - .toSet - - plan transform { - case l: MultiInstanceRelation if multiAppearance.contains(l) => l.newInstance() - } - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index c4a1f899d8a13..7d609b91389c6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -33,11 +33,9 @@ class PlanTest extends FunSuite { * we must normalize them to check if two different queries are identical. */ protected def normalizeExprIds(plan: LogicalPlan) = { - val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) - val minId = if (list.isEmpty) 0 else list.min plan transformAllExpressions { case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 05ac1623d78ed..fd121ce05698c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -122,6 +122,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case _ => } + @transient protected[sql] val cacheManager = new CacheManager(this) /** @@ -159,6 +160,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * DataTypes.StringType); * }}} */ + @transient val udf: UDFRegistration = new UDFRegistration(this) /** Returns true if the table is currently cached in-memory. */ diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index fbed0a782dd3e..28e90b9520b2c 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -39,6 +39,9 @@ log4j.appender.FA.Threshold = INFO log4j.additivity.parquet.hadoop.ParquetRecordReader=false log4j.logger.parquet.hadoop.ParquetRecordReader=OFF +log4j.additivity.parquet.hadoop.ParquetOutputCommitter=false +log4j.logger.parquet.hadoop.ParquetOutputCommitter=OFF + log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 02623f73c7f76..7be9215a443f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.test.TestSQLContext.sql class DataFrameSuite extends QueryTest { @@ -88,6 +89,15 @@ class DataFrameSuite extends QueryTest { testData.collect().toSeq) } + test("self join") { + val df1 = testData.select(testData("key")).as('df1) + val df2 = testData.select(testData("key")).as('df2) + + checkAnswer( + df1.join(df2, $"df1.key" === $"df2.key"), + sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) + } + test("selectExpr") { checkAnswer( testData.selectExpr("abs(key)", "value"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 081d94b6fc020..44ee5ab5975fb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -35,11 +35,9 @@ class PlanTest extends FunSuite { * we must normalize them to check if two different queries are identical. */ protected def normalizeExprIds(plan: LogicalPlan) = { - val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) - val minId = if (list.isEmpty) 0 else list.min plan transformAllExpressions { case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) } } From 44b2311d946981c8251cb7807d70c8e99db5bbed Mon Sep 17 00:00:00 2001 From: tianyi Date: Wed, 11 Feb 2015 12:50:17 -0800 Subject: [PATCH 072/817] [SPARK-3688][SQL]LogicalPlan can't resolve column correctlly This PR fixed the resolving problem described in https://issues.apache.org/jira/browse/SPARK-3688 ``` CREATE TABLE t1(x INT); CREATE TABLE t2(a STRUCT, k INT); SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k; ``` Author: tianyi Closes #4524 from tianyi/SPARK-3688 and squashes the following commits: 237a256 [tianyi] resolve a name with table.column pattern first. --- .../catalyst/plans/logical/LogicalPlan.scala | 50 ++++++++++++------- ...as hive-0-c6d02549aec166e16bfc44d5905fa33a | 0 ...as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f | 0 ...as hive-2-26f54240cf5b909086fc34a34d7fdb56 | 0 ...as hive-3-d08d5280027adea681001ad82a5a6974 | 0 ...as hive-4-22eb25b5be6daf72a6649adfe5041749 | 1 + .../hive/execution/HiveResolutionSuite.scala | 9 ++++ 7 files changed, 42 insertions(+), 18 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a create mode 100644 sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f create mode 100644 sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56 create mode 100644 sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-3-d08d5280027adea681001ad82a5a6974 create mode 100644 sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 7cf4b81274906..b23f8d03df574 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -128,6 +128,29 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { def resolve(name: String, resolver: Resolver): Option[NamedExpression] = resolve(name, output, resolver) + def resolveAsTableColumn( + nameParts: Array[String], + resolver: Resolver, + attribute: Attribute): List[(Attribute, List[String])] = { + if (attribute.qualifiers.find(resolver(_, nameParts.head)).nonEmpty && nameParts.size > 1) { + val remainingParts = nameParts.drop(1) + resolveAsColumn(remainingParts, resolver, attribute) + } else { + Nil + } + } + + def resolveAsColumn( + nameParts: Array[String], + resolver: Resolver, + attribute: Attribute): List[(Attribute, List[String])] = { + if (resolver(attribute.name, nameParts.head)) { + (attribute.withName(nameParts.head), nameParts.tail.toList) :: Nil + } else { + Nil + } + } + /** Performs attribute resolution given a name and a sequence of possible attributes. */ protected def resolve( name: String, @@ -136,24 +159,15 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { val parts = name.split("\\.") - // Collect all attributes that are output by this nodes children where either the first part - // matches the name or where the first part matches the scope and the second part matches the - // name. Return these matches along with any remaining parts, which represent dotted access to - // struct fields. - val options = input.flatMap { option => - // If the first part of the desired name matches a qualifier for this possible match, drop it. - val remainingParts = - if (option.qualifiers.find(resolver(_, parts.head)).nonEmpty && parts.size > 1) { - parts.drop(1) - } else { - parts - } - - if (resolver(option.name, remainingParts.head)) { - // Preserve the case of the user's attribute reference. - (option.withName(remainingParts.head), remainingParts.tail.toList) :: Nil - } else { - Nil + // We will try to resolve this name as `table.column` pattern first. + var options = input.flatMap { option => + resolveAsTableColumn(parts, resolver, option) + } + + // If none of attributes match `table.column` pattern, we try to resolve it as a column. + if(options.isEmpty) { + options = input.flatMap { option => + resolveAsColumn(parts, resolver, option) } } diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-0-c6d02549aec166e16bfc44d5905fa33a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-1-a8987ff8c7b9ca95bf8b32314694ed1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56 b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-2-26f54240cf5b909086fc34a34d7fdb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-3-d08d5280027adea681001ad82a5a6974 b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-3-d08d5280027adea681001ad82a5a6974 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/test ambiguousReferences resolved as hive-4-22eb25b5be6daf72a6649adfe5041749 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index ab5f9cdddf508..029c36aa89b26 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -99,6 +99,15 @@ class HiveResolutionSuite extends HiveComparisonTest { assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } + createQueryTest("test ambiguousReferences resolved as hive", + """ + |CREATE TABLE t1(x INT); + |CREATE TABLE t2(a STRUCT, k INT); + |INSERT OVERWRITE TABLE t1 SELECT 1 FROM src LIMIT 1; + |INSERT OVERWRITE TABLE t2 SELECT named_struct("x",1),1 FROM src LIMIT 1; + |SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k; + """.stripMargin) + /** * Negative examples. Currently only left here for documentation purposes. * TODO(marmbrus): Test that catalyst fails on these queries. From fa6bdc6e819f9338248b952ec578bcd791ddbf6d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 11 Feb 2015 15:26:31 -0800 Subject: [PATCH 073/817] [SPARK-3688][SQL] More inline comments for LogicalPlan. As a follow-up to https://github.com/apache/spark/pull/4524 Author: Reynold Xin Closes #4539 from rxin/SPARK-3688 and squashes the following commits: 5ac56c7 [Reynold Xin] exists da8eea4 [Reynold Xin] [SPARK-3688][SQL] More inline comments for LogicalPlan. --- .../spark/sql/catalyst/analysis/package.scala | 4 +- .../expressions/namedExpressions.scala | 11 ++ .../spark/sql/catalyst/plans/QueryPlan.scala | 5 + .../catalyst/plans/logical/LogicalPlan.scala | 102 +++++++++++------- .../catalyst/plans/logical/Statistics.scala | 35 ++++++ 5 files changed, 115 insertions(+), 42 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 3f672a3e0fd91..5dc9d0e566087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -25,8 +25,8 @@ package org.apache.spark.sql.catalyst package object analysis { /** - * Responsible for resolving which identifiers refer to the same entity. For example, by using - * case insensitive equality. + * Resolver should return true if the first string refers to the same entity as the second string. + * For example, by using case insensitive equality. */ type Resolver = (String, String) => Boolean diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 7f122e9d55734..f77c56311cc8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -40,6 +40,17 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId + + /** + * All possible qualifiers for the expression. + * + * For now, since we do not allow using original table name to qualify a column name once the + * table is aliased, this can only be: + * + * 1. Empty Seq: when an attribute doesn't have a qualifier, + * e.g. top level attributes aliased in the SELECT clause, or column from a LocalRelation. + * 2. Single element: either the table name or the alias name of the table. + */ def qualifiers: Seq[String] def toAttribute: Attribute diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 619f42859cbb8..17a88e07de15f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -152,6 +152,11 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** Prints out the schema in the tree format */ def printSchema(): Unit = println(schemaString) + /** + * A prefix string used when printing the plan. + * + * We use "!" to indicate an invalid plan, and "'" to indicate an unresolved plan. + */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" override def simpleString = statePrefix + super.simpleString diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index b23f8d03df574..8c4f09b58a4f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -18,41 +18,29 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, Resolver} -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.catalyst.trees -/** - * Estimates of various statistics. The default estimation logic simply lazily multiplies the - * corresponding statistic produced by the children. To override this behavior, override - * `statistics` and assign it an overridden version of `Statistics`. - * - * '''NOTE''': concrete and/or overridden versions of statistics fields should pay attention to the - * performance of the implementations. The reason is that estimations might get triggered in - * performance-critical processes, such as query plan planning. - * - * Note that we are using a BigInt here since it is easy to overflow a 64-bit integer in - * cardinality estimation (e.g. cartesian joins). - * - * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it - * defaults to the product of children's `sizeInBytes`. - */ -private[sql] case class Statistics(sizeInBytes: BigInt) abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => + /** + * Computes [[Statistics]] for this plan. The default implementation assumes the output + * cardinality is the product of of all child plan's cardinality, i.e. applies in the case + * of cartesian joins. + * + * [[LeafNode]]s must override this. + */ def statistics: Statistics = { if (children.size == 0) { throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") } - - Statistics( - sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) + Statistics(sizeInBytes = children.map(_.statistics.sizeInBytes).product) } /** @@ -128,26 +116,41 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { def resolve(name: String, resolver: Resolver): Option[NamedExpression] = resolve(name, output, resolver) - def resolveAsTableColumn( + /** + * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. + * + * This assumes `name` has multiple parts, where the 1st part is a qualifier + * (i.e. table name, alias, or subquery alias). + * See the comment above `candidates` variable in resolve() for semantics the returned data. + */ + private def resolveAsTableColumn( nameParts: Array[String], resolver: Resolver, - attribute: Attribute): List[(Attribute, List[String])] = { - if (attribute.qualifiers.find(resolver(_, nameParts.head)).nonEmpty && nameParts.size > 1) { - val remainingParts = nameParts.drop(1) + attribute: Attribute): Option[(Attribute, List[String])] = { + assert(nameParts.length > 1) + if (attribute.qualifiers.exists(resolver(_, nameParts.head))) { + // At least one qualifier matches. See if remaining parts match. + val remainingParts = nameParts.tail resolveAsColumn(remainingParts, resolver, attribute) } else { - Nil + None } } - def resolveAsColumn( + /** + * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. + * + * Different from resolveAsTableColumn, this assumes `name` does NOT start with a qualifier. + * See the comment above `candidates` variable in resolve() for semantics the returned data. + */ + private def resolveAsColumn( nameParts: Array[String], resolver: Resolver, - attribute: Attribute): List[(Attribute, List[String])] = { + attribute: Attribute): Option[(Attribute, List[String])] = { if (resolver(attribute.name, nameParts.head)) { - (attribute.withName(nameParts.head), nameParts.tail.toList) :: Nil + Option((attribute.withName(nameParts.head), nameParts.tail.toList)) } else { - Nil + None } } @@ -159,25 +162,44 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { val parts = name.split("\\.") - // We will try to resolve this name as `table.column` pattern first. - var options = input.flatMap { option => - resolveAsTableColumn(parts, resolver, option) + // A sequence of possible candidate matches. + // Each candidate is a tuple. The first element is a resolved attribute, followed by a list + // of parts that are to be resolved. + // For example, consider an example where "a" is the table name, "b" is the column name, + // and "c" is the struct field name, i.e. "a.b.c". In this case, Attribute will be "a.b", + // and the second element will be List("c"). + var candidates: Seq[(Attribute, List[String])] = { + // If the name has 2 or more parts, try to resolve it as `table.column` first. + if (parts.length > 1) { + input.flatMap { option => + resolveAsTableColumn(parts, resolver, option) + } + } else { + Seq.empty + } } // If none of attributes match `table.column` pattern, we try to resolve it as a column. - if(options.isEmpty) { - options = input.flatMap { option => - resolveAsColumn(parts, resolver, option) + if (candidates.isEmpty) { + candidates = input.flatMap { candidate => + resolveAsColumn(parts, resolver, candidate) } } - options.distinct match { + candidates.distinct match { // One match, no nested fields, use it. case Seq((a, Nil)) => Some(a) // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - Some(Alias(nestedFields.foldLeft(a: Expression)(UnresolvedGetField), nestedFields.last)()) + // The foldLeft adds UnresolvedGetField for every remaining parts of the name, + // and aliased it with the last part of the name. + // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. + // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias + // the final expression as "c". + val fieldExprs = nestedFields.foldLeft(a: Expression)(UnresolvedGetField) + val aliasName = nestedFields.last + Some(Alias(fieldExprs, aliasName)()) // No matches. case Seq() => @@ -186,8 +208,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // More than one match. case ambiguousReferences => - throw new TreeNodeException( - this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") + throw new AnalysisException( + s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala new file mode 100644 index 0000000000000..9ac4c3a2a56c8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.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.sql.catalyst.plans.logical + +/** + * Estimates of various statistics. The default estimation logic simply lazily multiplies the + * corresponding statistic produced by the children. To override this behavior, override + * `statistics` and assign it an overridden version of `Statistics`. + * + * '''NOTE''': concrete and/or overridden versions of statistics fields should pay attention to the + * performance of the implementations. The reason is that estimations might get triggered in + * performance-critical processes, such as query plan planning. + * + * Note that we are using a BigInt here since it is easy to overflow a 64-bit integer in + * cardinality estimation (e.g. cartesian joins). + * + * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it + * defaults to the product of children's `sizeInBytes`. + */ +private[sql] case class Statistics(sizeInBytes: BigInt) From d931b01dcaaf009dcf68dcfe83428bd7f9e857cc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 11 Feb 2015 18:32:48 -0800 Subject: [PATCH 074/817] [SQL] Two DataFrame fixes. - Removed DataFrame.apply for projection & filtering since they are extremely confusing. - Added implicits for RDD[Int], RDD[Long], and RDD[String] Author: Reynold Xin Closes #4543 from rxin/df-cleanup and squashes the following commits: 81ec915 [Reynold Xin] [SQL] More DataFrame fixes. --- .../org/apache/spark/sql/DataFrame.scala | 39 ++++--------- .../org/apache/spark/sql/DataFrameImpl.scala | 24 +++----- .../apache/spark/sql/IncomputableColumn.scala | 4 -- .../org/apache/spark/sql/SQLContext.scala | 54 +++++++++++++++--- .../spark/sql/DataFrameImplicitsSuite.scala | 55 +++++++++++++++++++ 5 files changed, 119 insertions(+), 57 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.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 17900c5ee3892..327cf87f30f4b 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 @@ -85,6 +85,14 @@ trait DataFrame extends RDDApi[Row] { protected[sql] def logicalPlan: LogicalPlan + override def toString = + try { + schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]") + } catch { + case NonFatal(e) => + s"Invalid tree; ${e.getMessage}:\n$queryExecution" + } + /** Left here for backward compatibility. */ @deprecated("1.3.0", "use toDataFrame") def toSchemaRDD: DataFrame = this @@ -92,13 +100,9 @@ trait DataFrame extends RDDApi[Row] { /** * Returns the object itself. Used to force an implicit conversion from RDD to DataFrame in Scala. */ - def toDataFrame: DataFrame = this - - override def toString = - try schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]") catch { - case NonFatal(e) => - s"Invalid tree; ${e.getMessage}:\n$queryExecution" - } + // This is declared with parentheses to prevent the Scala compiler from treating + // `rdd.toDataFrame("1")` as invoking this toDataFrame and then apply on the returned DataFrame. + def toDataFrame(): DataFrame = this /** * Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion @@ -234,16 +238,6 @@ trait DataFrame extends RDDApi[Row] { */ def col(colName: String): Column - /** - * Selects a set of expressions, wrapped in a Product. - * {{{ - * // The following two are equivalent: - * df.apply(($"colA", $"colB" + 1)) - * df.select($"colA", $"colB" + 1) - * }}} - */ - def apply(projection: Product): DataFrame - /** * Returns a new [[DataFrame]] with an alias set. */ @@ -317,17 +311,6 @@ trait DataFrame extends RDDApi[Row] { */ def where(condition: Column): DataFrame - /** - * Filters rows using the given condition. This is a shorthand meant for Scala. - * {{{ - * // The following are equivalent: - * peopleDf.filter($"age" > 15) - * peopleDf.where($"age" > 15) - * peopleDf($"age" > 15) - * }}} - */ - def apply(condition: Column): DataFrame - /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. * See [[GroupedData]] for all the available aggregate functions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 41da4424ae459..3863df53181a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -49,8 +49,10 @@ private[sql] class DataFrameImpl protected[sql]( extends DataFrame { /** - * A constructor that automatically analyzes the logical plan. This reports error eagerly - * as the [[DataFrame]] is constructed. + * A constructor that automatically analyzes the logical plan. + * + * This reports error eagerly as the [[DataFrame]] is constructed, unless + * [[SQLConf.dataFrameEagerAnalysis]] is turned off. */ def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { this(sqlContext, { @@ -158,7 +160,7 @@ private[sql] class DataFrameImpl protected[sql]( } override def show(): Unit = { - println(showString) + println(showString()) } override def join(right: DataFrame): DataFrame = { @@ -205,14 +207,6 @@ private[sql] class DataFrameImpl protected[sql]( Column(sqlContext, Project(Seq(expr), logicalPlan), expr) } - override def apply(projection: Product): DataFrame = { - require(projection.productArity >= 1) - select(projection.productIterator.map { - case c: Column => c - case o: Any => Column(Literal(o)) - }.toSeq :_*) - } - override def as(alias: String): DataFrame = Subquery(alias, logicalPlan) override def as(alias: Symbol): DataFrame = Subquery(alias.name, logicalPlan) @@ -259,10 +253,6 @@ private[sql] class DataFrameImpl protected[sql]( filter(condition) } - override def apply(condition: Column): DataFrame = { - filter(condition) - } - override def groupBy(cols: Column*): GroupedData = { new GroupedData(this, cols.map(_.expr)) } @@ -323,7 +313,7 @@ private[sql] class DataFrameImpl protected[sql]( override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) override def repartition(numPartitions: Int): DataFrame = { - sqlContext.applySchema(rdd.repartition(numPartitions), schema) + sqlContext.createDataFrame(rdd.repartition(numPartitions), schema) } override def distinct: DataFrame = Distinct(logicalPlan) @@ -401,7 +391,7 @@ private[sql] class DataFrameImpl protected[sql]( val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) new Iterator[String] { - override def hasNext() = iter.hasNext + override def hasNext = iter.hasNext override def next(): String = { JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) gen.flush() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 494e49c1317b6..4f9d92d97646f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -80,8 +80,6 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def col(colName: String): Column = err() - override def apply(projection: Product): DataFrame = err() - override def select(cols: Column*): DataFrame = err() override def select(col: String, cols: String*): DataFrame = err() @@ -98,8 +96,6 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def where(condition: Column): DataFrame = err() - override def apply(condition: Column): DataFrame = err() - override def groupBy(cols: Column*): GroupedData = err() override def groupBy(col1: String, cols: String*): GroupedData = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index fd121ce05698c..ca5e62f29569e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -180,21 +180,59 @@ class SQLContext(@transient val sparkContext: SparkContext) */ object implicits { // scalastyle:on - /** - * Creates a DataFrame from an RDD of case classes. - * - * @group userf - */ + + /** Creates a DataFrame from an RDD of case classes or tuples. */ implicit def rddToDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { self.createDataFrame(rdd) } - /** - * Creates a DataFrame from a local Seq of Product. - */ + /** Creates a DataFrame from a local Seq of Product. */ implicit def localSeqToDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { self.createDataFrame(data) } + + // Do NOT add more implicit conversions. They are likely to break source compatibility by + // making existing implicit conversions ambiguous. In particular, RDD[Double] is dangerous + // because of [[DoubleRDDFunctions]]. + + /** Creates a single column DataFrame from an RDD[Int]. */ + implicit def intRddToDataFrame(data: RDD[Int]): DataFrame = { + val dataType = IntegerType + val rows = data.mapPartitions { iter => + val row = new SpecificMutableRow(dataType :: Nil) + iter.map { v => + row.setInt(0, v) + row: Row + } + } + self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + } + + /** Creates a single column DataFrame from an RDD[Long]. */ + implicit def longRddToDataFrame(data: RDD[Long]): DataFrame = { + val dataType = LongType + val rows = data.mapPartitions { iter => + val row = new SpecificMutableRow(dataType :: Nil) + iter.map { v => + row.setLong(0, v) + row: Row + } + } + self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + } + + /** Creates a single column DataFrame from an RDD[String]. */ + implicit def stringRddToDataFrame(data: RDD[String]): DataFrame = { + val dataType = StringType + val rows = data.mapPartitions { iter => + val row = new SpecificMutableRow(dataType :: Nil) + iter.map { v => + row.setString(0, v) + row: Row + } + } + self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala new file mode 100644 index 0000000000000..8fa830dd9390f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala @@ -0,0 +1,55 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.TestSQLContext.{sparkContext => sc} +import org.apache.spark.sql.test.TestSQLContext.implicits._ + + +class DataFrameImplicitsSuite extends QueryTest { + + test("RDD of tuples") { + checkAnswer( + sc.parallelize(1 to 10).map(i => (i, i.toString)).toDataFrame("intCol", "strCol"), + (1 to 10).map(i => Row(i, i.toString))) + } + + test("Seq of tuples") { + checkAnswer( + (1 to 10).map(i => (i, i.toString)).toDataFrame("intCol", "strCol"), + (1 to 10).map(i => Row(i, i.toString))) + } + + test("RDD[Int]") { + checkAnswer( + sc.parallelize(1 to 10).toDataFrame("intCol"), + (1 to 10).map(i => Row(i))) + } + + test("RDD[Long]") { + checkAnswer( + sc.parallelize(1L to 10L).toDataFrame("longCol"), + (1L to 10L).map(i => Row(i))) + } + + test("RDD[String]") { + checkAnswer( + sc.parallelize(1 to 10).map(_.toString).toDataFrame("stringCol"), + (1 to 10).map(i => Row(i.toString))) + } +} From a38e23c30fb5d12f8f46a119d91a0620036e6800 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 11 Feb 2015 19:05:49 -0800 Subject: [PATCH 075/817] [SQL] Make dataframe more tolerant of being serialized Eases use in the spark-shell. Author: Michael Armbrust Closes #4545 from marmbrus/serialization and squashes the following commits: 04748e6 [Michael Armbrust] @scala.annotation.varargs b36e219 [Michael Armbrust] moreFixes --- .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- .../main/scala/org/apache/spark/sql/DataFrameImpl.scala | 4 ++-- sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala | 9 +++++++++ .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 +++- 4 files changed, 15 insertions(+), 4 deletions(-) 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 327cf87f30f4b..13aff760e9a5c 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 @@ -76,7 +76,7 @@ private[sql] object DataFrame { */ // TODO: Improve documentation. @Experimental -trait DataFrame extends RDDApi[Row] { +trait DataFrame extends RDDApi[Row] with Serializable { val sqlContext: SQLContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 3863df53181a3..4c6e19cace8ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -44,8 +44,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} * Internal implementation of [[DataFrame]]. Users of the API should use [[DataFrame]] directly. */ private[sql] class DataFrameImpl protected[sql]( - override val sqlContext: SQLContext, - val queryExecution: SQLContext#QueryExecution) + @transient override val sqlContext: SQLContext, + @transient val queryExecution: SQLContext#QueryExecution) extends DataFrame { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala index 6bf21dd1bc79b..7bc7683576b71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala @@ -166,6 +166,15 @@ object Dsl { ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Returns the first column that is not null. + * {{{ + * df.select(coalesce(df("a"), df("b"))) + * }}} + */ + @scala.annotation.varargs + def coalesce(e: Column*): Column = Coalesce(e.map(_.expr)) + /** * Unary minus, i.e. negate the expression. * {{{ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index ca5e62f29569e..8aae222acd927 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -129,9 +129,11 @@ class SQLContext(@transient val sparkContext: SparkContext) * A collection of methods that are considered experimental, but can be used to hook into * the query planner for advanced functionalities. */ + @transient val experimental: ExperimentalMethods = new ExperimentalMethods(this) /** Returns a [[DataFrame]] with no rows or columns. */ + @transient lazy val emptyDataFrame = DataFrame(this, NoRelation) /** @@ -178,7 +180,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * (Scala-specific) * Implicit methods available in Scala for converting common Scala objects into [[DataFrame]]s. */ - object implicits { + object implicits extends Serializable { // scalastyle:on /** Creates a DataFrame from an RDD of case classes or tuples. */ From 9a3ea49f74b9a41dba527a7e995735f488fe0847 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 12 Feb 2015 12:36:26 +0000 Subject: [PATCH 076/817] SPARK-5727 [BUILD] Remove Debian packaging (for master / 1.4 only) Author: Sean Owen Closes #4526 from srowen/SPARK-5727.2 and squashes the following commits: 83ba49c [Sean Owen] Remove Debian packaging --- assembly/pom.xml | 129 ------------------------------- assembly/src/deb/RELEASE | 2 - assembly/src/deb/control/control | 8 -- docs/building-spark.md | 10 --- repl/pom.xml | 2 - 5 files changed, 151 deletions(-) delete mode 100644 assembly/src/deb/RELEASE delete mode 100644 assembly/src/deb/control/control diff --git a/assembly/pom.xml b/assembly/pom.xml index c1bcdbb664dd0..fa9f56e556d8b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -36,10 +36,6 @@ scala-${scala.binary.version} spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} - spark - /usr/share/spark - root - 755 @@ -217,131 +213,6 @@ - - deb - - - - maven-antrun-plugin - - - prepare-package - - run - - - - - NOTE: Debian packaging is deprecated and is scheduled to be removed in Spark 1.4. - - - - - - - - org.codehaus.mojo - buildnumber-maven-plugin - 1.2 - - - validate - - create - - - 8 - - - - - - org.vafer - jdeb - 0.11 - - - package - - jdeb - - - ${project.build.directory}/${deb.pkg.name}_${project.version}-${buildNumber}_all.deb - false - gzip - - - ${spark.jar} - file - - perm - ${deb.user} - ${deb.user} - ${deb.install.path}/jars - - - - ${basedir}/src/deb/RELEASE - file - - perm - ${deb.user} - ${deb.user} - ${deb.install.path} - - - - ${basedir}/../conf - directory - - perm - ${deb.user} - ${deb.user} - ${deb.install.path}/conf - ${deb.bin.filemode} - - - - ${basedir}/../bin - directory - - perm - ${deb.user} - ${deb.user} - ${deb.install.path}/bin - ${deb.bin.filemode} - - - - ${basedir}/../sbin - directory - - perm - ${deb.user} - ${deb.user} - ${deb.install.path}/sbin - ${deb.bin.filemode} - - - - ${basedir}/../python - directory - - perm - ${deb.user} - ${deb.user} - ${deb.install.path}/python - ${deb.bin.filemode} - - - - - - - - - - kinesis-asl diff --git a/assembly/src/deb/RELEASE b/assembly/src/deb/RELEASE deleted file mode 100644 index aad50ee73aa45..0000000000000 --- a/assembly/src/deb/RELEASE +++ /dev/null @@ -1,2 +0,0 @@ -compute-classpath.sh uses the existence of this file to decide whether to put the assembly jar on the -classpath or instead to use classfiles in the source tree. \ No newline at end of file diff --git a/assembly/src/deb/control/control b/assembly/src/deb/control/control deleted file mode 100644 index a6b4471d485f4..0000000000000 --- a/assembly/src/deb/control/control +++ /dev/null @@ -1,8 +0,0 @@ -Package: [[deb.pkg.name]] -Version: [[version]]-[[buildNumber]] -Section: misc -Priority: extra -Architecture: all -Maintainer: Matei Zaharia -Description: [[name]] -Distribution: development diff --git a/docs/building-spark.md b/docs/building-spark.md index d3824fb61eef9..088da7da4980e 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -159,16 +159,6 @@ Thus, the full flow for running continuous-compilation of the `core` submodule m For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the [wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup). -# Building Spark Debian Packages - -_NOTE: Debian packaging is deprecated and is scheduled to be removed in Spark 1.4._ - -The Maven build includes support for building a Debian package containing the assembly 'fat-jar', PySpark, and the necessary scripts and configuration files. This can be created by specifying the following: - - mvn -Pdeb -DskipTests clean package - -The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions. - # Running Java 8 Test Suites Running only Java 8 tests and nothing else. diff --git a/repl/pom.xml b/repl/pom.xml index 3d4adf8fd5b03..b883344bf0ceb 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -33,8 +33,6 @@ repl - /usr/share/spark - root scala-2.10/src/main/scala scala-2.10/src/test/scala From 9a6efbccf9eb3ab13295dfd204df8e58e5112209 Mon Sep 17 00:00:00 2001 From: Oren Mazor Date: Thu, 12 Feb 2015 18:37:00 +0000 Subject: [PATCH 077/817] ignore cache paths for RAT tests RAT fails on cache paths. add to .rat-excludes Author: Oren Mazor Closes #4569 from orenmazor/apache_master and squashes the following commits: d0c9e7e [Oren Mazor] ignore cache paths for RAT tests --- .rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/.rat-excludes b/.rat-excludes index 769defbac11b7..a788e8273d8a2 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -1,4 +1,5 @@ target +cache .gitignore .gitattributes .project From 466b1f671b21f575d28f9c103f51765790914fe3 Mon Sep 17 00:00:00 2001 From: Andrew Rowson Date: Thu, 12 Feb 2015 18:41:39 +0000 Subject: [PATCH 078/817] [SPARK-5655] Don't chmod700 application files if running in YARN [Was previously PR4507] As per SPARK-5655, recently committed code chmod 700s all application files created on the local fs by a spark executor. This is both unnecessary and broken on YARN, where files created in the nodemanager's working directory are already owned by the user running the job and the 'yarn' group. Group read permission is also needed for the auxiliary shuffle service to be able to read the files, as this is running as the 'yarn' user. Author: Andrew Rowson Closes #4509 from growse/master and squashes the following commits: 7ca993c [Andrew Rowson] Moved chmod700 functionality into Utils.getOrCreateLocalRootDirs f57ce6b [Andrew Rowson] [SPARK-5655] Don't chmod700 application files if running in a YARN container --- core/src/main/scala/org/apache/spark/util/Utils.scala | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6af8dd555f2aa..c06bd6fab0cc9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -283,13 +283,6 @@ private[spark] object Utils extends Logging { dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString) if (dir.exists() || !dir.mkdirs()) { dir = null - } else { - // Restrict file permissions via chmod if available. - // For Windows this step is ignored. - if (!isWindows && !chmod700(dir)) { - dir.delete() - dir = null - } } } catch { case e: SecurityException => dir = null; } } @@ -703,7 +696,9 @@ private[spark] object Utils extends Logging { try { val rootDir = new File(root) if (rootDir.exists || rootDir.mkdirs()) { - Some(createDirectory(root).getAbsolutePath()) + val dir = createDirectory(root) + chmod700(dir) + Some(dir.getAbsolutePath) } else { logError(s"Failed to create dir in $root. Ignoring this directory.") None From 99bd5006650bb15ec5465ffee1ebaca81354a3df Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 12 Feb 2015 10:48:13 -0800 Subject: [PATCH 079/817] [SPARK-5757][MLLIB] replace SQL JSON usage in model import/export by json4s This PR detaches MLlib model import/export code from SQL's JSON support, and hence unblocks #4544 . yhuai Author: Xiangrui Meng Closes #4555 from mengxr/SPARK-5757 and squashes the following commits: b0415e8 [Xiangrui Meng] replace SQL JSON usage by json4s --- .../classification/ClassificationModel.scala | 16 ++---- .../classification/LogisticRegression.scala | 3 +- .../mllib/classification/NaiveBayes.scala | 18 +++---- .../spark/mllib/classification/SVM.scala | 6 +-- .../impl/GLMClassificationModel.scala | 17 ++++--- .../MatrixFactorizationModel.scala | 14 +++-- .../apache/spark/mllib/regression/Lasso.scala | 2 +- .../mllib/regression/LinearRegression.scala | 2 +- .../mllib/regression/RegressionModel.scala | 16 ++---- .../mllib/regression/RidgeRegression.scala | 2 +- .../regression/impl/GLMRegressionModel.scala | 11 ++-- .../spark/mllib/tree/DecisionTree.scala | 8 +-- .../mllib/tree/model/DecisionTreeModel.scala | 28 ++++------ .../mllib/tree/model/treeEnsembleModels.scala | 51 ++++++++----------- .../spark/mllib/util/modelSaveLoad.scala | 25 +++------ 15 files changed, 92 insertions(+), 127 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 348c1e8760a66..35a0db76f3a8c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -17,12 +17,12 @@ package org.apache.spark.mllib.classification +import org.json4s.{DefaultFormats, JValue} + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.util.Loader import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} /** * :: Experimental :: @@ -60,16 +60,10 @@ private[mllib] object ClassificationModel { /** * Helper method for loading GLM classification model metadata. - * - * @param modelClass String name for model class (used for error messages) * @return (numFeatures, numClasses) */ - def getNumFeaturesClasses(metadata: DataFrame, modelClass: String, path: String): (Int, Int) = { - metadata.select("numFeatures", "numClasses").take(1)(0) match { - case Row(nFeatures: Int, nClasses: Int) => (nFeatures, nClasses) - case _ => throw new Exception(s"$modelClass unable to load" + - s" numFeatures, numClasses from metadata: ${Loader.metadataPath(path)}") - } + def getNumFeaturesClasses(metadata: JValue): (Int, Int) = { + implicit val formats = DefaultFormats + ((metadata \ "numFeatures").extract[Int], (metadata \ "numClasses").extract[Int]) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 9a391bfff76a3..420d6e2861934 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -173,8 +173,7 @@ object LogisticRegressionModel extends Loader[LogisticRegressionModel] { val classNameV1_0 = "org.apache.spark.mllib.classification.LogisticRegressionModel" (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val (numFeatures, numClasses) = - ClassificationModel.getNumFeaturesClasses(metadata, classNameV1_0, path) + val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) val data = GLMClassificationModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0) // numFeatures, numClasses, weights are checked in model initialization val model = 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 d9ce2822dd391..f9142bc2268b7 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 @@ -18,15 +18,16 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{SparkContext, SparkException, Logging} +import org.apache.spark.{Logging, SparkContext, SparkException} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} - /** * Model for Naive Bayes Classifiers. * @@ -78,7 +79,7 @@ class NaiveBayesModel private[mllib] ( object NaiveBayesModel extends Loader[NaiveBayesModel] { - import Loader._ + import org.apache.spark.mllib.util.Loader._ private object SaveLoadV1_0 { @@ -95,10 +96,10 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { import sqlContext.implicits._ // Create JSON metadata. - val metadataRDD = - sc.parallelize(Seq((thisClassName, thisFormatVersion, data.theta(0).size, data.pi.size)), 1) - .toDataFrame("class", "version", "numFeatures", "numClasses") - metadataRDD.toJSON.saveAsTextFile(metadataPath(path)) + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("numFeatures" -> data.theta(0).length) ~ ("numClasses" -> data.pi.length))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) // Create Parquet data. val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) @@ -126,8 +127,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { val classNameV1_0 = SaveLoadV1_0.thisClassName (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val (numFeatures, numClasses) = - ClassificationModel.getNumFeaturesClasses(metadata, classNameV1_0, path) + val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) val model = SaveLoadV1_0.load(sc, path) assert(model.pi.size == numClasses, s"NaiveBayesModel.load expected $numClasses classes," + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 24d31e62ba500..cfc7f868a02f0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -23,10 +23,9 @@ import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} +import org.apache.spark.mllib.util.{DataValidators, Loader, Saveable} import org.apache.spark.rdd.RDD - /** * Model for Support Vector Machines (SVMs). * @@ -97,8 +96,7 @@ object SVMModel extends Loader[SVMModel] { val classNameV1_0 = "org.apache.spark.mllib.classification.SVMModel" (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val (numFeatures, numClasses) = - ClassificationModel.getNumFeaturesClasses(metadata, classNameV1_0, path) + val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) val data = GLMClassificationModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0) val model = new SVMModel(data.weights, data.intercept) assert(model.weights.size == numFeatures, s"SVMModel.load with numFeatures=$numFeatures" + 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 8d600572ed7f3..1d118963b49fc 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 @@ -17,10 +17,13 @@ package org.apache.spark.mllib.classification.impl +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.Loader -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} /** * Helper class for import/export of GLM classification models. @@ -52,16 +55,14 @@ private[classification] object GLMClassificationModel { import sqlContext.implicits._ // Create JSON metadata. - val metadataRDD = - sc.parallelize(Seq((modelClass, thisFormatVersion, numFeatures, numClasses)), 1) - .toDataFrame("class", "version", "numFeatures", "numClasses") - metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + val metadata = compact(render( + ("class" -> modelClass) ~ ("version" -> thisFormatVersion) ~ + ("numFeatures" -> numFeatures) ~ ("numClasses" -> numClasses))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) // Create Parquet data. val data = Data(weights, intercept, threshold) - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) - // TODO: repartition with 1 partition after SPARK-5532 gets fixed - dataRDD.saveAsParquetFile(Loader.dataPath(path)) + sc.parallelize(Seq(data), 1).saveAsParquetFile(Loader.dataPath(path)) } /** 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 16979c9ed43ca..a3a3b5d418c0a 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 @@ -22,6 +22,9 @@ import java.lang.{Integer => JavaInteger} import org.apache.hadoop.fs.Path import org.jblas.DoubleMatrix +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} @@ -153,7 +156,7 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { import org.apache.spark.mllib.util.Loader._ override def load(sc: SparkContext, path: String): MatrixFactorizationModel = { - val (loadedClassName, formatVersion, metadata) = loadMetadata(sc, path) + val (loadedClassName, formatVersion, _) = loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName (loadedClassName, formatVersion) match { case (className, "1.0") if className == classNameV1_0 => @@ -181,19 +184,20 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { val sc = model.userFeatures.sparkContext val sqlContext = new SQLContext(sc) import sqlContext.implicits._ - val metadata = (thisClassName, thisFormatVersion, model.rank) - val metadataRDD = sc.parallelize(Seq(metadata), 1).toDataFrame("class", "version", "rank") - metadataRDD.toJSON.saveAsTextFile(metadataPath(path)) + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) model.userFeatures.toDataFrame("id", "features").saveAsParquetFile(userPath(path)) model.productFeatures.toDataFrame("id", "features").saveAsParquetFile(productPath(path)) } def load(sc: SparkContext, path: String): MatrixFactorizationModel = { + implicit val formats = DefaultFormats val sqlContext = new SQLContext(sc) val (className, formatVersion, metadata) = loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) - val rank = metadata.select("rank").first().getInt(0) + val rank = (metadata \ "rank").extract[Int] val userFeatures = sqlContext.parquetFile(userPath(path)) .map { case Row(id: Int, features: Seq[Double]) => (id, features.toArray) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 1159e59fff5f6..e8b03816573cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -58,7 +58,7 @@ object LassoModel extends Loader[LassoModel] { val classNameV1_0 = "org.apache.spark.mllib.regression.LassoModel" (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val numFeatures = RegressionModel.getNumFeatures(metadata, classNameV1_0, path) + val numFeatures = RegressionModel.getNumFeatures(metadata) val data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) new LassoModel(data.weights, data.intercept) case _ => throw new Exception( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 0136dcfdceaef..6fa7ad52a5b33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -58,7 +58,7 @@ object LinearRegressionModel extends Loader[LinearRegressionModel] { val classNameV1_0 = "org.apache.spark.mllib.regression.LinearRegressionModel" (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val numFeatures = RegressionModel.getNumFeatures(metadata, classNameV1_0, path) + val numFeatures = RegressionModel.getNumFeatures(metadata) val data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) new LinearRegressionModel(data.weights, data.intercept) case _ => throw new Exception( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 843e59bdfbdd2..214ac4d0ed7dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -17,12 +17,12 @@ package org.apache.spark.mllib.regression +import org.json4s.{DefaultFormats, JValue} + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.util.Loader import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} @Experimental trait RegressionModel extends Serializable { @@ -55,16 +55,10 @@ private[mllib] object RegressionModel { /** * Helper method for loading GLM regression model metadata. - * - * @param modelClass String name for model class (used for error messages) * @return numFeatures */ - def getNumFeatures(metadata: DataFrame, modelClass: String, path: String): Int = { - metadata.select("numFeatures").take(1)(0) match { - case Row(nFeatures: Int) => nFeatures - case _ => throw new Exception(s"$modelClass unable to load" + - s" numFeatures from metadata: ${Loader.metadataPath(path)}") - } + def getNumFeatures(metadata: JValue): Int = { + implicit val formats = DefaultFormats + (metadata \ "numFeatures").extract[Int] } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index f2a5f1db1ece6..8838ca8c14718 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -59,7 +59,7 @@ object RidgeRegressionModel extends Loader[RidgeRegressionModel] { val classNameV1_0 = "org.apache.spark.mllib.regression.RidgeRegressionModel" (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val numFeatures = RegressionModel.getNumFeatures(metadata, classNameV1_0, path) + val numFeatures = RegressionModel.getNumFeatures(metadata) val data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) new RidgeRegressionModel(data.weights, data.intercept) case _ => throw new Exception( 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 838100e949ec2..f75de6f637640 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 @@ -17,6 +17,9 @@ package org.apache.spark.mllib.regression.impl +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.Loader @@ -48,10 +51,10 @@ private[regression] object GLMRegressionModel { import sqlContext.implicits._ // Create JSON metadata. - val metadataRDD = - sc.parallelize(Seq((modelClass, thisFormatVersion, weights.size)), 1) - .toDataFrame("class", "version", "numFeatures") - metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + val metadata = compact(render( + ("class" -> modelClass) ~ ("version" -> thisFormatVersion) ~ + ("numFeatures" -> weights.size))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) // Create Parquet data. val data = Data(weights, intercept) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b3e8ed9af8c51..9a586b9d9c68e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.tree -import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer - +import org.apache.spark.Logging import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD -import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Strategy @@ -32,13 +31,10 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.impl._ -import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.SparkContext._ - /** * :: Experimental :: 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 89ecf3773dd77..373192a20c0ea 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 @@ -19,6 +19,10 @@ package org.apache.spark.mllib.tree.model import scala.collection.mutable +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD @@ -184,10 +188,10 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] { import sqlContext.implicits._ // Create JSON metadata. - val metadataRDD = sc.parallelize( - Seq((thisClassName, thisFormatVersion, model.algo.toString, model.numNodes)), 1) - .toDataFrame("class", "version", "algo", "numNodes") - metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("algo" -> model.algo.toString) ~ ("numNodes" -> model.numNodes))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) // Create Parquet data. val nodes = model.topNode.subtreeIterator.toSeq @@ -269,20 +273,10 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] { } override def load(sc: SparkContext, path: String): DecisionTreeModel = { + implicit val formats = DefaultFormats val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) - val (algo: String, numNodes: Int) = try { - val algo_numNodes = metadata.select("algo", "numNodes").collect() - assert(algo_numNodes.length == 1) - algo_numNodes(0) match { - case Row(a: String, n: Int) => (a, n) - } - } catch { - // Catch both Error and Exception since the checks above can throw either. - case e: Throwable => - throw new Exception( - s"Unable to load DecisionTreeModel metadata from: ${Loader.metadataPath(path)}." - + s" Error message: ${e.getMessage}") - } + val algo = (metadata \ "algo").extract[String] + val numNodes = (metadata \ "numNodes").extract[Int] val classNameV1_0 = SaveLoadV1_0.thisClassName (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => 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 23bd46baabf65..dbd69dca60b92 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 @@ -20,18 +20,20 @@ package org.apache.spark.mllib.tree.model import scala.collection.mutable import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.Algo +import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ -import org.apache.spark.mllib.util.{Saveable, Loader} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} - +import org.apache.spark.sql.SQLContext /** * :: Experimental :: @@ -59,11 +61,11 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis object RandomForestModel extends Loader[RandomForestModel] { override def load(sc: SparkContext, path: String): RandomForestModel = { - val (loadedClassName, version, metadataRDD) = Loader.loadMetadata(sc, path) + val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(metadataRDD, path) + val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(jsonMetadata) assert(metadata.treeWeights.forall(_ == 1.0)) val trees = TreeEnsembleModel.SaveLoadV1_0.loadTrees(sc, path, metadata.treeAlgo) @@ -110,11 +112,11 @@ class GradientBoostedTreesModel( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { - val (loadedClassName, version, metadataRDD) = Loader.loadMetadata(sc, path) + val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => - val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(metadataRDD, path) + val metadata = TreeEnsembleModel.SaveLoadV1_0.readMetadata(jsonMetadata) assert(metadata.combiningStrategy == Sum.toString) val trees = TreeEnsembleModel.SaveLoadV1_0.loadTrees(sc, path, metadata.treeAlgo) @@ -252,7 +254,7 @@ private[tree] object TreeEnsembleModel { object SaveLoadV1_0 { - import DecisionTreeModel.SaveLoadV1_0.{NodeData, constructTrees} + import org.apache.spark.mllib.tree.model.DecisionTreeModel.SaveLoadV1_0.{NodeData, constructTrees} def thisFormatVersion = "1.0" @@ -276,11 +278,13 @@ private[tree] object TreeEnsembleModel { import sqlContext.implicits._ // Create JSON metadata. - val metadata = Metadata(model.algo.toString, model.trees(0).algo.toString, + implicit val format = DefaultFormats + val ensembleMetadata = Metadata(model.algo.toString, model.trees(0).algo.toString, model.combiningStrategy.toString, model.treeWeights) - val metadataRDD = sc.parallelize(Seq((className, thisFormatVersion, metadata)), 1) - .toDataFrame("class", "version", "metadata") - metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + val metadata = compact(render( + ("class" -> className) ~ ("version" -> thisFormatVersion) ~ + ("metadata" -> Extraction.decompose(ensembleMetadata)))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) // Create Parquet data. val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => @@ -290,24 +294,11 @@ private[tree] object TreeEnsembleModel { } /** - * Read metadata from the loaded metadata DataFrame. - * @param path Path for loading data, used for debug messages. + * Read metadata from the loaded JSON metadata. */ - def readMetadata(metadata: DataFrame, path: String): Metadata = { - try { - // We rely on the try-catch for schema checking rather than creating a schema just for this. - val metadataArray = metadata.select("metadata.algo", "metadata.treeAlgo", - "metadata.combiningStrategy", "metadata.treeWeights").collect() - assert(metadataArray.size == 1) - Metadata(metadataArray(0).getString(0), metadataArray(0).getString(1), - metadataArray(0).getString(2), metadataArray(0).getAs[Seq[Double]](3).toArray) - } catch { - // Catch both Error and Exception since the checks above can throw either. - case e: Throwable => - throw new Exception( - s"Unable to load TreeEnsembleModel metadata from: ${Loader.metadataPath(path)}." - + s" Error message: ${e.getMessage}") - } + def readMetadata(metadata: JValue): Metadata = { + implicit val formats = DefaultFormats + (metadata \ "metadata").extract[Metadata] } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index 56b77a7d12e83..4458340497f0b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -20,13 +20,13 @@ package org.apache.spark.mllib.util import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.types.{DataType, StructType, StructField} - +import org.apache.spark.sql.types.{DataType, StructField, StructType} /** * :: DeveloperApi :: @@ -120,20 +120,11 @@ private[mllib] object Loader { * Load metadata from the given path. * @return (class name, version, metadata) */ - def loadMetadata(sc: SparkContext, path: String): (String, String, DataFrame) = { - val sqlContext = new SQLContext(sc) - val metadata = sqlContext.jsonFile(metadataPath(path)) - val (clazz, version) = try { - val metadataArray = metadata.select("class", "version").take(1) - assert(metadataArray.size == 1) - metadataArray(0) match { - case Row(clazz: String, version: String) => (clazz, version) - } - } catch { - case e: Exception => - throw new Exception(s"Unable to load model metadata from: ${metadataPath(path)}") - } + def loadMetadata(sc: SparkContext, path: String): (String, String, JValue) = { + implicit val formats = DefaultFormats + val metadata = parse(sc.textFile(metadataPath(path)).first()) + val clazz = (metadata \ "class").extract[String] + val version = (metadata \ "version").extract[String] (clazz, version, metadata) } - } From bc57789bbb2a4fa7733594d30a5818f048aacfed Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 12 Feb 2015 20:14:45 +0000 Subject: [PATCH 080/817] SPARK-5776 JIRA version not of form x.y.z breaks merge_spark_pr.py Consider only x.y.z verisons from JIRA. CC JoshRosen who will probably know this script well. Alternative is to call the version "2.0.0" after all in JIRA. Author: Sean Owen Closes #4570 from srowen/SPARK-5776 and squashes the following commits: fffafde [Sean Owen] Consider only x.y.z verisons from JIRA --- dev/merge_spark_pr.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index dfa924d2aa0ba..3062e9c3c6651 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -244,6 +244,8 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): versions = asf_jira.project_versions("SPARK") versions = sorted(versions, key=lambda x: x.name, reverse=True) versions = filter(lambda x: x.raw['released'] is False, versions) + # Consider only x.y.z versions + versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions) default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) for v in default_fix_versions: From 6a1be026cf37e4c8bf39133dfb4a73f7caedcc26 Mon Sep 17 00:00:00 2001 From: Antonio Navarro Perez Date: Thu, 12 Feb 2015 12:46:17 -0800 Subject: [PATCH 081/817] [SQL][DOCS] Update sql documentation Updated examples using the new api and added DataFrame concept Author: Antonio Navarro Perez Closes #4560 from ajnavarro/ajnavarro-doc-sql-update and squashes the following commits: 82ebcf3 [Antonio Navarro Perez] Changed a missing JavaSQLContext to SQLContext. 8d5376a [Antonio Navarro Perez] fixed typo 8196b6b [Antonio Navarro Perez] [SQL][DOCS] Update sql documentation --- docs/sql-programming-guide.md | 168 +++++++++++++++++----------------- 1 file changed, 84 insertions(+), 84 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b2b007509c735..8022c5ecc2430 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -14,10 +14,10 @@ title: Spark SQL Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of +[DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame). DataFrames are composed of [Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with -a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) +a schema that describes the data types of each column in the row. A DataFrame is similar to a table +in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`. @@ -27,10 +27,10 @@ All of the examples on this page use sample data included in the Spark distribut
Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed of +[DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame). DataFrames are composed of [Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with -a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table -in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) +a schema that describes the data types of each column in the row. A DataFrame is similar to a table +in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
@@ -38,10 +38,10 @@ file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive]( Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed of +[DataFrame](api/python/pyspark.sql.html#pyspark.sql.DataFrame). DataFrames are composed of [Row](api/python/pyspark.sql.Row-class.html) objects, along with -a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) +a schema that describes the data types of each column in the row. A DataFrame is similar to a table +in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell. @@ -65,8 +65,8 @@ descendants. To create a basic SQLContext, all you need is a SparkContext. val sc: SparkContext // An existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. -import sqlContext.createSchemaRDD +// this is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ {% endhighlight %} In addition to the basic SQLContext, you can also create a HiveContext, which provides a @@ -84,12 +84,12 @@ feature parity with a HiveContext.
The entry point into all relational functionality in Spark is the -[JavaSQLContext](api/scala/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one -of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. +[SQLContext](api/scala/index.html#org.apache.spark.sql.api.SQLContext) class, or one +of its descendants. To create a basic SQLContext, all you need is a JavaSparkContext. {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); {% endhighlight %} In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict @@ -138,21 +138,21 @@ default is "hiveql", though "sql" is also available. Since the HiveQL parser is # Data Sources -Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. -A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table. -Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section -describes the various methods for loading data into a SchemaRDD. +Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. +A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. +Registering a DataFrame as a table allows you to run SQL queries over its data. This section +describes the various methods for loading data into a DataFrame. ## RDDs -Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first +Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This reflection based approach leads to more concise code and works well when you already know the schema while writing your Spark application. -The second method for creating SchemaRDDs is through a programmatic interface that allows you to +The second method for creating DataFrames is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows -you to construct SchemaRDDs when the columns and their types are not known until runtime. +you to construct DataFrames when the columns and their types are not known until runtime. ### Inferring the Schema Using Reflection
@@ -160,17 +160,17 @@ you to construct SchemaRDDs when the columns and their types are not known until
The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes -to a SchemaRDD. The case class +to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex -types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be +types such as Sequences or Arrays. This RDD can be implicitly converted to a DataFrame and then be registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} // sc is an existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. -import sqlContext.createSchemaRDD +// this is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ // Define the schema using a case class. // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, @@ -184,7 +184,7 @@ people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -194,7 +194,7 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) -into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table. +into a DataFrame. The BeanInfo, obtained using reflection, defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. @@ -230,7 +230,7 @@ for the JavaBean. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( @@ -247,13 +247,13 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m }); // Apply a schema to an RDD of JavaBeans and register it as a table. -JavaSchemaRDD schemaPeople = sqlContext.createDataFrame(people, Person.class); +DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. -JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. List teenagerNames = teenagers.map(new Function() { public String call(Row row) { @@ -267,7 +267,7 @@ List teenagerNames = teenagers.map(new Function() {
-Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes. Rows are constructed by passing a list of +Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, and the types are inferred by looking at the first row. Since we currently only look at the first row, it is important that there is no missing data in the first row of the RDD. In future versions we @@ -284,11 +284,11 @@ lines = sc.textFile("examples/src/main/resources/people.txt") parts = lines.map(lambda l: l.split(",")) people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) -# Infer the schema, and register the SchemaRDD as a table. +# Infer the schema, and register the DataFrame as a table. schemaPeople = sqlContext.inferSchema(people) schemaPeople.registerTempTable("people") -# SQL can be run over SchemaRDDs that have been registered as a table. +# SQL can be run over DataFrames that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # The results of SQL queries are RDDs and support all the normal RDD operations. @@ -310,7 +310,7 @@ for teenName in teenNames.collect(): When case classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), -a `SchemaRDD` can be created programmatically with three steps. +a `DataFrame` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of @@ -341,15 +341,15 @@ val schema = val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) // Apply the schema to the RDD. -val peopleSchemaRDD = sqlContext.createDataFrame(rowRDD, schema) +val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) -// Register the SchemaRDD as a table. -peopleSchemaRDD.registerTempTable("people") +// Register the DataFrames as a table. +peopleDataFrame.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val results = sqlContext.sql("SELECT name FROM people") -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. results.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -362,13 +362,13 @@ results.map(t => "Name: " + t(0)).collect().foreach(println) When JavaBean classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), -a `SchemaRDD` can be created programmatically with three steps. +a `DataFrame` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of `Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `JavaSQLContext`. +by `SQLContext`. For example: {% highlight java %} @@ -381,7 +381,7 @@ import org.apache.spark.sql.api.java.StructField import org.apache.spark.sql.api.java.Row // sc is an existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); @@ -406,15 +406,15 @@ JavaRDD rowRDD = people.map( }); // Apply the schema to the RDD. -JavaSchemaRDD peopleSchemaRDD = sqlContext.createDataFrame(rowRDD, schema); +DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); -// Register the SchemaRDD as a table. -peopleSchemaRDD.registerTempTable("people"); +// Register the DataFrame as a table. +peopleDataFrame.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. -JavaSchemaRDD results = sqlContext.sql("SELECT name FROM people"); +DataFrame results = sqlContext.sql("SELECT name FROM people"); -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. List names = results.map(new Function() { public String call(Row row) { @@ -431,7 +431,7 @@ List names = results.map(new Function() { When a dictionary of kwargs cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), -a `SchemaRDD` can be created programmatically with three steps. +a `DataFrame` can be created programmatically with three steps. 1. Create an RDD of tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of @@ -460,10 +460,10 @@ schema = StructType(fields) # Apply the schema to the RDD. schemaPeople = sqlContext.createDataFrame(people, schema) -# Register the SchemaRDD as a table. +# Register the DataFrame as a table. schemaPeople.registerTempTable("people") -# SQL can be run over SchemaRDDs that have been registered as a table. +# SQL can be run over DataFrames that have been registered as a table. results = sqlContext.sql("SELECT name FROM people") # The results of SQL queries are RDDs and support all the normal RDD operations. @@ -493,16 +493,16 @@ Using the data from the above example: {% highlight scala %} // sqlContext from the previous example is used in this example. -// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. -import sqlContext.createSchemaRDD +// This is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. -// The RDD is implicitly converted to a SchemaRDD by createSchemaRDD, allowing it to be stored using Parquet. +// The RDD is implicitly converted to a DataFrame by implicits, allowing it to be stored using Parquet. people.saveAsParquetFile("people.parquet") // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. -// The result of loading a Parquet file is also a SchemaRDD. +// The result of loading a Parquet file is also a DataFrame. val parquetFile = sqlContext.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. @@ -518,18 +518,18 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% highlight java %} // sqlContext from the previous example is used in this example. -JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example. +DataFrame schemaPeople = ... // The DataFrame from the previous example. -// JavaSchemaRDDs can be saved as Parquet files, maintaining the schema information. +// DataFrames can be saved as Parquet files, maintaining the schema information. schemaPeople.saveAsParquetFile("people.parquet"); // Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. -// The result of loading a parquet file is also a JavaSchemaRDD. -JavaSchemaRDD parquetFile = sqlContext.parquetFile("people.parquet"); +// The result of loading a parquet file is also a DataFrame. +DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); -JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); @@ -544,13 +544,13 @@ List teenagerNames = teenagers.map(new Function() { {% highlight python %} # sqlContext from the previous example is used in this example. -schemaPeople # The SchemaRDD from the previous example. +schemaPeople # The DataFrame from the previous example. -# SchemaRDDs can be saved as Parquet files, maintaining the schema information. +# DataFrames can be saved as Parquet files, maintaining the schema information. schemaPeople.saveAsParquetFile("people.parquet") # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. -# The result of loading a parquet file is also a SchemaRDD. +# The result of loading a parquet file is also a DataFrame. parquetFile = sqlContext.parquetFile("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. @@ -629,7 +629,7 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD. +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. @@ -646,7 +646,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) // A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. val path = "examples/src/main/resources/people.json" -// Create a SchemaRDD from the file(s) pointed to by path +// Create a DataFrame from the file(s) pointed to by path val people = sqlContext.jsonFile(path) // The inferred schema can be visualized using the printSchema() method. @@ -655,13 +655,13 @@ people.printSchema() // |-- age: integer (nullable = true) // |-- name: string (nullable = true) -// Register this SchemaRDD as a table. +// Register this DataFrame as a table. people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") -// Alternatively, a SchemaRDD can be created for a JSON dataset represented by +// Alternatively, a DataFrame can be created for a JSON dataset represented by // an RDD[String] storing one JSON object per string. val anotherPeopleRDD = sc.parallelize( """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) @@ -671,8 +671,8 @@ val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a JavaSchemaRDD. -This conversion can be done using one of two methods in a JavaSQLContext : +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. +This conversion can be done using one of two methods in a SQLContext : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -683,13 +683,13 @@ a regular multi-line JSON file will most often fail. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); // A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; -// Create a JavaSchemaRDD from the file(s) pointed to by path -JavaSchemaRDD people = sqlContext.jsonFile(path); +// Create a DataFrame from the file(s) pointed to by path +DataFrame people = sqlContext.jsonFile(path); // The inferred schema can be visualized using the printSchema() method. people.printSchema(); @@ -697,23 +697,23 @@ people.printSchema(); // |-- age: integer (nullable = true) // |-- name: string (nullable = true) -// Register this JavaSchemaRDD as a table. +// Register this DataFrame as a table. people.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlContext. -JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); +DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); -// Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by +// Alternatively, a DataFrame can be created for a JSON dataset represented by // an RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); -JavaSchemaRDD anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD); +DataFrame anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD); {% endhighlight %}
-Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD. +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. @@ -731,7 +731,7 @@ sqlContext = SQLContext(sc) # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. path = "examples/src/main/resources/people.json" -# Create a SchemaRDD from the file(s) pointed to by path +# Create a DataFrame from the file(s) pointed to by path people = sqlContext.jsonFile(path) # The inferred schema can be visualized using the printSchema() method. @@ -740,13 +740,13 @@ people.printSchema() # |-- age: integer (nullable = true) # |-- name: string (nullable = true) -# Register this SchemaRDD as a table. +# Register this DataFrame as a table. people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") -# Alternatively, a SchemaRDD can be created for a JSON dataset represented by +# Alternatively, a DataFrame can be created for a JSON dataset represented by # an RDD[String] storing one JSON object per string. anotherPeopleRDD = sc.parallelize([ '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) @@ -792,14 +792,14 @@ sqlContext.sql("FROM src SELECT key, value").collect().foreach(println)
-When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and +When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to -the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be +the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be expressed in HiveQL. {% highlight java %} // sc is an existing JavaSparkContext. -JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); +HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc); sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); @@ -841,7 +841,7 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `dataFrame.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. @@ -1161,7 +1161,7 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println) The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.DataFrame). From aa4ca8b873fd83e64e5faea6f7febcc830e30b02 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Feb 2015 13:11:28 -0800 Subject: [PATCH 082/817] [SQL] Improve error messages Author: Michael Armbrust Author: wangfei Closes #4558 from marmbrus/errorMessages and squashes the following commits: 5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages fa38881 [wangfei] fix for grouping__id f279a71 [wangfei] make right references for ScriptTransformation d29fbde [Michael Armbrust] extra case 1a797b4 [Michael Armbrust] comments d4e9015 [Michael Armbrust] add comment af9e668 [Michael Armbrust] no braces 34eb3a4 [Michael Armbrust] more work 6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures --- .../sql/catalyst/analysis/Analyzer.scala | 123 ++++++++++-------- .../catalyst/expressions/AttributeSet.scala | 6 +- .../expressions/namedExpressions.scala | 2 +- .../plans/logical/ScriptTransformation.scala | 6 +- .../spark/sql/catalyst/trees/TreeNode.scala | 9 ++ .../sql/catalyst/analysis/AnalysisSuite.scala | 79 +++++++---- .../BooleanSimplificationSuite.scala | 4 +- .../optimizer/ConstantFoldingSuite.scala | 4 +- .../optimizer/FilterPushdownSuite.scala | 12 +- .../catalyst/optimizer/OptimizeInSuite.scala | 4 +- .../optimizer/UnionPushdownSuite.scala | 4 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/execution/commands.scala | 4 +- 14 files changed, 164 insertions(+), 103 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2d1fa106a2aa9..58a7003977c93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.{ArrayType, StructField, StructType, IntegerType} +import org.apache.spark.sql.types._ /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing @@ -66,32 +66,82 @@ class Analyzer(catalog: Catalog, typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, - CheckResolution :: - CheckAggregation :: - Nil: _*), - Batch("AnalysisOperators", fixedPoint, - EliminateAnalysisOperators) + CheckResolution), + Batch("Remove SubQueries", fixedPoint, + EliminateSubQueries) ) /** * Makes sure all attributes and logical plans have been resolved. */ object CheckResolution extends Rule[LogicalPlan] { + def failAnalysis(msg: String) = { throw new AnalysisException(msg) } + def apply(plan: LogicalPlan): LogicalPlan = { - plan.transformUp { - case p if p.expressions.exists(!_.resolved) => - val missing = p.expressions.filterNot(_.resolved).map(_.prettyString).mkString(",") - val from = p.inputSet.map(_.name).mkString("{", ", ", "}") - - throw new AnalysisException(s"Cannot resolve '$missing' given input columns $from") - case p if !p.resolved && p.childrenResolved => - throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}") - } match { - // As a backstop, use the root node to check that the entire plan tree is resolved. - case p if !p.resolved => - throw new AnalysisException(s"Unresolved operator in the query plan ${p.simpleString}") - case p => p + // We transform up and order the rules so as to catch the first possible failure instead + // of the result of cascading resolution failures. + plan.foreachUp { + case operator: LogicalPlan => + operator transformExpressionsUp { + case a: Attribute if !a.resolved => + val from = operator.inputSet.map(_.name).mkString(", ") + failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") + + case c: Cast if !c.resolved => + failAnalysis( + s"invalid cast from ${c.child.dataType.simpleString} to ${c.dataType.simpleString}") + + case b: BinaryExpression if !b.resolved => + failAnalysis( + s"invalid expression ${b.prettyString} " + + s"between ${b.left.simpleString} and ${b.right.simpleString}") + } + + operator match { + case f: Filter if f.condition.dataType != BooleanType => + failAnalysis( + s"filter expression '${f.condition.prettyString}' " + + s"of type ${f.condition.dataType.simpleString} is not a boolean.") + + case aggregatePlan @ Aggregate(groupingExprs, aggregateExprs, child) => + def checkValidAggregateExpression(expr: Expression): Unit = expr match { + case _: AggregateExpression => // OK + case e: Attribute if !groupingExprs.contains(e) => + failAnalysis( + s"expression '${e.prettyString}' is neither present in the group by, " + + s"nor is it an aggregate function. " + + "Add to group by or wrap in first() if you don't care which value you get.") + case e if groupingExprs.contains(e) => // OK + case e if e.references.isEmpty => // OK + case e => e.children.foreach(checkValidAggregateExpression) + } + + val cleaned = aggregateExprs.map(_.transform { + // Should trim aliases around `GetField`s. These aliases are introduced while + // resolving struct field accesses, because `GetField` is not a `NamedExpression`. + // (Should we just turn `GetField` into a `NamedExpression`?) + case Alias(g, _) => g + }) + + cleaned.foreach(checkValidAggregateExpression) + + case o if o.children.nonEmpty && + !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => + val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") + val input = o.inputSet.map(_.prettyString).mkString(",") + + failAnalysis(s"resolved attributes $missingAttributes missing from $input") + + // Catch all + case o if !o.resolved => + failAnalysis( + s"unresolved operator ${operator.simpleString}") + + case _ => // Analysis successful! + } } + + plan } } @@ -192,37 +242,6 @@ class Analyzer(catalog: Catalog, } } - /** - * Checks for non-aggregated attributes with aggregation - */ - object CheckAggregation extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - plan.transform { - case aggregatePlan @ Aggregate(groupingExprs, aggregateExprs, child) => - def isValidAggregateExpression(expr: Expression): Boolean = expr match { - case _: AggregateExpression => true - case e: Attribute => groupingExprs.contains(e) - case e if groupingExprs.contains(e) => true - case e if e.references.isEmpty => true - case e => e.children.forall(isValidAggregateExpression) - } - - aggregateExprs.find { e => - !isValidAggregateExpression(e.transform { - // Should trim aliases around `GetField`s. These aliases are introduced while - // resolving struct field accesses, because `GetField` is not a `NamedExpression`. - // (Should we just turn `GetField` into a `NamedExpression`?) - case Alias(g: GetField, _) => g - }) - }.foreach { e => - throw new TreeNodeException(plan, s"Expression not in GROUP BY: $e") - } - - aggregatePlan - } - } - } - /** * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ @@ -230,7 +249,7 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transform { case i @ InsertIntoTable(UnresolvedRelation(tableIdentifier, alias), _, _, _) => i.copy( - table = EliminateAnalysisOperators(catalog.lookupRelation(tableIdentifier, alias))) + table = EliminateSubQueries(catalog.lookupRelation(tableIdentifier, alias))) case UnresolvedRelation(tableIdentifier, alias) => catalog.lookupRelation(tableIdentifier, alias) } @@ -477,7 +496,7 @@ class Analyzer(catalog: Catalog, * only required to provide scoping information for attributes and can be removed once analysis is * complete. */ -object EliminateAnalysisOperators extends Rule[LogicalPlan] { +object EliminateSubQueries extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 171845ad14e3e..a9ba0be596349 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -20,7 +20,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.Star protected class AttributeEquals(val a: Attribute) { - override def hashCode() = a.exprId.hashCode() + override def hashCode() = a match { + case ar: AttributeReference => ar.exprId.hashCode() + case a => a.hashCode() + } + override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match { case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId case (a1, a2) => a1 == a2 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index f77c56311cc8c..62c062be6d820 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -218,7 +218,7 @@ case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[E override def exprId: ExprId = ??? override def eval(input: Row): EvaluatedType = ??? override def nullable: Boolean = ??? - override def dataType: DataType = ??? + override def dataType: DataType = NullType } object VirtualColumn { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index cfe2c7a39a17c..ccf5291219add 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Attribute, Expression} /** * Transforms the input by forking and running the specified script. @@ -32,7 +32,9 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: LogicalPlan, - ioschema: ScriptInputOutputSchema) extends UnaryNode + ioschema: ScriptInputOutputSchema) extends UnaryNode { + override def references: AttributeSet = AttributeSet(input.flatMap(_.references)) +} /** * A placeholder for implementation specific input and output properties when passing data diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 2013ae4f7bd13..e0930b056d5fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -46,6 +46,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { children.foreach(_.foreach(f)) } + /** + * Runs the given function recursively on [[children]] then on this node. + * @param f the function to be applied to each node in the tree. + */ + def foreachUp(f: BaseType => Unit): Unit = { + children.foreach(_.foreach(f)) + f(this) + } + /** * Returns a Seq containing the result of applying the given function to each * node in this tree in a preorder traversal. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f011a5ff15ea9..e70c651e1486e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Literal, Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ @@ -108,24 +108,56 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { testRelation) } - test("throw errors for unresolved attributes during analysis") { - val e = intercept[AnalysisException] { - caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) + def errorTest( + name: String, + plan: LogicalPlan, + errorMessages: Seq[String], + caseSensitive: Boolean = true) = { + test(name) { + val error = intercept[AnalysisException] { + if(caseSensitive) { + caseSensitiveAnalyze(plan) + } else { + caseInsensitiveAnalyze(plan) + } + } + + errorMessages.foreach(m => assert(error.getMessage contains m)) } - assert(e.getMessage().toLowerCase.contains("cannot resolve")) } - test("throw errors for unresolved plans during analysis") { - case class UnresolvedTestPlan() extends LeafNode { - override lazy val resolved = false - override def output = Nil - } - val e = intercept[AnalysisException] { - caseSensitiveAnalyze(UnresolvedTestPlan()) - } - assert(e.getMessage().toLowerCase.contains("unresolved")) + errorTest( + "unresolved attributes", + testRelation.select('abcd), + "cannot resolve" :: "abcd" :: Nil) + + errorTest( + "bad casts", + testRelation.select(Literal(1).cast(BinaryType).as('badCast)), + "invalid cast" :: Literal(1).dataType.simpleString :: BinaryType.simpleString :: Nil) + + errorTest( + "non-boolean filters", + testRelation.where(Literal(1)), + "filter" :: "'1'" :: "not a boolean" :: Literal(1).dataType.simpleString :: Nil) + + errorTest( + "missing group by", + testRelation2.groupBy('a)('b), + "'b'" :: "group by" :: Nil + ) + + case class UnresolvedTestPlan() extends LeafNode { + override lazy val resolved = false + override def output = Nil } + errorTest( + "catch all unresolved plan", + UnresolvedTestPlan(), + "unresolved" :: Nil) + + test("divide should be casted into fractional types") { val testRelation2 = LocalRelation( AttributeReference("a", StringType)(), @@ -134,18 +166,15 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) - val expr0 = 'a / 2 - val expr1 = 'a / 'b - val expr2 = 'a / 'c - val expr3 = 'a / 'd - val expr4 = 'e / 'e - val plan = caseInsensitiveAnalyze(Project( - Alias(expr0, s"Analyzer($expr0)")() :: - Alias(expr1, s"Analyzer($expr1)")() :: - Alias(expr2, s"Analyzer($expr2)")() :: - Alias(expr3, s"Analyzer($expr3)")() :: - Alias(expr4, s"Analyzer($expr4)")() :: Nil, testRelation2)) + val plan = caseInsensitiveAnalyze( + testRelation2.select( + 'a / Literal(2) as 'div1, + 'a / 'b as 'div2, + 'a / 'c as 'div3, + 'a / 'd as 'div4, + 'e / 'e as 'div5)) val pl = plan.asInstanceOf[Project].projectList + assert(pl(0).dataType == DoubleType) assert(pl(1).dataType == DoubleType) assert(pl(2).dataType == DoubleType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 264a0eff37d34..72f06e26e05f1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -30,7 +30,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("AnalysisNodes", Once, - EliminateAnalysisOperators) :: + EliminateSubQueries) :: Batch("Constant Folding", FixedPoint(50), NullPropagation, ConstantFolding, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index e22c62505860a..ef10c0aece716 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, EliminateAnalysisOperators} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, EliminateSubQueries} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.PlanTest @@ -33,7 +33,7 @@ class ConstantFoldingSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("AnalysisNodes", Once, - EliminateAnalysisOperators) :: + EliminateSubQueries) :: Batch("ConstantFolding", Once, ConstantFolding, BooleanSimplification) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 1158b5dfc6147..0b74bacb18f4b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions.Explode import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} @@ -32,7 +32,7 @@ class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateAnalysisOperators) :: + EliminateSubQueries) :: Batch("Filter Pushdown", Once, CombineFilters, PushPredicateThroughProject, @@ -351,7 +351,7 @@ class FilterPushdownSuite extends PlanTest { } val optimized = Optimize(originalQuery.analyze) - comparePlans(analysis.EliminateAnalysisOperators(originalQuery.analyze), optimized) + comparePlans(analysis.EliminateSubQueries(originalQuery.analyze), optimized) } test("joins: conjunctive predicates") { @@ -370,7 +370,7 @@ class FilterPushdownSuite extends PlanTest { left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze - comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) + comparePlans(optimized, analysis.EliminateSubQueries(correctAnswer)) } test("joins: conjunctive predicates #2") { @@ -389,7 +389,7 @@ class FilterPushdownSuite extends PlanTest { left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze - comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) + comparePlans(optimized, analysis.EliminateSubQueries(correctAnswer)) } test("joins: conjunctive predicates #3") { @@ -412,7 +412,7 @@ class FilterPushdownSuite extends PlanTest { condition = Some("z.a".attr === "x.b".attr)) .analyze - comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) + comparePlans(optimized, analysis.EliminateSubQueries(correctAnswer)) } val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index da912ab382179..233e329cb2038 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.immutable.HashSet -import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.PlanTest @@ -34,7 +34,7 @@ class OptimizeInSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("AnalysisNodes", Once, - EliminateAnalysisOperators) :: + EliminateSubQueries) :: Batch("ConstantFolding", Once, ConstantFolding, BooleanSimplification, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala index dfef87bd9133d..a54751dfa9a12 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ @@ -29,7 +29,7 @@ class UnionPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateAnalysisOperators) :: + EliminateSubQueries) :: Batch("Union Pushdown", Once, UnionPushdown) :: Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bba8899651259..a1c8cf58f2357 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -806,10 +806,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("throw errors for non-aggregate attributes with aggregation") { def checkAggregation(query: String, isInvalidQuery: Boolean = true) { if (isInvalidQuery) { - val e = intercept[TreeNodeException[LogicalPlan]](sql(query).queryExecution.analyzed) - assert( - e.getMessage.startsWith("Expression not in GROUP BY"), - "Non-aggregate attribute(s) not detected\n") + val e = intercept[AnalysisException](sql(query).queryExecution.analyzed) + assert(e.getMessage contains "group by") } else { // Should not throw sql(query).queryExecution.analyzed diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7ae6ed6f841bf..ddc7b181d4d46 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -37,7 +37,7 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, SetCommand, QueryExecutionException} import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand} @@ -104,7 +104,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ @Experimental def analyze(tableName: String) { - val relation = EliminateAnalysisOperators(catalog.lookupRelation(Seq(tableName))) + val relation = EliminateSubQueries(catalog.lookupRelation(Seq(tableName))) relation match { case relation: MetastoreRelation => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index f6bea1c6a6fe1..ce0db1125c27f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.{DataFrame, SQLContext} @@ -175,7 +175,7 @@ case class CreateMetastoreDataSourceAsSelect( val resolved = ResolvedDataSource(sqlContext, Some(query.schema), provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) - EliminateAnalysisOperators(sqlContext.table(tableName).logicalPlan) match { + EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { case l @ LogicalRelation(i: InsertableRelation) => if (l.schema != createdRelation.schema) { val errorDescription = From 893d6fd7049daf3c4d01eb6a960801cd064d5f73 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 12 Feb 2015 14:35:44 -0800 Subject: [PATCH 083/817] [SPARK-5645] Added local read bytes/time to task metrics ksakellis I stumbled on your JIRA for this yesterday; I know it's assigned to you but I'd already done this for my own uses a while ago so thought I could help save you the work of doing it! Hopefully this doesn't duplicate any work you've already done. Here's a screenshot of what the UI looks like: ![image](https://cloud.githubusercontent.com/assets/1108612/6135352/c03e7276-b11c-11e4-8f11-c6aefe1f35b9.png) Based on a discussion with pwendell, I put the data read remotely in as an additional metric rather than showing it in brackets as you'd suggested, Kostas. The assumption here is that the average user doesn't care about the differentiation between local / remote data, so it's better not to pollute the UI. I also added data about the local read time, which I've found very helpful for debugging, but I didn't put it in the UI because I think it's probably something not a ton of people will need to use. With this change, the total read time and total write time shown in the UI will be equal, fixing a long-term source of user confusion: ![image](https://cloud.githubusercontent.com/assets/1108612/6135399/25f14490-b11d-11e4-8086-20be5f4002e6.png) Author: Kay Ousterhout Closes #4510 from kayousterhout/SPARK-5645 and squashes the following commits: 4a0182c [Kay Ousterhout] oops 5f5da1b [Kay Ousterhout] Small style fix 5da04cf [Kay Ousterhout] Addressed more comments from Kostas ba05149 [Kay Ousterhout] Remove parens a9dc685 [Kay Ousterhout] Kostas comment, test fix 33d2e2d [Kay Ousterhout] Merge remote-tracking branch 'upstream/master' into SPARK-5645 347e2cd [Kay Ousterhout] [SPARK-5645] Added local read bytes/time to task metrics --- .../org/apache/spark/ui/static/webui.css | 4 +- .../apache/spark/executor/TaskMetrics.scala | 21 ++++++ .../apache/spark/scheduler/JobLogger.scala | 4 +- .../storage/ShuffleBlockFetcherIterator.scala | 3 + .../scala/org/apache/spark/ui/ToolTips.scala | 8 ++- .../spark/ui/jobs/JobProgressListener.scala | 6 +- .../org/apache/spark/ui/jobs/StagePage.scala | 67 +++++++++++++++---- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../spark/ui/jobs/TaskDetailsClassNames.scala | 1 + .../org/apache/spark/ui/jobs/UIData.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 4 ++ .../ui/jobs/JobProgressListenerSuite.scala | 12 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 17 +++++ 13 files changed, 125 insertions(+), 28 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 68b33b5f0d7c7..6c37cc8b98236 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -196,7 +196,7 @@ span.additional-metric-title { /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ -.scheduler_delay, .deserialization_time, .fetch_wait_time, .serialization_time, -.getting_result_time { +.scheduler_delay, .deserialization_time, .fetch_wait_time, .shuffle_read_remote, +.serialization_time, .getting_result_time { display: none; } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index bf3f1e4fc7832..df36566bec4b1 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -202,6 +202,8 @@ class TaskMetrics extends Serializable { merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) merged.incRemoteBytesRead(depMetrics.remoteBytesRead) + merged.incLocalBytesRead(depMetrics.localBytesRead) + merged.incLocalReadTime(depMetrics.localReadTime) merged.incRecordsRead(depMetrics.recordsRead) } _shuffleReadMetrics = Some(merged) @@ -343,6 +345,25 @@ class ShuffleReadMetrics extends Serializable { private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value + /** + * Time the task spent (in milliseconds) reading local shuffle blocks (from the local disk). + */ + private var _localReadTime: Long = _ + def localReadTime = _localReadTime + private[spark] def incLocalReadTime(value: Long) = _localReadTime += value + + /** + * Shuffle data that was read from the local disk (as opposed to from a remote executor). + */ + private var _localBytesRead: Long = _ + def localBytesRead = _localBytesRead + private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value + + /** + * Total bytes fetched in the shuffle by this task (both remote and local). + */ + def totalBytesRead = _remoteBytesRead + _localBytesRead + /** * Number of blocks fetched in this shuffle by this task (remote or local) */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 3bb54855bae44..f9fc8aa30454e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -169,7 +169,9 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + - " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + + " LOCAL_READ_TIME=" + metrics.localReadTime + + " LOCAL_BYTES_READ=" + metrics.localBytesRead case None => "" } val writeMetrics = taskMetrics.shuffleWriteMetrics match { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index ab9ee4f0096bf..2ebb79989da43 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -228,12 +228,14 @@ final class ShuffleBlockFetcherIterator( * track in-memory are the ManagedBuffer references themselves. */ private[this] def fetchLocalBlocks() { + val startTime = System.currentTimeMillis val iter = localBlocks.iterator while (iter.hasNext) { val blockId = iter.next() try { val buf = blockManager.getBlockData(blockId) shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() results.put(new SuccessFetchResult(blockId, 0, buf)) } catch { @@ -244,6 +246,7 @@ final class ShuffleBlockFetcherIterator( return } } + shuffleMetrics.incLocalReadTime(System.currentTimeMillis - startTime) } private[this] def initialize(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 3a15e603b1969..cae6870c2ab20 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -37,8 +37,12 @@ private[spark] object ToolTips { "Bytes and records written to disk in order to be read by a shuffle in a future stage." val SHUFFLE_READ = - """Bytes and records read from remote executors. Typically less than shuffle write bytes - because this does not include shuffle data read locally.""" + """Total shuffle bytes and records read (includes both data read locally and data read from + remote executors). """ + + val SHUFFLE_READ_REMOTE_SIZE = + """Total shuffle bytes read from remote executors. This is a subset of the shuffle + read bytes; the remaining shuffle data is read locally. """ val GETTING_RESULT_TIME = """Time that the driver spends fetching task results from workers. If this is large, consider diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index f463f8d7c7215..0b6fe70bd2062 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -401,9 +401,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta val shuffleReadDelta = - (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L)) - stageData.shuffleReadBytes += shuffleReadDelta + (taskMetrics.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.totalBytesRead).getOrElse(0L)) + stageData.shuffleReadTotalBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta val shuffleReadRecordsDelta = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 05ffd5bc58fbb..d752434ad58ae 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -85,7 +85,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {if (stageData.hasShuffleRead) {
  • Shuffle read: - {s"${Utils.bytesToString(stageData.shuffleReadBytes)} / " + + {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + s"${stageData.shuffleReadRecords}"}
  • }} @@ -143,6 +143,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Shuffle Read Blocked Time +
  • + + + Shuffle Remote Reads + +
  • }}
  • metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble } - val shuffleReadBlockedQuantiles = Shuffle Read Blocked Time +: + val shuffleReadBlockedQuantiles = + + + Shuffle Read Blocked Time + + +: getFormattedTimeQuantiles(shuffleReadBlockedTimes) - val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble } - - val shuffleReadRecords = validTasks.map { case TaskUIData(_, metrics, _) => + val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble } + val shuffleReadTotalQuantiles = + + + Shuffle Read Size / Records + + +: + getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) - val shuffleReadQuantiles = Shuffle Read Size / Records (Remote) +: - getFormattedSizeQuantilesWithRecords(shuffleReadSizes, shuffleReadRecords) + val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + } + val shuffleReadRemoteQuantiles = + + + Shuffle Remote Reads + + +: + getFormattedSizeQuantiles(shuffleReadRemoteSizes) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble @@ -374,7 +404,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {shuffleReadBlockedQuantiles} - {shuffleReadQuantiles} + {shuffleReadTotalQuantiles} + + {shuffleReadRemoteQuantiles} + } else { Nil }, @@ -454,11 +487,15 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val shuffleReadBlockedTimeReadable = maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") - val shuffleReadSortable = maybeShuffleRead.map(_.remoteBytesRead.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead - .map(m => s"${Utils.bytesToString(m.remoteBytesRead)}").getOrElse("") + val totalShuffleBytes = maybeShuffleRead.map(_.totalBytesRead) + val shuffleReadSortable = totalShuffleBytes.map(_.toString).getOrElse("") + val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("") val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("") + val remoteShuffleBytes = maybeShuffleRead.map(_.remoteBytesRead) + val shuffleReadRemoteSortable = remoteShuffleBytes.map(_.toString).getOrElse("") + val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("") + val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics) val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten.toString).getOrElse("") val shuffleWriteReadable = maybeShuffleWrite @@ -536,6 +573,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {s"$shuffleReadReadable / $shuffleReadRecords"} + + {shuffleReadRemoteReadable} + }} {if (hasShuffleWrite) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 703d43f9c640d..5865850fa09b5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -138,7 +138,7 @@ private[ui] class StageTableBase( val inputReadWithUnit = if (inputRead > 0) Utils.bytesToString(inputRead) else "" val outputWrite = stageData.outputBytes val outputWriteWithUnit = if (outputWrite > 0) Utils.bytesToString(outputWrite) else "" - val shuffleRead = stageData.shuffleReadBytes + val shuffleRead = stageData.shuffleReadTotalBytes val shuffleReadWithUnit = if (shuffleRead > 0) Utils.bytesToString(shuffleRead) else "" val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala index 37cf2c207ba40..9bf67db8acde1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -28,6 +28,7 @@ private[spark] object TaskDetailsClassNames { val SCHEDULER_DELAY = "scheduler_delay" val TASK_DESERIALIZATION_TIME = "deserialization_time" val SHUFFLE_READ_BLOCKED_TIME = "fetch_wait_time" + val SHUFFLE_READ_REMOTE_SIZE = "shuffle_read_remote" val RESULT_SERIALIZATION_TIME = "serialization_time" val GETTING_RESULT_TIME = "getting_result_time" } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 69aac6c862de5..dbf1ceeda1878 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -80,7 +80,7 @@ private[jobs] object UIData { var inputRecords: Long = _ var outputBytes: Long = _ var outputRecords: Long = _ - var shuffleReadBytes: Long = _ + var shuffleReadTotalBytes: Long = _ var shuffleReadRecords : Long = _ var shuffleWriteBytes: Long = _ var shuffleWriteRecords: Long = _ @@ -96,7 +96,7 @@ private[jobs] object UIData { def hasInput = inputBytes > 0 def hasOutput = outputBytes > 0 - def hasShuffleRead = shuffleReadBytes > 0 + def hasShuffleRead = shuffleReadTotalBytes > 0 def hasShuffleWrite = shuffleWriteBytes > 0 def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b0b545640f5aa..58d37e2d667f7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -294,6 +294,8 @@ private[spark] object JsonProtocol { ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ + ("Local Read Time" -> shuffleReadMetrics.localReadTime) ~ + ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> shuffleReadMetrics.recordsRead) } @@ -674,6 +676,8 @@ private[spark] object JsonProtocol { metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int]) metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) + metrics.incLocalReadTime((json \ "Local Read Time").extractOpt[Long].getOrElse(0)) + metrics.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) metrics } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index e8405baa8e3ea..6019282d2fb70 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -227,6 +227,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) shuffleReadMetrics.incRemoteBytesRead(base + 1) + shuffleReadMetrics.incLocalBytesRead(base + 9) shuffleReadMetrics.incRemoteBlocksFetched(base + 2) shuffleWriteMetrics.incShuffleBytesWritten(base + 3) taskMetrics.setExecutorRunTime(base + 4) @@ -260,8 +261,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc var stage0Data = listener.stageIdToData.get((0, 0)).get var stage1Data = listener.stageIdToData.get((1, 0)).get - assert(stage0Data.shuffleReadBytes == 102) - assert(stage1Data.shuffleReadBytes == 201) + assert(stage0Data.shuffleReadTotalBytes == 220) + assert(stage1Data.shuffleReadTotalBytes == 410) assert(stage0Data.shuffleWriteBytes == 106) assert(stage1Data.shuffleWriteBytes == 203) assert(stage0Data.executorRunTime == 108) @@ -290,8 +291,11 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc stage0Data = listener.stageIdToData.get((0, 0)).get stage1Data = listener.stageIdToData.get((1, 0)).get - assert(stage0Data.shuffleReadBytes == 402) - assert(stage1Data.shuffleReadBytes == 602) + // Task 1235 contributed (100+1)+(100+9) = 210 shuffle bytes, and task 1234 contributed + // (300+1)+(300+9) = 610 total shuffle bytes, so the total for the stage is 820. + assert(stage0Data.shuffleReadTotalBytes == 820) + // Task 1236 contributed 410 shuffle bytes, and task 1237 contributed 810 shuffle bytes. + assert(stage1Data.shuffleReadTotalBytes == 1220) assert(stage0Data.shuffleWriteBytes == 406) assert(stage1Data.shuffleWriteBytes == 606) assert(stage0Data.executorRunTime == 408) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f3017dc42cd5c..c181baf6844b0 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -260,6 +260,19 @@ class JsonProtocolSuite extends FunSuite { assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("ShuffleReadMetrics: Local bytes read and time taken backwards compatibility") { + // Metrics about local shuffle bytes read and local read time were added in 1.3.1. + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = false, hasOutput = false, hasRecords = false) + assert(metrics.shuffleReadMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Local Bytes Read" } + .removeField { case (field, _) => field == "Local Read Time" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.shuffleReadMetrics.get.localBytesRead == 0) + assert(newMetrics.shuffleReadMetrics.get.localReadTime == 0) + } + test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") @@ -695,6 +708,8 @@ class JsonProtocolSuite extends FunSuite { sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1) + sr.incLocalReadTime(a + e) + sr.incLocalBytesRead(a + f) t.setShuffleReadMetrics(Some(sr)) } if (hasOutput) { @@ -941,6 +956,8 @@ class JsonProtocolSuite extends FunSuite { | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, + | "Local Read Time": 1000, + | "Local Bytes Read": 1100, | "Total Records Read" : 10 | }, | "Shuffle Write Metrics": { From 9c8076502f0c2c6a6dcdc6720d16b34132dfc06a Mon Sep 17 00:00:00 2001 From: Katsunori Kanda Date: Thu, 12 Feb 2015 14:38:42 -0800 Subject: [PATCH 084/817] [EC2] Update default Spark version to 1.2.1 Author: Katsunori Kanda Closes #4566 from potix2/ec2-update-version-1-2-1 and squashes the following commits: 77e7840 [Katsunori Kanda] [EC2] Update default Spark version to 1.2.1 --- ec2/spark_ec2.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0ea7365d75b83..35209aec5fcfe 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -42,7 +42,7 @@ from optparse import OptionParser from sys import stderr -SPARK_EC2_VERSION = "1.2.0" +SPARK_EC2_VERSION = "1.2.1" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) VALID_SPARK_VERSIONS = set([ @@ -58,6 +58,7 @@ "1.1.0", "1.1.1", "1.2.0", + "1.2.1", ]) DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION From 629d0143eeb3c153dac9c65e7b556723c6b4bfc7 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Thu, 12 Feb 2015 14:44:21 -0800 Subject: [PATCH 085/817] [SPARK-5765][Examples]Fixed word split problem in run-example and compute-classpath Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #4561 from gvramana/word_split and squashes the following commits: 285c8d4 [Venkata Ramana Gollamudi] Fixed word split problem in run-example and compute-classpath --- bin/compute-classpath.sh | 4 ++-- bin/run-example | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index a8c344b1ca594..f4f6b7b909490 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -76,7 +76,7 @@ fi num_jars=0 -for f in ${assembly_folder}/spark-assembly*hadoop*.jar; do +for f in "${assembly_folder}"/spark-assembly*hadoop*.jar; do if [[ ! -e "$f" ]]; then echo "Failed to find Spark assembly in $assembly_folder" 1>&2 echo "You need to build Spark before running this program." 1>&2 @@ -88,7 +88,7 @@ done if [ "$num_jars" -gt "1" ]; then echo "Found multiple Spark assembly jars in $assembly_folder:" 1>&2 - ls ${assembly_folder}/spark-assembly*hadoop*.jar 1>&2 + ls "${assembly_folder}"/spark-assembly*hadoop*.jar 1>&2 echo "Please remove all but one jar." 1>&2 exit 1 fi diff --git a/bin/run-example b/bin/run-example index c567acf9a6b5c..a106411392e06 100755 --- a/bin/run-example +++ b/bin/run-example @@ -42,7 +42,7 @@ fi JAR_COUNT=0 -for f in ${JAR_PATH}/spark-examples-*hadoop*.jar; do +for f in "${JAR_PATH}"/spark-examples-*hadoop*.jar; do if [[ ! -e "$f" ]]; then echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 echo "You need to build Spark before running this program" 1>&2 @@ -54,7 +54,7 @@ done if [ "$JAR_COUNT" -gt "1" ]; then echo "Found multiple Spark examples assembly jars in ${JAR_PATH}" 1>&2 - ls ${JAR_PATH}/spark-examples-*hadoop*.jar 1>&2 + ls "${JAR_PATH}"/spark-examples-*hadoop*.jar 1>&2 echo "Please remove all but one jar." 1>&2 exit 1 fi From 47c73d410ab533c3196184d2b6004081e79daeaa Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 12 Feb 2015 14:46:37 -0800 Subject: [PATCH 086/817] [SPARK-5762] Fix shuffle write time for sort-based shuffle mateiz was excluding the time to write this final file from the shuffle write time intentional? Author: Kay Ousterhout Closes #4559 from kayousterhout/SPARK-5762 and squashes the following commits: 5c6f3d9 [Kay Ousterhout] Use foreach 94e4237 [Kay Ousterhout] Removed open time metrics added inadvertently ace156c [Kay Ousterhout] Moved metrics to finally block d773276 [Kay Ousterhout] Use nano time 5a59906 [Kay Ousterhout] [SPARK-5762] Fix shuffle write time for sort-based shuffle --- .../org/apache/spark/util/collection/ExternalSorter.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index eaec5a71e6819..d69f2d9048055 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -723,6 +723,7 @@ private[spark] class ExternalSorter[K, V, C]( partitionWriters.foreach(_.commitAndClose()) var out: FileOutputStream = null var in: FileInputStream = null + val writeStartTime = System.nanoTime try { out = new FileOutputStream(outputFile, true) for (i <- 0 until numPartitions) { @@ -739,6 +740,8 @@ private[spark] class ExternalSorter[K, V, C]( if (in != null) { in.close() } + context.taskMetrics.shuffleWriteMetrics.foreach( + _.incShuffleWriteTime(System.nanoTime - writeStartTime)) } } else { // Either we're not bypassing merge-sort or we have only in-memory data; get an iterator by From 1d5663e92cdaaa3dabfa58fdd7aede7e4fa4ec63 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 12 Feb 2015 14:47:52 -0800 Subject: [PATCH 087/817] [SPARK-5760][SPARK-5761] Fix standalone rest protocol corner cases + revamp tests The changes are summarized in the commit message. Test or test-related code accounts for 90% of the lines changed. Author: Andrew Or Closes #4557 from andrewor14/rest-tests and squashes the following commits: b4dc980 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest-tests b55e40f [Andrew Or] Add test for unknown fields cc96993 [Andrew Or] private[spark] -> private[rest] 578cf45 [Andrew Or] Clean up test code a little d82d971 [Andrew Or] v1 -> serverVersion ea48f65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest-tests 00999a8 [Andrew Or] Revamp tests + fix a few corner cases --- .../deploy/rest/StandaloneRestClient.scala | 52 +- .../deploy/rest/StandaloneRestServer.scala | 105 ++- .../rest/StandaloneRestSubmitSuite.scala | 671 +++++++++++++----- 3 files changed, 589 insertions(+), 239 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index 115aa5278bb62..c4be1f19e8e9f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -19,10 +19,11 @@ package org.apache.spark.deploy.rest import java.io.{DataOutputStream, FileNotFoundException} import java.net.{HttpURLConnection, SocketException, URL} +import javax.servlet.http.HttpServletResponse import scala.io.Source -import com.fasterxml.jackson.databind.JsonMappingException +import com.fasterxml.jackson.core.JsonProcessingException import com.google.common.base.Charsets import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} @@ -155,10 +156,21 @@ private[spark] class StandaloneRestClient extends Logging { /** * Read the response from the server and return it as a validated [[SubmitRestProtocolResponse]]. * If the response represents an error, report the embedded message to the user. + * Exposed for testing. */ - private def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = { + private[rest] def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = { try { - val responseJson = Source.fromInputStream(connection.getInputStream).mkString + val dataStream = + if (connection.getResponseCode == HttpServletResponse.SC_OK) { + connection.getInputStream + } else { + connection.getErrorStream + } + // If the server threw an exception while writing a response, it will not have a body + if (dataStream == null) { + throw new SubmitRestProtocolException("Server returned empty body") + } + val responseJson = Source.fromInputStream(dataStream).mkString logDebug(s"Response from the server:\n$responseJson") val response = SubmitRestProtocolMessage.fromJson(responseJson) response.validate() @@ -177,7 +189,7 @@ private[spark] class StandaloneRestClient extends Logging { case unreachable @ (_: FileNotFoundException | _: SocketException) => throw new SubmitRestConnectionException( s"Unable to connect to server ${connection.getURL}", unreachable) - case malformed @ (_: SubmitRestProtocolException | _: JsonMappingException) => + case malformed @ (_: JsonProcessingException | _: SubmitRestProtocolException) => throw new SubmitRestProtocolException( "Malformed response received from server", malformed) } @@ -284,7 +296,27 @@ private[spark] object StandaloneRestClient { val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" - /** Submit an application, assuming Spark parameters are specified through system properties. */ + /** + * Submit an application, assuming Spark parameters are specified through the given config. + * This is abstracted to its own method for testing purposes. + */ + private[rest] def run( + appResource: String, + mainClass: String, + appArgs: Array[String], + conf: SparkConf, + env: Map[String, String] = sys.env): SubmitRestProtocolResponse = { + val master = conf.getOption("spark.master").getOrElse { + throw new IllegalArgumentException("'spark.master' must be set.") + } + val sparkProperties = conf.getAll.toMap + val environmentVariables = env.filter { case (k, _) => k.startsWith("SPARK_") } + val client = new StandaloneRestClient + val submitRequest = client.constructSubmitRequest( + appResource, mainClass, appArgs, sparkProperties, environmentVariables) + client.createSubmission(master, submitRequest) + } + def main(args: Array[String]): Unit = { if (args.size < 2) { sys.error("Usage: StandaloneRestClient [app resource] [main class] [app args*]") @@ -294,14 +326,6 @@ private[spark] object StandaloneRestClient { val mainClass = args(1) val appArgs = args.slice(2, args.size) val conf = new SparkConf - val master = conf.getOption("spark.master").getOrElse { - throw new IllegalArgumentException("'spark.master' must be set.") - } - val sparkProperties = conf.getAll.toMap - val environmentVariables = sys.env.filter { case (k, _) => k.startsWith("SPARK_") } - val client = new StandaloneRestClient - val submitRequest = client.constructSubmitRequest( - appResource, mainClass, appArgs, sparkProperties, environmentVariables) - client.createSubmission(master, submitRequest) + run(appResource, mainClass, appArgs, conf) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index acd3a2b5abe6c..f9e0478e4f874 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.rest -import java.io.{DataOutputStream, File} +import java.io.File import java.net.InetSocketAddress import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source import akka.actor.ActorRef -import com.fasterxml.jackson.databind.JsonMappingException -import com.google.common.base.Charsets +import com.fasterxml.jackson.core.JsonProcessingException import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -70,14 +69,14 @@ private[spark] class StandaloneRestServer( import StandaloneRestServer._ private var _server: Option[Server] = None - private val baseContext = s"/$PROTOCOL_VERSION/submissions" - - // A mapping from servlets to the URL prefixes they are responsible for - private val servletToContext = Map[StandaloneRestServlet, String]( - new SubmitRequestServlet(masterActor, masterUrl, masterConf) -> s"$baseContext/create/*", - new KillRequestServlet(masterActor, masterConf) -> s"$baseContext/kill/*", - new StatusRequestServlet(masterActor, masterConf) -> s"$baseContext/status/*", - new ErrorServlet -> "/*" // default handler + + // A mapping from URL prefixes to servlets that serve them. Exposed for testing. + protected val baseContext = s"/$PROTOCOL_VERSION/submissions" + protected val contextToServlet = Map[String, StandaloneRestServlet]( + s"$baseContext/create/*" -> new SubmitRequestServlet(masterActor, masterUrl, masterConf), + s"$baseContext/kill/*" -> new KillRequestServlet(masterActor, masterConf), + s"$baseContext/status/*" -> new StatusRequestServlet(masterActor, masterConf), + "/*" -> new ErrorServlet // default handler ) /** Start the server and return the bound port. */ @@ -99,7 +98,7 @@ private[spark] class StandaloneRestServer( server.setThreadPool(threadPool) val mainHandler = new ServletContextHandler mainHandler.setContextPath("/") - servletToContext.foreach { case (servlet, prefix) => + contextToServlet.foreach { case (prefix, servlet) => mainHandler.addServlet(new ServletHolder(servlet), prefix) } server.setHandler(mainHandler) @@ -113,7 +112,7 @@ private[spark] class StandaloneRestServer( } } -private object StandaloneRestServer { +private[rest] object StandaloneRestServer { val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION val SC_UNKNOWN_PROTOCOL_VERSION = 468 } @@ -121,20 +120,7 @@ private object StandaloneRestServer { /** * An abstract servlet for handling requests passed to the [[StandaloneRestServer]]. */ -private abstract class StandaloneRestServlet extends HttpServlet with Logging { - - /** Service a request. If an exception is thrown in the process, indicate server error. */ - protected override def service( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - try { - super.service(request, response) - } catch { - case e: Exception => - logError("Exception while handling request", e) - response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) - } - } +private[rest] abstract class StandaloneRestServlet extends HttpServlet with Logging { /** * Serialize the given response message to JSON and send it through the response servlet. @@ -146,11 +132,7 @@ private abstract class StandaloneRestServlet extends HttpServlet with Logging { val message = validateResponse(responseMessage, responseServlet) responseServlet.setContentType("application/json") responseServlet.setCharacterEncoding("utf-8") - responseServlet.setStatus(HttpServletResponse.SC_OK) - val content = message.toJson.getBytes(Charsets.UTF_8) - val out = new DataOutputStream(responseServlet.getOutputStream) - out.write(content) - out.close() + responseServlet.getWriter.write(message.toJson) } /** @@ -186,6 +168,19 @@ private abstract class StandaloneRestServlet extends HttpServlet with Logging { e } + /** + * Parse a submission ID from the relative path, assuming it is the first part of the path. + * For instance, we expect the path to take the form /[submission ID]/maybe/something/else. + * The returned submission ID cannot be empty. If the path is unexpected, return None. + */ + protected def parseSubmissionId(path: String): Option[String] = { + if (path == null || path.isEmpty) { + None + } else { + path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty) + } + } + /** * Validate the response to ensure that it is correctly constructed. * @@ -209,7 +204,7 @@ private abstract class StandaloneRestServlet extends HttpServlet with Logging { /** * A servlet for handling kill requests passed to the [[StandaloneRestServer]]. */ -private class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) +private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) extends StandaloneRestServlet { /** @@ -219,18 +214,15 @@ private class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) protected override def doPost( request: HttpServletRequest, response: HttpServletResponse): Unit = { - val submissionId = request.getPathInfo.stripPrefix("/") - val responseMessage = - if (submissionId.nonEmpty) { - handleKill(submissionId) - } else { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in kill request.") - } + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleKill).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in kill request.") + } sendResponse(responseMessage, response) } - private def handleKill(submissionId: String): KillSubmissionResponse = { + protected def handleKill(submissionId: String): KillSubmissionResponse = { val askTimeout = AkkaUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout) @@ -246,7 +238,7 @@ private class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling status requests passed to the [[StandaloneRestServer]]. */ -private class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) +private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) extends StandaloneRestServlet { /** @@ -256,18 +248,15 @@ private class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) protected override def doGet( request: HttpServletRequest, response: HttpServletResponse): Unit = { - val submissionId = request.getPathInfo.stripPrefix("/") - val responseMessage = - if (submissionId.nonEmpty) { - handleStatus(submissionId) - } else { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in status request.") - } + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleStatus).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in status request.") + } sendResponse(responseMessage, response) } - private def handleStatus(submissionId: String): SubmissionStatusResponse = { + protected def handleStatus(submissionId: String): SubmissionStatusResponse = { val askTimeout = AkkaUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout) @@ -287,7 +276,7 @@ private class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling submit requests passed to the [[StandaloneRestServer]]. */ -private class SubmitRequestServlet( +private[rest] class SubmitRequestServlet( masterActor: ActorRef, masterUrl: String, conf: SparkConf) @@ -313,7 +302,7 @@ private class SubmitRequestServlet( handleSubmit(requestMessageJson, requestMessage, responseServlet) } catch { // The client failed to provide a valid JSON, so this is not our fault - case e @ (_: JsonMappingException | _: SubmitRestProtocolException) => + case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) => responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) handleError("Malformed request: " + formatException(e)) } @@ -413,7 +402,7 @@ private class ErrorServlet extends StandaloneRestServlet { request: HttpServletRequest, response: HttpServletResponse): Unit = { val path = request.getPathInfo - val parts = path.stripPrefix("/").split("/").toSeq + val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList var versionMismatch = false var msg = parts match { @@ -423,10 +412,10 @@ private class ErrorServlet extends StandaloneRestServlet { case `serverVersion` :: Nil => // http://host:port/correct-version "Missing the /submissions prefix." - case `serverVersion` :: "submissions" :: Nil => - // http://host:port/correct-version/submissions + case `serverVersion` :: "submissions" :: tail => + // http://host:port/correct-version/submissions/* "Missing an action: please specify one of /create, /kill, or /status." - case unknownVersion :: _ => + case unknownVersion :: tail => // http://host:port/unknown-version/* versionMismatch = true s"Unknown protocol version '$unknownVersion'." diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 29aed89b67aa7..a345e06ecb7d2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -17,141 +17,412 @@ package org.apache.spark.deploy.rest -import java.io.{File, FileInputStream, FileOutputStream, PrintWriter} -import java.util.jar.{JarEntry, JarOutputStream} -import java.util.zip.ZipEntry +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import javax.servlet.http.HttpServletResponse -import scala.collection.mutable.ArrayBuffer -import scala.io.Source +import scala.collection.mutable -import akka.actor.ActorSystem -import com.google.common.io.ByteStreams -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import org.scalatest.exceptions.TestFailedException +import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import com.google.common.base.Charsets +import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.json4s.JsonAST._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} -import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.deploy.worker.Worker +import org.apache.spark.deploy.master.DriverState._ /** - * End-to-end tests for the REST application submission protocol in standalone mode. + * Tests for the REST application submission protocol used in standalone cluster mode. */ -class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - private val systemsToStop = new ArrayBuffer[ActorSystem] - private val masterRestUrl = startLocalCluster() +class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { private val client = new StandaloneRestClient - private val mainJar = StandaloneRestSubmitSuite.createJar() - private val mainClass = StandaloneRestApp.getClass.getName.stripSuffix("$") + private var actorSystem: Option[ActorSystem] = None + private var server: Option[StandaloneRestServer] = None - override def afterAll() { - systemsToStop.foreach(_.shutdown()) + override def afterEach() { + actorSystem.foreach(_.shutdown()) + server.foreach(_.stop()) } - test("simple submit until completion") { - val resultsFile = File.createTempFile("test-submit", ".txt") - val numbers = Seq(1, 2, 3) - val size = 500 - val submissionId = submitApplication(resultsFile, numbers, size) - waitUntilFinished(submissionId) - validateResult(resultsFile, numbers, size) + test("construct submit request") { + val appArgs = Array("one", "two", "three") + val sparkProperties = Map("spark.app.name" -> "pi") + val environmentVariables = Map("SPARK_ONE" -> "UN", "SPARK_TWO" -> "DEUX") + val request = client.constructSubmitRequest( + "my-app-resource", "my-main-class", appArgs, sparkProperties, environmentVariables) + assert(request.action === Utils.getFormattedClassName(request)) + assert(request.clientSparkVersion === SPARK_VERSION) + assert(request.appResource === "my-app-resource") + assert(request.mainClass === "my-main-class") + assert(request.appArgs === appArgs) + assert(request.sparkProperties === sparkProperties) + assert(request.environmentVariables === environmentVariables) } - test("kill empty submission") { - val response = client.killSubmission(masterRestUrl, "submission-that-does-not-exist") - val killResponse = getKillResponse(response) - val killSuccess = killResponse.success - assert(!killSuccess) + test("create submission") { + val submittedDriverId = "my-driver-id" + val submitMessage = "your driver is submitted" + val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) + val appArgs = Array("one", "two", "four") + val request = constructSubmitRequest(masterUrl, appArgs) + assert(request.appArgs === appArgs) + assert(request.sparkProperties("spark.master") === masterUrl) + val response = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response) + assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) + assert(submitResponse.serverSparkVersion === SPARK_VERSION) + assert(submitResponse.message === submitMessage) + assert(submitResponse.submissionId === submittedDriverId) + assert(submitResponse.success) + } + + test("create submission from main method") { + val submittedDriverId = "your-driver-id" + val submitMessage = "my driver is submitted" + val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.master", masterUrl) + conf.set("spark.app.name", "dreamer") + val appArgs = Array("one", "two", "six") + // main method calls this + val response = StandaloneRestClient.run("app-resource", "main-class", appArgs, conf) + val submitResponse = getSubmitResponse(response) + assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) + assert(submitResponse.serverSparkVersion === SPARK_VERSION) + assert(submitResponse.message === submitMessage) + assert(submitResponse.submissionId === submittedDriverId) + assert(submitResponse.success) } - test("kill running submission") { - val resultsFile = File.createTempFile("test-kill", ".txt") - val numbers = Seq(1, 2, 3) - val size = 500 - val submissionId = submitApplication(resultsFile, numbers, size) - val response = client.killSubmission(masterRestUrl, submissionId) + test("kill submission") { + val submissionId = "my-lyft-driver" + val killMessage = "your driver is killed" + val masterUrl = startDummyServer(killMessage = killMessage) + val response = client.killSubmission(masterUrl, submissionId) val killResponse = getKillResponse(response) - val killSuccess = killResponse.success - waitUntilFinished(submissionId) - val response2 = client.requestSubmissionStatus(masterRestUrl, submissionId) - val statusResponse = getStatusResponse(response2) - val statusSuccess = statusResponse.success - val driverState = statusResponse.driverState - assert(killSuccess) - assert(statusSuccess) - assert(driverState === DriverState.KILLED.toString) - // we should not see the expected results because we killed the submission - intercept[TestFailedException] { validateResult(resultsFile, numbers, size) } + assert(killResponse.action === Utils.getFormattedClassName(killResponse)) + assert(killResponse.serverSparkVersion === SPARK_VERSION) + assert(killResponse.message === killMessage) + assert(killResponse.submissionId === submissionId) + assert(killResponse.success) } - test("request status for empty submission") { - val response = client.requestSubmissionStatus(masterRestUrl, "submission-that-does-not-exist") + test("request submission status") { + val submissionId = "my-uber-driver" + val submissionState = KILLED + val submissionException = new Exception("there was an irresponsible mix of alcohol and cars") + val masterUrl = startDummyServer(state = submissionState, exception = Some(submissionException)) + val response = client.requestSubmissionStatus(masterUrl, submissionId) val statusResponse = getStatusResponse(response) - val statusSuccess = statusResponse.success - assert(!statusSuccess) + assert(statusResponse.action === Utils.getFormattedClassName(statusResponse)) + assert(statusResponse.serverSparkVersion === SPARK_VERSION) + assert(statusResponse.message.contains(submissionException.getMessage)) + assert(statusResponse.submissionId === submissionId) + assert(statusResponse.driverState === submissionState.toString) + assert(statusResponse.success) + } + + test("create then kill") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response1) + assert(submitResponse.success) + assert(submitResponse.submissionId != null) + // kill submission that was just created + val submissionId = submitResponse.submissionId + val response2 = client.killSubmission(masterUrl, submissionId) + val killResponse = getKillResponse(response2) + assert(killResponse.success) + assert(killResponse.submissionId === submissionId) + } + + test("create then request status") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val submitResponse = getSubmitResponse(response1) + assert(submitResponse.success) + assert(submitResponse.submissionId != null) + // request status of submission that was just created + val submissionId = submitResponse.submissionId + val response2 = client.requestSubmissionStatus(masterUrl, submissionId) + val statusResponse = getStatusResponse(response2) + assert(statusResponse.success) + assert(statusResponse.submissionId === submissionId) + assert(statusResponse.driverState === RUNNING.toString) + } + + test("create then kill then request status") { + val masterUrl = startSmartServer() + val request = constructSubmitRequest(masterUrl) + val response1 = client.createSubmission(masterUrl, request) + val response2 = client.createSubmission(masterUrl, request) + val submitResponse1 = getSubmitResponse(response1) + val submitResponse2 = getSubmitResponse(response2) + assert(submitResponse1.success) + assert(submitResponse2.success) + assert(submitResponse1.submissionId != null) + assert(submitResponse2.submissionId != null) + val submissionId1 = submitResponse1.submissionId + val submissionId2 = submitResponse2.submissionId + // kill only submission 1, but not submission 2 + val response3 = client.killSubmission(masterUrl, submissionId1) + val killResponse = getKillResponse(response3) + assert(killResponse.success) + assert(killResponse.submissionId === submissionId1) + // request status for both submissions: 1 should be KILLED but 2 should be RUNNING still + val response4 = client.requestSubmissionStatus(masterUrl, submissionId1) + val response5 = client.requestSubmissionStatus(masterUrl, submissionId2) + val statusResponse1 = getStatusResponse(response4) + val statusResponse2 = getStatusResponse(response5) + assert(statusResponse1.submissionId === submissionId1) + assert(statusResponse2.submissionId === submissionId2) + assert(statusResponse1.driverState === KILLED.toString) + assert(statusResponse2.driverState === RUNNING.toString) + } + + test("kill or request status before create") { + val masterUrl = startSmartServer() + val doesNotExist = "does-not-exist" + // kill a non-existent submission + val response1 = client.killSubmission(masterUrl, doesNotExist) + val killResponse = getKillResponse(response1) + assert(!killResponse.success) + assert(killResponse.submissionId === doesNotExist) + // request status for a non-existent submission + val response2 = client.requestSubmissionStatus(masterUrl, doesNotExist) + val statusResponse = getStatusResponse(response2) + assert(!statusResponse.success) + assert(statusResponse.submissionId === doesNotExist) + } + + /* ---------------------------------------- * + | Aberrant client / server behavior | + * ---------------------------------------- */ + + test("good request paths") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val json = constructSubmitRequest(masterUrl).toJson + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill" + val statusRequestPath = s"$httpUrl/$v/submissions/status" + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", json) + val (response2, code2) = sendHttpRequestWithResponse(s"$killRequestPath/anything", "POST") + val (response3, code3) = sendHttpRequestWithResponse(s"$killRequestPath/any/thing", "POST") + val (response4, code4) = sendHttpRequestWithResponse(s"$statusRequestPath/anything", "GET") + val (response5, code5) = sendHttpRequestWithResponse(s"$statusRequestPath/any/thing", "GET") + // these should all succeed and the responses should be of the correct types + getSubmitResponse(response1) + val killResponse1 = getKillResponse(response2) + val killResponse2 = getKillResponse(response3) + val statusResponse1 = getStatusResponse(response4) + val statusResponse2 = getStatusResponse(response5) + assert(killResponse1.submissionId === "anything") + assert(killResponse2.submissionId === "any") + assert(statusResponse1.submissionId === "anything") + assert(statusResponse2.submissionId === "any") + assert(code1 === HttpServletResponse.SC_OK) + assert(code2 === HttpServletResponse.SC_OK) + assert(code3 === HttpServletResponse.SC_OK) + assert(code4 === HttpServletResponse.SC_OK) + assert(code5 === HttpServletResponse.SC_OK) + } + + test("good request paths, bad requests") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill" + val statusRequestPath = s"$httpUrl/$v/submissions/status" + val goodJson = constructSubmitRequest(masterUrl).toJson + val badJson1 = goodJson.replaceAll("action", "fraction") // invalid JSON + val badJson2 = goodJson.substring(goodJson.size / 2) // malformed JSON + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST") // missing JSON + val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson1) + val (response3, code3) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson2) + val (response4, code4) = sendHttpRequestWithResponse(killRequestPath, "POST") // missing ID + val (response5, code5) = sendHttpRequestWithResponse(s"$killRequestPath/", "POST") + val (response6, code6) = sendHttpRequestWithResponse(statusRequestPath, "GET") // missing ID + val (response7, code7) = sendHttpRequestWithResponse(s"$statusRequestPath/", "GET") + // these should all fail as error responses + getErrorResponse(response1) + getErrorResponse(response2) + getErrorResponse(response3) + getErrorResponse(response4) + getErrorResponse(response5) + getErrorResponse(response6) + getErrorResponse(response7) + assert(code1 === HttpServletResponse.SC_BAD_REQUEST) + assert(code2 === HttpServletResponse.SC_BAD_REQUEST) + assert(code3 === HttpServletResponse.SC_BAD_REQUEST) + assert(code4 === HttpServletResponse.SC_BAD_REQUEST) + assert(code5 === HttpServletResponse.SC_BAD_REQUEST) + assert(code6 === HttpServletResponse.SC_BAD_REQUEST) + assert(code7 === HttpServletResponse.SC_BAD_REQUEST) + } + + test("bad request paths") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET") + val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET") + val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET") + val (response4, code4) = sendHttpRequestWithResponse(s"$httpUrl/$v/", "GET") + val (response5, code5) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions", "GET") + val (response6, code6) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/", "GET") + val (response7, code7) = sendHttpRequestWithResponse(s"$httpUrl/$v/submissions/bad", "GET") + val (response8, code8) = sendHttpRequestWithResponse(s"$httpUrl/bad-version", "GET") + assert(code1 === HttpServletResponse.SC_BAD_REQUEST) + assert(code2 === HttpServletResponse.SC_BAD_REQUEST) + assert(code3 === HttpServletResponse.SC_BAD_REQUEST) + assert(code4 === HttpServletResponse.SC_BAD_REQUEST) + assert(code5 === HttpServletResponse.SC_BAD_REQUEST) + assert(code6 === HttpServletResponse.SC_BAD_REQUEST) + assert(code7 === HttpServletResponse.SC_BAD_REQUEST) + assert(code8 === StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) + // all responses should be error responses + val errorResponse1 = getErrorResponse(response1) + val errorResponse2 = getErrorResponse(response2) + val errorResponse3 = getErrorResponse(response3) + val errorResponse4 = getErrorResponse(response4) + val errorResponse5 = getErrorResponse(response5) + val errorResponse6 = getErrorResponse(response6) + val errorResponse7 = getErrorResponse(response7) + val errorResponse8 = getErrorResponse(response8) + // only the incompatible version response should have server protocol version set + assert(errorResponse1.highestProtocolVersion === null) + assert(errorResponse2.highestProtocolVersion === null) + assert(errorResponse3.highestProtocolVersion === null) + assert(errorResponse4.highestProtocolVersion === null) + assert(errorResponse5.highestProtocolVersion === null) + assert(errorResponse6.highestProtocolVersion === null) + assert(errorResponse7.highestProtocolVersion === null) + assert(errorResponse8.highestProtocolVersion === StandaloneRestServer.PROTOCOL_VERSION) + } + + test("server returns unknown fields") { + val masterUrl = startSmartServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val oldJson = constructSubmitRequest(masterUrl).toJson + val oldFields = parse(oldJson).asInstanceOf[JObject].obj + val newFields = oldFields ++ Seq( + JField("tomato", JString("not-a-fruit")), + JField("potato", JString("not-po-tah-to")) + ) + val newJson = pretty(render(JObject(newFields))) + // send two requests, one with the unknown fields and the other without + val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST", oldJson) + val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", newJson) + val submitResponse1 = getSubmitResponse(response1) + val submitResponse2 = getSubmitResponse(response2) + assert(code1 === HttpServletResponse.SC_OK) + assert(code2 === HttpServletResponse.SC_OK) + // only the response to the modified request should have unknown fields set + assert(submitResponse1.unknownFields === null) + assert(submitResponse2.unknownFields === Array("tomato", "potato")) + } + + test("client handles faulty server") { + val masterUrl = startFaultyServer() + val httpUrl = masterUrl.replace("spark://", "http://") + val v = StandaloneRestServer.PROTOCOL_VERSION + val submitRequestPath = s"$httpUrl/$v/submissions/create" + val killRequestPath = s"$httpUrl/$v/submissions/kill/anything" + val statusRequestPath = s"$httpUrl/$v/submissions/status/anything" + val json = constructSubmitRequest(masterUrl).toJson + // server returns malformed response unwittingly + // client should throw an appropriate exception to indicate server failure + val conn1 = sendHttpRequest(submitRequestPath, "POST", json) + intercept[SubmitRestProtocolException] { client.readResponse(conn1) } + // server attempts to send invalid response, but fails internally on validation + // client should receive an error response as server is able to recover + val conn2 = sendHttpRequest(killRequestPath, "POST") + val response2 = client.readResponse(conn2) + getErrorResponse(response2) + assert(conn2.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + // server explodes internally beyond recovery + // client should throw an appropriate exception to indicate server failure + val conn3 = sendHttpRequest(statusRequestPath, "GET") + intercept[SubmitRestProtocolException] { client.readResponse(conn3) } // empty response + assert(conn3.getResponseCode === HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + } + + /* --------------------- * + | Helper methods | + * --------------------- */ + + /** Start a dummy server that responds to requests using the specified parameters. */ + private def startDummyServer( + submitId: String = "fake-driver-id", + submitMessage: String = "driver is submitted", + killMessage: String = "driver is killed", + state: DriverState = FINISHED, + exception: Option[Exception] = None): String = { + startServer(new DummyMaster(submitId, submitMessage, killMessage, state, exception)) + } + + /** Start a smarter dummy server that keeps track of submitted driver states. */ + private def startSmartServer(): String = { + startServer(new SmarterMaster) + } + + /** Start a dummy server that is faulty in many ways... */ + private def startFaultyServer(): String = { + startServer(new DummyMaster, faulty = true) } /** - * Start a local cluster containing one Master and a few Workers. - * Do not use [[org.apache.spark.deploy.LocalSparkCluster]] here because we want the REST URL. - * Return the Master's REST URL to which applications should be submitted. + * Start a [[StandaloneRestServer]] that communicates with the given actor. + * If `faulty` is true, start an [[FaultyStandaloneRestServer]] instead. + * Return the master URL that corresponds to the address of this server. */ - private def startLocalCluster(): String = { - val conf = new SparkConf(false) - .set("spark.master.rest.enabled", "true") - .set("spark.master.rest.port", "0") - val (numWorkers, coresPerWorker, memPerWorker) = (2, 1, 512) - val localHostName = Utils.localHostName() - val (masterSystem, masterPort, _, _masterRestPort) = - Master.startSystemAndActor(localHostName, 0, 0, conf) - val masterRestPort = _masterRestPort.getOrElse { fail("REST server not started on Master!") } - val masterUrl = "spark://" + localHostName + ":" + masterPort - val masterRestUrl = "spark://" + localHostName + ":" + masterRestPort - (1 to numWorkers).foreach { n => - val (workerSystem, _) = Worker.startSystemAndActor( - localHostName, 0, 0, coresPerWorker, memPerWorker, Array(masterUrl), null, Some(n)) - systemsToStop.append(workerSystem) - } - systemsToStop.append(masterSystem) - masterRestUrl + private def startServer(makeFakeMaster: => Actor, faulty: Boolean = false): String = { + val name = "test-standalone-rest-protocol" + val conf = new SparkConf + val localhost = Utils.localHostName() + val securityManager = new SecurityManager(conf) + val (_actorSystem, _) = AkkaUtils.createActorSystem(name, localhost, 0, conf, securityManager) + val fakeMasterRef = _actorSystem.actorOf(Props(makeFakeMaster)) + val _server = + if (faulty) { + new FaultyStandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + } else { + new StandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + } + val port = _server.start() + // set these to clean them up after every test + actorSystem = Some(_actorSystem) + server = Some(_server) + s"spark://$localhost:$port" } - /** Submit the [[StandaloneRestApp]] and return the corresponding submission ID. */ - private def submitApplication(resultsFile: File, numbers: Seq[Int], size: Int): String = { - val appArgs = Seq(resultsFile.getAbsolutePath) ++ numbers.map(_.toString) ++ Seq(size.toString) + /** Create a submit request with real parameters using Spark submit. */ + private def constructSubmitRequest( + masterUrl: String, + appArgs: Array[String] = Array.empty): CreateSubmissionRequest = { + val mainClass = "main-class-not-used" + val mainJar = "dummy-jar-not-used.jar" val commandLineArgs = Array( "--deploy-mode", "cluster", - "--master", masterRestUrl, + "--master", masterUrl, "--name", mainClass, "--class", mainClass, mainJar) ++ appArgs val args = new SparkSubmitArguments(commandLineArgs) val (_, _, sparkProperties, _) = SparkSubmit.prepareSubmitEnvironment(args) - val request = client.constructSubmitRequest( - mainJar, mainClass, appArgs.toArray, sparkProperties.toMap, Map.empty) - val response = client.createSubmission(masterRestUrl, request) - val submitResponse = getSubmitResponse(response) - val submissionId = submitResponse.submissionId - assert(submissionId != null, "Application submission was unsuccessful!") - submissionId - } - - /** Wait until the given submission has finished running up to the specified timeout. */ - private def waitUntilFinished(submissionId: String, maxSeconds: Int = 30): Unit = { - var finished = false - val expireTime = System.currentTimeMillis + maxSeconds * 1000 - while (!finished) { - val response = client.requestSubmissionStatus(masterRestUrl, submissionId) - val statusResponse = getStatusResponse(response) - val driverState = statusResponse.driverState - finished = - driverState != DriverState.SUBMITTED.toString && - driverState != DriverState.RUNNING.toString - if (System.currentTimeMillis > expireTime) { - fail(s"Driver $submissionId did not finish within $maxSeconds seconds.") - } - } + client.constructSubmitRequest( + mainJar, mainClass, appArgs, sparkProperties.toMap, Map.empty) } /** Return the response as a submit response, or fail with error otherwise. */ @@ -181,85 +452,151 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterAll with Bef } } - /** Validate whether the application produced the corrupt output. */ - private def validateResult(resultsFile: File, numbers: Seq[Int], size: Int): Unit = { - val lines = Source.fromFile(resultsFile.getAbsolutePath).getLines().toSeq - val unexpectedContent = - if (lines.nonEmpty) { - "[\n" + lines.map { l => " " + l }.mkString("\n") + "\n]" - } else { - "[EMPTY]" - } - assert(lines.size === 2, s"Unexpected content in file: $unexpectedContent") - assert(lines(0).toInt === numbers.sum, s"Sum of ${numbers.mkString(",")} is incorrect") - assert(lines(1).toInt === (size / 2) + 1, "Result of Spark job is incorrect") + /** Return the response as an error response, or fail if the response was not an error. */ + private def getErrorResponse(response: SubmitRestProtocolResponse): ErrorResponse = { + response match { + case e: ErrorResponse => e + case r => fail(s"Expected error response. Actual: ${r.toJson}") + } } -} - -private object StandaloneRestSubmitSuite { - private val pathPrefix = this.getClass.getPackage.getName.replaceAll("\\.", "/") /** - * Create a jar that contains all the class files needed for running the [[StandaloneRestApp]]. - * Return the absolute path to that jar. + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. */ - def createJar(): String = { - val jarFile = File.createTempFile("test-standalone-rest-protocol", ".jar") - val jarFileStream = new FileOutputStream(jarFile) - val jarStream = new JarOutputStream(jarFileStream, new java.util.jar.Manifest) - jarStream.putNextEntry(new ZipEntry(pathPrefix)) - getClassFiles.foreach { cf => - jarStream.putNextEntry(new JarEntry(pathPrefix + "/" + cf.getName)) - val in = new FileInputStream(cf) - ByteStreams.copy(in, jarStream) - in.close() + private def sendHttpRequest( + url: String, + method: String, + body: String = ""): HttpURLConnection = { + val conn = new URL(url).openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod(method) + if (body.nonEmpty) { + conn.setDoOutput(true) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(Charsets.UTF_8)) + out.close() } - jarStream.close() - jarFileStream.close() - jarFile.getAbsolutePath + conn } /** - * Return a list of class files compiled for [[StandaloneRestApp]]. - * This includes all the anonymous classes used in the application. + * Send an HTTP request to the given URL using the method and the body specified. + * Return a 2-tuple of the response message from the server and the response code. */ - private def getClassFiles: Seq[File] = { - val className = Utils.getFormattedClassName(StandaloneRestApp) - val clazz = StandaloneRestApp.getClass - val basePath = clazz.getProtectionDomain.getCodeSource.getLocation.toURI.getPath - val baseDir = new File(basePath + "/" + pathPrefix) - baseDir.listFiles().filter(_.getName.contains(className)) + private def sendHttpRequestWithResponse( + url: String, + method: String, + body: String = ""): (SubmitRestProtocolResponse, Int) = { + val conn = sendHttpRequest(url, method, body) + (client.readResponse(conn), conn.getResponseCode) } } /** - * Sample application to be submitted to the cluster using the REST gateway. - * All relevant classes will be packaged into a jar at run time. + * A mock standalone Master that responds with dummy messages. + * In all responses, the success parameter is always true. */ -object StandaloneRestApp { - // Usage: [path to results file] [num1] [num2] [num3] [rddSize] - // The first line of the results file should be (num1 + num2 + num3) - // The second line should be (rddSize / 2) + 1 - def main(args: Array[String]) { - assert(args.size == 5, s"Expected exactly 5 arguments: ${args.mkString(",")}") - val resultFile = new File(args(0)) - val writer = new PrintWriter(resultFile) - try { - val conf = new SparkConf() - val sc = new SparkContext(conf) - val firstLine = args(1).toInt + args(2).toInt + args(3).toInt - val secondLine = sc.parallelize(1 to args(4).toInt) - .map { i => (i / 2, i) } - .reduceByKey(_ + _) - .count() - writer.println(firstLine) - writer.println(secondLine) - } catch { - case e: Exception => - writer.println(e) - e.getStackTrace.foreach { l => writer.println(" " + l) } - } finally { - writer.close() +private class DummyMaster( + submitId: String = "fake-driver-id", + submitMessage: String = "submitted", + killMessage: String = "killed", + state: DriverState = FINISHED, + exception: Option[Exception] = None) + extends Actor { + + override def receive = { + case RequestSubmitDriver(driverDesc) => + sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage) + case RequestKillDriver(driverId) => + sender ! KillDriverResponse(driverId, success = true, killMessage) + case RequestDriverStatus(driverId) => + sender ! DriverStatusResponse(found = true, Some(state), None, None, exception) + } +} + +/** + * A mock standalone Master that keeps track of drivers that have been submitted. + * + * If a driver is submitted, its state is immediately set to RUNNING. + * If an existing driver is killed, its state is immediately set to KILLED. + * If an existing driver's status is requested, its state is returned in the response. + * Submits are always successful while kills and status requests are successful only + * if the driver was submitted in the past. + */ +private class SmarterMaster extends Actor { + private var counter: Int = 0 + private val submittedDrivers = new mutable.HashMap[String, DriverState] + + override def receive = { + case RequestSubmitDriver(driverDesc) => + val driverId = s"driver-$counter" + submittedDrivers(driverId) = RUNNING + counter += 1 + sender ! SubmitDriverResponse(success = true, Some(driverId), "submitted") + + case RequestKillDriver(driverId) => + val success = submittedDrivers.contains(driverId) + if (success) { + submittedDrivers(driverId) = KILLED + } + sender ! KillDriverResponse(driverId, success, "killed") + + case RequestDriverStatus(driverId) => + val found = submittedDrivers.contains(driverId) + val state = submittedDrivers.get(driverId) + sender ! DriverStatusResponse(found, state, None, None, None) + } +} + +/** + * A [[StandaloneRestServer]] that is faulty in many ways. + * + * When handling a submit request, the server returns a malformed JSON. + * When handling a kill request, the server returns an invalid JSON. + * When handling a status request, the server throws an internal exception. + * The purpose of this class is to test that client handles these cases gracefully. + */ +private class FaultyStandaloneRestServer( + host: String, + requestedPort: Int, + masterActor: ActorRef, + masterUrl: String, + masterConf: SparkConf) + extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) { + + protected override val contextToServlet = Map[String, StandaloneRestServlet]( + s"$baseContext/create/*" -> new MalformedSubmitServlet, + s"$baseContext/kill/*" -> new InvalidKillServlet, + s"$baseContext/status/*" -> new ExplodingStatusServlet, + "/*" -> new ErrorServlet + ) + + /** A faulty servlet that produces malformed responses. */ + class MalformedSubmitServlet extends SubmitRequestServlet(masterActor, masterUrl, masterConf) { + protected override def sendResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): Unit = { + val badJson = responseMessage.toJson.drop(10).dropRight(20) + responseServlet.getWriter.write(badJson) + } + } + + /** A faulty servlet that produces invalid responses. */ + class InvalidKillServlet extends KillRequestServlet(masterActor, masterConf) { + protected override def handleKill(submissionId: String): KillSubmissionResponse = { + val k = super.handleKill(submissionId) + k.submissionId = null + k + } + } + + /** A faulty status servlet that explodes. */ + class ExplodingStatusServlet extends StatusRequestServlet(masterActor, masterConf) { + private def explode: Int = 1 / 0 + protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { + val s = super.handleStatus(submissionId) + s.workerId = explode.toString + s } } } From 947b8bd82ec0f4c45910e6d781df4661f56e4587 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Thu, 12 Feb 2015 14:50:16 -0800 Subject: [PATCH 088/817] [SPARK-5759][Yarn]ExecutorRunnable should catch YarnException while NMClient start contain... some time since some reasons, it lead to some exception while NMClient start some containers.example:we do not config spark_shuffle on some machines, so it will throw a exception: java.lang.Error: org.apache.hadoop.yarn.exceptions.InvalidAuxServiceException: The auxService:spark_shuffle does not exist. because YarnAllocator use ThreadPoolExecutor to start Container, so we can not find which container or hostname throw exception. I think we should catch YarnException in ExecutorRunnable when start container. if there are some exceptions, we can know the container id or hostname of failed container. Author: lianhuiwang Closes #4554 from lianhuiwang/SPARK-5759 and squashes the following commits: caf5a99 [lianhuiwang] use SparkException to warp exception c02140f [lianhuiwang] ExecutorRunnable should catch YarnException while NMClient start container --- .../apache/spark/deploy/yarn/ExecutorRunnable.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 6d5b8fda76ab8..c1d3f7320f53c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.network.util.JavaUtils class ExecutorRunnable( @@ -109,7 +109,13 @@ class ExecutorRunnable( } // Send the start request to the ContainerManager - nmClient.startContainer(container, ctx) + try { + nmClient.startContainer(container, ctx) + } catch { + case ex: Exception => + throw new SparkException(s"Exception while starting container ${container.getId}" + + s" on host $hostname", ex) + } } private def prepareCommand( From 26c816e7388eaa336a59183029f86548f1cc279c Mon Sep 17 00:00:00 2001 From: "David Y. Ross" Date: Thu, 12 Feb 2015 14:52:38 -0800 Subject: [PATCH 089/817] SPARK-5747: Fix wordsplitting bugs in make-distribution.sh The `$MVN` command variable may have spaces, so when referring to it, must wrap in quotes. Author: David Y. Ross Closes #4540 from dyross/dyr-fix-make-distribution2 and squashes the following commits: 5a41596 [David Y. Ross] SPARK-5747: Fix wordsplitting bugs in make-distribution.sh --- make-distribution.sh | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 051c87c0894ae..dd990d4b96e46 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -98,9 +98,9 @@ done if [ -z "$JAVA_HOME" ]; then # Fall back on JAVA_HOME from rpm, if found if [ $(command -v rpm) ]; then - RPM_JAVA_HOME=$(rpm -E %java_home 2>/dev/null) + RPM_JAVA_HOME="$(rpm -E %java_home 2>/dev/null)" if [ "$RPM_JAVA_HOME" != "%java_home" ]; then - JAVA_HOME=$RPM_JAVA_HOME + JAVA_HOME="$RPM_JAVA_HOME" echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm" fi fi @@ -113,24 +113,24 @@ fi if [ $(command -v git) ]; then GITREV=$(git rev-parse --short HEAD 2>/dev/null || :) - if [ ! -z $GITREV ]; then + if [ ! -z "$GITREV" ]; then GITREVSTRING=" (git revision $GITREV)" fi unset GITREV fi -if [ ! $(command -v $MVN) ] ; then +if [ ! $(command -v "$MVN") ] ; then echo -e "Could not locate Maven command: '$MVN'." echo -e "Specify the Maven command with the --mvn flag" exit -1; fi -VERSION=$($MVN help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) -SPARK_HADOOP_VERSION=$($MVN help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ +VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) -SPARK_HIVE=$($MVN help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\ +SPARK_HIVE=$("$MVN" help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\ | grep -v "INFO"\ | fgrep --count "hive";\ # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\ @@ -147,7 +147,7 @@ if [[ ! "$JAVA_VERSION" =~ "1.6" && -z "$SKIP_JAVA_TEST" ]]; then echo "Output from 'java -version' was:" echo "$JAVA_VERSION" read -p "Would you like to continue anyways? [y,n]: " -r - if [[ ! $REPLY =~ ^[Yy]$ ]]; then + if [[ ! "$REPLY" =~ ^[Yy]$ ]]; then echo "Okay, exiting." exit 1 fi @@ -232,7 +232,7 @@ cp -r "$SPARK_HOME/ec2" "$DISTDIR" if [ "$SPARK_TACHYON" == "true" ]; then TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` - pushd $TMPD > /dev/null + pushd "$TMPD" > /dev/null echo "Fetching tachyon tgz" TACHYON_DL="${TACHYON_TGZ}.part" @@ -259,7 +259,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then fi popd > /dev/null - rm -rf $TMPD + rm -rf "$TMPD" fi if [ "$MAKE_TGZ" == "true" ]; then From 0bf031582588723dd5a4ca42e6f9f36bc2da1a0b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 12 Feb 2015 14:54:38 -0800 Subject: [PATCH 090/817] [SPARK-5780] [PySpark] Mute the logging during unit tests There a bunch of logging coming from driver and worker, it's noisy and scaring, and a lots of exception in it, people are confusing about the tests are failing or not. This PR will mute the logging during tests, only show them if any one failed. Author: Davies Liu Closes #4572 from davies/mute and squashes the following commits: 1e9069c [Davies Liu] mute the logging during python tests --- python/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/run-tests b/python/run-tests index 58a26dd8ff088..077ad60d764a3 100755 --- a/python/run-tests +++ b/python/run-tests @@ -35,7 +35,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" | tee -a $LOG_FILE - SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 2>&1 | tee -a $LOG_FILE + SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 >> $LOG_FILE 2>&1 FAILED=$((PIPESTATUS[0]||$FAILED)) From c352ffbdb9112714c176a747edff6115e9369e58 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 12 Feb 2015 15:17:25 -0800 Subject: [PATCH 091/817] [SPARK-5758][SQL] Use LongType as the default type for integers in JSON schema inference. Author: Yin Huai Closes #4544 from yhuai/jsonUseLongTypeByDefault and squashes the following commits: 6e2ffc2 [Yin Huai] Use LongType as the default type for integers in JSON schema inference. --- .../org/apache/spark/sql/json/JsonRDD.scala | 12 ++++++++---- .../spark/sql/api/java/JavaApplySchemaSuite.java | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 16 ++++++++-------- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 1043eefcfc6a5..7dfb30402136e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.json -import java.io.StringWriter -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} -import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException, JsonFactory} +import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException} import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD @@ -178,7 +177,12 @@ private[sql] object JsonRDD extends Logging { } private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = { - ScalaReflection.typeOfObject orElse { + // For Integer values, use LongType by default. + val useLongType: PartialFunction[Any, DataType] = { + case value: IntegerType.JvmType => LongType + } + + useLongType orElse ScalaReflection.typeOfObject orElse { // Since we do not have a data type backed by BigInteger, // when we see a Java BigInteger, we use DecimalType. case value: java.math.BigInteger => DecimalType.Unlimited diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 2e6e977fdc752..643b891ab1b63 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -164,7 +164,7 @@ public void applySchemaToJSON() { fields.add(DataTypes.createStructField("bigInteger", DataTypes.createDecimalType(), true)); fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, true)); fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, true)); - fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, true)); + fields.add(DataTypes.createStructField("integer", DataTypes.LongType, true)); fields.add(DataTypes.createStructField("long", DataTypes.LongType, true)); fields.add(DataTypes.createStructField("null", DataTypes.StringType, true)); fields.add(DataTypes.createStructField("string", DataTypes.StringType, true)); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index fde4b47438c56..b5f13f8bd5e80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -222,7 +222,7 @@ class JsonSuite extends QueryTest { StructField("bigInteger", DecimalType.Unlimited, true) :: StructField("boolean", BooleanType, true) :: StructField("double", DoubleType, true) :: - StructField("integer", IntegerType, true) :: + StructField("integer", LongType, true) :: StructField("long", LongType, true) :: StructField("null", StringType, true) :: StructField("string", StringType, true) :: Nil) @@ -252,7 +252,7 @@ class JsonSuite extends QueryTest { StructField("arrayOfBigInteger", ArrayType(DecimalType.Unlimited, false), true) :: StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: - StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: + StructField("arrayOfInteger", ArrayType(LongType, false), true) :: StructField("arrayOfLong", ArrayType(LongType, false), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: StructField("arrayOfString", ArrayType(StringType, false), true) :: @@ -265,7 +265,7 @@ class JsonSuite extends QueryTest { StructField("field1", BooleanType, true) :: StructField("field2", DecimalType.Unlimited, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( - StructField("field1", ArrayType(IntegerType, false), true) :: + StructField("field1", ArrayType(LongType, false), true) :: StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonDF.schema) @@ -486,7 +486,7 @@ class JsonSuite extends QueryTest { val jsonDF = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(IntegerType, false), true) :: + StructField("array", ArrayType(LongType, false), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -540,7 +540,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(IntegerType, false), true) :: + StructField("c", ArrayType(LongType, false), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: Nil) @@ -560,7 +560,7 @@ class JsonSuite extends QueryTest { StructField("bigInteger", DecimalType.Unlimited, true) :: StructField("boolean", BooleanType, true) :: StructField("double", DoubleType, true) :: - StructField("integer", IntegerType, true) :: + StructField("integer", LongType, true) :: StructField("long", LongType, true) :: StructField("null", StringType, true) :: StructField("string", StringType, true) :: Nil) @@ -781,12 +781,12 @@ class JsonSuite extends QueryTest { ArrayType(ArrayType(ArrayType(ArrayType(StringType, false), false), true), false), true) :: StructField("field2", ArrayType(ArrayType( - StructType(StructField("Test", IntegerType, true) :: Nil), false), true), true) :: + StructType(StructField("Test", LongType, true) :: Nil), false), true), true) :: StructField("field3", ArrayType(ArrayType( StructType(StructField("Test", StringType, true) :: Nil), true), false), true) :: StructField("field4", - ArrayType(ArrayType(ArrayType(IntegerType, false), true), false), true) :: Nil) + ArrayType(ArrayType(ArrayType(LongType, false), true), false), true) :: Nil) assert(schema === jsonDF.schema) From ee04a8b19be8330bfc48f470ef365622162c915f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 12 Feb 2015 15:19:19 -0800 Subject: [PATCH 092/817] [SPARK-5573][SQL] Add explode to dataframes Author: Michael Armbrust Closes #4546 from marmbrus/explode and squashes the following commits: eefd33a [Michael Armbrust] whitespace a8d496c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into explode 4af740e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explode dc86a5c [Michael Armbrust] simple version d633d01 [Michael Armbrust] add scala specific 950707a [Michael Armbrust] fix comments ba8854c [Michael Armbrust] [SPARK-5573][SQL] Add explode to dataframes --- .../sql/catalyst/expressions/generators.scala | 19 ++++++++++ .../org/apache/spark/sql/DataFrame.scala | 38 +++++++++++++++++++ .../org/apache/spark/sql/DataFrameImpl.scala | 30 ++++++++++++++- .../apache/spark/sql/IncomputableColumn.scala | 9 +++++ .../org/apache/spark/sql/DataFrameSuite.scala | 25 ++++++++++++ 5 files changed, 119 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 43b6482c0171c..0983d274def3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -73,6 +73,25 @@ abstract class Generator extends Expression { } } +/** + * A generator that produces its output using the provided lambda function. + */ +case class UserDefinedGenerator( + schema: Seq[Attribute], + function: Row => TraversableOnce[Row], + children: Seq[Expression]) + extends Generator{ + + override protected def makeOutput(): Seq[Attribute] = schema + + override def eval(input: Row): TraversableOnce[Row] = { + val inputRow = new InterpretedProjection(children) + function(inputRow(input)) + } + + override def toString = s"UserDefinedGenerator(${children.mkString(",")})" +} + /** * Given an input array produces a sequence of rows for each value in the array. */ 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 13aff760e9a5c..65257882f4e7c 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.annotation.{DeveloperApi, Experimental} @@ -441,6 +442,43 @@ trait DataFrame extends RDDApi[Row] with Serializable { sample(withReplacement, fraction, Utils.random.nextLong) } + /** + * (Scala-specific) Returns a new [[DataFrame]] where each row has been expanded to zero or more + * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of + * the input row are implicitly joined with each row that is output by the function. + * + * The following example uses this function to count the number of books which contain + * a given word: + * + * {{{ + * case class Book(title: String, words: String) + * val df: RDD[Book] + * + * case class Word(word: String) + * val allWords = df.explode('words) { + * case Row(words: String) => words.split(" ").map(Word(_)) + * } + * + * val bookCountPerWord = allWords.groupBy("word").agg(countDistinct("title")) + * }}} + */ + def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame + + + /** + * (Scala-specific) Returns a new [[DataFrame]] where a single column has been expanded to zero + * or more rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. All + * columns of the input row are implicitly joined with each value that is output by the function. + * + * {{{ + * df.explode("words", "word")(words: String => words.split(" ")) + * }}} + */ + def explode[A, B : TypeTag]( + inputColumn: String, + outputColumn: String)( + f: A => TraversableOnce[B]): DataFrame + ///////////////////////////////////////////////////////////////////////////// /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 4c6e19cace8ca..bb5c6226a2217 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -21,6 +21,7 @@ import java.io.CharArrayWriter import scala.language.implicitConversions import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import scala.collection.JavaConversions._ import com.fasterxml.jackson.core.JsonFactory @@ -29,7 +30,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{SqlParser, ScalaReflection} +import org.apache.spark.sql.catalyst.{expressions, SqlParser, ScalaReflection} import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} @@ -39,7 +40,6 @@ import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{NumericType, StructType} - /** * Internal implementation of [[DataFrame]]. Users of the API should use [[DataFrame]] directly. */ @@ -282,6 +282,32 @@ private[sql] class DataFrameImpl protected[sql]( Sample(fraction, withReplacement, seed, logicalPlan) } + override def explode[A <: Product : TypeTag] + (input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { + val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + val attributes = schema.toAttributes + val rowFunction = + f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) + val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) + + Generate(generator, join = true, outer = false, None, logicalPlan) + } + + override def explode[A, B : TypeTag]( + inputColumn: String, + outputColumn: String)( + f: A => TraversableOnce[B]): DataFrame = { + val dataType = ScalaReflection.schemaFor[B].dataType + val attributes = AttributeReference(outputColumn, dataType)() :: Nil + def rowFunction(row: Row) = { + f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) + } + val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) + + Generate(generator, join = true, outer = false, None, logicalPlan) + + } + ///////////////////////////////////////////////////////////////////////////// // RDD API ///////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 4f9d92d97646f..19c8e3b4f4708 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD @@ -110,6 +111,14 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = err() + override def explode[A <: Product : TypeTag] + (input: Column*)(f: Row => TraversableOnce[A]): DataFrame = err() + + override def explode[A, B : TypeTag]( + inputColumn: String, + outputColumn: String)( + f: A => TraversableOnce[B]): DataFrame = err() + ///////////////////////////////////////////////////////////////////////////// override def head(n: Int): Array[Row] = err() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 7be9215a443f0..33b35f376b270 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -98,6 +98,31 @@ class DataFrameSuite extends QueryTest { sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) } + test("simple explode") { + val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDataFrame("words") + + checkAnswer( + df.explode("words", "word") { word: String => word.split(" ").toSeq }.select('word), + Row("a") :: Row("b") :: Row("c") :: Row("d") ::Row("e") :: Nil + ) + } + + test("explode") { + val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDataFrame("number", "letters") + val df2 = + df.explode('letters) { + case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq + } + + checkAnswer( + df2 + .select('_1 as 'letter, 'number) + .groupBy('letter) + .agg('letter, countDistinct('number)), + Row("a", 3) :: Row("b", 2) :: Row("c", 1) :: Nil + ) + } + test("selectExpr") { checkAnswer( testData.selectExpr("abs(key)", "value"), From d5fc51491808630d0328a5937dbf349e00de361f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 12 Feb 2015 15:22:07 -0800 Subject: [PATCH 093/817] [SPARK-5755] [SQL] remove unnecessary Add explain extended select +key from src; before: == Parsed Logical Plan == 'Project [(0 + 'key) AS _c0#8] 'UnresolvedRelation [src], None == Analyzed Logical Plan == Project [(0 + key#10) AS _c0#8] MetastoreRelation test, src, None == Optimized Logical Plan == Project [(0 + key#10) AS _c0#8] MetastoreRelation test, src, None == Physical Plan == Project [(0 + key#10) AS _c0#8] HiveTableScan [key#10], (MetastoreRelation test, src, None), None after this patch: == Parsed Logical Plan == 'Project ['key] 'UnresolvedRelation [src], None == Analyzed Logical Plan == Project [key#10] MetastoreRelation test, src, None == Optimized Logical Plan == Project [key#10] MetastoreRelation test, src, None == Physical Plan == HiveTableScan [key#10], (MetastoreRelation test, src, None), None Author: Daoyuan Wang Closes #4551 from adrian-wang/positive and squashes the following commits: 0821ae4 [Daoyuan Wang] remove unnecessary Add --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index f3c9e63652a8e..5269460e5b6bc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1099,7 +1099,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Cast(nodeToExpr(arg), DateType) /* Arithmetic */ - case Token("+", child :: Nil) => Add(Literal(0), nodeToExpr(child)) + case Token("+", child :: Nil) => nodeToExpr(child) case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) From ada993e954e2825c0fe13326fc23b0e1a567cd55 Mon Sep 17 00:00:00 2001 From: Vladimir Grigor Date: Thu, 12 Feb 2015 23:26:24 +0000 Subject: [PATCH 094/817] [SPARK-5335] Fix deletion of security groups within a VPC Please see https://issues.apache.org/jira/browse/SPARK-5335. The fix itself is in e58a8b01a8bedcbfbbc6d04b1c1489255865cf87 commit. Two earlier commits are fixes of another VPC related bug waiting to be merged. I should have created former bug fix in own branch then this fix would not have former fixes. :( This code is released under the project's license. Author: Vladimir Grigor Author: Vladimir Grigor Closes #4122 from voukka/SPARK-5335_delete_sg_vpc and squashes the following commits: 090dca9 [Vladimir Grigor] fixes as per review: removed printing of group_id and added comment 730ec05 [Vladimir Grigor] fix for SPARK-5335: Destroying cluster in VPC with "--delete-groups" fails to remove security groups --- ec2/spark_ec2.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 35209aec5fcfe..c59ab565c6862 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1174,11 +1174,12 @@ def real_main(): time.sleep(30) # Yes, it does have to be this long :-( for group in groups: try: - conn.delete_security_group(group.name) - print "Deleted security group " + group.name + # It is needed to use group_id to make it work with VPC + conn.delete_security_group(group_id=group.id) + print "Deleted security group %s" % group.name except boto.exception.EC2ResponseError: success = False - print "Failed to delete security group " + group.name + print "Failed to delete security group %s" % group.name # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails From c025a468826e9b9f62032e207daa9d42d9dba3ca Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 12 Feb 2015 15:32:17 -0800 Subject: [PATCH 095/817] [SQL] Move SaveMode to SQL package. Author: Yin Huai Closes #4542 from yhuai/moveSaveMode and squashes the following commits: 65a4425 [Yin Huai] Move SaveMode to sql package. --- python/pyspark/sql/dataframe.py | 2 +- .../main/java/org/apache/spark/sql/{sources => }/SaveMode.java | 2 +- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 1 - .../main/scala/org/apache/spark/sql/IncomputableColumn.scala | 1 - .../main/scala/org/apache/spark/sql/json/JSONRelation.scala | 2 +- .../main/scala/org/apache/spark/sql/parquet/newParquet.scala | 3 +-- sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala | 2 +- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 2 +- .../scala/org/apache/spark/sql/sources/SaveLoadSuite.scala | 2 +- .../scala/org/apache/spark/sql/hive/execution/commands.scala | 2 +- .../apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java | 2 +- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 2 -- 12 files changed, 9 insertions(+), 14 deletions(-) rename sql/core/src/main/java/org/apache/spark/sql/{sources => }/SaveMode.java (97%) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 3eb56ed74cc6f..b6f052ee44ae2 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -149,7 +149,7 @@ def insertInto(self, tableName, overwrite=False): def _java_save_mode(self, mode): """Returns the Java save mode based on the Python save mode represented by a string. """ - jSaveMode = self._sc._jvm.org.apache.spark.sql.sources.SaveMode + jSaveMode = self._sc._jvm.org.apache.spark.sql.SaveMode jmode = jSaveMode.ErrorIfExists mode = mode.lower() if mode == "append": diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java similarity index 97% rename from sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java rename to sql/core/src/main/java/org/apache/spark/sql/SaveMode.java index 3109f5716da2c..a40be526d0d11 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/SaveMode.java +++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.sources; +package org.apache.spark.sql; /** * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. 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 65257882f4e7c..4f8f19e2c1465 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 @@ -27,7 +27,6 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.sources.SaveMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index 19c8e3b4f4708..cba3b77011cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedSt import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.sources.SaveMode import org.apache.spark.sql.types.StructType private[sql] class IncomputableColumn(protected[sql] val expr: Expression) extends Column { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index 51ff2443f3717..24848634de9cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -21,7 +21,7 @@ import java.io.IOException import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType 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 aef9c10fbcd01..3a9f0600617be 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 @@ -44,9 +44,8 @@ import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ +import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLConf, SQLContext} import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} -import org.apache.spark.sql.types.StructType._ -import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext} import org.apache.spark.{Partition => SparkPartition, TaskContext, SerializableWritable, Logging, SparkException} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index d3d72089c3303..8cac9c0fdf7fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.sources import scala.language.implicitConversions import org.apache.spark.Logging -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation 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 37fda7ba6e5d0..0c4b706eeebae 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{SaveMode, DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index a51004567175c..607488ccfdd6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.util -import org.apache.spark.sql.{SQLConf, DataFrame} +import org.apache.spark.sql.{SaveMode, SQLConf, DataFrame} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index ce0db1125c27f..0aa5f7f7b88bd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 313e84756b6bb..53ddecf57958b 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.spark.sql.sources.SaveMode; +import org.apache.spark.sql.SaveMode; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index f94aabd29ad23..2916724f66e24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -18,8 +18,6 @@ package org.apache.spark.sql.hive import java.io.File - -import org.apache.spark.sql.sources.SaveMode import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils From 1d0596a16e1d3add2631f5d8169aeec2876a1362 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 12 Feb 2015 18:08:01 -0800 Subject: [PATCH 096/817] [SPARK-3299][SQL]Public API in SQLContext to list tables https://issues.apache.org/jira/browse/SPARK-3299 Author: Yin Huai Closes #4547 from yhuai/tables and squashes the following commits: 6c8f92e [Yin Huai] Add tableNames. acbb281 [Yin Huai] Update Python test. 7793dcb [Yin Huai] Fix scala test. 572870d [Yin Huai] Address comments. aba2e88 [Yin Huai] Format. 12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing existing tables. --- python/pyspark/sql/context.py | 34 ++++++++ .../spark/sql/catalyst/analysis/Catalog.scala | 37 +++++++++ .../org/apache/spark/sql/SQLContext.scala | 36 +++++++++ .../apache/spark/sql/ListTablesSuite.scala | 76 ++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 5 ++ .../spark/sql/hive/ListTablesSuite.scala | 77 +++++++++++++++++++ 6 files changed, 265 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index db4bcbece2c1b..082f1b691b196 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -621,6 +621,40 @@ def table(self, tableName): """ return DataFrame(self._ssql_ctx.table(tableName), self) + def tables(self, dbName=None): + """Returns a DataFrame containing names of tables in the given database. + + If `dbName` is not specified, the current database will be used. + + The returned DataFrame has two columns, tableName and isTemporary + (a column with BooleanType indicating if a table is a temporary one or not). + + >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> df2 = sqlCtx.tables() + >>> df2.filter("tableName = 'table1'").first() + Row(tableName=u'table1', isTemporary=True) + """ + if dbName is None: + return DataFrame(self._ssql_ctx.tables(), self) + else: + return DataFrame(self._ssql_ctx.tables(dbName), self) + + def tableNames(self, dbName=None): + """Returns a list of names of tables in the database `dbName`. + + If `dbName` is not specified, the current database will be used. + + >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> "table1" in sqlCtx.tableNames() + True + >>> "table1" in sqlCtx.tableNames("db") + True + """ + if dbName is None: + return [name for name in self._ssql_ctx.tableNames()] + else: + return [name for name in self._ssql_ctx.tableNames(dbName)] + def cacheTable(self, tableName): """Caches the specified table in-memory.""" self._ssql_ctx.cacheTable(tableName) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index df8d03b86c533..f57eab24607f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -34,6 +34,12 @@ trait Catalog { tableIdentifier: Seq[String], alias: Option[String] = None): LogicalPlan + /** + * Returns tuples of (tableName, isTemporary) for all tables in the given database. + * isTemporary is a Boolean value indicates if a table is a temporary or not. + */ + def getTables(databaseName: Option[String]): Seq[(String, Boolean)] + def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit def unregisterTable(tableIdentifier: Seq[String]): Unit @@ -101,6 +107,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { // properly qualified with this alias. alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) } + + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + tables.map { + case (name, _) => (name, true) + }.toSeq + } } /** @@ -137,6 +149,27 @@ trait OverrideCatalog extends Catalog { withAlias.getOrElse(super.lookupRelation(tableIdentifier, alias)) } + abstract override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val dbName = if (!caseSensitive) { + if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None + } else { + databaseName + } + + val temporaryTables = overrides.filter { + // If a temporary table does not have an associated database, we should return its name. + case ((None, _), _) => true + // If a temporary table does have an associated database, we should return it if the database + // matches the given database name. + case ((db: Some[String], _), _) if db == dbName => true + case _ => false + }.map { + case ((_, tableName), _) => (tableName, true) + }.toSeq + + temporaryTables ++ super.getTables(databaseName) + } + override def registerTable( tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { @@ -172,6 +205,10 @@ object EmptyCatalog extends Catalog { throw new UnsupportedOperationException } + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + throw new UnsupportedOperationException + } + def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { throw new UnsupportedOperationException } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 8aae222acd927..0f8af75fe740d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -774,6 +774,42 @@ class SQLContext(@transient val sparkContext: SparkContext) def table(tableName: String): DataFrame = DataFrame(this, catalog.lookupRelation(Seq(tableName))) + /** + * Returns a [[DataFrame]] containing names of existing tables in the given database. + * The returned DataFrame has two columns, tableName and isTemporary (a column with BooleanType + * indicating if a table is a temporary one or not). + */ + def tables(): DataFrame = { + createDataFrame(catalog.getTables(None)).toDataFrame("tableName", "isTemporary") + } + + /** + * Returns a [[DataFrame]] containing names of existing tables in the current database. + * The returned DataFrame has two columns, tableName and isTemporary (a column with BooleanType + * indicating if a table is a temporary one or not). + */ + def tables(databaseName: String): DataFrame = { + createDataFrame(catalog.getTables(Some(databaseName))).toDataFrame("tableName", "isTemporary") + } + + /** + * Returns an array of names of tables in the current database. + */ + def tableNames(): Array[String] = { + catalog.getTables(None).map { + case (tableName, _) => tableName + }.toArray + } + + /** + * Returns an array of names of tables in the given database. + */ + def tableNames(databaseName: String): Array[String] = { + catalog.getTables(Some(databaseName)).map { + case (tableName, _) => tableName + }.toArray + } + protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala new file mode 100644 index 0000000000000..5fc35349e166e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -0,0 +1,76 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} + +class ListTablesSuite extends QueryTest with BeforeAndAfter { + + import org.apache.spark.sql.test.TestSQLContext.implicits._ + + val df = + sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDataFrame("key", "value") + + before { + df.registerTempTable("ListTablesSuiteTable") + } + + after { + catalog.unregisterTable(Seq("ListTablesSuiteTable")) + } + + test("get all tables") { + checkAnswer( + tables().filter("tableName = 'ListTablesSuiteTable'"), + Row("ListTablesSuiteTable", true)) + + catalog.unregisterTable(Seq("ListTablesSuiteTable")) + assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) + } + + test("getting all Tables with a database name has no impact on returned table names") { + checkAnswer( + tables("DB").filter("tableName = 'ListTablesSuiteTable'"), + Row("ListTablesSuiteTable", true)) + + catalog.unregisterTable(Seq("ListTablesSuiteTable")) + assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) + } + + test("query the returned DataFrame of tables") { + val tableDF = tables() + val schema = StructType( + StructField("tableName", StringType, true) :: + StructField("isTemporary", BooleanType, false) :: Nil) + assert(schema === tableDF.schema) + + tableDF.registerTempTable("tables") + checkAnswer( + sql("SELECT isTemporary, tableName from tables WHERE tableName = 'ListTablesSuiteTable'"), + Row(true, "ListTablesSuiteTable") + ) + checkAnswer( + tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), + Row("tables", true)) + dropTempTable("tables") + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c78369d12cf55..eb1ee54247bea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -198,6 +198,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + val dbName = databaseName.getOrElse(hive.sessionState.getCurrentDatabase) + client.getAllTables(dbName).map(tableName => (tableName, false)) + } + /** * Create table with specified database, table name, table description and schema * @param databaseName Database Name diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala new file mode 100644 index 0000000000000..068aa03330c33 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -0,0 +1,77 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.hive + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.Row + +class ListTablesSuite extends QueryTest with BeforeAndAfterAll { + + import org.apache.spark.sql.hive.test.TestHive.implicits._ + + val df = + sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDataFrame("key", "value") + + override def beforeAll(): Unit = { + // The catalog in HiveContext is a case insensitive one. + catalog.registerTable(Seq("ListTablesSuiteTable"), df.logicalPlan) + catalog.registerTable(Seq("ListTablesSuiteDB", "InDBListTablesSuiteTable"), df.logicalPlan) + sql("CREATE TABLE HiveListTablesSuiteTable (key int, value string)") + sql("CREATE DATABASE IF NOT EXISTS ListTablesSuiteDB") + sql("CREATE TABLE ListTablesSuiteDB.HiveInDBListTablesSuiteTable (key int, value string)") + } + + override def afterAll(): Unit = { + catalog.unregisterTable(Seq("ListTablesSuiteTable")) + catalog.unregisterTable(Seq("ListTablesSuiteDB", "InDBListTablesSuiteTable")) + sql("DROP TABLE IF EXISTS HiveListTablesSuiteTable") + sql("DROP TABLE IF EXISTS ListTablesSuiteDB.HiveInDBListTablesSuiteTable") + sql("DROP DATABASE IF EXISTS ListTablesSuiteDB") + } + + test("get all tables of current database") { + val allTables = tables() + // We are using default DB. + checkAnswer( + allTables.filter("tableName = 'listtablessuitetable'"), + Row("listtablessuitetable", true)) + assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0) + checkAnswer( + allTables.filter("tableName = 'hivelisttablessuitetable'"), + Row("hivelisttablessuitetable", false)) + assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0) + } + + test("getting all tables with a database name") { + val allTables = tables("ListTablesSuiteDB") + checkAnswer( + allTables.filter("tableName = 'listtablessuitetable'"), + Row("listtablessuitetable", true)) + checkAnswer( + allTables.filter("tableName = 'indblisttablessuitetable'"), + Row("indblisttablessuitetable", true)) + assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) + checkAnswer( + allTables.filter("tableName = 'hiveindblisttablessuitetable'"), + Row("hiveindblisttablessuitetable", false)) + } +} From 2aea892ebd4d6c802defeef35ef7ebfe42c06eba Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 12 Feb 2015 20:37:55 -0800 Subject: [PATCH 097/817] [SQL] Fix docs of SQLContext.tables Author: Yin Huai Closes #4579 from yhuai/tablesDoc and squashes the following commits: 7f8964c [Yin Huai] Fix doc. --- .../main/scala/org/apache/spark/sql/SQLContext.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0f8af75fe740d..2165949d32c6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -775,8 +775,8 @@ class SQLContext(@transient val sparkContext: SparkContext) DataFrame(this, catalog.lookupRelation(Seq(tableName))) /** - * Returns a [[DataFrame]] containing names of existing tables in the given database. - * The returned DataFrame has two columns, tableName and isTemporary (a column with BooleanType + * Returns a [[DataFrame]] containing names of existing tables in the current database. + * The returned DataFrame has two columns, tableName and isTemporary (a Boolean * indicating if a table is a temporary one or not). */ def tables(): DataFrame = { @@ -784,8 +784,8 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Returns a [[DataFrame]] containing names of existing tables in the current database. - * The returned DataFrame has two columns, tableName and isTemporary (a column with BooleanType + * Returns a [[DataFrame]] containing names of existing tables in the given database. + * The returned DataFrame has two columns, tableName and isTemporary (a Boolean * indicating if a table is a temporary one or not). */ def tables(databaseName: String): DataFrame = { @@ -793,7 +793,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Returns an array of names of tables in the current database. + * Returns the names of tables in the current database as an array. */ def tableNames(): Array[String] = { catalog.getTables(None).map { @@ -802,7 +802,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Returns an array of names of tables in the given database. + * Returns the names of tables in the given database as an array. */ def tableNames(databaseName: String): Array[String] = { catalog.getTables(Some(databaseName)).map { From 1c8633f3fe9d814c83384e339b958740c250c00c Mon Sep 17 00:00:00 2001 From: tianyi Date: Thu, 12 Feb 2015 22:18:39 -0800 Subject: [PATCH 098/817] [SPARK-3365][SQL]Wrong schema generated for List type This PR fix the issue SPARK-3365. The reason is Spark generated wrong schema for the type `List` in `ScalaReflection.scala` for example: the generated schema for type `Seq[String]` is: ``` {"name":"x","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}` ``` the generated schema for type `List[String]` is: ``` {"name":"x","type":{"type":"struct","fields":[]},"nullable":true,"metadata":{}}` ``` Author: tianyi Closes #4581 from tianyi/SPARK-3365 and squashes the following commits: a097e86 [tianyi] change the order of resolution in ScalaReflection.scala --- .../spark/sql/catalyst/ScalaReflection.scala | 30 +++++++++---------- .../sql/catalyst/ScalaReflectionSuite.scala | 5 ++++ 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 5d9c331ca5178..11fd443733658 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -122,6 +122,21 @@ trait ScalaReflection { case t if t <:< typeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) + // Need to decide if we actually need a special type here. + case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) + case t if t <:< typeOf[Array[_]] => + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + case t if t <:< typeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + case t if t <:< typeOf[Map[_, _]] => + val TypeRef(_, _, Seq(keyType, valueType)) = t + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) case t if t <:< typeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t @@ -144,21 +159,6 @@ trait ScalaReflection { schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) StructField(p.name.toString, dataType, nullable) }), nullable = true) - // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< typeOf[Array[_]] => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Map[_, _]] => - val TypeRef(_, _, Seq(keyType, valueType)) = t - val Schema(valueDataType, valueNullable) = schemaFor(valueType) - Schema(MapType(schemaFor(keyType).dataType, - valueDataType, valueContainsNull = valueNullable), nullable = true) case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index d0f547d187ecb..eee00e3f7ea76 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -61,6 +61,7 @@ case class OptionalData( case class ComplexData( arrayField: Seq[Int], arrayField1: Array[Int], + arrayField2: List[Int], arrayFieldContainsNull: Seq[java.lang.Integer], mapField: Map[Int, Long], mapFieldValueContainsNull: Map[Int, java.lang.Long], @@ -137,6 +138,10 @@ class ScalaReflectionSuite extends FunSuite { "arrayField1", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField( + "arrayField2", + ArrayType(IntegerType, containsNull = false), + nullable = true), StructField( "arrayFieldContainsNull", ArrayType(IntegerType, containsNull = true), From 1768bd51438670c493ca3ca02988aee3ae31e87e Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Fri, 13 Feb 2015 10:27:23 +0000 Subject: [PATCH 099/817] [SPARK-4832][Deploy]some other processes might take the daemon pid Some other processes might use the pid saved in pid file. In that case we should ignore it and launch daemons. JIRA is down for maintenance. I will file one once it return. Author: WangTaoTheTonic Author: WangTaoTheTonic Closes #3683 from WangTaoTheTonic/otherproc and squashes the following commits: daa86a1 [WangTaoTheTonic] some bash style fix 8befee7 [WangTaoTheTonic] handle the mistake scenario cf4ecc6 [WangTaoTheTonic] remove redundant condition f36cfb4 [WangTaoTheTonic] some other processes might take the pid --- sbin/spark-daemon.sh | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 89608bc41b71d..ec6d0b5a40ef2 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -129,8 +129,9 @@ case $option in mkdir -p "$SPARK_PID_DIR" if [ -f $pid ]; then - if kill -0 `cat $pid` > /dev/null 2>&1; then - echo $command running as process `cat $pid`. Stop it first. + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + echo "$command running as process $TARGET_ID. Stop it first." exit 1 fi fi @@ -141,7 +142,7 @@ case $option in fi spark_rotate_log "$log" - echo starting $command, logging to $log + echo "starting $command, logging to $log" if [ $option == spark-submit ]; then source "$SPARK_HOME"/bin/utils.sh gatherSparkSubmitOpts "$@" @@ -154,7 +155,7 @@ case $option in echo $newpid > $pid sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see - if ! kill -0 $newpid >/dev/null 2>&1; then + if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then echo "failed to launch $command:" tail -2 "$log" | sed 's/^/ /' echo "full log in $log" @@ -164,14 +165,15 @@ case $option in (stop) if [ -f $pid ]; then - if kill -0 `cat $pid` > /dev/null 2>&1; then - echo stopping $command - kill `cat $pid` + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + echo "stopping $command" + kill "$TARGET_ID" else - echo no $command to stop + echo "no $command to stop" fi else - echo no $command to stop + echo "no $command to stop" fi ;; From c0ccd2564182695ea5771524840bf1a99d5aa842 Mon Sep 17 00:00:00 2001 From: uncleGen Date: Fri, 13 Feb 2015 09:43:10 -0800 Subject: [PATCH 100/817] [SPARK-5732][CORE]:Add an option to print the spark version in spark script. Naturally, we may need to add an option to print the spark version in spark script. It is pretty common in script tool. ![9](https://cloud.githubusercontent.com/assets/7402327/6183331/cab1b74e-b38e-11e4-9daa-e26e6015cff3.JPG) Author: uncleGen Author: genmao.ygm Closes #4522 from uncleGen/master-clean-150211 and squashes the following commits: 9f2127c [genmao.ygm] revert the behavior of "-v" 015ddee [uncleGen] minor changes 463f02c [uncleGen] minor changes --- .../org/apache/spark/deploy/SparkSubmit.scala | 18 +++++++++++++++--- .../spark/deploy/SparkSubmitArguments.scala | 4 ++++ 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 80cc0587286b1..54399e99c98f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -37,6 +37,7 @@ import org.apache.ivy.core.settings.IvySettings import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver} +import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ import org.apache.spark.executor._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} @@ -83,14 +84,25 @@ object SparkSubmit { // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(1) private[spark] var printStream: PrintStream = System.err - private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) - private[spark] def printErrorAndExit(str: String) = { + private[spark] def printWarning(str: String): Unit = printStream.println("Warning: " + str) + private[spark] def printErrorAndExit(str: String): Unit = { printStream.println("Error: " + str) printStream.println("Run with --help for usage help or --verbose for debug output") exitFn() } + private[spark] def printVersionAndExit(): Unit = { + printStream.println("""Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version %s + /_/ + """.format(SPARK_VERSION)) + printStream.println("Type --help for more information.") + exitFn() + } - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { printStream.println(appArgs) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index fa38070c6fcfe..82e66a374249c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -417,6 +417,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St verbose = true parse(tail) + case ("--version") :: tail => + SparkSubmit.printVersionAndExit() + case EQ_SEPARATED_OPT(opt, value) :: tail => parse(opt :: value :: tail) @@ -485,6 +488,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | | --help, -h Show this help message and exit | --verbose, -v Print additional debug output + | --version, Print the version of current Spark | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). From e1a1ff8108463ca79299ec0eb555a0c8db9dffa0 Mon Sep 17 00:00:00 2001 From: sboeschhuawei Date: Fri, 13 Feb 2015 09:45:57 -0800 Subject: [PATCH 101/817] [SPARK-5503][MLLIB] Example code for Power Iteration Clustering Author: sboeschhuawei Closes #4495 from javadba/picexamples and squashes the following commits: 3c84b14 [sboeschhuawei] PIC Examples updates from Xiangrui's comments round 5 2878675 [sboeschhuawei] Fourth round with xiangrui on PICExample d7ac350 [sboeschhuawei] Updates to PICExample from Xiangrui's comments round 3 d7f0cba [sboeschhuawei] Updates to PICExample from Xiangrui's comments round 3 cef28f4 [sboeschhuawei] Further updates to PICExample from Xiangrui's comments f7ff43d [sboeschhuawei] Update to PICExample from Xiangrui's comments efeec45 [sboeschhuawei] Update to PICExample from Xiangrui's comments 03e8de4 [sboeschhuawei] Added PICExample c509130 [sboeschhuawei] placeholder for pic examples 5864d4a [sboeschhuawei] placeholder for pic examples --- .../PowerIterationClusteringExample.scala | 160 ++++++++++++++++++ 1 file changed, 160 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala new file mode 100644 index 0000000000000..b2373adba1fd4 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.log4j.{Level, Logger} +import scopt.OptionParser + +import org.apache.spark.mllib.clustering.PowerIterationClustering +import org.apache.spark.rdd.RDD +import org.apache.spark.{SparkConf, SparkContext} + +/** + * An example Power Iteration Clustering http://www.icml2010.org/papers/387.pdf app. + * Takes an input of K concentric circles and the number of points in the innermost circle. + * The output should be K clusters - each cluster containing precisely the points associated + * with each of the input circles. + * + * Run with + * {{{ + * ./bin/run-example mllib.PowerIterationClusteringExample [options] + * + * Where options include: + * k: Number of circles/clusters + * n: Number of sampled points on innermost circle.. There are proportionally more points + * within the outer/larger circles + * maxIterations: Number of Power Iterations + * outerRadius: radius of the outermost of the concentric circles + * }}} + * + * Here is a sample run and output: + * + * ./bin/run-example mllib.PowerIterationClusteringExample + * -k 3 --n 30 --maxIterations 15 + * + * Cluster assignments: 1 -> [0,1,2,3,4],2 -> [5,6,7,8,9,10,11,12,13,14], + * 0 -> [15,16,17,18,19,20,21,22,23,24,25,26,27,28,29] + * + * + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object PowerIterationClusteringExample { + + case class Params( + input: String = null, + k: Int = 3, + numPoints: Int = 5, + maxIterations: Int = 10, + outerRadius: Double = 3.0 + ) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("PIC Circles") { + head("PowerIterationClusteringExample: an example PIC app using concentric circles.") + opt[Int]('k', "k") + .text(s"number of circles (/clusters), default: ${defaultParams.k}") + .action((x, c) => c.copy(k = x)) + opt[Int]('n', "n") + .text(s"number of points in smallest circle, default: ${defaultParams.numPoints}") + .action((x, c) => c.copy(numPoints = x)) + opt[Int]("maxIterations") + .text(s"number of iterations, default: ${defaultParams.maxIterations}") + .action((x, c) => c.copy(maxIterations = x)) + opt[Int]('r', "r") + .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") + .action((x, c) => c.copy(numPoints = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf() + .setMaster("local") + .setAppName(s"PowerIterationClustering with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val circlesRdd = generateCirclesRdd(sc, params.k, params.numPoints, params.outerRadius) + val model = new PowerIterationClustering() + .setK(params.k) + .setMaxIterations(params.maxIterations) + .run(circlesRdd) + + val clusters = model.assignments.collect.groupBy(_._2).mapValues(_.map(_._1)) + val assignments = clusters.toList.sortBy { case (k, v) => v.length} + val assignmentsStr = assignments + .map { case (k, v) => + s"$k -> ${v.sorted.mkString("[", ",", "]")}" + }.mkString(",") + val sizesStr = assignments.map { + _._2.size + }.sorted.mkString("(", ",", ")") + println(s"Cluster assignments: $assignmentsStr\ncluster sizes: $sizesStr") + + sc.stop() + } + + def generateCircle(radius: Double, n: Int) = { + Seq.tabulate(n) { i => + val theta = 2.0 * math.Pi * i / n + (radius * math.cos(theta), radius * math.sin(theta)) + } + } + + def generateCirclesRdd(sc: SparkContext, + nCircles: Int = 3, + nPoints: Int = 30, + outerRadius: Double): RDD[(Long, Long, Double)] = { + + val radii = Array.tabulate(nCircles) { cx => outerRadius / (nCircles - cx)} + val groupSizes = Array.tabulate(nCircles) { cx => (cx + 1) * nPoints} + val points = (0 until nCircles).flatMap { cx => + generateCircle(radii(cx), groupSizes(cx)) + }.zipWithIndex + val rdd = sc.parallelize(points) + val distancesRdd = rdd.cartesian(rdd).flatMap { case (((x0, y0), i0), ((x1, y1), i1)) => + if (i0 < i1) { + Some((i0.toLong, i1.toLong, gaussianSimilarity((x0, y0), (x1, y1), 1.0))) + } else { + None + } + } + distancesRdd + } + + /** + * Gaussian Similarity: http://en.wikipedia.org/wiki/Radial_basis_function_kernel + */ + def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double), sigma: Double) = { + val coeff = 1.0 / (math.sqrt(2.0 * math.Pi) * sigma) + val expCoeff = -1.0 / 2.0 * math.pow(sigma, 2.0) + val ssquares = (p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2) + coeff * math.exp(expCoeff * ssquares) + // math.exp((p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2)) + } + + +} From fc6d3e796a3c600e2f7827562455d555e59775ae Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Fri, 13 Feb 2015 09:47:26 -0800 Subject: [PATCH 102/817] [SPARK-5783] Better eventlog-parsing error messages Author: Ryan Williams Closes #4573 from ryan-williams/history and squashes the following commits: a8647ec [Ryan Williams] fix test calls to .replay() 98aa3fe [Ryan Williams] include filename in history-parsing error message 8deecf0 [Ryan Williams] add line number to history-parsing error message b668b52 [Ryan Williams] add log info line to history-eventlog parsing --- .../apache/spark/deploy/history/FsHistoryProvider.scala | 3 ++- .../scala/org/apache/spark/deploy/master/Master.scala | 2 +- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 9 ++++++--- .../org/apache/spark/scheduler/ReplayListenerSuite.scala | 4 ++-- 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 868c63d30a202..885fa0fdbf85b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -247,6 +247,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { val logPath = eventLog.getPath() + logInfo(s"Replaying log path: $logPath") val (logInput, sparkVersion) = if (isLegacyLogDirectory(eventLog)) { openLegacyEventLog(logPath) @@ -256,7 +257,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val appListener = new ApplicationEventListener bus.addListener(appListener) - bus.replay(logInput, sparkVersion) + bus.replay(logInput, sparkVersion, logPath.toString) new FsApplicationHistoryInfo( logPath.getName(), appListener.appId.getOrElse(logPath.getName()), diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 53e453990f8c7..8cc6ec1e8192c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -761,7 +761,7 @@ private[spark] class Master( val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") try { - replayBus.replay(logInput, sparkVersion) + replayBus.replay(logInput, sparkVersion, eventLogFile) } finally { logInput.close() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 584f4e7789d1a..d9c3a10dc5413 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -40,21 +40,24 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * * @param logData Stream containing event log data. * @param version Spark version that generated the events. + * @param sourceName Filename (or other source identifier) from whence @logData is being read */ - def replay(logData: InputStream, version: String) { + def replay(logData: InputStream, version: String, sourceName: String) { var currentLine: String = null + var lineNumber: Int = 1 try { val lines = Source.fromInputStream(logData).getLines() lines.foreach { line => currentLine = line postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + lineNumber += 1 } } catch { case ioe: IOException => throw ioe case e: Exception => - logError("Exception in parsing Spark event log.", e) - logError("Malformed line: %s\n".format(currentLine)) + logError(s"Exception parsing Spark event log: $sourceName", e) + logError(s"Malformed line #$lineNumber: $currentLine\n") } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7e360cc6082ec..702c4cb3bdef9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -61,7 +61,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, SPARK_VERSION) + replayer.replay(logData, SPARK_VERSION, logFilePath.toString) } finally { logData.close() } @@ -120,7 +120,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, version) + replayer.replay(logData, version, eventLog.getPath().toString) } finally { logData.close() } From 077eec2d9dba197f51004ee4a322d0fa71424ea0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 13 Feb 2015 09:53:57 -0800 Subject: [PATCH 103/817] [SPARK-5735] Replace uses of EasyMock with Mockito This patch replaces all uses of EasyMock with Mockito. There are two motivations for this: 1. We should use a single mocking framework in our tests in order to keep things consistent. 2. EasyMock may be responsible for non-deterministic unit test failures due to its Objensis dependency (see SPARK-5626 for more details). Most of these changes are fairly mechanical translations of Mockito code to EasyMock, although I made a small change that strengthens the assertions in one test in KinesisReceiverSuite. Author: Josh Rosen Closes #4578 from JoshRosen/SPARK-5735-remove-easymock and squashes the following commits: 0ab192b [Josh Rosen] Import sorting plus two minor changes to more closely match old semantics. 977565b [Josh Rosen] Remove EasyMock from build. fae1d8f [Josh Rosen] Remove EasyMock usage in KinesisReceiverSuite. 7cca486 [Josh Rosen] Remove EasyMock usage in MesosSchedulerBackendSuite fc5e94d [Josh Rosen] Remove EasyMock in CacheManagerSuite --- core/pom.xml | 10 - .../org/apache/spark/CacheManagerSuite.scala | 42 ++- .../mesos/MesosSchedulerBackendSuite.scala | 125 +++++---- extras/kinesis-asl/pom.xml | 5 - .../kinesis/KinesisReceiverSuite.scala | 263 +++++++++--------- pom.xml | 13 - 6 files changed, 207 insertions(+), 251 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 66180035e61f1..c993781c0e0d6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -329,16 +329,6 @@ scalacheck_${scala.binary.version} test - - org.easymock - easymockclassextension - test - - - asm - asm - test - junit junit diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index d7d9dc7b50f30..4b25c200a695a 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -17,16 +17,18 @@ package org.apache.spark +import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.mock.EasyMockSugar +import org.scalatest.mock.MockitoSugar -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.RDD import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects -class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { - var sc : SparkContext = _ +class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter + with MockitoSugar { + var blockManager: BlockManager = _ var cacheManager: CacheManager = _ var split: Partition = _ @@ -57,10 +59,6 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar }.cache() } - after { - sc.stop() - } - test("get uncached rdd") { // Do not mock this test, because attempting to match Array[Any], which is not covariant, // in blockManager.put is a losing battle. You have been warned. @@ -75,29 +73,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get cached rdd") { - expecting { - val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) - blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) - } + val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) + when(blockManager.get(RDDBlockId(0, 0))).thenReturn(Some(result)) - whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, 0) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(5, 6, 7)) - } + val context = new TaskContextImpl(0, 0, 0, 0) + val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + assert(value.toList === List(5, 6, 7)) } test("get uncached local rdd") { - expecting { - // Local computation should not persist the resulting value, so don't expect a put(). - blockManager.get(RDDBlockId(0, 0)).andReturn(None) - } + // Local computation should not persist the resulting value, so don't expect a put(). + when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, 0, true) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } + val context = new TaskContextImpl(0, 0, 0, 0, true) + val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + assert(value.toList === List(1, 2, 3, 4)) } test("verify task metrics updated correctly") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index 46ab02bfef780..8cd302e2b4313 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -17,45 +17,47 @@ package org.apache.spark.scheduler.mesos -import org.apache.spark.executor.MesosExecutorBackend -import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} -import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus, - TaskDescription, WorkerOffer, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend} -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, _} -import org.apache.mesos.Protos.Value.Scalar -import org.easymock.{Capture, EasyMock} import java.nio.ByteBuffer -import java.util.Collections import java.util -import org.scalatest.mock.EasyMockSugar +import java.util.Collections import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { +import org.apache.mesos.SchedulerDriver +import org.apache.mesos.Protos._ +import org.apache.mesos.Protos.Value.Scalar +import org.mockito.Mockito._ +import org.mockito.Matchers._ +import org.mockito.{ArgumentCaptor, Matchers} +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} +import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} + +class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { test("check spark-class location correctly") { val conf = new SparkConf conf.set("spark.mesos.executor.home" , "/mesos-home") - val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - EasyMock.replay(listenerBus) - - val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.getSparkHome()).andReturn(Option("/spark-home")).anyTimes() - EasyMock.expect(sc.conf).andReturn(conf).anyTimes() - EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() - EasyMock.expect(sc.listenerBus).andReturn(listenerBus) - EasyMock.replay(sc) - val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) + val listenerBus = mock[LiveListenerBus] + listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + + val sc = mock[SparkContext] + when(sc.getSparkHome()).thenReturn(Option("/spark-home")) + + when(sc.conf).thenReturn(conf) + when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) + when(sc.executorMemory).thenReturn(100) + when(sc.listenerBus).thenReturn(listenerBus) + val taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") @@ -84,20 +86,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } - val driver = EasyMock.createMock(classOf[SchedulerDriver]) - val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] - val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - EasyMock.replay(listenerBus) + val listenerBus = mock[LiveListenerBus] + listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() - EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() - EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() - EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() - EasyMock.expect(sc.listenerBus).andReturn(listenerBus) - EasyMock.replay(sc) + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/path")) + when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) + when(sc.conf).thenReturn(new SparkConf) + when(sc.listenerBus).thenReturn(listenerBus) val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 @@ -121,25 +122,29 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea 2 )) val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) + when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val capture = new Capture[util.Collection[TaskInfo]] - EasyMock.expect( + val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + when( driver.launchTasks( - EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), - EasyMock.capture(capture), - EasyMock.anyObject(classOf[Filters]) + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + capture.capture(), + any(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)).once - EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) + ).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(mesosOffers.get(1).getId)).thenReturn(Status.valueOf(1)) + when(driver.declineOffer(mesosOffers.get(2).getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, mesosOffers) - EasyMock.verify(driver) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + capture.capture(), + any(classOf[Filters]) + ) + verify(driver, times(1)).declineOffer(mesosOffers.get(1).getId) + verify(driver, times(1)).declineOffer(mesosOffers.get(2).getId) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -151,15 +156,13 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea // Unwanted resources offered on an existing node. Make sure they are declined val mesosOffers2 = new java.util.ArrayList[Offer] mesosOffers2.add(createOffer(1, minMem, minCpu)) - EasyMock.reset(taskScheduler) - EasyMock.reset(driver) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) - EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) + reset(taskScheduler) + reset(driver) + when(taskScheduler.resourceOffers(any(classOf[Seq[WorkerOffer]]))).thenReturn(Seq(Seq())) + when(taskScheduler.CPUS_PER_TASK).thenReturn(2) + when(driver.declineOffer(mesosOffers2.get(0).getId)).thenReturn(Status.valueOf(1)) backend.resourceOffers(driver, mesosOffers2) - EasyMock.verify(driver) + verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId) } } diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index c815eda52bda7..216661b8bc73a 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -67,11 +67,6 @@ scalacheck_${scala.binary.version} test - - org.easymock - easymockclassextension - test - com.novocode junit-interface diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index 41dbd64c2b1fa..f56898af029c1 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -20,7 +20,6 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions.seqAsJavaList -import org.apache.spark.annotation.Experimental import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.Milliseconds import org.apache.spark.streaming.Seconds @@ -28,9 +27,11 @@ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.TestSuiteBase import org.apache.spark.streaming.util.Clock import org.apache.spark.streaming.util.ManualClock + +import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers -import org.scalatest.mock.EasyMockSugar +import org.scalatest.mock.MockitoSugar import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException @@ -42,10 +43,10 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason import com.amazonaws.services.kinesis.model.Record /** - * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor + * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor */ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter - with EasyMockSugar { + with MockitoSugar { val app = "TestKinesisReceiver" val stream = "mySparkStream" @@ -73,6 +74,14 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft currentClockMock = mock[Clock] } + override def afterFunction(): Unit = { + super.afterFunction() + // Since this suite was originally written using EasyMock, add this to preserve the old + // mocking semantics (see SPARK-5735 for more details) + verifyNoMoreInteractions(receiverMock, checkpointerMock, checkpointClockMock, + checkpointStateMock, currentClockMock) + } + test("kinesis utils api") { val ssc = new StreamingContext(master, framework, batchDuration) // Tests the API, does not actually test data receiving @@ -83,193 +92,175 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft } test("process records including store and checkpoint") { - val expectedCheckpointIntervalMillis = 10 - expecting { - receiverMock.isStopped().andReturn(false).once() - receiverMock.store(record1.getData().array()).once() - receiverMock.store(record2.getData().array()).once() - checkpointStateMock.shouldCheckpoint().andReturn(true).once() - checkpointerMock.checkpoint().once() - checkpointStateMock.advanceCheckpoint().once() - } - whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, - checkpointStateMock) - recordProcessor.processRecords(batch, checkpointerMock) - } + when(receiverMock.isStopped()).thenReturn(false) + when(checkpointStateMock.shouldCheckpoint()).thenReturn(true) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, times(1)).store(record1.getData().array()) + verify(receiverMock, times(1)).store(record2.getData().array()) + verify(checkpointStateMock, times(1)).shouldCheckpoint() + verify(checkpointerMock, times(1)).checkpoint() + verify(checkpointStateMock, times(1)).advanceCheckpoint() } test("shouldn't store and checkpoint when receiver is stopped") { - expecting { - receiverMock.isStopped().andReturn(true).once() - } - whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, - checkpointStateMock) - recordProcessor.processRecords(batch, checkpointerMock) - } + when(receiverMock.isStopped()).thenReturn(true) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + + verify(receiverMock, times(1)).isStopped() } test("shouldn't checkpoint when exception occurs during store") { - expecting { - receiverMock.isStopped().andReturn(false).once() - receiverMock.store(record1.getData().array()).andThrow(new RuntimeException()).once() - } - whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { - intercept[RuntimeException] { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, - checkpointStateMock) - recordProcessor.processRecords(batch, checkpointerMock) - } + when(receiverMock.isStopped()).thenReturn(false) + when(receiverMock.store(record1.getData().array())).thenThrow(new RuntimeException()) + + intercept[RuntimeException] { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) } + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, times(1)).store(record1.getData().array()) } test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") { - expecting { - currentClockMock.currentTime().andReturn(0).once() - } - whenExecuting(currentClockMock) { + when(currentClockMock.currentTime()).thenReturn(0) + val checkpointIntervalMillis = 10 - val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + val checkpointState = + new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) - } + + verify(currentClockMock, times(1)).currentTime() } test("should checkpoint if we have exceeded the checkpoint interval") { - expecting { - currentClockMock.currentTime().andReturn(0).once() - } - whenExecuting(currentClockMock) { - val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) - assert(checkpointState.shouldCheckpoint()) - } + when(currentClockMock.currentTime()).thenReturn(0) + + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) + assert(checkpointState.shouldCheckpoint()) + + verify(currentClockMock, times(1)).currentTime() } test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { - expecting { - currentClockMock.currentTime().andReturn(0).once() - } - whenExecuting(currentClockMock) { - val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) - assert(!checkpointState.shouldCheckpoint()) - } + when(currentClockMock.currentTime()).thenReturn(0) + + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) + assert(!checkpointState.shouldCheckpoint()) + + verify(currentClockMock, times(1)).currentTime() } test("should add to time when advancing checkpoint") { - expecting { - currentClockMock.currentTime().andReturn(0).once() - } - whenExecuting(currentClockMock) { - val checkpointIntervalMillis = 10 - val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) - assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) - checkpointState.advanceCheckpoint() - assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis)) - } + when(currentClockMock.currentTime()).thenReturn(0) + + val checkpointIntervalMillis = 10 + val checkpointState = + new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + checkpointState.advanceCheckpoint() + assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis)) + + verify(currentClockMock, times(1)).currentTime() } test("shutdown should checkpoint if the reason is TERMINATE") { - expecting { - checkpointerMock.checkpoint().once() - } - whenExecuting(checkpointerMock, checkpointStateMock) { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, - checkpointStateMock) - val reason = ShutdownReason.TERMINATE - recordProcessor.shutdown(checkpointerMock, reason) - } + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) + val reason = ShutdownReason.TERMINATE + recordProcessor.shutdown(checkpointerMock, reason) + + verify(checkpointerMock, times(1)).checkpoint() } test("shutdown should not checkpoint if the reason is something other than TERMINATE") { - expecting { - } - whenExecuting(checkpointerMock, checkpointStateMock) { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, - checkpointStateMock) - recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) - recordProcessor.shutdown(checkpointerMock, null) - } + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) + recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) + recordProcessor.shutdown(checkpointerMock, null) + + verify(checkpointerMock, never()).checkpoint() } test("retry success on first attempt") { val expectedIsStopped = false - expecting { - receiverMock.isStopped().andReturn(expectedIsStopped).once() - } - whenExecuting(receiverMock) { - val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) - assert(actualVal == expectedIsStopped) - } + when(receiverMock.isStopped()).thenReturn(expectedIsStopped) + + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + + verify(receiverMock, times(1)).isStopped() } test("retry success on second attempt after a Kinesis throttling exception") { val expectedIsStopped = false - expecting { - receiverMock.isStopped().andThrow(new ThrottlingException("error message")) - .andReturn(expectedIsStopped).once() - } - whenExecuting(receiverMock) { - val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) - assert(actualVal == expectedIsStopped) - } + when(receiverMock.isStopped()) + .thenThrow(new ThrottlingException("error message")) + .thenReturn(expectedIsStopped) + + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + + verify(receiverMock, times(2)).isStopped() } test("retry success on second attempt after a Kinesis dependency exception") { val expectedIsStopped = false - expecting { - receiverMock.isStopped().andThrow(new KinesisClientLibDependencyException("error message")) - .andReturn(expectedIsStopped).once() - } - whenExecuting(receiverMock) { - val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) - assert(actualVal == expectedIsStopped) - } + when(receiverMock.isStopped()) + .thenThrow(new KinesisClientLibDependencyException("error message")) + .thenReturn(expectedIsStopped) + + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + + verify(receiverMock, times(2)).isStopped() } test("retry failed after a shutdown exception") { - expecting { - checkpointerMock.checkpoint().andThrow(new ShutdownException("error message")).once() - } - whenExecuting(checkpointerMock) { - intercept[ShutdownException] { - KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) - } + when(checkpointerMock.checkpoint()).thenThrow(new ShutdownException("error message")) + + intercept[ShutdownException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) } + + verify(checkpointerMock, times(1)).checkpoint() } test("retry failed after an invalid state exception") { - expecting { - checkpointerMock.checkpoint().andThrow(new InvalidStateException("error message")).once() - } - whenExecuting(checkpointerMock) { - intercept[InvalidStateException] { - KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) - } + when(checkpointerMock.checkpoint()).thenThrow(new InvalidStateException("error message")) + + intercept[InvalidStateException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) } + + verify(checkpointerMock, times(1)).checkpoint() } test("retry failed after unexpected exception") { - expecting { - checkpointerMock.checkpoint().andThrow(new RuntimeException("error message")).once() - } - whenExecuting(checkpointerMock) { - intercept[RuntimeException] { - KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) - } + when(checkpointerMock.checkpoint()).thenThrow(new RuntimeException("error message")) + + intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) } + + verify(checkpointerMock, times(1)).checkpoint() } test("retry failed after exhausing all retries") { val expectedErrorMessage = "final try error message" - expecting { - checkpointerMock.checkpoint().andThrow(new ThrottlingException("error message")) - .andThrow(new ThrottlingException(expectedErrorMessage)).once() - } - whenExecuting(checkpointerMock) { - val exception = intercept[RuntimeException] { - KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) - } - exception.getMessage().shouldBe(expectedErrorMessage) + when(checkpointerMock.checkpoint()) + .thenThrow(new ThrottlingException("error message")) + .thenThrow(new ThrottlingException(expectedErrorMessage)) + + val exception = intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) } + exception.getMessage().shouldBe(expectedErrorMessage) + + verify(checkpointerMock, times(2)).checkpoint() } } diff --git a/pom.xml b/pom.xml index 56e37d42265c0..53372d5cfc624 100644 --- a/pom.xml +++ b/pom.xml @@ -619,19 +619,6 @@ 2.2.1 test - - org.easymock - easymockclassextension - 3.1 - test - - - - asm - asm - 3.3.1 - test - org.mockito mockito-all From 9f31db061019414a964aac432e946eac61f8307c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Emre=20Sevin=C3=A7?= Date: Fri, 13 Feb 2015 12:31:27 -0800 Subject: [PATCH 104/817] SPARK-5805 Fixed the type error in documentation. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fixes SPARK-5805 : Fix the type error in the final example given in MLlib - Clustering documentation. Author: Emre Sevinç Closes #4596 from emres/SPARK-5805 and squashes the following commits: 1029f66 [Emre Sevinç] SPARK-5805 Fixed the type error in documentation. --- docs/mllib-clustering.md | 62 ++++++++++++++++++++-------------------- 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 99ed6b60e3f00..0450e6fd3e07e 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -14,7 +14,7 @@ Clustering is an unsupervised learning problem whereby we aim to group subsets of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression -models are trained for each cluster). +models are trained for each cluster). MLlib supports the following models: @@ -25,7 +25,7 @@ most commonly used clustering algorithms that clusters the data points into a predefined number of clusters. The MLlib implementation includes a parallelized variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). -The implementation in MLlib has the following parameters: +The implementation in MLlib has the following parameters: * *k* is the number of desired clusters. * *maxIterations* is the maximum number of iterations to run. @@ -35,12 +35,12 @@ initialization via k-means\|\|. guaranteed to find a globally optimal solution, and when run multiple times on a given dataset, the algorithm returns the best clustering result). * *initializationSteps* determines the number of steps in the k-means\|\| algorithm. -* *epsilon* determines the distance threshold within which we consider k-means to have converged. +* *epsilon* determines the distance threshold within which we consider k-means to have converged. ### Gaussian mixture A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) -represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, +represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, each with its own probability. The MLlib implementation uses the [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) algorithm to induce the maximum-likelihood model given a set of samples. The implementation @@ -221,8 +221,8 @@ print("Within Set Sum of Squared Error = " + str(WSSSE))
    In the following example after loading and parsing data, we use a -[GaussianMixture](api/scala/index.html#org.apache.spark.mllib.clustering.GaussianMixture) -object to cluster the data into two clusters. The number of desired clusters is passed +[GaussianMixture](api/scala/index.html#org.apache.spark.mllib.clustering.GaussianMixture) +object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then output the parameters of the mixture model. {% highlight scala %} @@ -238,7 +238,7 @@ val gmm = new GaussianMixture().setK(2).run(parsedData) // output parameters of max-likelihood model for (i <- 0 until gmm.k) { - println("weight=%f\nmu=%s\nsigma=\n%s\n" format + println("weight=%f\nmu=%s\nsigma=\n%s\n" format (gmm.weights(i), gmm.gaussians(i).mu, gmm.gaussians(i).sigma)) } @@ -298,7 +298,7 @@ public class GaussianMixtureExample {
    In the following example after loading and parsing data, we use a [GaussianMixture](api/python/pyspark.mllib.html#pyspark.mllib.clustering.GaussianMixture) -object to cluster the data into two clusters. The number of desired clusters is passed +object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then output the parameters of the mixture model. {% highlight python %} @@ -326,7 +326,7 @@ for i in range(2): In the following example, we load word count vectors representing a corpus of documents. We then use [LDA](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) -to infer three topics from the documents. The number of desired clusters is passed +to infer three topics from the documents. The number of desired clusters is passed to the algorithm. We then output the topics, represented as probability distributions over words.
    @@ -428,27 +428,27 @@ a dependency. ## Streaming clustering -When data arrive in a stream, we may want to estimate clusters dynamically, -updating them as new data arrive. MLlib provides support for streaming k-means clustering, -with parameters to control the decay (or "forgetfulness") of the estimates. The algorithm -uses a generalization of the mini-batch k-means update rule. For each batch of data, we assign +When data arrive in a stream, we may want to estimate clusters dynamically, +updating them as new data arrive. MLlib provides support for streaming k-means clustering, +with parameters to control the decay (or "forgetfulness") of the estimates. The algorithm +uses a generalization of the mini-batch k-means update rule. For each batch of data, we assign all points to their nearest cluster, compute new cluster centers, then update each cluster using: `\begin{equation} c_{t+1} = \frac{c_tn_t\alpha + x_tm_t}{n_t\alpha+m_t} \end{equation}` `\begin{equation} - n_{t+1} = n_t + m_t + n_{t+1} = n_t + m_t \end{equation}` -Where `$c_t$` is the previous center for the cluster, `$n_t$` is the number of points assigned -to the cluster thus far, `$x_t$` is the new cluster center from the current batch, and `$m_t$` -is the number of points added to the cluster in the current batch. The decay factor `$\alpha$` -can be used to ignore the past: with `$\alpha$=1` all data will be used from the beginning; -with `$\alpha$=0` only the most recent data will be used. This is analogous to an -exponentially-weighted moving average. +Where `$c_t$` is the previous center for the cluster, `$n_t$` is the number of points assigned +to the cluster thus far, `$x_t$` is the new cluster center from the current batch, and `$m_t$` +is the number of points added to the cluster in the current batch. The decay factor `$\alpha$` +can be used to ignore the past: with `$\alpha$=1` all data will be used from the beginning; +with `$\alpha$=0` only the most recent data will be used. This is analogous to an +exponentially-weighted moving average. -The decay can be specified using a `halfLife` parameter, which determines the +The decay can be specified using a `halfLife` parameter, which determines the correct decay factor `a` such that, for data acquired at time `t`, its contribution by time `t + halfLife` will have dropped to 0.5. The unit of time can be specified either as `batches` or `points` and the update rule @@ -472,9 +472,9 @@ import org.apache.spark.mllib.clustering.StreamingKMeans {% endhighlight %} -Then we make an input stream of vectors for training, as well as a stream of labeled data -points for testing. We assume a StreamingContext `ssc` has been created, see -[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info. +Then we make an input stream of vectors for training, as well as a stream of labeled data +points for testing. We assume a StreamingContext `ssc` has been created, see +[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info. {% highlight scala %} @@ -496,24 +496,24 @@ val model = new StreamingKMeans() {% endhighlight %} -Now register the streams for training and testing and start the job, printing +Now register the streams for training and testing and start the job, printing the predicted cluster assignments on new data points as they arrive. {% highlight scala %} model.trainOn(trainingData) -model.predictOnValues(testData).print() +model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() ssc.start() ssc.awaitTermination() - + {% endhighlight %} -As you add new text files with data the cluster centers will update. Each training +As you add new text files with data the cluster centers will update. Each training point should be formatted as `[x1, x2, x3]`, and each test data point -should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier -(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir` -the model will update. Anytime a text file is placed in `/testing/data/dir` +should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier +(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir` +the model will update. Anytime a text file is placed in `/testing/data/dir` you will see predictions. With new data, the cluster centers will change!
    From 378c7eb0d6a9a86da5b9d75a41400128abbe67fb Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 13 Feb 2015 12:43:53 -0800 Subject: [PATCH 105/817] [HOTFIX] Ignore DirectKafkaStreamSuite. --- .../spark/streaming/kafka/DirectKafkaStreamSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index b25c2120d54f7..926094449e7fc 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -67,7 +67,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } - test("basic stream receiving with multiple topics and smallest starting offset") { + ignore("basic stream receiving with multiple topics and smallest starting offset") { val topics = Set("basic1", "basic2", "basic3") val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => @@ -113,7 +113,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase ssc.stop() } - test("receiving from largest starting offset") { + ignore("receiving from largest starting offset") { val topic = "largest" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) @@ -158,7 +158,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } - test("creating stream by offset") { + ignore("creating stream by offset") { val topic = "offset" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) @@ -204,7 +204,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Test to verify the offset ranges can be recovered from the checkpoints - test("offset recovery") { + ignore("offset recovery") { val topic = "recovery" createTopic(topic) testDir = Utils.createTempDir() From 5d3cc6b3d7e6ede3be68885e37e3c4ca3a608982 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 13 Feb 2015 13:10:29 -0800 Subject: [PATCH 106/817] [HOTFIX] Fix build break in MesosSchedulerBackendSuite --- .../spark/scheduler/mesos/MesosSchedulerBackendSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index 8cd302e2b4313..afbaa9ade811f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -35,7 +35,8 @@ import org.scalatest.mock.MockitoSugar import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, + TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} From 2cbb3e433ae334d5c318f05b987af314c854fbcc Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 13 Feb 2015 13:46:50 -0800 Subject: [PATCH 107/817] [SPARK-5642] [SQL] Apply column pruning on unused aggregation fields select k from (select key k, max(value) v from src group by k) t Author: Daoyuan Wang Author: Michael Armbrust Closes #4415 from adrian-wang/groupprune and squashes the following commits: 5d2d8a3 [Daoyuan Wang] address Michael's comments 61f8ef7 [Daoyuan Wang] add a unit test 80ddcc6 [Daoyuan Wang] keep project b69d385 [Daoyuan Wang] add a prune rule for grouping set --- .../sql/catalyst/optimizer/Optimizer.scala | 9 +++++ .../optimizer/FilterPushdownSuite.scala | 37 ++++++++++++++++++- 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0da081ed1a6e2..1a75fcf3545bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -119,6 +119,15 @@ object ColumnPruning extends Rule[LogicalPlan] { case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => a.copy(child = Project(a.references.toSeq, child)) + case p @ Project(projectList, a @ Aggregate(groupingExpressions, aggregateExpressions, child)) + if (a.outputSet -- p.references).nonEmpty => + Project( + projectList, + Aggregate( + groupingExpressions, + aggregateExpressions.filter(e => p.references.contains(e)), + child)) + // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of all references required either above or to evaluate the condition. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 0b74bacb18f4b..55c6766520a1e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries -import org.apache.spark.sql.catalyst.expressions.Explode +import org.apache.spark.sql.catalyst.expressions.{Count, Explode} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ @@ -37,7 +37,8 @@ class FilterPushdownSuite extends PlanTest { CombineFilters, PushPredicateThroughProject, PushPredicateThroughJoin, - PushPredicateThroughGenerate) :: Nil + PushPredicateThroughGenerate, + ColumnPruning) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -58,6 +59,38 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("column pruning for group") { + val originalQuery = + testRelation + .groupBy('a)('a, Count('b)) + .select('a) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .select('a) + .groupBy('a)('a) + .select('a).analyze + + comparePlans(optimized, correctAnswer) + } + + test("column pruning for group with alias") { + val originalQuery = + testRelation + .groupBy('a)('a as 'c, Count('b)) + .select('c) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .select('a) + .groupBy('a)('a as 'c) + .select('c).analyze + + comparePlans(optimized, correctAnswer) + } + // After this line is unimplemented. test("simple push down") { val originalQuery = From 2e0c084528409e1c565e6945521a33c0835ebbee Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 13 Feb 2015 13:51:06 -0800 Subject: [PATCH 108/817] [SPARK-5789][SQL]Throw a better error message if JsonRDD.parseJson encounters unrecoverable parsing errors. Author: Yin Huai Closes #4582 from yhuai/jsonErrorMessage and squashes the following commits: 152dbd4 [Yin Huai] Update error message. 1466256 [Yin Huai] Throw a better error message when a JSON object in the input dataset span multiple records (lines for files or strings for an RDD of strings). --- .../src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 7dfb30402136e..3b8dde1823370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -306,6 +306,10 @@ private[sql] object JsonRDD extends Logging { val parsed = mapper.readValue(record, classOf[Object]) match { case map: java.util.Map[_, _] => scalafy(map).asInstanceOf[Map[String, Any]] :: Nil case list: java.util.List[_] => scalafy(list).asInstanceOf[Seq[Map[String, Any]]] + case _ => + sys.error( + s"Failed to parse record $record. Please make sure that each line of the file " + + "(or each string in the RDD) is a valid JSON object or an array of JSON objects.") } parsed From cc56c8729a76af85aa6eb5d2f99787cca5e5b38f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 13 Feb 2015 15:09:27 -0800 Subject: [PATCH 109/817] [SPARK-5806] re-organize sections in mllib-clustering.md Put example code close to the algorithm description. Author: Xiangrui Meng Closes #4598 from mengxr/SPARK-5806 and squashes the following commits: a137872 [Xiangrui Meng] re-organize sections in mllib-clustering.md --- docs/mllib-clustering.md | 156 ++++++++++++++++++--------------------- docs/mllib-guide.md | 8 +- 2 files changed, 77 insertions(+), 87 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 0450e6fd3e07e..09b56576699e0 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -4,12 +4,6 @@ title: Clustering - MLlib displayTitle: MLlib - Clustering --- -* Table of contents -{:toc} - - -## Clustering - Clustering is an unsupervised learning problem whereby we aim to group subsets of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical @@ -18,7 +12,10 @@ models are trained for each cluster). MLlib supports the following models: -### k-means +* Table of contents +{:toc} + +## K-means [k-means](http://en.wikipedia.org/wiki/K-means_clustering) is one of the most commonly used clustering algorithms that clusters the data points into a @@ -37,72 +34,7 @@ a given dataset, the algorithm returns the best clustering result). * *initializationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. -### Gaussian mixture - -A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) -represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, -each with its own probability. The MLlib implementation uses the -[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) - algorithm to induce the maximum-likelihood model given a set of samples. The implementation -has the following parameters: - -* *k* is the number of desired clusters. -* *convergenceTol* is the maximum change in log-likelihood at which we consider convergence achieved. -* *maxIterations* is the maximum number of iterations to perform without reaching convergence. -* *initialModel* is an optional starting point from which to start the EM algorithm. If this parameter is omitted, a random starting point will be constructed from the data. - -### Power Iteration Clustering - -Power iteration clustering is a scalable and efficient algorithm for clustering points given pointwise mutual affinity values. Internally the algorithm: - -* accepts a [Graph](api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points. -* calculates the principal eigenvalue and eigenvector -* Clusters each of the input points according to their principal eigenvector component value - -Details of this algorithm are found within [Power Iteration Clustering, Lin and Cohen]{www.icml2010.org/papers/387.pdf} - -Example outputs for a dataset inspired by the paper - but with five clusters instead of three- have he following output from our implementation: - -

    - The Property Graph - -

    - -### Latent Dirichlet Allocation (LDA) - -[Latent Dirichlet Allocation (LDA)](http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation) -is a topic model which infers topics from a collection of text documents. -LDA can be thought of as a clustering algorithm as follows: - -* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset. -* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts. -* Rather than estimating a clustering using a traditional distance, LDA uses a function based - on a statistical model of how text documents are generated. - -LDA takes in a collection of documents as vectors of word counts. -It learns clustering using [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) -on the likelihood function. After fitting on the documents, LDA provides: - -* Topics: Inferred topics, each of which is a probability distribution over terms (words). -* Topic distributions for documents: For each document in the training set, LDA gives a probability distribution over topics. - -LDA takes the following parameters: - -* `k`: Number of topics (i.e., cluster centers) -* `maxIterations`: Limit on the number of iterations of EM used for learning -* `docConcentration`: Hyperparameter for prior over documents' distributions over topics. Currently must be > 1, where larger values encourage smoother inferred distributions. -* `topicConcentration`: Hyperparameter for prior over topics' distributions over terms (words). Currently must be > 1, where larger values encourage smoother inferred distributions. -* `checkpointInterval`: If using checkpointing (set in the Spark configuration), this parameter specifies the frequency with which checkpoints will be created. If `maxIterations` is large, using checkpointing can help reduce shuffle file sizes on disk and help with failure recovery. - -*Note*: LDA is a new feature with some missing functionality. In particular, it does not yet -support prediction on new documents, and it does not have a Python API. These will be added in the future. - -### Examples - -#### k-means +**Examples**
    @@ -216,7 +148,21 @@ print("Within Set Sum of Squared Error = " + str(WSSSE))
    -#### GaussianMixture +## Gaussian mixture + +A [Gaussian Mixture Model](http://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) +represents a composite distribution whereby points are drawn from one of *k* Gaussian sub-distributions, +each with its own probability. The MLlib implementation uses the +[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) + algorithm to induce the maximum-likelihood model given a set of samples. The implementation +has the following parameters: + +* *k* is the number of desired clusters. +* *convergenceTol* is the maximum change in log-likelihood at which we consider convergence achieved. +* *maxIterations* is the maximum number of iterations to perform without reaching convergence. +* *initialModel* is an optional starting point from which to start the EM algorithm. If this parameter is omitted, a random starting point will be constructed from the data. + +**Examples**
    @@ -322,7 +268,56 @@ for i in range(2):
    -#### Latent Dirichlet Allocation (LDA) Example +## Power iteration clustering (PIC) + +Power iteration clustering (PIC) is a scalable and efficient algorithm for clustering points given pointwise mutual affinity values. Internally the algorithm: + +* accepts a [Graph](api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points. +* calculates the principal eigenvalue and eigenvector +* Clusters each of the input points according to their principal eigenvector component value + +Details of this algorithm are found within [Power Iteration Clustering, Lin and Cohen]{www.icml2010.org/papers/387.pdf} + +Example outputs for a dataset inspired by the paper - but with five clusters instead of three- have he following output from our implementation: + +

    + The Property Graph + +

    + +## Latent Dirichlet allocation (LDA) + +[Latent Dirichlet allocation (LDA)](http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation) +is a topic model which infers topics from a collection of text documents. +LDA can be thought of as a clustering algorithm as follows: + +* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset. +* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts. +* Rather than estimating a clustering using a traditional distance, LDA uses a function based + on a statistical model of how text documents are generated. + +LDA takes in a collection of documents as vectors of word counts. +It learns clustering using [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +on the likelihood function. After fitting on the documents, LDA provides: + +* Topics: Inferred topics, each of which is a probability distribution over terms (words). +* Topic distributions for documents: For each document in the training set, LDA gives a probability distribution over topics. + +LDA takes the following parameters: + +* `k`: Number of topics (i.e., cluster centers) +* `maxIterations`: Limit on the number of iterations of EM used for learning +* `docConcentration`: Hyperparameter for prior over documents' distributions over topics. Currently must be > 1, where larger values encourage smoother inferred distributions. +* `topicConcentration`: Hyperparameter for prior over topics' distributions over terms (words). Currently must be > 1, where larger values encourage smoother inferred distributions. +* `checkpointInterval`: If using checkpointing (set in the Spark configuration), this parameter specifies the frequency with which checkpoints will be created. If `maxIterations` is large, using checkpointing can help reduce shuffle file sizes on disk and help with failure recovery. + +*Note*: LDA is a new feature with some missing functionality. In particular, it does not yet +support prediction on new documents, and it does not have a Python API. These will be added in the future. + +**Examples** In the following example, we load word count vectors representing a corpus of documents. We then use [LDA](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) @@ -419,14 +414,7 @@ public class JavaLDAExample {
    - -In order to run the above application, follow the instructions -provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) -section of the Spark -Quick Start guide. Be sure to also include *spark-mllib* to your build file as -a dependency. - -## Streaming clustering +## Streaming k-means When data arrive in a stream, we may want to estimate clusters dynamically, updating them as new data arrive. MLlib provides support for streaming k-means clustering, @@ -454,7 +442,7 @@ at time `t`, its contribution by time `t + halfLife` will have dropped to 0.5. The unit of time can be specified either as `batches` or `points` and the update rule will be adjusted accordingly. -### Examples +**Examples** This example shows how to estimate clusters on streaming data. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 3d32d03e35c62..5091dbf329b97 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -24,9 +24,11 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) * [Clustering](mllib-clustering.html) - * k-means - * Gaussian mixture - * power iteration + * [k-means](mllib-clustering.html#k-means) + * [Gaussian mixture](mllib-clustering.html#gaussian-mixture) + * [power iteration clustering (PIC)](mllib-clustering.html#power-iteration-clustering-pic) + * [latent Dirichlet allocation (LDA)](mllib-clustering.html#latent-dirichlet-allocation-lda) + * [streaming k-means](mllib-clustering.html#streaming-k-means) * [Dimensionality reduction](mllib-dimensionality-reduction.html) * singular value decomposition (SVD) * principal component analysis (PCA) From d50a91d529b0913364b483c511397d4af308a435 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 13 Feb 2015 16:43:49 -0800 Subject: [PATCH 110/817] [SPARK-5803][MLLIB] use ArrayBuilder to build primitive arrays because ArrayBuffer is not specialized. Author: Xiangrui Meng Closes #4594 from mengxr/SPARK-5803 and squashes the following commits: 1261bd5 [Xiangrui Meng] merge master a4ea872 [Xiangrui Meng] use ArrayBuilder to build primitive arrays --- .../org/apache/spark/mllib/feature/Word2Vec.scala | 6 +++--- .../org/apache/spark/mllib/tree/DecisionTree.scala | 10 +++++----- .../org/apache/spark/mllib/util/NumericParser.scala | 10 +++++----- 3 files changed, 13 insertions(+), 13 deletions(-) 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 a3e40200bc063..59a79e5c6a4ac 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 @@ -21,7 +21,7 @@ import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -272,7 +272,7 @@ class Word2Vec extends Serializable with Logging { def hasNext: Boolean = iter.hasNext def next(): Array[Int] = { - var sentence = new ArrayBuffer[Int] + val sentence = ArrayBuilder.make[Int] var sentenceLength = 0 while (iter.hasNext && sentenceLength < MAX_SENTENCE_LENGTH) { val word = bcVocabHash.value.get(iter.next()) @@ -283,7 +283,7 @@ class Word2Vec extends Serializable with Logging { case None => } } - sentence.toArray + sentence.result() } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 9a586b9d9c68e..f1f85994e61b2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.tree -import scala.collection.mutable import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable +import scala.collection.mutable.ArrayBuilder import org.apache.spark.Logging import org.apache.spark.annotation.Experimental @@ -1136,7 +1136,7 @@ object DecisionTree extends Serializable with Logging { logDebug("stride = " + stride) // iterate `valueCount` to find splits - val splits = new ArrayBuffer[Double] + val splitsBuilder = ArrayBuilder.make[Double] var index = 1 // currentCount: sum of counts of values that have been visited var currentCount = valueCounts(0)._2 @@ -1154,13 +1154,13 @@ object DecisionTree extends Serializable with Logging { // makes the gap between currentCount and targetCount smaller, // previous value is a split threshold. if (previousGap < currentGap) { - splits.append(valueCounts(index - 1)._1) + splitsBuilder += valueCounts(index - 1)._1 targetCount += stride } index += 1 } - splits.toArray + splitsBuilder.result() } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala index f7cba6c6cb628..308f7f3578e21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/NumericParser.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import java.util.StringTokenizer -import scala.collection.mutable.{ArrayBuffer, ListBuffer} +import scala.collection.mutable.{ArrayBuilder, ListBuffer} import org.apache.spark.SparkException @@ -51,7 +51,7 @@ private[mllib] object NumericParser { } private def parseArray(tokenizer: StringTokenizer): Array[Double] = { - val values = ArrayBuffer.empty[Double] + val values = ArrayBuilder.make[Double] var parsing = true var allowComma = false var token: String = null @@ -67,14 +67,14 @@ private[mllib] object NumericParser { } } else { // expecting a number - values.append(parseDouble(token)) + values += parseDouble(token) allowComma = true } } if (parsing) { throw new SparkException(s"An array must end with ']'.") } - values.toArray + values.result() } private def parseTuple(tokenizer: StringTokenizer): Seq[_] = { @@ -114,7 +114,7 @@ private[mllib] object NumericParser { try { java.lang.Double.parseDouble(s) } catch { - case e: Throwable => + case e: NumberFormatException => throw new SparkException(s"Cannot parse a double from: $s", e) } } From 4f4c6d5a5db04a56906bacdc85d7e5589b6edada Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 13 Feb 2015 16:45:59 -0800 Subject: [PATCH 111/817] [SPARK-5730][ML] add doc groups to spark.ml components This PR adds three groups to the ScalaDoc: `param`, `setParam`, and `getParam`. Params will show up in the generated Scala API doc as the top group. Setters/getters will be at the bottom. Preview: ![screen shot 2015-02-13 at 2 47 49 pm](https://cloud.githubusercontent.com/assets/829644/6196657/5740c240-b38f-11e4-94bb-bd8ef5a796c5.png) Author: Xiangrui Meng Closes #4600 from mengxr/SPARK-5730 and squashes the following commits: febed9a [Xiangrui Meng] add doc groups to spark.ml components --- .../org/apache/spark/ml/Transformer.scala | 3 + .../spark/ml/classification/Classifier.scala | 2 + .../classification/LogisticRegression.scala | 6 ++ .../ProbabilisticClassifier.scala | 2 + .../BinaryClassificationEvaluator.scala | 12 ++- .../apache/spark/ml/feature/HashingTF.scala | 11 ++- .../spark/ml/feature/StandardScaler.scala | 6 ++ .../spark/ml/impl/estimator/Predictor.scala | 7 ++ .../scala/org/apache/spark/ml/package.scala | 14 +++ .../apache/spark/ml/param/sharedParams.scala | 70 ++++++++++++--- .../apache/spark/ml/recommendation/ALS.scala | 90 +++++++++++++++++-- .../ml/regression/LinearRegression.scala | 3 + .../spark/ml/tuning/CrossValidator.scala | 35 +++++++- 13 files changed, 235 insertions(+), 26 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index cd95c16aa768d..2ec2ccdb8c8b3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -62,7 +62,10 @@ abstract class Transformer extends PipelineStage with Params { private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] extends Transformer with HasInputCol with HasOutputCol with Logging { + /** @group setParam */ def setInputCol(value: String): T = set(inputCol, value).asInstanceOf[T] + + /** @group setParam */ def setOutputCol(value: String): T = set(outputCol, value).asInstanceOf[T] /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 1bf8eb4640d11..124ab30f279c1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -66,6 +66,7 @@ private[spark] abstract class Classifier[ extends Predictor[FeaturesType, E, M] with ClassifierParams { + /** @group setParam */ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] @@ -87,6 +88,7 @@ private[spark] abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] extends PredictionModel[FeaturesType, M] with ClassifierParams { + /** @group setParam */ def setRawPredictionCol(value: String): M = set(rawPredictionCol, value).asInstanceOf[M] /** Number of classes (values which the label can take). */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index c146fe244c66e..a9a5af5f0f2d7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -49,8 +49,13 @@ class LogisticRegression setMaxIter(100) setThreshold(0.5) + /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) + + /** @group setParam */ def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) override protected def train(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { @@ -93,6 +98,7 @@ class LogisticRegressionModel private[ml] ( setThreshold(0.5) + /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) private val margin: Vector => Double = (features) => { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 1202528ca654e..38518785dceae 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -61,6 +61,7 @@ private[spark] abstract class ProbabilisticClassifier[ M <: ProbabilisticClassificationModel[FeaturesType, M]] extends Classifier[FeaturesType, E, M] with ProbabilisticClassifierParams { + /** @group setParam */ def setProbabilityCol(value: String): E = set(probabilityCol, value).asInstanceOf[E] } @@ -82,6 +83,7 @@ private[spark] abstract class ProbabilisticClassificationModel[ M <: ProbabilisticClassificationModel[FeaturesType, M]] extends ClassificationModel[FeaturesType, M] with ProbabilisticClassifierParams { + /** @group setParam */ def setProbabilityCol(value: String): M = set(probabilityCol, value).asInstanceOf[M] /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index f21a30627e540..2360f4479f1c2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -35,13 +35,23 @@ import org.apache.spark.sql.types.DoubleType class BinaryClassificationEvaluator extends Evaluator with Params with HasRawPredictionCol with HasLabelCol { - /** param for metric name in evaluation */ + /** + * param for metric name in evaluation + * @group param + */ val metricName: Param[String] = new Param(this, "metricName", "metric name in evaluation (areaUnderROC|areaUnderPR)", Some("areaUnderROC")) + + /** @group getParam */ def getMetricName: String = get(metricName) + + /** @group setParam */ def setMetricName(value: String): this.type = set(metricName, value) + /** @group setParam */ def setScoreCol(value: String): this.type = set(rawPredictionCol, value) + + /** @group setParam */ def setLabelCol(value: String): this.type = set(labelCol, value) override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index 0956062643f23..6131ba8832691 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -31,11 +31,18 @@ import org.apache.spark.sql.types.DataType @AlphaComponent class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { - /** number of features */ + /** + * number of features + * @group param + */ val numFeatures = new IntParam(this, "numFeatures", "number of features", Some(1 << 18)) - def setNumFeatures(value: Int) = set(numFeatures, value) + + /** @group getParam */ def getNumFeatures: Int = get(numFeatures) + /** @group setParam */ + def setNumFeatures(value: Int) = set(numFeatures, value) + override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 4745a7ae95679..7623ec59ae0eb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -39,7 +39,10 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with @AlphaComponent class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerParams { + /** @group setParam */ def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { @@ -75,7 +78,10 @@ class StandardScalerModel private[ml] ( scaler: feature.StandardScalerModel) extends Model[StandardScalerModel] with StandardScalerParams { + /** @group setParam */ def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 89b53f3890ea3..e416c1eb585fc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -85,8 +85,13 @@ private[spark] abstract class Predictor[ M <: PredictionModel[FeaturesType, M]] extends Estimator[M] with PredictorParams { + /** @group setParam */ def setLabelCol(value: String): Learner = set(labelCol, value).asInstanceOf[Learner] + + /** @group setParam */ def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] + + /** @group setParam */ def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] override def fit(dataset: DataFrame, paramMap: ParamMap): M = { @@ -160,8 +165,10 @@ private[spark] abstract class Predictor[ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] extends Model[M] with PredictorParams { + /** @group setParam */ def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] + /** @group setParam */ def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/package.scala b/mllib/src/main/scala/org/apache/spark/ml/package.scala index 51cd48c90432a..b45bd1499b72e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -20,5 +20,19 @@ package org.apache.spark /** * Spark ML is an ALPHA component that adds a new set of machine learning APIs to let users quickly * assemble and configure practical machine learning pipelines. + * + * @groupname param Parameters + * @groupdesc param A list of (hyper-)parameter keys this algorithm can take. Users can set and get + * the parameter values through setters and getters, respectively. + * @groupprio param -5 + * + * @groupname setParam Parameter setters + * @groupprio setParam 5 + * + * @groupname getParam Parameter getters + * @groupprio getParam 6 + * + * @groupname Ungrouped Members + * @groupprio Ungrouped 0 */ package object ml diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 32fc74462ef4a..1a70322b4cace 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -24,67 +24,117 @@ package org.apache.spark.ml.param */ private[ml] trait HasRegParam extends Params { - /** param for regularization parameter */ + /** + * param for regularization parameter + * @group param + */ val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") + + /** @group getParam */ def getRegParam: Double = get(regParam) } private[ml] trait HasMaxIter extends Params { - /** param for max number of iterations */ + /** + * param for max number of iterations + * @group param + */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") + + /** @group getParam */ def getMaxIter: Int = get(maxIter) } private[ml] trait HasFeaturesCol extends Params { - /** param for features column name */ + /** + * param for features column name + * @group param + */ val featuresCol: Param[String] = new Param(this, "featuresCol", "features column name", Some("features")) + + /** @group getParam */ def getFeaturesCol: String = get(featuresCol) } private[ml] trait HasLabelCol extends Params { - /** param for label column name */ + /** + * param for label column name + * @group param + */ val labelCol: Param[String] = new Param(this, "labelCol", "label column name", Some("label")) + + /** @group getParam */ def getLabelCol: String = get(labelCol) } private[ml] trait HasPredictionCol extends Params { - /** param for prediction column name */ + /** + * param for prediction column name + * @group param + */ val predictionCol: Param[String] = new Param(this, "predictionCol", "prediction column name", Some("prediction")) + + /** @group getParam */ def getPredictionCol: String = get(predictionCol) } private[ml] trait HasRawPredictionCol extends Params { - /** param for raw prediction column name */ + /** + * param for raw prediction column name + * @group param + */ val rawPredictionCol: Param[String] = new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", Some("rawPrediction")) + + /** @group getParam */ def getRawPredictionCol: String = get(rawPredictionCol) } private[ml] trait HasProbabilityCol extends Params { - /** param for predicted class conditional probabilities column name */ + /** + * param for predicted class conditional probabilities column name + * @group param + */ val probabilityCol: Param[String] = new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", Some("probability")) + + /** @group getParam */ def getProbabilityCol: String = get(probabilityCol) } private[ml] trait HasThreshold extends Params { - /** param for threshold in (binary) prediction */ + /** + * param for threshold in (binary) prediction + * @group param + */ val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") + + /** @group getParam */ def getThreshold: Double = get(threshold) } private[ml] trait HasInputCol extends Params { - /** param for input column name */ + /** + * param for input column name + * @group param + */ val inputCol: Param[String] = new Param(this, "inputCol", "input column name") + + /** @group getParam */ def getInputCol: String = get(inputCol) } private[ml] trait HasOutputCol extends Params { - /** param for output column name */ + /** + * param for output column name + * @group param + */ val outputCol: Param[String] = new Param(this, "outputCol", "output column name") + + /** @group getParam */ def getOutputCol: String = get(outputCol) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index bf5737177ceee..aac487745f77d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -49,43 +49,89 @@ import org.apache.spark.util.random.XORShiftRandom private[recommendation] trait ALSParams extends Params with HasMaxIter with HasRegParam with HasPredictionCol { - /** Param for rank of the matrix factorization. */ + /** + * Param for rank of the matrix factorization. + * @group param + */ val rank = new IntParam(this, "rank", "rank of the factorization", Some(10)) + + /** @group getParam */ def getRank: Int = get(rank) - /** Param for number of user blocks. */ + /** + * Param for number of user blocks. + * @group param + */ val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", Some(10)) + + /** @group getParam */ def getNumUserBlocks: Int = get(numUserBlocks) - /** Param for number of item blocks. */ + /** + * Param for number of item blocks. + * @group param + */ val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", Some(10)) + + /** @group getParam */ def getNumItemBlocks: Int = get(numItemBlocks) - /** Param to decide whether to use implicit preference. */ + /** + * Param to decide whether to use implicit preference. + * @group param + */ val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference", Some(false)) + + /** @group getParam */ def getImplicitPrefs: Boolean = get(implicitPrefs) - /** Param for the alpha parameter in the implicit preference formulation. */ + /** + * Param for the alpha parameter in the implicit preference formulation. + * @group param + */ val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", Some(1.0)) + + /** @group getParam */ def getAlpha: Double = get(alpha) - /** Param for the column name for user ids. */ + /** + * Param for the column name for user ids. + * @group param + */ val userCol = new Param[String](this, "userCol", "column name for user ids", Some("user")) + + /** @group getParam */ def getUserCol: String = get(userCol) - /** Param for the column name for item ids. */ + /** + * Param for the column name for item ids. + * @group param + */ val itemCol = new Param[String](this, "itemCol", "column name for item ids", Some("item")) + + /** @group getParam */ def getItemCol: String = get(itemCol) - /** Param for the column name for ratings. */ + /** + * Param for the column name for ratings. + * @group param + */ val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) + + /** @group getParam */ def getRatingCol: String = get(ratingCol) + /** + * Param for whether to apply nonnegativity constraints. + * @group param + */ val nonnegative = new BooleanParam( this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + + /** @group getParam */ val getNonnegative: Boolean = get(nonnegative) /** @@ -181,20 +227,46 @@ class ALS extends Estimator[ALSModel] with ALSParams { import org.apache.spark.ml.recommendation.ALS.Rating + /** @group setParam */ def setRank(value: Int): this.type = set(rank, value) + + /** @group setParam */ def setNumUserBlocks(value: Int): this.type = set(numUserBlocks, value) + + /** @group setParam */ def setNumItemBlocks(value: Int): this.type = set(numItemBlocks, value) + + /** @group setParam */ def setImplicitPrefs(value: Boolean): this.type = set(implicitPrefs, value) + + /** @group setParam */ def setAlpha(value: Double): this.type = set(alpha, value) + + /** @group setParam */ def setUserCol(value: String): this.type = set(userCol, value) + + /** @group setParam */ def setItemCol(value: String): this.type = set(itemCol, value) + + /** @group setParam */ def setRatingCol(value: String): this.type = set(ratingCol, value) + + /** @group setParam */ def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) + + /** @group setParam */ def setNonnegative(value: Boolean): this.type = set(nonnegative, value) - /** Sets both numUserBlocks and numItemBlocks to the specific value. */ + /** + * Sets both numUserBlocks and numItemBlocks to the specific value. + * @group setParam + */ def setNumBlocks(value: Int): this.type = { setNumUserBlocks(value) setNumItemBlocks(value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index d5a7bdafcb623..65f6627a0c351 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -44,7 +44,10 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress setRegParam(0.1) setMaxIter(100) + /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) + + /** @group setParam */ def setMaxIter(value: Int): this.type = set(maxIter, value) override protected def train(dataset: DataFrame, paramMap: ParamMap): LinearRegressionModel = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 324b1ba784387..b139bc8dcb44c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -31,22 +31,42 @@ import org.apache.spark.sql.types.StructType * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends Params { - /** param for the estimator to be cross-validated */ + /** + * param for the estimator to be cross-validated + * @group param + */ val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") + + /** @group getParam */ def getEstimator: Estimator[_] = get(estimator) - /** param for estimator param maps */ + /** + * param for estimator param maps + * @group param + */ val estimatorParamMaps: Param[Array[ParamMap]] = new Param(this, "estimatorParamMaps", "param maps for the estimator") + + /** @group getParam */ def getEstimatorParamMaps: Array[ParamMap] = get(estimatorParamMaps) - /** param for the evaluator for selection */ + /** + * param for the evaluator for selection + * @group param + */ val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") + + /** @group getParam */ def getEvaluator: Evaluator = get(evaluator) - /** param for number of folds for cross validation */ + /** + * param for number of folds for cross validation + * @group param + */ val numFolds: IntParam = new IntParam(this, "numFolds", "number of folds for cross validation", Some(3)) + + /** @group getParam */ def getNumFolds: Int = get(numFolds) } @@ -59,9 +79,16 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP private val f2jBLAS = new F2jBLAS + /** @group setParam */ def setEstimator(value: Estimator[_]): this.type = set(estimator, value) + + /** @group setParam */ def setEstimatorParamMaps(value: Array[ParamMap]): this.type = set(estimatorParamMaps, value) + + /** @group setParam */ def setEvaluator(value: Evaluator): this.type = set(evaluator, value) + + /** @group setParam */ def setNumFolds(value: Int): this.type = set(numFolds, value) override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = { From d06d5ee9b33505774ef1e5becc01b47492f1a2dc Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 13 Feb 2015 17:45:31 -0800 Subject: [PATCH 112/817] [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite This patch fixes two difficult-to-reproduce Jenkins test failures in InputOutputMetricsSuite (SPARK-5227 and SPARK-5679). The problem was that WholeTextFileRecordReaderSuite modifies the `fs.local.block.size` Hadoop configuration and this change was affecting subsequent test suites due to Hadoop's caching of FileSystem instances (see HADOOP-8490 for more details). The fix implemented here is to disable FileSystem caching in WholeTextFileRecordReaderSuite. Author: Josh Rosen Closes #4599 from JoshRosen/inputoutputsuite-fix and squashes the following commits: 47dc447 [Josh Rosen] [SPARK-5227] [SPARK-5679] Disable FileSystem cache in WholeTextFileRecordReaderSuite --- .../spark/input/WholeTextFileRecordReaderSuite.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 98b0a16ce88ba..2e58c159a2ed8 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.util.Utils import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec} @@ -42,7 +42,15 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { private var factory: CompressionCodecFactory = _ override def beforeAll() { - sc = new SparkContext("local", "test") + // Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which + // can cause Filesystem.get(Configuration) to return a cached instance created with a different + // configuration than the one passed to get() (see HADOOP-8490 for more details). This caused + // hard-to-reproduce test failures, since any suites that were run after this one would inherit + // the new value of "fs.local.block.size" (see SPARK-5227 and SPARK-5679). To work around this, + // we disable FileSystem caching in this suite. + val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true") + + sc = new SparkContext("local", "test", conf) // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) From 0ce4e430a81532dc317136f968f28742e087d840 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 13 Feb 2015 20:12:52 -0800 Subject: [PATCH 113/817] SPARK-3290 [GRAPHX] No unpersist callls in SVDPlusPlus This just unpersist()s each RDD in this code that was cache()ed. Author: Sean Owen Closes #4234 from srowen/SPARK-3290 and squashes the following commits: 66c1e11 [Sean Owen] unpersist() each RDD that was cache()ed --- .../apache/spark/graphx/lib/SVDPlusPlus.scala | 40 +++++++++++++++---- 1 file changed, 32 insertions(+), 8 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index f58587e10a820..112ed09ef46dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -72,17 +72,22 @@ object SVDPlusPlus { // construct graph var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() + materialize(g) + edges.unpersist() // Calculate initial bias and norm val t0 = g.aggregateMessages[(Long, Double)]( ctx => { ctx.sendToSrc((1L, ctx.attr)); ctx.sendToDst((1L, ctx.attr)) }, (g1, g2) => (g1._1 + g2._1, g1._2 + g2._2)) - g = g.outerJoinVertices(t0) { + val gJoinT0 = g.outerJoinVertices(t0) { (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[(Long, Double)]) => (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) - } + }.cache() + materialize(gJoinT0) + g.unpersist() + g = gJoinT0 def sendMsgTrainF(conf: Conf, u: Double) (ctx: EdgeContext[ @@ -114,12 +119,15 @@ object SVDPlusPlus { val t1 = g.aggregateMessages[DoubleMatrix]( ctx => ctx.sendToSrc(ctx.dstAttr._2), (g1, g2) => g1.addColumnVector(g2)) - g = g.outerJoinVertices(t1) { + val gJoinT1 = g.outerJoinVertices(t1) { (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[DoubleMatrix]) => if (msg.isDefined) (vd._1, vd._1 .addColumnVector(msg.get.mul(vd._4)), vd._3, vd._4) else vd - } + }.cache() + materialize(gJoinT1) + g.unpersist() + g = gJoinT1 // Phase 2, update p for user nodes and q, y for item nodes g.cache() @@ -127,13 +135,16 @@ object SVDPlusPlus { sendMsgTrainF(conf, u), (g1: (DoubleMatrix, DoubleMatrix, Double), g2: (DoubleMatrix, DoubleMatrix, Double)) => (g1._1.addColumnVector(g2._1), g1._2.addColumnVector(g2._2), g1._3 + g2._3)) - g = g.outerJoinVertices(t2) { + val gJoinT2 = g.outerJoinVertices(t2) { (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[(DoubleMatrix, DoubleMatrix, Double)]) => (vd._1.addColumnVector(msg.get._1), vd._2.addColumnVector(msg.get._2), vd._3 + msg.get._3, vd._4) - } + }.cache() + materialize(gJoinT2) + g.unpersist() + g = gJoinT2 } // calculate error on training set @@ -147,13 +158,26 @@ object SVDPlusPlus { val err = (ctx.attr - pred) * (ctx.attr - pred) ctx.sendToDst(err) } + g.cache() val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _) - g = g.outerJoinVertices(t3) { + val gJoinT3 = g.outerJoinVertices(t3) { (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd - } + }.cache() + materialize(gJoinT3) + g.unpersist() + g = gJoinT3 (g, u) } + + /** + * Forces materialization of a Graph by count()ing its RDDs. + */ + private def materialize(g: Graph[_,_]): Unit = { + g.vertices.count() + g.edges.count() + } + } From e98dfe627c5d0201464cdd0f363f391ea84c389a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 13 Feb 2015 23:03:22 -0800 Subject: [PATCH 114/817] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin Author: Davies Liu Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames --- .../examples/ml/CrossValidatorExample.scala | 4 +- .../examples/ml/DeveloperApiExample.scala | 4 +- .../spark/examples/ml/MovieLensALS.scala | 6 +- .../examples/ml/SimpleParamsExample.scala | 6 +- .../ml/SimpleTextClassificationPipeline.scala | 4 +- .../spark/examples/mllib/DatasetExample.scala | 8 +- .../spark/examples/sql/RDDRelation.scala | 10 +- .../examples/sql/hive/HiveFromSpark.scala | 2 +- .../org/apache/spark/ml/Transformer.scala | 6 +- .../spark/ml/classification/Classifier.scala | 16 +- .../classification/LogisticRegression.scala | 33 ++- .../ProbabilisticClassifier.scala | 6 +- .../spark/ml/feature/StandardScaler.scala | 4 +- .../spark/ml/impl/estimator/Predictor.scala | 4 +- .../apache/spark/ml/recommendation/ALS.scala | 6 +- .../mllib/classification/NaiveBayes.scala | 2 +- .../impl/GLMClassificationModel.scala | 2 +- .../MatrixFactorizationModel.scala | 4 +- .../regression/impl/GLMRegressionModel.scala | 2 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../mllib/tree/model/treeEnsembleModels.scala | 2 +- .../spark/ml/recommendation/ALSSuite.scala | 4 +- python/docs/pyspark.sql.rst | 8 + python/pyspark/mllib/tests.py | 2 +- python/pyspark/sql/__init__.py | 3 +- python/pyspark/sql/context.py | 34 ++- python/pyspark/sql/dataframe.py | 221 ++++-------------- python/pyspark/sql/functions.py | 170 ++++++++++++++ python/pyspark/sql/tests.py | 38 ++- python/run-tests | 3 +- .../apache/spark/repl/SparkILoopInit.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 2 +- .../org/apache/spark/repl/SparkILoop.scala | 2 +- .../sql/catalyst/analysis/unresolved.scala | 2 +- .../scala/org/apache/spark/sql/Column.scala | 21 +- .../org/apache/spark/sql/DataFrame.scala | 25 +- .../apache/spark/sql/DataFrameHolder.scala | 30 +++ .../org/apache/spark/sql/DataFrameImpl.scala | 6 +- .../org/apache/spark/sql/GroupedData.scala | 19 +- .../apache/spark/sql/IncomputableColumn.scala | 6 +- .../org/apache/spark/sql/SQLContext.scala | 35 ++- .../spark/sql/UserDefinedFunction.scala | 4 +- .../spark/sql/{Dsl.scala => functions.scala} | 21 +- .../spark/sql/parquet/ParquetTest.scala | 2 +- .../apache/spark/sql/api/java/JavaDsl.java | 2 +- .../apache/spark/sql/CachedTableSuite.scala | 7 +- .../spark/sql/ColumnExpressionSuite.scala | 10 +- .../spark/sql/DataFrameImplicitsSuite.scala | 10 +- .../org/apache/spark/sql/DataFrameSuite.scala | 51 ++-- .../org/apache/spark/sql/JoinSuite.scala | 3 +- .../apache/spark/sql/ListTablesSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../sql/ScalaReflectionRelationSuite.scala | 10 +- .../scala/org/apache/spark/sql/TestData.scala | 46 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 2 +- .../spark/sql/UserDefinedTypeSuite.scala | 4 +- .../columnar/InMemoryColumnarQuerySuite.scala | 8 +- .../columnar/PartitionBatchPruningSuite.scala | 5 +- .../spark/sql/execution/PlannerSuite.scala | 3 +- .../org/apache/spark/sql/json/JsonSuite.scala | 7 +- .../spark/sql/parquet/ParquetIOSuite.scala | 6 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 8 +- .../spark/sql/hive/ListTablesSuite.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 17 +- .../sql/hive/execution/HiveQuerySuite.scala | 12 +- .../hive/execution/HiveResolutionSuite.scala | 6 +- .../hive/execution/HiveTableScanSuite.scala | 3 +- .../sql/hive/execution/HiveUdfSuite.scala | 10 +- .../sql/hive/execution/SQLQuerySuite.scala | 11 +- .../spark/sql/parquet/parquetSuites.scala | 6 +- 70 files changed, 596 insertions(+), 456 deletions(-) create mode 100644 python/pyspark/sql/functions.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala rename sql/core/src/main/scala/org/apache/spark/sql/{Dsl.scala => functions.scala} (97%) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index a2893f78e0fec..f0241943ef410 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -90,7 +90,7 @@ object CrossValidatorExample { crossval.setNumFolds(2) // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. - val cvModel = crossval.fit(training) + val cvModel = crossval.fit(training.toDF) // Prepare test documents, which are unlabeled. val test = sc.parallelize(Seq( @@ -100,7 +100,7 @@ object CrossValidatorExample { Document(7L, "apache hadoop"))) // Make predictions on test documents. cvModel uses the best model found (lrModel). - cvModel.transform(test) + cvModel.transform(test.toDF) .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index aed44238939c7..54aadd2288817 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -58,7 +58,7 @@ object DeveloperApiExample { lr.setMaxIter(10) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model = lr.fit(training) + val model = lr.fit(training.toDF) // Prepare test data. val test = sc.parallelize(Seq( @@ -67,7 +67,7 @@ object DeveloperApiExample { LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test) + val sumPredictions: Double = model.transform(test.toDF) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala index 836ea2e01201e..adaf796dc1896 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala @@ -137,9 +137,9 @@ object MovieLensALS { .setRegParam(params.regParam) .setNumBlocks(params.numBlocks) - val model = als.fit(training) + val model = als.fit(training.toDF) - val predictions = model.transform(test).cache() + val predictions = model.transform(test.toDF).cache() // Evaluate the model. // TODO: Create an evaluator to compute RMSE. @@ -158,7 +158,7 @@ object MovieLensALS { // Inspect false positives. predictions.registerTempTable("prediction") - sc.textFile(params.movies).map(Movie.parseMovie).registerTempTable("movie") + sc.textFile(params.movies).map(Movie.parseMovie).toDF.registerTempTable("movie") sqlContext.sql( """ |SELECT userId, prediction.movieId, title, rating, prediction diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 80c9f5ff5781e..c5bb5515b1930 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -58,7 +58,7 @@ object SimpleParamsExample { .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model1 = lr.fit(training) + val model1 = lr.fit(training.toDF) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -77,7 +77,7 @@ object SimpleParamsExample { // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. - val model2 = lr.fit(training, paramMapCombined) + val model2 = lr.fit(training.toDF, paramMapCombined) println("Model 2 was fit using parameters: " + model2.fittingParamMap) // Prepare test data. @@ -90,7 +90,7 @@ object SimpleParamsExample { // LogisticRegression.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test) + model2.transform(test.toDF) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 968cb292120d8..8b47f88e48df1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -69,7 +69,7 @@ object SimpleTextClassificationPipeline { .setStages(Array(tokenizer, hashingTF, lr)) // Fit the pipeline to training documents. - val model = pipeline.fit(training) + val model = pipeline.fit(training.toDF) // Prepare test documents, which are unlabeled. val test = sc.parallelize(Seq( @@ -79,7 +79,7 @@ object SimpleTextClassificationPipeline { Document(7L, "apache hadoop"))) // Make predictions on test documents. - model.transform(test) + model.transform(test.toDF) .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala index 89b6255991a38..c98c68a02f2be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala @@ -81,18 +81,18 @@ object DatasetExample { println(s"Loaded ${origData.count()} instances from file: ${params.input}") // Convert input data to DataFrame explicitly. - val df: DataFrame = origData.toDataFrame + val df: DataFrame = origData.toDF println(s"Inferred schema:\n${df.schema.prettyJson}") println(s"Converted to DataFrame with ${df.count()} records") - // Select columns, using implicit conversion to DataFrames. - val labelsDf: DataFrame = origData.select("label") + // Select columns + val labelsDf: DataFrame = df.select("label") val labels: RDD[Double] = labelsDf.map { case Row(v: Double) => v } val numLabels = labels.count() val meanLabel = labels.fold(0.0)(_ + _) / numLabels println(s"Selected label column with average value $meanLabel") - val featuresDf: DataFrame = origData.select("features") + val featuresDf: DataFrame = df.select("features") val features: RDD[Vector] = featuresDf.map { case Row(v: Vector) => v } val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( (summary, feat) => summary.add(feat), diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 1eac3c8d03e39..79d3d5a24ceaf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.sql import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ // One method for defining the schema of an RDD is to make a case class with the desired column // names and types. @@ -34,10 +34,10 @@ object RDDRelation { // Importing the SQL context gives access to all the SQL functions and implicit conversions. import sqlContext.implicits._ - val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) + val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. - rdd.registerTempTable("records") + df.registerTempTable("records") // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") @@ -55,10 +55,10 @@ object RDDRelation { rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) // Queries can also be written using a LINQ-like Scala DSL. - rdd.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println) + df.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println) // Write out an RDD as a parquet file. - rdd.saveAsParquetFile("pair.parquet") + df.saveAsParquetFile("pair.parquet") // Read in parquet file. Parquet files are self-describing so the schmema is preserved. val parquetFile = sqlContext.parquetFile("pair.parquet") diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 15754cdfcc35e..7128deba54da7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -68,7 +68,7 @@ object HiveFromSpark { // You can also register RDDs as temporary tables within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.registerTempTable("records") + rdd.toDF.registerTempTable("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 2ec2ccdb8c8b3..9a5848684b179 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -23,7 +23,7 @@ import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** @@ -100,7 +100,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap - dataset.select($"*", callUDF( - this.createTransformFunc(map), outputDataType, dataset(map(inputCol))).as(map(outputCol))) + dataset.withColumn(map(outputCol), + callUDF(this.createTransformFunc(map), outputDataType, dataset(map(inputCol)))) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 124ab30f279c1..c5fc89f935432 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -182,24 +182,22 @@ private[ml] object ClassificationModel { if (map(model.rawPredictionCol) != "") { // output raw prediction val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.select($"*", - callUDF(features2raw, new VectorUDT, - col(map(model.featuresCol))).as(map(model.rawPredictionCol))) + tmpData = tmpData.withColumn(map(model.rawPredictionCol), + callUDF(features2raw, new VectorUDT, col(map(model.featuresCol)))) numColsOutput += 1 if (map(model.predictionCol) != "") { val raw2pred: Vector => Double = (rawPred) => { rawPred.toArray.zipWithIndex.maxBy(_._1)._2 } - tmpData = tmpData.select($"*", callUDF(raw2pred, DoubleType, - col(map(model.rawPredictionCol))).as(map(model.predictionCol))) + tmpData = tmpData.withColumn(map(model.predictionCol), + callUDF(raw2pred, DoubleType, col(map(model.rawPredictionCol)))) numColsOutput += 1 } } else if (map(model.predictionCol) != "") { // output prediction val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.select($"*", - callUDF(features2pred, DoubleType, - col(map(model.featuresCol))).as(map(model.predictionCol))) + tmpData = tmpData.withColumn(map(model.predictionCol), + callUDF(features2pred, DoubleType, col(map(model.featuresCol)))) numColsOutput += 1 } (numColsOutput, tmpData) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index a9a5af5f0f2d7..21f61d80dd95a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -22,7 +22,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -130,44 +130,39 @@ class LogisticRegressionModel private[ml] ( var numColsOutput = 0 if (map(rawPredictionCol) != "") { val features2raw: Vector => Vector = (features) => predictRaw(features) - tmpData = tmpData.select($"*", - callUDF(features2raw, new VectorUDT, col(map(featuresCol))).as(map(rawPredictionCol))) + tmpData = tmpData.withColumn(map(rawPredictionCol), + callUDF(features2raw, new VectorUDT, col(map(featuresCol)))) numColsOutput += 1 } if (map(probabilityCol) != "") { if (map(rawPredictionCol) != "") { - val raw2prob: Vector => Vector = { (rawPreds: Vector) => + val raw2prob = udf { (rawPreds: Vector) => val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) - Vectors.dense(1.0 - prob1, prob1) + Vectors.dense(1.0 - prob1, prob1): Vector } - tmpData = tmpData.select($"*", - callUDF(raw2prob, new VectorUDT, col(map(rawPredictionCol))).as(map(probabilityCol))) + tmpData = tmpData.withColumn(map(probabilityCol), raw2prob(col(map(rawPredictionCol)))) } else { - val features2prob: Vector => Vector = (features: Vector) => predictProbabilities(features) - tmpData = tmpData.select($"*", - callUDF(features2prob, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) + val features2prob = udf { (features: Vector) => predictProbabilities(features) : Vector } + tmpData = tmpData.withColumn(map(probabilityCol), features2prob(col(map(featuresCol)))) } numColsOutput += 1 } if (map(predictionCol) != "") { val t = map(threshold) if (map(probabilityCol) != "") { - val predict: Vector => Double = { probs: Vector => + val predict = udf { probs: Vector => if (probs(1) > t) 1.0 else 0.0 } - tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, col(map(probabilityCol))).as(map(predictionCol))) + tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(probabilityCol)))) } else if (map(rawPredictionCol) != "") { - val predict: Vector => Double = { rawPreds: Vector => + val predict = udf { rawPreds: Vector => val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) if (prob1 > t) 1.0 else 0.0 } - tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, col(map(rawPredictionCol))).as(map(predictionCol))) + tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(rawPredictionCol)))) } else { - val predict: Vector => Double = (features: Vector) => this.predict(features) - tmpData = tmpData.select($"*", - callUDF(predict, DoubleType, col(map(featuresCol))).as(map(predictionCol))) + val predict = udf { features: Vector => this.predict(features) } + tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(featuresCol)))) } numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 38518785dceae..bd8caac855981 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} @@ -122,8 +122,8 @@ private[spark] abstract class ProbabilisticClassificationModel[ val features2probs: FeaturesType => Vector = (features) => { tmpModel.predictProbabilities(features) } - outputData.select($"*", - callUDF(features2probs, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) + outputData.withColumn(map(probabilityCol), + callUDF(features2probs, new VectorUDT, col(map(featuresCol)))) } else { if (numColsOutput == 0) { this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 7623ec59ae0eb..ddbd648d64f23 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -23,7 +23,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} /** @@ -88,7 +88,7 @@ class StandardScalerModel private[ml] ( transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) - dataset.select($"*", scale(col(map(inputCol))).as(map(outputCol))) + dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) } private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index e416c1eb585fc..7daeff980f0ea 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -216,7 +216,7 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel val pred: FeaturesType => Double = (features) => { tmpModel.predict(features) } - dataset.select($"*", callUDF(pred, DoubleType, col(map(featuresCol))).as(map(predictionCol))) + dataset.withColumn(map(predictionCol), callUDF(pred, DoubleType, col(map(featuresCol)))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + " since no output columns were set.") diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index aac487745f77d..8d70e4347c4c9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -36,7 +36,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructField, StructType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -170,8 +170,8 @@ class ALSModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { import dataset.sqlContext.implicits._ val map = this.paramMap ++ paramMap - val users = userFactors.toDataFrame("id", "features") - val items = itemFactors.toDataFrame("id", "features") + val users = userFactors.toDF("id", "features") + val items = itemFactors.toDF("id", "features") // Register a UDF for DataFrame, and then // create a new column named map(predictionCol) by running the predict UDF. 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 f9142bc2268b7..dd7a9469d5041 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 @@ -102,7 +102,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) // Create Parquet data. - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF dataRDD.saveAsParquetFile(dataPath(path)) } 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 1d118963b49fc..0a358f2e4f39c 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 @@ -62,7 +62,7 @@ private[classification] object GLMClassificationModel { // Create Parquet data. val data = Data(weights, intercept, threshold) - sc.parallelize(Seq(data), 1).saveAsParquetFile(Loader.dataPath(path)) + sc.parallelize(Seq(data), 1).toDF.saveAsParquetFile(Loader.dataPath(path)) } /** 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 a3a3b5d418c0a..c399496568bfb 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 @@ -187,8 +187,8 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) - model.userFeatures.toDataFrame("id", "features").saveAsParquetFile(userPath(path)) - model.productFeatures.toDataFrame("id", "features").saveAsParquetFile(productPath(path)) + model.userFeatures.toDF("id", "features").saveAsParquetFile(userPath(path)) + model.productFeatures.toDF("id", "features").saveAsParquetFile(productPath(path)) } def load(sc: SparkContext, path: String): MatrixFactorizationModel = { 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 f75de6f637640..7b27aaa322b00 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 @@ -58,7 +58,7 @@ private[regression] object GLMRegressionModel { // Create Parquet data. val data = Data(weights, intercept) - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF // TODO: repartition with 1 partition after SPARK-5532 gets fixed dataRDD.saveAsParquetFile(Loader.dataPath(path)) } 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 373192a20c0ea..5dac62b0c42f0 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 @@ -197,7 +197,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] { val nodes = model.topNode.subtreeIterator.toSeq val dataRDD: DataFrame = sc.parallelize(nodes) .map(NodeData.apply(0, _)) - .toDataFrame + .toDF dataRDD.saveAsParquetFile(Loader.dataPath(path)) } 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 dbd69dca60b92..e507f247cca76 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 @@ -289,7 +289,7 @@ private[tree] object TreeEnsembleModel { // Create Parquet data. val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => tree.topNode.subtreeIterator.toSeq.map(node => NodeData(treeId, node)) - }.toDataFrame + }.toDF dataRDD.saveAsParquetFile(Loader.dataPath(path)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index cb7d57de35c34..b118a8dcf1363 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -358,8 +358,8 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { .setNumUserBlocks(numUserBlocks) .setNumItemBlocks(numItemBlocks) val alpha = als.getAlpha - val model = als.fit(training) - val predictions = model.transform(test) + val model = als.fit(training.toDF) + val predictions = model.transform(test.toDF) .select("rating", "prediction") .map { case Row(rating: Float, prediction: Float) => (rating.toDouble, prediction.toDouble) diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index 80c6f02a9df41..e03379e521a07 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -16,3 +16,11 @@ pyspark.sql.types module :members: :undoc-members: :show-inheritance: + + +pyspark.sql.functions module +------------------------ +.. automodule:: pyspark.sql.functions + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 49e5c9d58e5db..06207a076eece 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -335,7 +335,7 @@ def test_infer_schema(self): sqlCtx = SQLContext(self.sc) rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) srdd = sqlCtx.inferSchema(rdd) - schema = srdd.schema() + schema = srdd.schema field = [f for f in schema.fields if f.name == "features"][0] self.assertEqual(field.dataType, self.udt) vectors = srdd.map(lambda p: p.features).collect() diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 0a5ba00393aab..b9ffd6945ea7e 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -34,9 +34,8 @@ from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row -from pyspark.sql.dataframe import DataFrame, GroupedData, Column, Dsl, SchemaRDD +from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD __all__ = [ 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', - 'Dsl', ] diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 082f1b691b196..7683c1b4dfa4e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -38,6 +38,25 @@ __all__ = ["SQLContext", "HiveContext"] +def _monkey_patch_RDD(sqlCtx): + def toDF(self, schema=None, sampleRatio=None): + """ + Convert current :class:`RDD` into a :class:`DataFrame` + + This is a shorthand for `sqlCtx.createDataFrame(rdd, schema, sampleRatio)` + + :param schema: a StructType or list of names of columns + :param samplingRatio: the sample ratio of rows used for inferring + :return: a DataFrame + + >>> rdd.toDF().collect() + [Row(name=u'Alice', age=1)] + """ + return sqlCtx.createDataFrame(self, schema, sampleRatio) + + RDD.toDF = toDF + + class SQLContext(object): """Main entry point for Spark SQL functionality. @@ -49,15 +68,20 @@ class SQLContext(object): def __init__(self, sparkContext, sqlContext=None): """Create a new SQLContext. + It will add a method called `toDF` to :class:`RDD`, which could be + used to convert an RDD into a DataFrame, it's a shorthand for + :func:`SQLContext.createDataFrame`. + :param sparkContext: The SparkContext to wrap. :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new SQLContext in the JVM, instead we make all calls to this object. >>> from datetime import datetime + >>> sqlCtx = SQLContext(sc) >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) - >>> df = sqlCtx.createDataFrame(allTypes) + >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() @@ -70,6 +94,7 @@ def __init__(self, sparkContext, sqlContext=None): self._jsc = self._sc._jsc self._jvm = self._sc._jvm self._scala_SQLContext = sqlContext + _monkey_patch_RDD(self) @property def _ssql_ctx(self): @@ -442,7 +467,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema()) + >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema) >>> sqlCtx.registerRDDAsTable(df3, "table2") >>> df4 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " @@ -495,7 +520,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - >>> df3 = sqlCtx.jsonRDD(json, df1.schema()) + >>> df3 = sqlCtx.jsonRDD(json, df1.schema) >>> sqlCtx.registerRDDAsTable(df3, "table2") >>> df4 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " @@ -800,7 +825,8 @@ def _test(): Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) - globs['df'] = sqlCtx.createDataFrame(rdd) + _monkey_patch_RDD(sqlCtx) + globs['df'] = rdd.toDF() jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b6f052ee44ae2..1438fe5285cc5 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -21,21 +21,19 @@ import random import os from tempfile import NamedTemporaryFile -from itertools import imap from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD, _prepare_for_python_RDD -from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \ - UTF8Deserializer +from pyspark.rdd import RDD +from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql.types import * from pyspark.sql.types import _create_cls, _parse_datatype_json_string -__all__ = ["DataFrame", "GroupedData", "Column", "Dsl", "SchemaRDD"] +__all__ = ["DataFrame", "GroupedData", "Column", "SchemaRDD"] class DataFrame(object): @@ -76,6 +74,7 @@ def __init__(self, jdf, sql_ctx): self.sql_ctx = sql_ctx self._sc = sql_ctx and sql_ctx._sc self.is_cached = False + self._schema = None # initialized lazily @property def rdd(self): @@ -86,7 +85,7 @@ def rdd(self): if not hasattr(self, '_lazy_rdd'): jrdd = self._jdf.javaToPython() rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) - schema = self.schema() + schema = self.schema def applySchema(it): cls = _create_cls(schema) @@ -216,14 +215,17 @@ def save(self, path=None, source=None, mode="append", **options): self._sc._gateway._gateway_client) self._jdf.save(source, jmode, joptions) + @property def schema(self): """Returns the schema of this DataFrame (represented by a L{StructType}). - >>> df.schema() + >>> df.schema StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) """ - return _parse_datatype_json_string(self._jdf.schema().json()) + if self._schema is None: + self._schema = _parse_datatype_json_string(self._jdf.schema().json()) + return self._schema def printSchema(self): """Prints out the schema in the tree format. @@ -284,7 +286,7 @@ def collect(self): with open(tempFile.name, 'rb') as tempFile: rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile)) os.unlink(tempFile.name) - cls = _create_cls(self.schema()) + cls = _create_cls(self.schema) return [cls(r) for r in rs] def limit(self, num): @@ -310,14 +312,26 @@ def take(self, num): return self.limit(num).collect() def map(self, f): - """ Return a new RDD by applying a function to each Row, it's a - shorthand for df.rdd.map() + """ Return a new RDD by applying a function to each Row + + It's a shorthand for df.rdd.map() >>> df.map(lambda p: p.name).collect() [u'Alice', u'Bob'] """ return self.rdd.map(f) + def flatMap(self, f): + """ Return a new RDD by first applying a function to all elements of this, + and then flattening the results. + + It's a shorthand for df.rdd.flatMap() + + >>> df.flatMap(lambda p: p.name).collect() + [u'A', u'l', u'i', u'c', u'e', u'B', u'o', u'b'] + """ + return self.rdd.flatMap(f) + def mapPartitions(self, f, preservesPartitioning=False): """ Return a new RDD by applying a function to each partition. @@ -378,21 +392,6 @@ def sample(self, withReplacement, fraction, seed=None): rdd = self._jdf.sample(withReplacement, fraction, long(seed)) return DataFrame(rdd, self.sql_ctx) - # def takeSample(self, withReplacement, num, seed=None): - # """Return a fixed-size sampled subset of this DataFrame. - # - # >>> df = sqlCtx.inferSchema(rdd) - # >>> df.takeSample(False, 2, 97) - # [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')] - # """ - # seed = seed if seed is not None else random.randint(0, sys.maxint) - # with SCCallSiteSync(self.context) as css: - # bytesInJava = self._jdf \ - # .takeSampleToPython(withReplacement, num, long(seed)) \ - # .iterator() - # cls = _create_cls(self.schema()) - # return map(cls, self._collect_iterator_through_file(bytesInJava)) - @property def dtypes(self): """Return all column names and their data types as a list. @@ -400,7 +399,7 @@ def dtypes(self): >>> df.dtypes [('age', 'int'), ('name', 'string')] """ - return [(str(f.name), f.dataType.simpleString()) for f in self.schema().fields] + return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] @property def columns(self): @@ -409,7 +408,7 @@ def columns(self): >>> df.columns [u'age', u'name'] """ - return [f.name for f in self.schema().fields] + return [f.name for f in self.schema.fields] def join(self, other, joinExprs=None, joinType=None): """ @@ -586,8 +585,8 @@ def agg(self, *exprs): >>> df.agg({"age": "max"}).collect() [Row(MAX(age#0)=5)] - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.min(df.age)).collect() + >>> from pyspark.sql import functions as F + >>> df.agg(F.min(df.age)).collect() [Row(MIN(age#0)=2)] """ return self.groupBy().agg(*exprs) @@ -616,18 +615,18 @@ def subtract(self, other): """ return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) - def addColumn(self, colName, col): + def withColumn(self, colName, col): """ Return a new :class:`DataFrame` by adding a column. - >>> df.addColumn('age2', df.age + 2).collect() + >>> df.withColumn('age2', df.age + 2).collect() [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] """ return self.select('*', col.alias(colName)) - def renameColumn(self, existing, new): + def withColumnRenamed(self, existing, new): """ Rename an existing column to a new name - >>> df.renameColumn('age', 'age2').collect() + >>> df.withColumnRenamed('age', 'age2').collect() [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] """ cols = [Column(_to_java_column(c), self.sql_ctx).alias(new) @@ -635,11 +634,11 @@ def renameColumn(self, existing, new): for c in self.columns] return self.select(*cols) - def to_pandas(self): + def toPandas(self): """ Collect all the rows and return a `pandas.DataFrame`. - >>> df.to_pandas() # doctest: +SKIP + >>> df.toPandas() # doctest: +SKIP age name 0 2 Alice 1 5 Bob @@ -687,10 +686,11 @@ def agg(self, *exprs): name to aggregate methods. >>> gdf = df.groupBy(df.name) - >>> gdf.agg({"age": "max"}).collect() - [Row(name=u'Bob', MAX(age#0)=5), Row(name=u'Alice', MAX(age#0)=2)] - >>> from pyspark.sql import Dsl - >>> gdf.agg(Dsl.min(df.age)).collect() + >>> gdf.agg({"*": "count"}).collect() + [Row(name=u'Bob', COUNT(1)=1), Row(name=u'Alice', COUNT(1)=1)] + + >>> from pyspark.sql import functions as F + >>> gdf.agg(F.min(df.age)).collect() [Row(MIN(age#0)=5), Row(MIN(age#0)=2)] """ assert exprs, "exprs should not be empty" @@ -742,12 +742,12 @@ def sum(self): def _create_column_from_literal(literal): sc = SparkContext._active_spark_context - return sc._jvm.Dsl.lit(literal) + return sc._jvm.functions.lit(literal) def _create_column_from_name(name): sc = SparkContext._active_spark_context - return sc._jvm.Dsl.col(name) + return sc._jvm.functions.col(name) def _to_java_column(col): @@ -767,9 +767,9 @@ def _(self): return _ -def _dsl_op(name, doc=''): +def _func_op(name, doc=''): def _(self): - jc = getattr(self._sc._jvm.Dsl, name)(self._jc) + jc = getattr(self._sc._jvm.functions, name)(self._jc) return Column(jc, self.sql_ctx) _.__doc__ = doc return _ @@ -818,7 +818,7 @@ def __init__(self, jc, sql_ctx=None): super(Column, self).__init__(jc, sql_ctx) # arithmetic operators - __neg__ = _dsl_op("negate") + __neg__ = _func_op("negate") __add__ = _bin_op("plus") __sub__ = _bin_op("minus") __mul__ = _bin_op("multiply") @@ -842,7 +842,7 @@ def __init__(self, jc, sql_ctx=None): # so use bitwise operators as boolean operators __and__ = _bin_op('and') __or__ = _bin_op('or') - __invert__ = _dsl_op('not') + __invert__ = _func_op('not') __rand__ = _bin_op("and") __ror__ = _bin_op("or") @@ -920,11 +920,11 @@ def __repr__(self): else: return 'Column<%s>' % self._jdf.toString() - def to_pandas(self): + def toPandas(self): """ Return a pandas.Series from the column - >>> df.age.to_pandas() # doctest: +SKIP + >>> df.age.toPandas() # doctest: +SKIP 0 2 1 5 dtype: int64 @@ -934,123 +934,6 @@ def to_pandas(self): return pd.Series(data) -def _aggregate_func(name, doc=""): - """ Create a function for aggregator by name""" - def _(col): - sc = SparkContext._active_spark_context - jc = getattr(sc._jvm.Dsl, name)(_to_java_column(col)) - return Column(jc) - _.__name__ = name - _.__doc__ = doc - return staticmethod(_) - - -class UserDefinedFunction(object): - def __init__(self, func, returnType): - self.func = func - self.returnType = returnType - self._broadcast = None - self._judf = self._create_judf() - - def _create_judf(self): - f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) - ser = AutoBatchedSerializer(PickleSerializer()) - command = (func, None, ser, ser) - sc = SparkContext._active_spark_context - pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) - ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) - jdt = ssql_ctx.parseDataType(self.returnType.json()) - judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, - includes, sc.pythonExec, broadcast_vars, - sc._javaAccumulator, jdt) - return judf - - def __del__(self): - if self._broadcast is not None: - self._broadcast.unpersist() - self._broadcast = None - - def __call__(self, *cols): - sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) - return Column(jc) - - -class Dsl(object): - """ - A collections of builtin aggregators - """ - DSLS = { - 'lit': 'Creates a :class:`Column` of literal value.', - 'col': 'Returns a :class:`Column` based on the given column name.', - 'column': 'Returns a :class:`Column` based on the given column name.', - 'upper': 'Converts a string expression to upper case.', - 'lower': 'Converts a string expression to upper case.', - 'sqrt': 'Computes the square root of the specified float value.', - 'abs': 'Computes the absolutle value.', - - 'max': 'Aggregate function: returns the maximum value of the expression in a group.', - 'min': 'Aggregate function: returns the minimum value of the expression in a group.', - 'first': 'Aggregate function: returns the first value in a group.', - 'last': 'Aggregate function: returns the last value in a group.', - 'count': 'Aggregate function: returns the number of items in a group.', - 'sum': 'Aggregate function: returns the sum of all values in the expression.', - 'avg': 'Aggregate function: returns the average of the values in a group.', - 'mean': 'Aggregate function: returns the average of the values in a group.', - 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', - } - - for _name, _doc in DSLS.items(): - locals()[_name] = _aggregate_func(_name, _doc) - del _name, _doc - - @staticmethod - def countDistinct(col, *cols): - """ Return a new Column for distinct count of (col, *cols) - - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.countDistinct(df.age, df.name).alias('c')).collect() - [Row(c=2)] - - >>> df.agg(Dsl.countDistinct("age", "name").alias('c')).collect() - [Row(c=2)] - """ - sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = sc._jvm.Dsl.countDistinct(_to_java_column(col), - sc._jvm.PythonUtils.toSeq(jcols)) - return Column(jc) - - @staticmethod - def approxCountDistinct(col, rsd=None): - """ Return a new Column for approxiate distinct count of (col, *cols) - - >>> from pyspark.sql import Dsl - >>> df.agg(Dsl.approxCountDistinct(df.age).alias('c')).collect() - [Row(c=2)] - """ - sc = SparkContext._active_spark_context - if rsd is None: - jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col)) - else: - jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col), rsd) - return Column(jc) - - @staticmethod - def udf(f, returnType=StringType()): - """Create a user defined function (UDF) - - >>> slen = Dsl.udf(lambda s: len(s), IntegerType()) - >>> df.select(slen(df.name).alias('slen')).collect() - [Row(slen=5), Row(slen=3)] - """ - return UserDefinedFunction(f, returnType) - - def _test(): import doctest from pyspark.context import SparkContext @@ -1059,11 +942,9 @@ def _test(): globs = pyspark.sql.dataframe.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) - rdd2 = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]) - rdd3 = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]) - globs['df'] = sqlCtx.inferSchema(rdd2) - globs['df2'] = sqlCtx.inferSchema(rdd3) + globs['sqlCtx'] = SQLContext(sc) + globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() + globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py new file mode 100644 index 0000000000000..39aa550eeb5ad --- /dev/null +++ b/python/pyspark/sql/functions.py @@ -0,0 +1,170 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +A collections of builtin functions +""" + +from itertools import imap + +from py4j.java_collections import ListConverter + +from pyspark import SparkContext +from pyspark.rdd import _prepare_for_python_RDD +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.sql.types import StringType +from pyspark.sql.dataframe import Column, _to_java_column + + +__all__ = ['countDistinct', 'approxCountDistinct', 'udf'] + + +def _create_function(name, doc=""): + """ Create a function for aggregator by name""" + def _(col): + sc = SparkContext._active_spark_context + jc = getattr(sc._jvm.functions, name)(_to_java_column(col)) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return _ + + +_functions = { + 'lit': 'Creates a :class:`Column` of literal value.', + 'col': 'Returns a :class:`Column` based on the given column name.', + 'column': 'Returns a :class:`Column` based on the given column name.', + 'upper': 'Converts a string expression to upper case.', + 'lower': 'Converts a string expression to upper case.', + 'sqrt': 'Computes the square root of the specified float value.', + 'abs': 'Computes the absolutle value.', + + 'max': 'Aggregate function: returns the maximum value of the expression in a group.', + 'min': 'Aggregate function: returns the minimum value of the expression in a group.', + 'first': 'Aggregate function: returns the first value in a group.', + 'last': 'Aggregate function: returns the last value in a group.', + 'count': 'Aggregate function: returns the number of items in a group.', + 'sum': 'Aggregate function: returns the sum of all values in the expression.', + 'avg': 'Aggregate function: returns the average of the values in a group.', + 'mean': 'Aggregate function: returns the average of the values in a group.', + 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', +} + + +for _name, _doc in _functions.items(): + globals()[_name] = _create_function(_name, _doc) +del _name, _doc +__all__ += _functions.keys() + + +def countDistinct(col, *cols): + """ Return a new Column for distinct count of `col` or `cols` + + >>> df.agg(countDistinct(df.age, df.name).alias('c')).collect() + [Row(c=2)] + + >>> df.agg(countDistinct("age", "name").alias('c')).collect() + [Row(c=2)] + """ + sc = SparkContext._active_spark_context + jcols = ListConverter().convert([_to_java_column(c) for c in cols], sc._gateway._gateway_client) + jc = sc._jvm.functions.countDistinct(_to_java_column(col), sc._jvm.PythonUtils.toSeq(jcols)) + return Column(jc) + + +def approxCountDistinct(col, rsd=None): + """ Return a new Column for approximate distinct count of `col` + + >>> df.agg(approxCountDistinct(df.age).alias('c')).collect() + [Row(c=2)] + """ + sc = SparkContext._active_spark_context + if rsd is None: + jc = sc._jvm.functions.approxCountDistinct(_to_java_column(col)) + else: + jc = sc._jvm.functions.approxCountDistinct(_to_java_column(col), rsd) + return Column(jc) + + +class UserDefinedFunction(object): + """ + User defined function in Python + """ + def __init__(self, func, returnType): + self.func = func + self.returnType = returnType + self._broadcast = None + self._judf = self._create_judf() + + def _create_judf(self): + f = self.func # put it in closure `func` + func = lambda _, it: imap(lambda x: f(*x), it) + ser = AutoBatchedSerializer(PickleSerializer()) + command = (func, None, ser, ser) + sc = SparkContext._active_spark_context + pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) + ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) + jdt = ssql_ctx.parseDataType(self.returnType.json()) + judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, + includes, sc.pythonExec, broadcast_vars, + sc._javaAccumulator, jdt) + return judf + + def __del__(self): + if self._broadcast is not None: + self._broadcast.unpersist() + self._broadcast = None + + def __call__(self, *cols): + sc = SparkContext._active_spark_context + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + sc._gateway._gateway_client) + jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) + return Column(jc) + + +def udf(f, returnType=StringType()): + """Create a user defined function (UDF) + + >>> slen = udf(lambda s: len(s), IntegerType()) + >>> df.select(slen(df.name).alias('slen')).collect() + [Row(slen=5), Row(slen=3)] + """ + return UserDefinedFunction(f, returnType) + + +def _test(): + import doctest + from pyspark.context import SparkContext + from pyspark.sql import Row, SQLContext + import pyspark.sql.dataframe + globs = pyspark.sql.dataframe.__dict__.copy() + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlCtx'] = SQLContext(sc) + globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() + globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() + (failure_count, test_count) = doctest.testmod( + pyspark.sql.dataframe, globs=globs, + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 43e5c3a1b00fa..aa80bca34655d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -96,7 +96,7 @@ def setUpClass(cls): cls.sqlCtx = SQLContext(cls.sc) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] rdd = cls.sc.parallelize(cls.testData) - cls.df = cls.sqlCtx.createDataFrame(rdd) + cls.df = rdd.toDF() @classmethod def tearDownClass(cls): @@ -138,7 +138,7 @@ def test_basic_functions(self): df = self.sqlCtx.jsonRDD(rdd) df.count() df.collect() - df.schema() + df.schema # cache and checkpoint self.assertFalse(df.is_cached) @@ -155,11 +155,11 @@ def test_basic_functions(self): def test_apply_schema_to_row(self): df = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""])) - df2 = self.sqlCtx.createDataFrame(df.map(lambda x: x), df.schema()) + df2 = self.sqlCtx.createDataFrame(df.map(lambda x: x), df.schema) self.assertEqual(df.collect(), df2.collect()) rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) - df3 = self.sqlCtx.createDataFrame(rdd, df.schema()) + df3 = self.sqlCtx.createDataFrame(rdd, df.schema) self.assertEqual(10, df3.count()) def test_serialize_nested_array_and_map(self): @@ -195,7 +195,7 @@ def test_infer_schema(self): self.assertEqual(1, result.head()[0]) df2 = self.sqlCtx.createDataFrame(rdd, samplingRatio=1.0) - self.assertEqual(df.schema(), df2.schema()) + self.assertEqual(df.schema, df2.schema) self.assertEqual({}, df2.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.map(lambda r: r.s).collect()) df2.registerTempTable("test2") @@ -204,8 +204,7 @@ def test_infer_schema(self): def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] - rdd = self.sc.parallelize(d) - df = self.sqlCtx.createDataFrame(rdd) + df = self.sc.parallelize(d).toDF() k, v = df.head().m.items()[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -213,8 +212,7 @@ def test_struct_in_map(self): def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) - rdd = self.sc.parallelize([row]) - df = self.sqlCtx.createDataFrame(rdd) + df = self.sc.parallelize([row]).toDF() df.registerTempTable("test") row = self.sqlCtx.sql("select l, d from test").head() self.assertEqual(1, row.asDict()["l"][0].a) @@ -223,9 +221,8 @@ def test_convert_row_to_dict(self): def test_infer_schema_with_udt(self): from pyspark.sql.tests import ExamplePoint, ExamplePointUDT row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) - rdd = self.sc.parallelize([row]) - df = self.sqlCtx.createDataFrame(rdd) - schema = df.schema() + df = self.sc.parallelize([row]).toDF() + schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), ExamplePointUDT) df.registerTempTable("labeled_point") @@ -238,15 +235,14 @@ def test_apply_schema_with_udt(self): rdd = self.sc.parallelize([row]) schema = StructType([StructField("label", DoubleType(), False), StructField("point", ExamplePointUDT(), False)]) - df = self.sqlCtx.createDataFrame(rdd, schema) + df = rdd.toDF(schema) point = df.head().point self.assertEquals(point, ExamplePoint(1.0, 2.0)) def test_parquet_with_udt(self): from pyspark.sql.tests import ExamplePoint row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) - rdd = self.sc.parallelize([row]) - df0 = self.sqlCtx.createDataFrame(rdd) + df0 = self.sc.parallelize([row]).toDF() output_dir = os.path.join(self.tempdir.name, "labeled_point") df0.saveAsParquetFile(output_dir) df1 = self.sqlCtx.parquetFile(output_dir) @@ -280,10 +276,11 @@ def test_aggregator(self): self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0])) self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) - from pyspark.sql import Dsl - self.assertEqual((0, u'99'), tuple(g.agg(Dsl.first(df.key), Dsl.last(df.value)).first())) - self.assertTrue(95 < g.agg(Dsl.approxCountDistinct(df.key)).first()[0]) - self.assertEqual(100, g.agg(Dsl.countDistinct(df.value)).first()[0]) + from pyspark.sql import functions + self.assertEqual((0, u'99'), + tuple(g.agg(functions.first(df.key), functions.last(df.value)).first())) + self.assertTrue(95 < g.agg(functions.approxCountDistinct(df.key)).first()[0]) + self.assertEqual(100, g.agg(functions.countDistinct(df.value)).first()[0]) def test_save_and_load(self): df = self.df @@ -339,8 +336,7 @@ def setUpClass(cls): cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) cls.sqlCtx = HiveContext(cls.sc, _scala_HiveContext) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] - rdd = cls.sc.parallelize(cls.testData) - cls.df = cls.sqlCtx.inferSchema(rdd) + cls.df = cls.sc.parallelize(cls.testData).toDF() @classmethod def tearDownClass(cls): diff --git a/python/run-tests b/python/run-tests index 077ad60d764a3..a2c2f37a54eda 100755 --- a/python/run-tests +++ b/python/run-tests @@ -35,7 +35,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" | tee -a $LOG_FILE - SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 >> $LOG_FILE 2>&1 + SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 > $LOG_FILE 2>&1 FAILED=$((PIPESTATUS[0]||$FAILED)) @@ -67,6 +67,7 @@ function run_sql_tests() { run_test "pyspark/sql/types.py" run_test "pyspark/sql/context.py" run_test "pyspark/sql/dataframe.py" + run_test "pyspark/sql/functions.py" run_test "pyspark/sql/tests.py" } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 0cf2de6d399b0..05faef8786d2c 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -137,7 +137,7 @@ private[repl] trait SparkILoopInit { command("import org.apache.spark.SparkContext._") command("import sqlContext.implicits._") command("import sqlContext.sql") - command("import org.apache.spark.sql.Dsl._") + command("import org.apache.spark.sql.functions._") } } diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 201f2672d5474..529914a2b6141 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -262,7 +262,7 @@ class ReplSuite extends FunSuite { |val sqlContext = new org.apache.spark.sql.SQLContext(sc) |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDataFrame.collect() + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 1bd2a6991404b..7a5e94da5cbf3 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -77,7 +77,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter) command("import org.apache.spark.SparkContext._") command("import sqlContext.implicits._") command("import sqlContext.sql") - command("import org.apache.spark.sql.Dsl._") + command("import org.apache.spark.sql.functions._") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index f959a50564011..a7cd4124e56f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -152,7 +152,7 @@ case class MultiAlias(child: Expression, names: Seq[String]) override lazy val resolved = false - override def newInstance = this + override def newInstance() = this override def withNullability(newNullability: Boolean) = this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 9d5d6e78bd487..f6ecee1af8aad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql -import scala.annotation.tailrec import scala.language.implicitConversions -import org.apache.spark.sql.Dsl.lit +import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Subquery, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField import org.apache.spark.sql.types._ @@ -127,7 +126,7 @@ trait Column extends DataFrame { * df.select( -df("amount") ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.select( negate(col("amount") ); * }}} */ @@ -140,7 +139,7 @@ trait Column extends DataFrame { * df.filter( !df("isActive") ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.filter( not(df.col("isActive")) ); * }} */ @@ -153,7 +152,7 @@ trait Column extends DataFrame { * df.filter( df("colA") === df("colB") ) * * // Java - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").equalTo(col("colB")) ); * }}} */ @@ -168,7 +167,7 @@ trait Column extends DataFrame { * df.filter( df("colA") === df("colB") ) * * // Java - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").equalTo(col("colB")) ); * }}} */ @@ -182,7 +181,7 @@ trait Column extends DataFrame { * df.select( !(df("colA") === df("colB")) ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").notEqual(col("colB")) ); * }}} */ @@ -198,7 +197,7 @@ trait Column extends DataFrame { * df.select( !(df("colA") === df("colB")) ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").notEqual(col("colB")) ); * }}} */ @@ -213,7 +212,7 @@ trait Column extends DataFrame { * people.select( people("age") > 21 ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * people.select( people("age").gt(21) ); * }}} */ @@ -228,7 +227,7 @@ trait Column extends DataFrame { * people.select( people("age") > lit(21) ) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * people.select( people("age").gt(21) ); * }}} */ 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 4f8f19e2c1465..e21e989f36c65 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 @@ -48,7 +48,7 @@ private[sql] object DataFrame { * }}} * * Once created, it can be manipulated using the various domain-specific-language (DSL) functions - * defined in: [[DataFrame]] (this class), [[Column]], [[Dsl]] for the DSL. + * defined in: [[DataFrame]] (this class), [[Column]], [[functions]] for the DSL. * * To select a column from the data frame, use the apply method: * {{{ @@ -94,27 +94,27 @@ trait DataFrame extends RDDApi[Row] with Serializable { } /** Left here for backward compatibility. */ - @deprecated("1.3.0", "use toDataFrame") + @deprecated("1.3.0", "use toDF") def toSchemaRDD: DataFrame = this /** * Returns the object itself. Used to force an implicit conversion from RDD to DataFrame in Scala. */ // This is declared with parentheses to prevent the Scala compiler from treating - // `rdd.toDataFrame("1")` as invoking this toDataFrame and then apply on the returned DataFrame. - def toDataFrame(): DataFrame = this + // `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. + def toDF(): DataFrame = this /** * Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion * from a RDD of tuples into a [[DataFrame]] with meaningful names. For example: * {{{ * val rdd: RDD[(Int, String)] = ... - * rdd.toDataFrame // this implicit conversion creates a DataFrame with column name _1 and _2 - * rdd.toDataFrame("id", "name") // this creates a DataFrame with column name "id" and "name" + * rdd.toDF // this implicit conversion creates a DataFrame with column name _1 and _2 + * rdd.toDF("id", "name") // this creates a DataFrame with column name "id" and "name" * }}} */ @scala.annotation.varargs - def toDataFrame(colNames: String*): DataFrame + def toDF(colNames: String*): DataFrame /** Returns the schema of this [[DataFrame]]. */ def schema: StructType @@ -132,7 +132,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { def explain(extended: Boolean): Unit /** Only prints the physical plan to the console for debugging purpose. */ - def explain(): Unit = explain(false) + def explain(): Unit = explain(extended = false) /** * Returns true if the `collect` and `take` methods can be run locally @@ -179,11 +179,11 @@ trait DataFrame extends RDDApi[Row] with Serializable { * * {{{ * // Scala: - * import org.apache.spark.sql.dsl._ + * import org.apache.spark.sql.functions._ * df1.join(df2, "outer", $"df1Key" === $"df2Key") * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df1.join(df2, "outer", col("df1Key") === col("df2Key")); * }}} * @@ -483,12 +483,12 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new [[DataFrame]] by adding a column. */ - def addColumn(colName: String, col: Column): DataFrame + def withColumn(colName: String, col: Column): DataFrame /** * Returns a new [[DataFrame]] with a column renamed. */ - def renameColumn(existingName: String, newName: String): DataFrame + def withColumnRenamed(existingName: String, newName: String): DataFrame /** * Returns the first `n` rows. @@ -520,6 +520,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Returns a new RDD by applying a function to each partition of this DataFrame. */ override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] + /** * Applies a function `f` to all rows. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala new file mode 100644 index 0000000000000..a3187fe3230fd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameHolder.scala @@ -0,0 +1,30 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +/** + * A container for a [[DataFrame]], used for implicit conversions. + */ +private[sql] case class DataFrameHolder(df: DataFrame) { + + // This is declared with parentheses to prevent the Scala compiler from treating + // `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. + def toDF(): DataFrame = df + + def toDF(colNames: String*): DataFrame = df.toDF(colNames :_*) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index bb5c6226a2217..7b7efbe3477b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -94,7 +94,7 @@ private[sql] class DataFrameImpl protected[sql]( } } - override def toDataFrame(colNames: String*): DataFrame = { + override def toDF(colNames: String*): DataFrame = { require(schema.size == colNames.size, "The number of columns doesn't match.\n" + "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + @@ -229,11 +229,11 @@ private[sql] class DataFrameImpl protected[sql]( }: _*) } - override def addColumn(colName: String, col: Column): DataFrame = { + override def withColumn(colName: String, col: Column): DataFrame = { select(Column("*"), col.as(colName)) } - override def renameColumn(existingName: String, newName: String): DataFrame = { + override def withColumnRenamed(existingName: String, newName: String): DataFrame = { val colNames = schema.map { field => val name = field.name if (name == existingName) Column(name).as(newName) else Column(name) 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 3c20676355c9d..0868013fe7c96 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 @@ -20,8 +20,8 @@ package org.apache.spark.sql import scala.language.implicitConversions import scala.collection.JavaConversions._ +import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.{Literal => LiteralExpr} import org.apache.spark.sql.catalyst.plans.logical.Aggregate @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate */ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { - private[this] implicit def toDataFrame(aggExprs: Seq[NamedExpression]): DataFrame = { + private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.toString)() @@ -52,7 +52,12 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio case "max" => Max case "min" => Min case "sum" => Sum - case "count" | "size" => Count + case "count" | "size" => + // Turn count(*) into count(1) + (inputExpr: Expression) => inputExpr match { + case s: Star => Count(Literal(1)) + case _ => Count(inputExpr) + } } } @@ -115,17 +120,17 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio * Compute aggregates by specifying a series of aggregate columns. Unlike other methods in this * class, the resulting [[DataFrame]] won't automatically include the grouping columns. * - * The available aggregate methods are defined in [[org.apache.spark.sql.Dsl]]. + * The available aggregate methods are defined in [[org.apache.spark.sql.functions]]. * * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department * * // Scala: - * import org.apache.spark.sql.dsl._ + * import org.apache.spark.sql.functions._ * df.groupBy("department").agg($"department", max($"age"), sum($"expense")) * * // Java: - * import static org.apache.spark.sql.Dsl.*; + * import static org.apache.spark.sql.functions.*; * df.groupBy("department").agg(col("department"), max(col("age")), sum(col("expense"))); * }}} */ @@ -142,7 +147,7 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio * Count the number of rows for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - def count(): DataFrame = Seq(Alias(Count(LiteralExpr(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`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index cba3b77011cc3..fc37cfa7a899f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -50,7 +50,7 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten protected[sql] override def logicalPlan: LogicalPlan = err() - override def toDataFrame(colNames: String*): DataFrame = err() + override def toDF(colNames: String*): DataFrame = err() override def schema: StructType = err() @@ -86,9 +86,9 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def selectExpr(exprs: String*): DataFrame = err() - override def addColumn(colName: String, col: Column): DataFrame = err() + override def withColumn(colName: String, col: Column): DataFrame = err() - override def renameColumn(existingName: String, newName: String): DataFrame = err() + override def withColumnRenamed(existingName: String, newName: String): DataFrame = err() override def filter(condition: Column): DataFrame = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 2165949d32c6f..a1736d0277f1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -183,14 +183,25 @@ class SQLContext(@transient val sparkContext: SparkContext) object implicits extends Serializable { // scalastyle:on + /** Converts $"col name" into an [[Column]]. */ + implicit class StringToColumn(val sc: StringContext) { + def $(args: Any*): ColumnName = { + new ColumnName(sc.s(args :_*)) + } + } + + /** An implicit conversion that turns a Scala `Symbol` into a [[Column]]. */ + implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) + /** Creates a DataFrame from an RDD of case classes or tuples. */ - implicit def rddToDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { - self.createDataFrame(rdd) + implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = { + DataFrameHolder(self.createDataFrame(rdd)) } /** Creates a DataFrame from a local Seq of Product. */ - implicit def localSeqToDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { - self.createDataFrame(data) + implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder = + { + DataFrameHolder(self.createDataFrame(data)) } // Do NOT add more implicit conversions. They are likely to break source compatibility by @@ -198,7 +209,7 @@ class SQLContext(@transient val sparkContext: SparkContext) // because of [[DoubleRDDFunctions]]. /** Creates a single column DataFrame from an RDD[Int]. */ - implicit def intRddToDataFrame(data: RDD[Int]): DataFrame = { + implicit def intRddToDataFrameHolder(data: RDD[Int]): DataFrameHolder = { val dataType = IntegerType val rows = data.mapPartitions { iter => val row = new SpecificMutableRow(dataType :: Nil) @@ -207,11 +218,11 @@ class SQLContext(@transient val sparkContext: SparkContext) row: Row } } - self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + DataFrameHolder(self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil))) } /** Creates a single column DataFrame from an RDD[Long]. */ - implicit def longRddToDataFrame(data: RDD[Long]): DataFrame = { + implicit def longRddToDataFrameHolder(data: RDD[Long]): DataFrameHolder = { val dataType = LongType val rows = data.mapPartitions { iter => val row = new SpecificMutableRow(dataType :: Nil) @@ -220,11 +231,11 @@ class SQLContext(@transient val sparkContext: SparkContext) row: Row } } - self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + DataFrameHolder(self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil))) } /** Creates a single column DataFrame from an RDD[String]. */ - implicit def stringRddToDataFrame(data: RDD[String]): DataFrame = { + implicit def stringRddToDataFrame(data: RDD[String]): DataFrameHolder = { val dataType = StringType val rows = data.mapPartitions { iter => val row = new SpecificMutableRow(dataType :: Nil) @@ -233,7 +244,7 @@ class SQLContext(@transient val sparkContext: SparkContext) row: Row } } - self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)) + DataFrameHolder(self.createDataFrame(rows, StructType(StructField("_1", dataType) :: Nil))) } } @@ -780,7 +791,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * indicating if a table is a temporary one or not). */ def tables(): DataFrame = { - createDataFrame(catalog.getTables(None)).toDataFrame("tableName", "isTemporary") + createDataFrame(catalog.getTables(None)).toDF("tableName", "isTemporary") } /** @@ -789,7 +800,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * indicating if a table is a temporary one or not). */ def tables(databaseName: String): DataFrame = { - createDataFrame(catalog.getTables(Some(databaseName))).toDataFrame("tableName", "isTemporary") + createDataFrame(catalog.getTables(Some(databaseName))).toDF("tableName", "isTemporary") } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala index c60d4070942a9..ee94a5fdbe376 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.PythonUDF import org.apache.spark.sql.types.DataType /** - * A user-defined function. To create one, use the `udf` functions in [[Dsl]]. + * A user-defined function. To create one, use the `udf` functions in [[functions]]. * As an example: * {{{ * // Defined a UDF that returns true or false based on some numeric score. @@ -45,7 +45,7 @@ case class UserDefinedFunction(f: AnyRef, dataType: DataType) { } /** - * A user-defined Python function. To create one, use the `pythonUDF` functions in [[Dsl]]. + * A user-defined Python function. To create one, use the `pythonUDF` functions in [[functions]]. * This is used by Python API. */ private[sql] case class UserDefinedPythonFunction( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala rename to sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 7bc7683576b71..4a0ec0b72ce81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -28,17 +29,9 @@ import org.apache.spark.sql.types._ /** * Domain specific functions available for [[DataFrame]]. */ -object Dsl { - - /** An implicit conversion that turns a Scala `Symbol` into a [[Column]]. */ - implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) - - /** Converts $"col name" into an [[Column]]. */ - implicit class StringToColumn(val sc: StringContext) extends AnyVal { - def $(args: Any*): ColumnName = { - new ColumnName(sc.s(args :_*)) - } - } +// scalastyle:off +object functions { +// scalastyle:on private[this] implicit def toColumn(expr: Expression): Column = Column(expr) @@ -104,7 +97,11 @@ object Dsl { def sumDistinct(columnName: String): Column = sumDistinct(Column(columnName)) /** Aggregate function: returns the number of items in a group. */ - def count(e: Column): Column = Count(e.expr) + def count(e: Column): Column = e.expr match { + // Turn count(*) into count(1) + case s: Star => Count(Literal(1)) + case _ => Count(e.expr) + } /** Aggregate function: returns the number of items in a group. */ def count(columnName: String): Column = count(Column(columnName)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index 8d3e094e3344d..538d774eb97eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -90,7 +90,7 @@ trait ParquetTest { (f: String => Unit): Unit = { import sqlContext.implicits._ withTempPath { file => - sparkContext.parallelize(data).saveAsParquetFile(file.getCanonicalPath) + sparkContext.parallelize(data).toDF().saveAsParquetFile(file.getCanonicalPath) f(file.getCanonicalPath) } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java index 639436368c4a3..05233dc5ffc58 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java @@ -23,7 +23,7 @@ import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.types.DataTypes; -import static org.apache.spark.sql.Dsl.*; +import static org.apache.spark.sql.functions.*; /** * This test doesn't actually run anything. It is here to check the API compatibility for Java. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 1318750a4a3b0..691dae0a0561b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -25,8 +25,9 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.storage.{StorageLevel, RDDBlockId} case class BigData(s: String) @@ -34,8 +35,6 @@ case class BigData(s: String) class CachedTableSuite extends QueryTest { TestData // Load test tables. - import org.apache.spark.sql.test.TestSQLContext.implicits._ - def rddIdOf(tableName: String): Int = { val executedPlan = table(tableName).queryExecution.executedPlan executedPlan.collect { @@ -95,7 +94,7 @@ class CachedTableSuite extends QueryTest { test("too big for memory") { val data = "*" * 10000 - sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") + sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF().registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) table("bigData").unpersist(blocking = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index e3e6f652ed3ed..a63d733ece627 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types.{BooleanType, IntegerType, StructField, StructType} @@ -68,7 +68,7 @@ class ColumnExpressionSuite extends QueryTest { } test("collect on column produced by a binary operator") { - val df = Seq((1, 2, 3)).toDataFrame("a", "b", "c") + val df = Seq((1, 2, 3)).toDF("a", "b", "c") checkAnswer(df("a") + df("b"), Seq(Row(3))) checkAnswer(df("a") + df("b").as("c"), Seq(Row(3))) } @@ -79,7 +79,7 @@ class ColumnExpressionSuite extends QueryTest { test("star qualified by data frame object") { // This is not yet supported. - val df = testData.toDataFrame + val df = testData.toDF val goldAnswer = df.collect().toSeq checkAnswer(df.select(df("*")), goldAnswer) @@ -156,13 +156,13 @@ class ColumnExpressionSuite extends QueryTest { test("isNull") { checkAnswer( - nullStrings.toDataFrame.where($"s".isNull), + nullStrings.toDF.where($"s".isNull), nullStrings.collect().toSeq.filter(r => r.getString(1) eq null)) } test("isNotNull") { checkAnswer( - nullStrings.toDataFrame.where($"s".isNotNull), + nullStrings.toDF.where($"s".isNotNull), nullStrings.collect().toSeq.filter(r => r.getString(1) ne null)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala index 8fa830dd9390f..2d2367d6e7292 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala @@ -25,31 +25,31 @@ class DataFrameImplicitsSuite extends QueryTest { test("RDD of tuples") { checkAnswer( - sc.parallelize(1 to 10).map(i => (i, i.toString)).toDataFrame("intCol", "strCol"), + sc.parallelize(1 to 10).map(i => (i, i.toString)).toDF("intCol", "strCol"), (1 to 10).map(i => Row(i, i.toString))) } test("Seq of tuples") { checkAnswer( - (1 to 10).map(i => (i, i.toString)).toDataFrame("intCol", "strCol"), + (1 to 10).map(i => (i, i.toString)).toDF("intCol", "strCol"), (1 to 10).map(i => Row(i, i.toString))) } test("RDD[Int]") { checkAnswer( - sc.parallelize(1 to 10).toDataFrame("intCol"), + sc.parallelize(1 to 10).toDF("intCol"), (1 to 10).map(i => Row(i))) } test("RDD[Long]") { checkAnswer( - sc.parallelize(1L to 10L).toDataFrame("longCol"), + sc.parallelize(1L to 10L).toDF("longCol"), (1L to 10L).map(i => Row(i))) } test("RDD[String]") { checkAnswer( - sc.parallelize(1 to 10).map(_.toString).toDataFrame("stringCol"), + sc.parallelize(1 to 10).map(_.toString).toDF("stringCol"), (1 to 10).map(i => Row(i.toString))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 33b35f376b270..f0cd43632ec3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.TestData._ import scala.language.postfixOps -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery @@ -99,7 +99,7 @@ class DataFrameSuite extends QueryTest { } test("simple explode") { - val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDataFrame("words") + val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words") checkAnswer( df.explode("words", "word") { word: String => word.split(" ").toSeq }.select('word), @@ -108,7 +108,7 @@ class DataFrameSuite extends QueryTest { } test("explode") { - val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDataFrame("number", "letters") + val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters") val df2 = df.explode('letters) { case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq @@ -141,15 +141,30 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } - test("agg") { + test("groupBy") { checkAnswer( testData2.groupBy("a").agg($"a", sum($"b")), - Seq(Row(1,3), Row(2,3), Row(3,3)) + Seq(Row(1, 3), Row(2, 3), Row(3, 3)) ) checkAnswer( testData2.groupBy("a").agg($"a", sum($"b").as("totB")).agg(sum('totB)), Row(9) ) + checkAnswer( + testData2.groupBy("a").agg(col("a"), count("*")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil + ) + checkAnswer( + testData2.groupBy("a").agg(Map("*" -> "count")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil + ) + checkAnswer( + testData2.groupBy("a").agg(Map("b" -> "sum")), + Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil + ) + } + + test("agg without groups") { checkAnswer( testData2.agg(sum('b)), Row(9) @@ -218,20 +233,20 @@ class DataFrameSuite extends QueryTest { Seq(Row(3,1), Row(3,2), Row(2,1), Row(2,2), Row(1,1), Row(1,2))) checkAnswer( - arrayData.orderBy('data.getItem(0).asc), - arrayData.toDataFrame.collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq) + arrayData.toDF.orderBy('data.getItem(0).asc), + arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq) checkAnswer( - arrayData.orderBy('data.getItem(0).desc), - arrayData.toDataFrame.collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq) + arrayData.toDF.orderBy('data.getItem(0).desc), + arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq) checkAnswer( - arrayData.orderBy('data.getItem(1).asc), - arrayData.toDataFrame.collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq) + arrayData.toDF.orderBy('data.getItem(1).asc), + arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq) checkAnswer( - arrayData.orderBy('data.getItem(1).desc), - arrayData.toDataFrame.collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq) + arrayData.toDF.orderBy('data.getItem(1).desc), + arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq) } test("limit") { @@ -240,11 +255,11 @@ class DataFrameSuite extends QueryTest { testData.take(10).toSeq) checkAnswer( - arrayData.limit(1), + arrayData.toDF.limit(1), arrayData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) checkAnswer( - mapData.limit(1), + mapData.toDF.limit(1), mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) } @@ -378,7 +393,7 @@ class DataFrameSuite extends QueryTest { } test("addColumn") { - val df = testData.toDataFrame.addColumn("newCol", col("key") + 1) + val df = testData.toDF.withColumn("newCol", col("key") + 1) checkAnswer( df, testData.collect().map { case Row(key: Int, value: String) => @@ -388,8 +403,8 @@ class DataFrameSuite extends QueryTest { } test("renameColumn") { - val df = testData.toDataFrame.addColumn("newCol", col("key") + 1) - .renameColumn("value", "valueRenamed") + val df = testData.toDF.withColumn("newCol", col("key") + 1) + .withColumnRenamed("value", "valueRenamed") checkAnswer( df, testData.collect().map { case Row(key: Int, value: String) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index f0c939dbb195f..fd73065c4ada3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ class JoinSuite extends QueryTest with BeforeAndAfterEach { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 5fc35349e166e..282b98a987dd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -28,7 +28,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.test.TestSQLContext.implicits._ val df = - sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDataFrame("key", "value") + sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDF("key", "value") before { df.registerTempTable("ListTablesSuiteTable") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a1c8cf58f2357..97684f75e79fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types._ @@ -1034,10 +1034,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) - rdd1.registerTempTable("nulldata1") + rdd1.toDF.registerTempTable("nulldata1") val nullCheckData2 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i))) - rdd2.registerTempTable("nulldata2") + rdd2.toDF.registerTempTable("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1046,7 +1046,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("Multi-column COUNT(DISTINCT ...)") { val data = TestData(1,"val_1") :: TestData(2,"val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.registerTempTable("distinctData") + rdd.toDF.registerTempTable("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 93782619826f0..9a48f8d0634cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -82,7 +82,7 @@ class ScalaReflectionRelationSuite extends FunSuite { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerTempTable("reflectData") + rdd.toDF.registerTempTable("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -93,7 +93,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerTempTable("reflectNullData") + rdd.toDF.registerTempTable("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) } @@ -101,7 +101,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerTempTable("reflectOptionalData") + rdd.toDF.registerTempTable("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) } @@ -109,7 +109,7 @@ class ScalaReflectionRelationSuite extends FunSuite { // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) - rdd.registerTempTable("reflectBinary") + rdd.toDF.registerTempTable("reflectBinary") val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) @@ -128,7 +128,7 @@ class ScalaReflectionRelationSuite extends FunSuite { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerTempTable("reflectComplexData") + rdd.toDF.registerTempTable("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === new GenericRow(Array[Any]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 0ed437edd05fd..c511eb1469167 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test._ import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -29,11 +29,11 @@ case class TestData(key: Int, value: String) object TestData { val testData = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))).toDataFrame + (1 to 100).map(i => TestData(i, i.toString))).toDF testData.registerTempTable("testData") val negativeData = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(-i, (-i).toString))).toDataFrame + (1 to 100).map(i => TestData(-i, (-i).toString))).toDF negativeData.registerTempTable("negativeData") case class LargeAndSmallInts(a: Int, b: Int) @@ -44,7 +44,7 @@ object TestData { LargeAndSmallInts(2147483645, 1) :: LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: - LargeAndSmallInts(3, 2) :: Nil).toDataFrame + LargeAndSmallInts(3, 2) :: Nil).toDF largeAndSmallInts.registerTempTable("largeAndSmallInts") case class TestData2(a: Int, b: Int) @@ -55,7 +55,7 @@ object TestData { TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil, 2).toDataFrame + TestData2(3, 2) :: Nil, 2).toDF testData2.registerTempTable("testData2") case class DecimalData(a: BigDecimal, b: BigDecimal) @@ -67,7 +67,7 @@ object TestData { DecimalData(2, 1) :: DecimalData(2, 2) :: DecimalData(3, 1) :: - DecimalData(3, 2) :: Nil).toDataFrame + DecimalData(3, 2) :: Nil).toDF decimalData.registerTempTable("decimalData") case class BinaryData(a: Array[Byte], b: Int) @@ -77,14 +77,14 @@ object TestData { BinaryData("22".getBytes(), 5) :: BinaryData("122".getBytes(), 3) :: BinaryData("121".getBytes(), 2) :: - BinaryData("123".getBytes(), 4) :: Nil).toDataFrame + BinaryData("123".getBytes(), 4) :: Nil).toDF binaryData.registerTempTable("binaryData") case class TestData3(a: Int, b: Option[Int]) val testData3 = TestSQLContext.sparkContext.parallelize( TestData3(1, None) :: - TestData3(2, Some(2)) :: Nil).toDataFrame + TestData3(2, Some(2)) :: Nil).toDF testData3.registerTempTable("testData3") val emptyTableData = logical.LocalRelation($"a".int, $"b".int) @@ -97,7 +97,7 @@ object TestData { UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: - UpperCaseData(6, "F") :: Nil).toDataFrame + UpperCaseData(6, "F") :: Nil).toDF upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) @@ -106,7 +106,7 @@ object TestData { LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: - LowerCaseData(4, "d") :: Nil).toDataFrame + LowerCaseData(4, "d") :: Nil).toDF lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) @@ -114,7 +114,7 @@ object TestData { TestSQLContext.sparkContext.parallelize( ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) - arrayData.registerTempTable("arrayData") + arrayData.toDF.registerTempTable("arrayData") case class MapData(data: scala.collection.Map[Int, String]) val mapData = @@ -124,18 +124,18 @@ object TestData { MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - mapData.registerTempTable("mapData") + mapData.toDF.registerTempTable("mapData") case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) - repeatedData.registerTempTable("repeatedData") + repeatedData.toDF.registerTempTable("repeatedData") val nullableRepeatedData = TestSQLContext.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - nullableRepeatedData.registerTempTable("nullableRepeatedData") + nullableRepeatedData.toDF.registerTempTable("nullableRepeatedData") case class NullInts(a: Integer) val nullInts = @@ -144,7 +144,7 @@ object TestData { NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil - ) + ).toDF nullInts.registerTempTable("nullInts") val allNulls = @@ -152,7 +152,7 @@ object TestData { NullInts(null) :: NullInts(null) :: NullInts(null) :: - NullInts(null) :: Nil) + NullInts(null) :: Nil).toDF allNulls.registerTempTable("allNulls") case class NullStrings(n: Int, s: String) @@ -160,11 +160,11 @@ object TestData { TestSQLContext.sparkContext.parallelize( NullStrings(1, "abc") :: NullStrings(2, "ABC") :: - NullStrings(3, null) :: Nil).toDataFrame + NullStrings(3, null) :: Nil).toDF nullStrings.registerTempTable("nullStrings") case class TableName(tableName: String) - TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerTempTable("tableName") + TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).toDF.registerTempTable("tableName") val unparsedStrings = TestSQLContext.sparkContext.parallelize( @@ -177,22 +177,22 @@ object TestData { val timestamps = TestSQLContext.sparkContext.parallelize((1 to 3).map { i => TimestampField(new Timestamp(i)) }) - timestamps.registerTempTable("timestamps") + timestamps.toDF.registerTempTable("timestamps") case class IntField(i: Int) // An RDD with 4 elements and 8 partitions val withEmptyParts = TestSQLContext.sparkContext.parallelize((1 to 4).map(IntField), 8) - withEmptyParts.registerTempTable("withEmptyParts") + withEmptyParts.toDF.registerTempTable("withEmptyParts") case class Person(id: Int, name: String, age: Int) case class Salary(personId: Int, salary: Double) val person = TestSQLContext.sparkContext.parallelize( Person(0, "mike", 30) :: - Person(1, "jim", 20) :: Nil) + Person(1, "jim", 20) :: Nil).toDF person.registerTempTable("person") val salary = TestSQLContext.sparkContext.parallelize( Salary(0, 2000.0) :: - Salary(1, 1000.0) :: Nil) + Salary(1, 1000.0) :: Nil).toDF salary.registerTempTable("salary") case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) @@ -200,6 +200,6 @@ object TestData { TestSQLContext.sparkContext.parallelize( ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) - :: Nil).toDataFrame + :: Nil).toDF complexData.registerTempTable("complexData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 95923f9aad931..be105c6e83594 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql -import org.apache.spark.sql.Dsl.StringToColumn import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ +import TestSQLContext.implicits._ case class FunctionResult(f1: String, f2: String) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 3c1657cd5fc3a..5f21d990e2e5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -66,7 +66,7 @@ class UserDefinedTypeSuite extends QueryTest { val points = Seq( MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))) - val pointsRDD: RDD[MyLabeledPoint] = sparkContext.parallelize(points) + val pointsRDD = sparkContext.parallelize(points).toDF() test("register user type: MyDenseVector for MyLabeledPoint") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 86b1b5fda1c0f..38b0f666ab90b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.columnar -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -28,8 +29,6 @@ class InMemoryColumnarQuerySuite extends QueryTest { // Make sure the tables are loaded. TestData - import org.apache.spark.sql.test.TestSQLContext.implicits._ - test("simple columnar query") { val plan = executePlan(testData.logicalPlan).executedPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) @@ -39,7 +38,8 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("default size avoids broadcast") { // TODO: Improve this test when we have better statistics - sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).registerTempTable("sizeTst") + sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) + .toDF().registerTempTable("sizeTst") cacheTable("sizeTst") assert( table("sizeTst").queryExecution.logical.statistics.sizeInBytes > diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 55a9f735b3506..e57bb06e7263b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -21,13 +21,12 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { val originalColumnBatchSize = conf.columnBatchSize val originalInMemoryPartitionPruning = conf.inMemoryPartitionPruning - import org.apache.spark.sql.test.TestSQLContext.implicits._ - override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch setConf(SQLConf.COLUMN_BATCH_SIZE, "10") @@ -35,7 +34,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be val pruningData = sparkContext.makeRDD((1 to 100).map { key => val string = if (((key - 1) / 10) % 2 == 0) null else key.toString TestData(key, string) - }, 5) + }, 5).toDF() pruningData.registerTempTable("pruningData") // Enable in-memory partition pruning diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index c3210733f1d42..523be56df65ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution import org.scalatest.FunSuite import org.apache.spark.sql.{SQLConf, execution} -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.test.TestSQLContext.planner._ import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index b5f13f8bd5e80..c94e44bd7c397 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -21,11 +21,12 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLConf} @@ -822,7 +823,7 @@ class JsonSuite extends QueryTest { val df1 = createDataFrame(rowRDD1, schema1) df1.registerTempTable("applySchema1") - val df2 = df1.toDataFrame + val df2 = df1.toDF val result = df2.toJSON.collect() assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") @@ -843,7 +844,7 @@ class JsonSuite extends QueryTest { val df3 = createDataFrame(rowRDD2, schema2) df3.registerTempTable("applySchema2") - val df4 = df3.toDataFrame + val df4 = df3.toDF val result2 = df4.toJSON.collect() assert(result2(1) === "{\"f1\":{\"f11\":2,\"f12\":false},\"f2\":{\"B2\":null}}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index c8ebbbc7d2eac..c306330818c0a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -33,11 +33,12 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf} -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types.DecimalType // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport @@ -64,6 +65,7 @@ private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteS * A test suite that tests basic Parquet I/O. */ class ParquetIOSuite extends QueryTest with ParquetTest { + val sqlContext = TestSQLContext /** @@ -99,12 +101,12 @@ class ParquetIOSuite extends QueryTest with ParquetTest { } test(s"$prefix: fixed-length decimals") { - import org.apache.spark.sql.test.TestSQLContext.implicits._ def makeDecimalRDD(decimal: DecimalType): DataFrame = sparkContext .parallelize(0 to 1000) .map(i => Tuple1(i / 100.0)) + .toDF // Parquet doesn't allow column names with spaces, have to add an alias here .select($"_1" cast decimal as "dec") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 89b18c3439cf6..9fcb04ca23590 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -37,7 +37,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ val testData = TestHive.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))) + (1 to 100).map(i => TestData(i, i.toString))).toDF before { // Since every we are doing tests for DDL statements, @@ -56,7 +56,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { // Make sure the table has also been updated. checkAnswer( sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq.map(Row.fromTuple) + testData.collect().toSeq ) // Add more data. @@ -65,7 +65,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { // Make sure the table has been updated. checkAnswer( sql("SELECT * FROM createAndInsertTest"), - testData.toDataFrame.collect().toSeq ++ testData.toDataFrame.collect().toSeq + testData.toDF.collect().toSeq ++ testData.toDF.collect().toSeq ) // Now overwrite. @@ -74,7 +74,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { // Make sure the registered table has also been updated. checkAnswer( sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq.map(Row.fromTuple) + testData.collect().toSeq ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala index 068aa03330c33..321b784a3f842 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -29,7 +29,7 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { import org.apache.spark.sql.hive.test.TestHive.implicits._ val df = - sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDataFrame("key", "value") + sparkContext.parallelize((1 to 10).map(i => (i,s"str$i"))).toDF("key", "value") override def beforeAll(): Unit = { // The catalog in HiveContext is a case insensitive one. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 2916724f66e24..addf887ab9162 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -28,17 +28,14 @@ import org.apache.spark.sql.catalyst.util import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ - -/* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.implicits._ /** * Tests for persisting tables created though the data sources API into the metastore. */ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { - import org.apache.spark.sql.hive.test.TestHive.implicits._ - override def afterEach(): Unit = { reset() if (tempPath.exists()) Utils.deleteRecursively(tempPath) @@ -154,7 +151,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("check change without refresh") { val tempDir = File.createTempFile("sparksql", "json") tempDir.delete() - sparkContext.parallelize(("a", "b") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + sparkContext.parallelize(("a", "b") :: Nil).toDF + .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql( s""" @@ -170,7 +168,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row("a", "b")) FileUtils.deleteDirectory(tempDir) - sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF + .toJSON.saveAsTextFile(tempDir.getCanonicalPath) // Schema is cached so the new column does not show. The updated values in existing columns // will show. @@ -190,7 +189,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("drop, change, recreate") { val tempDir = File.createTempFile("sparksql", "json") tempDir.delete() - sparkContext.parallelize(("a", "b") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + sparkContext.parallelize(("a", "b") :: Nil).toDF + .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql( s""" @@ -206,7 +206,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row("a", "b")) FileUtils.deleteDirectory(tempDir) - sparkContext.parallelize(("a", "b", "c") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + sparkContext.parallelize(("a", "b", "c") :: Nil).toDF + .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql("DROP TABLE jsonTable") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 405b200d05412..d01dbf80ef66d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.{SparkFiles, SparkException} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.Dsl._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -567,7 +567,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.registerTempTable("REGisteredTABle") + testData.toDF.registerTempTable("REGisteredTABle") assertResult(Array(Row(2, "str2"))) { sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -592,7 +592,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).registerTempTable("having_test") + TestHive.sparkContext.parallelize(fixture).toDF.registerTempTable("having_test") val results = sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() @@ -740,7 +740,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.registerTempTable("test_describe_commands2") + testData.toDF.registerTempTable("test_describe_commands2") assertResult( Array( @@ -900,8 +900,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { - sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") - sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") + sparkContext.makeRDD(Seq.empty[LogEntry]).toDF.registerTempTable("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).toDF.registerTempTable("logFiles") sql( """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 029c36aa89b26..6fc4cc14265ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -77,7 +77,7 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .registerTempTable("caseSensitivityTest") + .toDF.registerTempTable("caseSensitivityTest") val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), @@ -88,14 +88,14 @@ class HiveResolutionSuite extends HiveComparisonTest { ignore("case insensitivity with scala reflection joins") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .registerTempTable("caseSensitivityTest") + .toDF.registerTempTable("caseSensitivityTest") sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .registerTempTable("nestedRepeatedTest") + .toDF.registerTempTable("nestedRepeatedTest") assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 8fb5e050a237a..ab53c6309e089 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.Row -import org.apache.spark.sql.Dsl._ +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._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 1e99003d3e9b5..245161d2ebbca 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -111,7 +111,7 @@ class HiveUdfSuite extends QueryTest { test("UDFIntegerToString") { val testData = TestHive.sparkContext.parallelize( - IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil) + IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF testData.registerTempTable("integerTable") sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") @@ -127,7 +127,7 @@ class HiveUdfSuite extends QueryTest { val testData = TestHive.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: - ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil) + ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF testData.registerTempTable("listListIntTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") @@ -142,7 +142,7 @@ class HiveUdfSuite extends QueryTest { test("UDFListString") { val testData = TestHive.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: - ListStringCaseClass(Seq("d", "e")) :: Nil) + ListStringCaseClass(Seq("d", "e")) :: Nil).toDF testData.registerTempTable("listStringTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") @@ -156,7 +156,7 @@ class HiveUdfSuite extends QueryTest { test("UDFStringString") { val testData = TestHive.sparkContext.parallelize( - StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil) + StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF testData.registerTempTable("stringTable") sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") @@ -173,7 +173,7 @@ class HiveUdfSuite extends QueryTest { ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: - Nil) + Nil).toDF testData.registerTempTable("TwoListTable") sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 9a6e8650a0ec4..978825938395f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.HiveShim 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._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLConf} @@ -34,9 +35,6 @@ case class Nested3(f3: Int) */ class SQLQuerySuite extends QueryTest { - import org.apache.spark.sql.hive.test.TestHive.implicits._ - val sqlCtx = TestHive - test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { checkAnswer( sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"), @@ -176,7 +174,8 @@ class SQLQuerySuite extends QueryTest { } test("double nested data") { - sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil).registerTempTable("nested") + sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil) + .toDF().registerTempTable("nested") checkAnswer( sql("SELECT f1.f2.f3 FROM nested"), Row(1)) @@ -199,7 +198,7 @@ class SQLQuerySuite extends QueryTest { } test("SPARK-4825 save join to table") { - val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)) + val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF() sql("CREATE TABLE test1 (key INT, value STRING)") testData.insertInto("test1") sql("CREATE TABLE test2 (key INT, value STRING)") @@ -279,7 +278,7 @@ class SQLQuerySuite extends QueryTest { val rowRdd = sparkContext.parallelize(row :: Nil) - sqlCtx.createDataFrame(rowRdd, schema).registerTempTable("testTable") + TestHive.createDataFrame(rowRdd, schema).registerTempTable("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index a7479a5b95864..e246cbb6d77f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -27,6 +27,8 @@ import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.execution.PhysicalRDD import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.implicits._ + // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -152,7 +154,6 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll var normalTableDir: File = null var partitionedTableDirWithKey: File = null - import org.apache.spark.sql.hive.test.TestHive.implicits._ override def beforeAll(): Unit = { partitionedTableDir = File.createTempFile("parquettests", "sparksql") @@ -167,12 +168,14 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll val partDir = new File(partitionedTableDir, s"p=$p") sparkContext.makeRDD(1 to 10) .map(i => ParquetData(i, s"part-$p")) + .toDF() .saveAsParquetFile(partDir.getCanonicalPath) } sparkContext .makeRDD(1 to 10) .map(i => ParquetData(i, s"part-1")) + .toDF() .saveAsParquetFile(new File(normalTableDir, "normal").getCanonicalPath) partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") @@ -183,6 +186,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll val partDir = new File(partitionedTableDirWithKey, s"p=$p") sparkContext.makeRDD(1 to 10) .map(i => ParquetDataWithKey(p, i, s"part-$p")) + .toDF() .saveAsParquetFile(partDir.getCanonicalPath) } } From f80e2629bb74bc62960c61ff313f7e7802d61319 Mon Sep 17 00:00:00 2001 From: gasparms Date: Sat, 14 Feb 2015 20:10:29 +0000 Subject: [PATCH 115/817] [SPARK-5800] Streaming Docs. Change linked files according the selected language Currently, Spark Streaming Programming Guide after updateStateByKey explanation links to file stateful_network_wordcount.py and note "For the complete Scala code ..." for any language tab selected. This is an incoherence. I've changed the guide and link its pertinent example file. JavaStatefulNetworkWordCount.java example was not created so I added to the commit. Author: gasparms Closes #4589 from gasparms/feature/streaming-guide and squashes the following commits: 7f37f89 [gasparms] More style changes ec202b0 [gasparms] Follow spark style guide f527328 [gasparms] Improve example to look like scala example 4d8785c [gasparms] Remove throw exception e92e6b8 [gasparms] Fix incoherence 92db405 [gasparms] Fix Streaming Programming Guide. Change files according the selected language --- docs/streaming-programming-guide.md | 21 +++- .../JavaStatefulNetworkWordCount.java | 115 ++++++++++++++++++ 2 files changed, 132 insertions(+), 4 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 96fb12ce5e0b9..997de9511ca3e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -878,6 +878,12 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi val runningCounts = pairs.updateStateByKey[Int](updateFunction _) {% endhighlight %} +The update function will be called for each word, with `newValues` having a sequence of 1's (from +the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete +Scala code, take a look at the example +[StatefulNetworkWordCount.scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache +/spark/examples/streaming/StatefulNetworkWordCount.scala). +
    @@ -899,6 +905,13 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi JavaPairDStream runningCounts = pairs.updateStateByKey(updateFunction); {% endhighlight %} +The update function will be called for each word, with `newValues` having a sequence of 1's (from +the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete +Java code, take a look at the example +[JavaStatefulNetworkWordCount.java]({{site +.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming +/JavaStatefulNetworkWordCount.java). +
    @@ -916,14 +929,14 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi runningCounts = pairs.updateStateByKey(updateFunction) {% endhighlight %} -
    -
    - The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete -Scala code, take a look at the example +Python code, take a look at the example [stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py). +
    +
    + Note that using `updateStateByKey` requires the checkpoint directory to be configured, which is discussed in detail in the [checkpointing](#checkpointing) section. diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java new file mode 100644 index 0000000000000..09491fe300822 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import java.util.Arrays; +import java.util.List; +import java.util.regex.Pattern; + +import scala.Tuple2; + +import com.google.common.base.Optional; +import com.google.common.collect.Lists; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.StorageLevels; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + + +/** + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every + * second starting with initial value of word count. + * Usage: JavaStatefulNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive + * data. + *

    + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example + * org.apache.spark.examples.streaming.JavaStatefulNetworkWordCount localhost 9999` + */ +public class JavaStatefulNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: JavaStatefulNetworkWordCount "); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Update the cumulative count function + final Function2, Optional, Optional> updateFunction = new + Function2, Optional, Optional>() { + @Override + public Optional call(List values, Optional state) { + Integer newSum = state.or(0); + for (Integer value : values) { + newSum += value; + } + return Optional.of(newSum); + } + }; + + // Create the context with a 1 second batch size + SparkConf sparkConf = new SparkConf().setAppName("JavaStatefulNetworkWordCount"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1)); + ssc.checkpoint("."); + + // Initial RDD input to updateStateByKey + List> tuples = Arrays.asList(new Tuple2("hello", 1), + new Tuple2("world", 1)); + JavaPairRDD initialRDD = ssc.sc().parallelizePairs(tuples); + + JavaReceiverInputDStream lines = ssc.socketTextStream( + args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER_2); + + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + + JavaPairDStream wordsDstream = words.mapToPair(new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }); + + // This will give a Dstream made of state (which is the cumulative count of the words) + JavaPairDStream stateDstream = wordsDstream.updateStateByKey(updateFunction, + new HashPartitioner(ssc.sc().defaultParallelism()), initialRDD); + + stateDstream.print(); + ssc.start(); + ssc.awaitTermination(); + } +} From 15a2ab5f89d56e67c84e7163d28d93e72583393c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 14 Feb 2015 20:12:29 +0000 Subject: [PATCH 116/817] Revise formatting of previous commit f80e2629bb74bc62960c61ff313f7e7802d61319 --- .../JavaStatefulNetworkWordCount.java | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java index 09491fe300822..d46c7107c7a21 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -39,7 +39,6 @@ import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; - /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every * second starting with initial value of word count. @@ -65,17 +64,17 @@ public static void main(String[] args) { StreamingExamples.setStreamingLogLevels(); // Update the cumulative count function - final Function2, Optional, Optional> updateFunction = new - Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - Integer newSum = state.or(0); - for (Integer value : values) { - newSum += value; - } - return Optional.of(newSum); - } - }; + final Function2, Optional, Optional> updateFunction = + new Function2, Optional, Optional>() { + @Override + public Optional call(List values, Optional state) { + Integer newSum = state.or(0); + for (Integer value : values) { + newSum += value; + } + return Optional.of(newSum); + } + }; // Create the context with a 1 second batch size SparkConf sparkConf = new SparkConf().setAppName("JavaStatefulNetworkWordCount"); @@ -97,12 +96,13 @@ public Iterable call(String x) { } }); - JavaPairDStream wordsDstream = words.mapToPair(new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }); + JavaPairDStream wordsDstream = words.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }); // This will give a Dstream made of state (which is the cumulative count of the words) JavaPairDStream stateDstream = wordsDstream.updateStateByKey(updateFunction, From ed5f4bb7cb2c934b818d1e8b8b4e6a0056119c80 Mon Sep 17 00:00:00 2001 From: gli Date: Sat, 14 Feb 2015 20:43:27 +0000 Subject: [PATCH 117/817] SPARK-5822 [BUILD] cannot import src/main/scala & src/test/scala into eclipse as source folder When import the whole project into eclipse as maven project, found that the src/main/scala & src/test/scala can not be set as source folder as default behavior, so add a "add-source" goal in scala-maven-plugin to let this work. Author: gli Closes #4531 from ligangty/addsource and squashes the following commits: 4e4db4c [gli] [IDE] cannot import src/main/scala & src/test/scala into eclipse as source folder --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 53372d5cfc624..6810d71be4230 100644 --- a/pom.xml +++ b/pom.xml @@ -1083,6 +1083,12 @@ scala-maven-plugin 3.2.0 + + eclipse-add-source + + add-source + + scala-compile-first process-resources From c771e475c449fe07cf45f37bdca2ba6ce9600bfc Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 15 Feb 2015 14:42:20 +0000 Subject: [PATCH 118/817] [SPARK-5827][SQL] Add missing import in the example of SqlContext If one tries an example by using copy&paste, throw an exception. Author: Takeshi Yamamuro Closes #4615 from maropu/AddMissingImportInSqlContext and squashes the following commits: ab21b66 [Takeshi Yamamuro] Add missing import in the example of SqlContext --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a1736d0277f1c..6d1914808aeed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -286,6 +286,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Example: * {{{ * import org.apache.spark.sql._ + * import org.apache.spark.sql.types._ * val sqlContext = new org.apache.spark.sql.SQLContext(sc) * * val schema = @@ -377,6 +378,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Example: * {{{ * import org.apache.spark.sql._ + * import org.apache.spark.sql.types._ * val sqlContext = new org.apache.spark.sql.SQLContext(sc) * * val schema = From 61eb12674b90143388a01c22bf51cb7d02ab0447 Mon Sep 17 00:00:00 2001 From: martinzapletal Date: Sun, 15 Feb 2015 09:10:03 -0800 Subject: [PATCH 119/817] [MLLIB][SPARK-5502] User guide for isotonic regression User guide for isotonic regression added to docs/mllib-regression.md including code examples for Scala and Java. Author: martinzapletal Closes #4536 from zapletal-martin/SPARK-5502 and squashes the following commits: 67fe773 [martinzapletal] SPARK-5502 reworded model prediction rules to use more general language rather than the code/implementation specific terms 80bd4c3 [martinzapletal] SPARK-5502 created docs page for isotonic regression, added links to the page, updated data and examples 7d8136e [martinzapletal] SPARK-5502 Added documentation for Isotonic regression including examples for Scala and Java 504b5c3 [martinzapletal] SPARK-5502 Added documentation for Isotonic regression including examples for Scala and Java --- .../mllib/sample_isotonic_regression_data.txt | 100 +++++++++++ docs/mllib-classification-regression.md | 3 +- docs/mllib-guide.md | 1 + docs/mllib-isotonic-regression.md | 155 ++++++++++++++++++ 4 files changed, 258 insertions(+), 1 deletion(-) create mode 100644 data/mllib/sample_isotonic_regression_data.txt create mode 100644 docs/mllib-isotonic-regression.md diff --git a/data/mllib/sample_isotonic_regression_data.txt b/data/mllib/sample_isotonic_regression_data.txt new file mode 100644 index 0000000000000..d257b509d4d37 --- /dev/null +++ b/data/mllib/sample_isotonic_regression_data.txt @@ -0,0 +1,100 @@ +0.24579296,0.01 +0.28505864,0.02 +0.31208567,0.03 +0.35900051,0.04 +0.35747068,0.05 +0.16675166,0.06 +0.17491076,0.07 +0.04181540,0.08 +0.04793473,0.09 +0.03926568,0.10 +0.12952575,0.11 +0.00000000,0.12 +0.01376849,0.13 +0.13105558,0.14 +0.08873024,0.15 +0.12595614,0.16 +0.15247323,0.17 +0.25956145,0.18 +0.20040796,0.19 +0.19581846,0.20 +0.15757267,0.21 +0.13717491,0.22 +0.19020908,0.23 +0.19581846,0.24 +0.20091790,0.25 +0.16879143,0.26 +0.18510964,0.27 +0.20040796,0.28 +0.29576747,0.29 +0.43396226,0.30 +0.53391127,0.31 +0.52116267,0.32 +0.48546660,0.33 +0.49209587,0.34 +0.54156043,0.35 +0.59765426,0.36 +0.56144824,0.37 +0.58592555,0.38 +0.52983172,0.39 +0.50178480,0.40 +0.52626211,0.41 +0.58286588,0.42 +0.64660887,0.43 +0.68077511,0.44 +0.74298827,0.45 +0.64864865,0.46 +0.67261601,0.47 +0.65782764,0.48 +0.69811321,0.49 +0.63029067,0.50 +0.61601224,0.51 +0.63233044,0.52 +0.65323814,0.53 +0.65323814,0.54 +0.67363590,0.55 +0.67006629,0.56 +0.51555329,0.57 +0.50892402,0.58 +0.33299337,0.59 +0.36206017,0.60 +0.43090260,0.61 +0.45996940,0.62 +0.56348802,0.63 +0.54920959,0.64 +0.48393677,0.65 +0.48495665,0.66 +0.46965834,0.67 +0.45181030,0.68 +0.45843957,0.69 +0.47118817,0.70 +0.51555329,0.71 +0.58031617,0.72 +0.55481897,0.73 +0.56297807,0.74 +0.56603774,0.75 +0.57929628,0.76 +0.64762876,0.77 +0.66241713,0.78 +0.69301377,0.79 +0.65119837,0.80 +0.68332483,0.81 +0.66598674,0.82 +0.73890872,0.83 +0.73992861,0.84 +0.84242733,0.85 +0.91330954,0.86 +0.88016318,0.87 +0.90719021,0.88 +0.93115757,0.89 +0.93115757,0.90 +0.91942886,0.91 +0.92911780,0.92 +0.95665477,0.93 +0.95002550,0.94 +0.96940337,0.95 +1.00000000,0.96 +0.89801122,0.97 +0.90311066,0.98 +0.90362060,0.99 +0.83477817,1.0 \ No newline at end of file diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 719cc95767b00..5b9b4dd83b774 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -23,7 +23,7 @@ the supported algorithms for each type of problem. Multiclass Classificationdecision trees, naive Bayes - Regressionlinear least squares, Lasso, ridge regression, decision trees + Regressionlinear least squares, Lasso, ridge regression, decision trees, isotonic regression @@ -35,3 +35,4 @@ More details for these methods can be found here: * [linear regression (least squares, Lasso, ridge)](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) * [Decision trees](mllib-decision-tree.html) * [Naive Bayes](mllib-naive-bayes.html) +* [Isotonic regression](mllib-isotonic-regression.html) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 5091dbf329b97..fbe809b3478e5 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -21,6 +21,7 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [naive Bayes](mllib-naive-bayes.html) * [decision trees](mllib-decision-tree.html) * [ensembles of trees](mllib-ensembles.html) (Random Forests and Gradient-Boosted Trees) + * [isotonic regression](mllib-isotonic-regression.html) * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) * [Clustering](mllib-clustering.html) diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md new file mode 100644 index 0000000000000..12fb29d426741 --- /dev/null +++ b/docs/mllib-isotonic-regression.md @@ -0,0 +1,155 @@ +--- +layout: global +title: Naive Bayes - MLlib +displayTitle: MLlib - Regression +--- + +## Isotonic regression +[Isotonic regression](http://en.wikipedia.org/wiki/Isotonic_regression) +belongs to the family of regression algorithms. Formally isotonic regression is a problem where +given a finite set of real numbers `$Y = {y_1, y_2, ..., y_n}$` representing observed responses +and `$X = {x_1, x_2, ..., x_n}$` the unknown response values to be fitted +finding a function that minimises + +`\begin{equation} + f(x) = \sum_{i=1}^n w_i (y_i - x_i)^2 +\end{equation}` + +with respect to complete order subject to +`$x_1\le x_2\le ...\le x_n$` where `$w_i$` are positive weights. +The resulting function is called isotonic regression and it is unique. +It can be viewed as least squares problem under order restriction. +Essentially isotonic regression is a +[monotonic function](http://en.wikipedia.org/wiki/Monotonic_function) +best fitting the original data points. + +MLlib supports a +[pool adjacent violators algorithm](http://doi.org/10.1198/TECH.2010.10111) +which uses an approach to +[parallelizing isotonic regression](http://doi.org/10.1007/978-3-642-99789-1_10). +The training input is a RDD of tuples of three double values that represent +label, feature and weight in this order. Additionally IsotonicRegression algorithm has one +optional parameter called $isotonic$ defaulting to true. +This argument specifies if the isotonic regression is +isotonic (monotonically increasing) or antitonic (monotonically decreasing). + +Training returns an IsotonicRegressionModel that can be used to predict +labels for both known and unknown features. The result of isotonic regression +is treated as piecewise linear function. The rules for prediction therefore are: + +* If the prediction input exactly matches a training feature + then associated prediction is returned. In case there are multiple predictions with the same + feature then one of them is returned. Which one is undefined + (same as java.util.Arrays.binarySearch). +* If the prediction input is lower or higher than all training features + then prediction with lowest or highest feature is returned respectively. + In case there are multiple predictions with the same feature + then the lowest or highest is returned respectively. +* If the prediction input falls between two training features then prediction is treated + as piecewise linear function and interpolated value is calculated from the + predictions of the two closest features. In case there are multiple values + with the same feature then the same rules as in previous point are used. + +### Examples + +

    +
    +Data are read from a file where each line has a format label,feature +i.e. 4710.28,500.00. The data are split to training and testing set. +Model is created using the training set and a mean squared error is calculated from the predicted +labels and real labels in the test set. + +{% highlight scala %} +import org.apache.spark.mllib.regression.IsotonicRegression + +val data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt") + +// Create label, feature, weight tuples from input data with weight set to default value 1.0. +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + (parts(0), parts(1), 1.0) +} + +// Split data into training (60%) and test (40%) sets. +val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0) +val test = splits(1) + +// Create isotonic regression model from training data. +// Isotonic parameter defaults to true so it is only shown for demonstration +val model = new IsotonicRegression().setIsotonic(true).run(training) + +// Create tuples of predicted and real labels. +val predictionAndLabel = test.map { point => + val predictedLabel = model.predict(point._2) + (predictedLabel, point._1) +} + +// Calculate mean squared error between predicted and real labels. +val meanSquaredError = predictionAndLabel.map{case(p, l) => math.pow((p - l), 2)}.mean() +println("Mean Squared Error = " + meanSquaredError) +{% endhighlight %} +
    + +
    +Data are read from a file where each line has a format label,feature +i.e. 4710.28,500.00. The data are split to training and testing set. +Model is created using the training set and a mean squared error is calculated from the predicted +labels and real labels in the test set. + +{% highlight java %} +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.IsotonicRegressionModel; +import scala.Tuple2; +import scala.Tuple3; + +JavaRDD data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt"); + +// Create label, feature, weight tuples from input data with weight set to default value 1.0. +JavaRDD> parsedData = data.map( + new Function>() { + public Tuple3 call(String line) { + String[] parts = line.split(","); + return new Tuple3<>(new Double(parts[0]), new Double(parts[1]), 1.0); + } + } +); + +// Split data into training (60%) and test (40%) sets. +JavaRDD>[] splits = parsedData.randomSplit(new double[] {0.6, 0.4}, 11L); +JavaRDD> training = splits[0]; +JavaRDD> test = splits[1]; + +// Create isotonic regression model from training data. +// Isotonic parameter defaults to true so it is only shown for demonstration +IsotonicRegressionModel model = new IsotonicRegression().setIsotonic(true).run(training); + +// Create tuples of predicted and real labels. +JavaPairRDD predictionAndLabel = test.mapToPair( + new PairFunction, Double, Double>() { + @Override public Tuple2 call(Tuple3 point) { + Double predictedLabel = model.predict(point._2()); + return new Tuple2(predictedLabel, point._1()); + } + } +); + +// Calculate mean squared error between predicted and real labels. +Double meanSquaredError = new JavaDoubleRDD(predictionAndLabel.map( + new Function, Object>() { + @Override public Object call(Tuple2 pl) { + return Math.pow(pl._1() - pl._2(), 2); + } + } +).rdd()).mean(); + +System.out.println("Mean Squared Error = " + meanSquaredError); +{% endhighlight %} +
    +
    \ No newline at end of file From 836577b382695558f5c97d94ee725d0156ebfad2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 15 Feb 2015 09:15:48 -0800 Subject: [PATCH 120/817] SPARK-5669 [BUILD] Spark assembly includes incompatibly licensed libgfortran, libgcc code via JBLAS Exclude libgfortran, libgcc bundled by JBLAS for Windows. This much is simple, and solves the essential license issue. But the more important question is whether MLlib works on Windows then. Author: Sean Owen Closes #4453 from srowen/SPARK-5669 and squashes the following commits: 734dd86 [Sean Owen] Exclude libgfortran, libgcc bundled by JBLAS, affecting Windows / OS X / Linux 32-bit (not Linux 64-bit) --- assembly/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/assembly/pom.xml b/assembly/pom.xml index fa9f56e556d8b..fbb6e94839d42 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -114,6 +114,16 @@ META-INF/*.RSA + + + org.jblas:jblas + + + lib/Linux/i386/** + lib/Mac OS X/** + lib/Windows/** + + From cd4a15366244657c4b7936abe5054754534366f2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 15 Feb 2015 20:29:26 -0800 Subject: [PATCH 121/817] [SPARK-5769] Set params in constructors and in setParams in Python ML pipelines This PR allow Python users to set params in constructors and in setParams, where we use decorator `keyword_only` to force keyword arguments. The trade-off is discussed in the design doc of SPARK-4586. Generated doc: ![screen shot 2015-02-12 at 3 06 58 am](https://cloud.githubusercontent.com/assets/829644/6166491/9cfcd06a-b265-11e4-99ea-473d866634fc.png) CC: davies rxin Author: Xiangrui Meng Closes #4564 from mengxr/py-pipeline-kw and squashes the following commits: fedf720 [Xiangrui Meng] use toDF d565f2c [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into py-pipeline-kw cbc15d3 [Xiangrui Meng] fix style 5032097 [Xiangrui Meng] update pipeline signature 950774e [Xiangrui Meng] simplify keyword_only and update constructor/setParams signatures fdde5fc [Xiangrui Meng] fix style c9384b8 [Xiangrui Meng] fix sphinx doc 8e59180 [Xiangrui Meng] add setParams and make constructors take params, where we force keyword args --- .../ml/simple_text_classification_pipeline.py | 44 +++++------- python/docs/conf.py | 4 ++ python/pyspark/ml/classification.py | 44 +++++++++--- python/pyspark/ml/feature.py | 72 +++++++++++++++---- python/pyspark/ml/param/__init__.py | 8 +++ python/pyspark/ml/pipeline.py | 19 ++++- python/pyspark/ml/util.py | 15 ++++ 7 files changed, 153 insertions(+), 53 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index c7df3d7b74767..b4d9355b681f6 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -36,43 +36,33 @@ sqlCtx = SQLContext(sc) # Prepare training documents, which are labeled. - LabeledDocument = Row('id', 'text', 'label') - training = sqlCtx.inferSchema( - sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) - .map(lambda x: LabeledDocument(*x))) + LabeledDocument = Row("id", "text", "label") + training = sc.parallelize([(0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0)]) \ + .map(lambda x: LabeledDocument(*x)).toDF() # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. - tokenizer = Tokenizer() \ - .setInputCol("text") \ - .setOutputCol("words") - hashingTF = HashingTF() \ - .setInputCol(tokenizer.getOutputCol()) \ - .setOutputCol("features") - lr = LogisticRegression() \ - .setMaxIter(10) \ - .setRegParam(0.01) - pipeline = Pipeline() \ - .setStages([tokenizer, hashingTF, lr]) + tokenizer = Tokenizer(inputCol="text", outputCol="words") + hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") + lr = LogisticRegression(maxIter=10, regParam=0.01) + pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) # Fit the pipeline to training documents. model = pipeline.fit(training) # Prepare test documents, which are unlabeled. - Document = Row('id', 'text') - test = sqlCtx.inferSchema( - sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) - .map(lambda x: Document(*x))) + Document = Row("id", "text") + test = sc.parallelize([(4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop")]) \ + .map(lambda x: Document(*x)).toDF() # Make predictions on test documents and print columns of interest. prediction = model.transform(test) - prediction.registerTempTable("prediction") - selected = sqlCtx.sql("SELECT id, text, prediction from prediction") + selected = prediction.select("id", "text", "prediction") for row in selected.collect(): print row diff --git a/python/docs/conf.py b/python/docs/conf.py index b00dce95d65b4..cbbf7ffb08992 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -97,6 +97,10 @@ # If true, keep warnings as "system message" paragraphs in the built documents. #keep_warnings = False +# -- Options for autodoc -------------------------------------------------- + +# Look at the first line of the docstring for function and method signatures. +autodoc_docstring_signature = True # -- Options for HTML output ---------------------------------------------- diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 6bd2aa8e47837..b6de7493d7523 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark.ml.util import inherit_doc +from pyspark.ml.util import inherit_doc, keyword_only from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam @@ -32,22 +32,46 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> from pyspark.sql import Row >>> from pyspark.mllib.linalg import Vectors - >>> dataset = sqlCtx.inferSchema(sc.parallelize([ \ - Row(label=1.0, features=Vectors.dense(1.0)), \ - Row(label=0.0, features=Vectors.sparse(1, [], []))])) - >>> lr = LogisticRegression() \ - .setMaxIter(5) \ - .setRegParam(0.01) - >>> model = lr.fit(dataset) - >>> test0 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.dense(-1.0))])) + >>> df = sc.parallelize([ + ... Row(label=1.0, features=Vectors.dense(1.0)), + ... Row(label=0.0, features=Vectors.sparse(1, [], []))]).toDF() + >>> lr = LogisticRegression(maxIter=5, regParam=0.01) + >>> model = lr.fit(df) + >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0))]).toDF() >>> print model.transform(test0).head().prediction 0.0 - >>> test1 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))])) + >>> test1 = sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))]).toDF() >>> print model.transform(test1).head().prediction 1.0 + >>> lr.setParams("vector") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. """ _java_class = "org.apache.spark.ml.classification.LogisticRegression" + @keyword_only + def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", + maxIter=100, regParam=0.1): + """ + __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + maxIter=100, regParam=0.1) + """ + super(LogisticRegression, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", + maxIter=100, regParam=0.1): + """ + setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + maxIter=100, regParam=0.1) + Sets params for logistic regression. + """ + kwargs = self.setParams._input_kwargs + return self._set_params(**kwargs) + def _create_model(self, java_model): return LogisticRegressionModel(java_model) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index e088acd0ca82d..f1ddbb478dd9c 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -16,7 +16,7 @@ # from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures -from pyspark.ml.util import inherit_doc +from pyspark.ml.util import inherit_doc, keyword_only from pyspark.ml.wrapper import JavaTransformer __all__ = ['Tokenizer', 'HashingTF'] @@ -29,18 +29,45 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): splits it by white spaces. >>> from pyspark.sql import Row - >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(text="a b c")])) - >>> tokenizer = Tokenizer() \ - .setInputCol("text") \ - .setOutputCol("words") - >>> print tokenizer.transform(dataset).head() + >>> df = sc.parallelize([Row(text="a b c")]).toDF() + >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") + >>> print tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(dataset, {tokenizer.outputCol: "tokens"}).head() + >>> # Change a parameter. + >>> print tokenizer.setParams(outputCol="tokens").transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Temporarily modify a parameter. + >>> print tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> print tokenizer.transform(df).head() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + >>> # Must use keyword arguments to specify params. + >>> tokenizer.setParams("text") + Traceback (most recent call last): + ... + TypeError: Method setParams forces keyword arguments. """ _java_class = "org.apache.spark.ml.feature.Tokenizer" + @keyword_only + def __init__(self, inputCol="input", outputCol="output"): + """ + __init__(self, inputCol="input", outputCol="output") + """ + super(Tokenizer, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inputCol="input", outputCol="output"): + """ + setParams(self, inputCol="input", outputCol="output") + Sets params for this Tokenizer. + """ + kwargs = self.setParams._input_kwargs + return self._set_params(**kwargs) + @inherit_doc class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): @@ -49,20 +76,37 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): hashing trick. >>> from pyspark.sql import Row - >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(words=["a", "b", "c"])])) - >>> hashingTF = HashingTF() \ - .setNumFeatures(10) \ - .setInputCol("words") \ - .setOutputCol("features") - >>> print hashingTF.transform(dataset).head().features + >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() + >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") + >>> print hashingTF.transform(df).head().features + (10,[7,8,9],[1.0,1.0,1.0]) + >>> print hashingTF.setParams(outputCol="freqs").transform(df).head().freqs (10,[7,8,9],[1.0,1.0,1.0]) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(dataset, params).head().vector + >>> print hashingTF.transform(df, params).head().vector (5,[2,3,4],[1.0,1.0,1.0]) """ _java_class = "org.apache.spark.ml.feature.HashingTF" + @keyword_only + def __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + """ + __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + """ + super(HashingTF, self).__init__() + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + """ + setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + Sets params for this HashingTF. + """ + kwargs = self.setParams._input_kwargs + return self._set_params(**kwargs) + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 5566792cead48..e3a53dd780c4c 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -80,3 +80,11 @@ def _dummy(): dummy = Params() dummy.uid = "undefined" return dummy + + def _set_params(self, **kwargs): + """ + Sets params. + """ + for param, value in kwargs.iteritems(): + self.paramMap[getattr(self, param)] = value + return self diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 2d239f8c802a0..18d8a58f357bd 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -18,7 +18,7 @@ from abc import ABCMeta, abstractmethod from pyspark.ml.param import Param, Params -from pyspark.ml.util import inherit_doc +from pyspark.ml.util import inherit_doc, keyword_only __all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] @@ -89,10 +89,16 @@ class Pipeline(Estimator): identity transformer. """ - def __init__(self): + @keyword_only + def __init__(self, stages=[]): + """ + __init__(self, stages=[]) + """ super(Pipeline, self).__init__() #: Param for pipeline stages. self.stages = Param(self, "stages", "pipeline stages") + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) def setStages(self, value): """ @@ -110,6 +116,15 @@ def getStages(self): if self.stages in self.paramMap: return self.paramMap[self.stages] + @keyword_only + def setParams(self, stages=[]): + """ + setParams(self, stages=[]) + Sets params for Pipeline. + """ + kwargs = self.setParams._input_kwargs + return self._set_params(**kwargs) + def fit(self, dataset, params={}): paramMap = self._merge_params(params) stages = paramMap[self.stages] diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index b1caa84b6306a..81d3f0882b8a9 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -15,6 +15,7 @@ # limitations under the License. # +from functools import wraps import uuid @@ -32,6 +33,20 @@ def inherit_doc(cls): return cls +def keyword_only(func): + """ + A decorator that forces keyword arguments in the wrapped method + and saves actual input keyword arguments in `_input_kwargs`. + """ + @wraps(func) + def wrapper(*args, **kwargs): + if len(args) > 1: + raise TypeError("Method %s forces keyword arguments." % func.__name__) + wrapper._input_kwargs = kwargs + return func(*args, **kwargs) + return wrapper + + class Identifiable(object): """ Object with a unique ID. From acf2558dc92901c342262c35eebb95f2a9b7a9ae Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 15 Feb 2015 20:41:27 -0800 Subject: [PATCH 122/817] SPARK-5815 [MLLIB] Deprecate SVDPlusPlus APIs that expose DoubleMatrix from JBLAS Deprecate SVDPlusPlus.run and introduce SVDPlusPlus.runSVDPlusPlus with return type that doesn't include DoubleMatrix CC mengxr Author: Sean Owen Closes #4614 from srowen/SPARK-5815 and squashes the following commits: 288cb05 [Sean Owen] Clarify deprecation plans in scaladoc 497458e [Sean Owen] Deprecate SVDPlusPlus.run and introduce SVDPlusPlus.runSVDPlusPlus with return type that doesn't include DoubleMatrix --- .../apache/spark/graphx/lib/SVDPlusPlus.scala | 25 +++++++++++++++++++ .../spark/graphx/lib/SVDPlusPlusSuite.scala | 2 +- 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 112ed09ef46dc..fc84cfbe64184 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -17,6 +17,8 @@ package org.apache.spark.graphx.lib +import org.apache.spark.annotation.Experimental + import scala.util.Random import org.jblas.DoubleMatrix import org.apache.spark.rdd._ @@ -38,6 +40,8 @@ object SVDPlusPlus { extends Serializable /** + * :: Experimental :: + * * Implement SVD++ based on "Factorization Meets the Neighborhood: * a Multifaceted Collaborative Filtering Model", * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. @@ -45,12 +49,33 @@ object SVDPlusPlus { * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), * see the details on page 6. * + * This method temporarily replaces `run()`, and replaces `DoubleMatrix` in `run()`'s return + * value with `Array[Double]`. In 1.4.0, this method will be deprecated, but will be copied + * to replace `run()`, which will then be undeprecated. + * * @param edges edges for constructing the graph * * @param conf SVDPlusPlus parameters * * @return a graph with vertex attributes containing the trained model */ + @Experimental + def runSVDPlusPlus(edges: RDD[Edge[Double]], conf: Conf) + : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = + { + val (graph, u) = run(edges, conf) + // Convert DoubleMatrix to Array[Double]: + val newVertices = graph.vertices.mapValues(v => (v._1.toArray, v._2.toArray, v._3, v._4)) + (Graph(newVertices, graph.edges), u) + } + + /** + * This method is deprecated in favor of `runSVDPlusPlus()`, which replaces `DoubleMatrix` + * with `Array[Double]` in its return value. This method is deprecated. It will effectively + * be removed in 1.4.0 when `runSVDPlusPlus()` is copied to replace `run()`, and hence the + * return type of this method changes. + */ + @deprecated("Call runSVDPlusPlus", "1.3.0") def run(edges: RDD[Edge[Double]], conf: Conf) : (Graph[(DoubleMatrix, DoubleMatrix, Double, Double), Double], Double) = { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index e01df56e94de9..9987a4b1a3c25 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -32,7 +32,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = SVDPlusPlus.run(edges, conf) + var (graph, u) = SVDPlusPlus.runSVDPlusPlus(edges, conf) graph.cache() val err = graph.vertices.collect().map{ case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 From c78a12c4cc4d4312c4ee1069d3b218882d32d678 Mon Sep 17 00:00:00 2001 From: Peter Rudenko Date: Sun, 15 Feb 2015 20:51:32 -0800 Subject: [PATCH 123/817] [Ml] SPARK-5796 Don't transform data on a last estimator in Pipeline If it's a last estimator in Pipeline there's no need to transform data, since there's no next stage that would consume this data. Author: Peter Rudenko Closes #4590 from petro-rudenko/patch-1 and squashes the following commits: d13ec33 [Peter Rudenko] [Ml] SPARK-5796 Don't transform data on a last estimator in Pipeline --- mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index bb291e6e1fd7d..5607ed21afe18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -114,7 +114,9 @@ class Pipeline extends Estimator[PipelineModel] { throw new IllegalArgumentException( s"Do not support stage $stage of type ${stage.getClass}") } - curDataset = transformer.transform(curDataset, paramMap) + if (index < indexOfLastEstimator) { + curDataset = transformer.transform(curDataset, paramMap) + } transformers += transformer } else { transformers += stage.asInstanceOf[Transformer] From d51d6ba1547ae75ac76c9e6d8ea99e937eb7d09f Mon Sep 17 00:00:00 2001 From: Peter Rudenko Date: Mon, 16 Feb 2015 00:07:23 -0800 Subject: [PATCH 124/817] [Ml] SPARK-5804 Explicitly manage cache in Crossvalidator k-fold loop On a big dataset explicitly unpersist train and validation folds allows to load more data into memory in the next loop iteration. On my environment (single node 8Gb worker RAM, 2 GB dataset file, 3 folds for cross validation), saved more than 5 minutes. Author: Peter Rudenko Closes #4595 from petro-rudenko/patch-2 and squashes the following commits: 66a7cfb [Peter Rudenko] Move validationDataset cache to declaration c5f3265 [Peter Rudenko] [Ml] SPARK-5804 Explicitly manage cache in Crossvalidator k-fold loop --- .../main/scala/org/apache/spark/ml/tuning/CrossValidator.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index b139bc8dcb44c..b07a68269cc2b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -108,6 +108,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP // multi-model training logDebug(s"Train split $splitIndex with multiple sets of parameters.") val models = est.fit(trainingDataset, epm).asInstanceOf[Seq[Model[_]]] + trainingDataset.unpersist() var i = 0 while (i < numModels) { val metric = eval.evaluate(models(i).transform(validationDataset, epm(i)), map) @@ -115,6 +116,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP metrics(i) += metric i += 1 } + validationDataset.unpersist() } f2jBLAS.dscal(numModels, 1.0 / map(numFolds), metrics, 1) logInfo(s"Average cross-validation metrics: ${metrics.toSeq}") From 199a9e80275ac70582ea32f0f2f5a0a15b168785 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 16 Feb 2015 01:33:37 -0800 Subject: [PATCH 125/817] [Minor] [SQL] Renames stringRddToDataFrame to stringRddToDataFrameHolder for consistency [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4613) Author: Cheng Lian Closes #4613 from liancheng/df-implicit-rename and squashes the following commits: db8bdd3 [Cheng Lian] Renames stringRddToDataFrame to stringRddToDataFrameHolder for consistency --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 6d1914808aeed..d20b42de22706 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -235,7 +235,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** Creates a single column DataFrame from an RDD[String]. */ - implicit def stringRddToDataFrame(data: RDD[String]): DataFrameHolder = { + implicit def stringRddToDataFrameHolder(data: RDD[String]): DataFrameHolder = { val dataType = StringType val rows = data.mapPartitions { iter => val row = new SpecificMutableRow(dataType :: Nil) From 3ce58cf9c0ffe8b867ca79b404fe3fa291cf0e56 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 16 Feb 2015 01:38:31 -0800 Subject: [PATCH 126/817] [SPARK-4553] [SPARK-5767] [SQL] Wires Parquet data source with the newly introduced write support for data source API This PR migrates the Parquet data source to the new data source write support API. Now users can also overwriting and appending to existing tables. Notice that inserting into partitioned tables is not supported yet. When Parquet data source is enabled, insertion to Hive Metastore Parquet tables is also fullfilled by the Parquet data source. This is done by the newly introduced `HiveMetastoreCatalog.ParquetConversions` rule, which is a "proper" implementation of the original hacky `HiveStrategies.ParquetConversion`. The latter is still preserved, and can be removed together with the old Parquet support in the future. TODO: - [x] Update outdated comments in `newParquet.scala`. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4563) Author: Cheng Lian Closes #4563 from liancheng/parquet-refining and squashes the following commits: fa98d27 [Cheng Lian] Fixes test cases which should disable off Parquet data source 2476e82 [Cheng Lian] Fixes compilation error introduced during rebasing a83d290 [Cheng Lian] Passes Hive Metastore partitioning information to ParquetRelation2 --- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/parquet/ParquetTest.scala | 38 +- .../apache/spark/sql/parquet/newParquet.scala | 265 ++++++----- .../sql/parquet/ParquetFilterSuite.scala | 343 +++++++------- .../spark/sql/parquet/ParquetIOSuite.scala | 417 ++++++++++-------- .../ParquetPartitionDiscoverySuite.scala | 202 ++++++++- .../spark/sql/parquet/ParquetQuerySuite.scala | 147 +++--- .../apache/spark/sql/hive/HiveContext.scala | 11 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 127 ++++-- .../spark/sql/hive/HiveStrategies.scala | 22 +- .../spark/sql/parquet/parquetSuites.scala | 247 +++++++---- 12 files changed, 1148 insertions(+), 675 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d20b42de22706..b42a52ebd2f16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -446,7 +446,7 @@ class SQLContext(@transient val sparkContext: SparkContext) baseRelationToDataFrame(parquet.ParquetRelation2(path +: paths, Map.empty)(this)) } else { DataFrame(this, parquet.ParquetRelation( - paths.mkString(","), Some(sparkContext.hadoopConfiguration), this)) + (path +: paths).mkString(","), Some(sparkContext.hadoopConfiguration), this)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 28cd17fde46ab..7dd8bea49b8a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -647,6 +647,6 @@ private[parquet] object FileSystemHelper { sys.error("ERROR: attempting to append to set of Parquet files and found file" + s"that does not match name pattern: $other") case _ => 0 - }.reduceLeft((a, b) => if (a < b) b else a) + }.reduceOption(_ max _).getOrElse(0) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index 538d774eb97eb..d0856df8d4f43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -23,8 +23,8 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Try -import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} import org.apache.spark.util.Utils /** @@ -37,7 +37,8 @@ import org.apache.spark.util.Utils trait ParquetTest { val sqlContext: SQLContext - import sqlContext._ + import sqlContext.implicits.{localSeqToDataFrameHolder, rddToDataFrameHolder} + import sqlContext.{conf, sparkContext} protected def configuration = sparkContext.hadoopConfiguration @@ -49,11 +50,11 @@ trait ParquetTest { */ protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { val (keys, values) = pairs.unzip - val currentValues = keys.map(key => Try(getConf(key)).toOption) - (keys, values).zipped.foreach(setConf) + val currentValues = keys.map(key => Try(conf.getConf(key)).toOption) + (keys, values).zipped.foreach(conf.setConf) try f finally { keys.zip(currentValues).foreach { - case (key, Some(value)) => setConf(key, value) + case (key, Some(value)) => conf.setConf(key, value) case (key, None) => conf.unsetConf(key) } } @@ -88,7 +89,6 @@ trait ParquetTest { protected def withParquetFile[T <: Product: ClassTag: TypeTag] (data: Seq[T]) (f: String => Unit): Unit = { - import sqlContext.implicits._ withTempPath { file => sparkContext.parallelize(data).toDF().saveAsParquetFile(file.getCanonicalPath) f(file.getCanonicalPath) @@ -102,14 +102,14 @@ trait ParquetTest { protected def withParquetRDD[T <: Product: ClassTag: TypeTag] (data: Seq[T]) (f: DataFrame => Unit): Unit = { - withParquetFile(data)(path => f(parquetFile(path))) + withParquetFile(data)(path => f(sqlContext.parquetFile(path))) } /** * Drops temporary table `tableName` after calling `f`. */ protected def withTempTable(tableName: String)(f: => Unit): Unit = { - try f finally dropTempTable(tableName) + try f finally sqlContext.dropTempTable(tableName) } /** @@ -125,4 +125,26 @@ trait ParquetTest { withTempTable(tableName)(f) } } + + protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( + data: Seq[T], path: File): Unit = { + data.toDF().save(path.getCanonicalPath, "org.apache.spark.sql.parquet", SaveMode.Overwrite) + } + + protected def makePartitionDir( + basePath: File, + defaultPartitionName: String, + partitionCols: (String, Any)*): File = { + val partNames = partitionCols.map { case (k, v) => + val valueString = if (v == null || v == "") defaultPartitionName else v.toString + s"$k=$valueString" + } + + val partDir = partNames.foldLeft(basePath) { (parent, child) => + new File(parent, child) + } + + assert(partDir.mkdirs(), s"Couldn't create directory $partDir") + partDir + } } 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 3a9f0600617be..9279f5a903f55 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 @@ -20,7 +20,7 @@ import java.io.IOException import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} import java.math.{BigDecimal => JBigDecimal} import java.text.SimpleDateFormat -import java.util.{List => JList, Date} +import java.util.{Date, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer @@ -34,8 +34,9 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter -import parquet.hadoop.{ParquetInputFormat, _} +import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{ParquetInputFormat, _} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil @@ -44,21 +45,36 @@ import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLConf, SQLContext} import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} -import org.apache.spark.{Partition => SparkPartition, TaskContext, SerializableWritable, Logging, SparkException} - +import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode} +import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWritable, SparkException, TaskContext} /** - * Allows creation of parquet based tables using the syntax - * `CREATE TEMPORARY TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option - * required is `path`, which should be the location of a collection of, optionally partitioned, - * parquet files. + * Allows creation of Parquet based tables using the syntax: + * {{{ + * CREATE TEMPORARY TABLE ... USING org.apache.spark.sql.parquet OPTIONS (...) + * }}} + * + * Supported options include: + * + * - `path`: Required. When reading Parquet files, `path` should point to the location of the + * Parquet file(s). It can be either a single raw Parquet file, or a directory of Parquet files. + * In the latter case, this data source tries to discover partitioning information if the the + * directory is structured in the same style of Hive partitioned tables. When writing Parquet + * file, `path` should point to the destination folder. + * + * - `mergeSchema`: Optional. Indicates whether we should merge potentially different (but + * compatible) schemas stored in all Parquet part-files. + * + * - `partition.defaultName`: Optional. Partition name used when a value of a partition column is + * null or empty string. This is similar to the `hive.exec.default.partition.name` configuration + * in Hive. */ class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { + private def checkPath(parameters: Map[String, String]): String = { parameters.getOrElse("path", sys.error("'path' must be specified for parquet tables.")) } @@ -70,6 +86,7 @@ class DefaultSource ParquetRelation2(Seq(checkPath(parameters)), parameters, None)(sqlContext) } + /** Returns a new base relation with the given parameters and schema. */ override def createRelation( sqlContext: SQLContext, parameters: Map[String, String], @@ -77,6 +94,7 @@ class DefaultSource ParquetRelation2(Seq(checkPath(parameters)), parameters, Some(schema))(sqlContext) } + /** Returns a new base relation with the given parameters and save given data into it. */ override def createRelation( sqlContext: SQLContext, mode: SaveMode, @@ -85,33 +103,19 @@ class DefaultSource val path = checkPath(parameters) val filesystemPath = new Path(path) val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - val doSave = if (fs.exists(filesystemPath)) { - mode match { - case SaveMode.Append => - sys.error(s"Append mode is not supported by ${this.getClass.getCanonicalName}") - case SaveMode.Overwrite => - fs.delete(filesystemPath, true) - true - case SaveMode.ErrorIfExists => - sys.error(s"path $path already exists.") - case SaveMode.Ignore => false - } - } else { - true + val doInsertion = (mode, fs.exists(filesystemPath)) match { + case (SaveMode.ErrorIfExists, true) => + sys.error(s"path $path already exists.") + case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => + true + case (SaveMode.Ignore, exists) => + !exists } - val relation = if (doSave) { - // Only save data when the save mode is not ignore. - ParquetRelation.createEmpty( - path, - data.schema.toAttributes, - false, - sqlContext.sparkContext.hadoopConfiguration, - sqlContext) - - val createdRelation = createRelation(sqlContext, parameters, data.schema) - createdRelation.asInstanceOf[ParquetRelation2].insert(data, true) - + val relation = if (doInsertion) { + val createdRelation = + createRelation(sqlContext, parameters, data.schema).asInstanceOf[ParquetRelation2] + createdRelation.insert(data, overwrite = mode == SaveMode.Overwrite) createdRelation } else { // If the save mode is Ignore, we will just create the relation based on existing data. @@ -122,37 +126,31 @@ class DefaultSource } } -private[parquet] case class Partition(values: Row, path: String) +private[sql] case class Partition(values: Row, path: String) -private[parquet] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) +private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) /** * An alternative to [[ParquetRelation]] that plugs in using the data sources API. This class is - * currently not intended as a full replacement of the parquet support in Spark SQL though it is - * likely that it will eventually subsume the existing physical plan implementation. - * - * Compared with the current implementation, this class has the following notable differences: - * - * Partitioning: Partitions are auto discovered and must be in the form of directories `key=value/` - * located at `path`. Currently only a single partitioning column is supported and it must - * be an integer. This class supports both fully self-describing data, which contains the partition - * key, and data where the partition key is only present in the folder structure. The presence - * of the partitioning key in the data is also auto-detected. The `null` partition is not yet - * supported. + * intended as a full replacement of the Parquet support in Spark SQL. The old implementation will + * be deprecated and eventually removed once this version is proved to be stable enough. * - * Metadata: The metadata is automatically discovered by reading the first parquet file present. - * There is currently no support for working with files that have different schema. Additionally, - * when parquet metadata caching is turned on, the FileStatus objects for all data will be cached - * to improve the speed of interactive querying. When data is added to a table it must be dropped - * and recreated to pick up any changes. + * Compared with the old implementation, this class has the following notable differences: * - * Statistics: Statistics for the size of the table are automatically populated during metadata - * discovery. + * - Partitioning discovery: Hive style multi-level partitions are auto discovered. + * - Metadata discovery: Parquet is a format comes with schema evolving support. This data source + * can detect and merge schemas from all Parquet part-files as long as they are compatible. + * Also, metadata and [[FileStatus]]es are cached for better performance. + * - Statistics: Statistics for the size of the table are automatically populated during schema + * discovery. */ @DeveloperApi -case class ParquetRelation2 - (paths: Seq[String], parameters: Map[String, String], maybeSchema: Option[StructType] = None) - (@transient val sqlContext: SQLContext) +case class ParquetRelation2( + paths: Seq[String], + parameters: Map[String, String], + maybeSchema: Option[StructType] = None, + maybePartitionSpec: Option[PartitionSpec] = None)( + @transient val sqlContext: SQLContext) extends CatalystScan with InsertableRelation with SparkHadoopMapReduceUtil @@ -175,43 +173,90 @@ case class ParquetRelation2 override def equals(other: Any) = other match { case relation: ParquetRelation2 => + // If schema merging is required, we don't compare the actual schemas since they may evolve. + val schemaEquality = if (shouldMergeSchemas) { + shouldMergeSchemas == relation.shouldMergeSchemas + } else { + schema == relation.schema + } + paths.toSet == relation.paths.toSet && + schemaEquality && maybeMetastoreSchema == relation.maybeMetastoreSchema && - (shouldMergeSchemas == relation.shouldMergeSchemas || schema == relation.schema) + maybePartitionSpec == relation.maybePartitionSpec + case _ => false } private[sql] def sparkContext = sqlContext.sparkContext - @transient private val fs = FileSystem.get(sparkContext.hadoopConfiguration) - private class MetadataCache { + // `FileStatus` objects of all "_metadata" files. private var metadataStatuses: Array[FileStatus] = _ + + // `FileStatus` objects of all "_common_metadata" files. private var commonMetadataStatuses: Array[FileStatus] = _ + + // Parquet footer cache. private var footers: Map[FileStatus, Footer] = _ - private var parquetSchema: StructType = _ + // `FileStatus` objects of all data files (Parquet part-files). var dataStatuses: Array[FileStatus] = _ + + // Partition spec of this table, including names, data types, and values of each partition + // column, and paths of each partition. var partitionSpec: PartitionSpec = _ + + // Schema of the actual Parquet files, without partition columns discovered from partition + // directory paths. + var parquetSchema: StructType = _ + + // Schema of the whole table, including partition columns. var schema: StructType = _ - var dataSchemaIncludesPartitionKeys: Boolean = _ + // Indicates whether partition columns are also included in Parquet data file schema. If not, + // we need to fill in partition column values into read rows when scanning the table. + var partitionKeysIncludedInParquetSchema: Boolean = _ + + def prepareMetadata(path: Path, schema: StructType, conf: Configuration): Unit = { + conf.set( + ParquetOutputFormat.COMPRESSION, + ParquetRelation + .shortParquetCompressionCodecNames + .getOrElse( + sqlContext.conf.parquetCompressionCodec.toUpperCase, + CompressionCodecName.UNCOMPRESSED).name()) + + ParquetRelation.enableLogForwarding() + ParquetTypesConverter.writeMetaData(schema.toAttributes, path, conf) + } + + /** + * Refreshes `FileStatus`es, footers, partition spec, and table schema. + */ def refresh(): Unit = { - val baseStatuses = { - val statuses = paths.distinct.map(p => fs.getFileStatus(fs.makeQualified(new Path(p)))) - // Support either reading a collection of raw Parquet part-files, or a collection of folders - // containing Parquet files (e.g. partitioned Parquet table). - assert(statuses.forall(!_.isDir) || statuses.forall(_.isDir)) - statuses.toArray - } + val fs = FileSystem.get(sparkContext.hadoopConfiguration) + + // Support either reading a collection of raw Parquet part-files, or a collection of folders + // containing Parquet files (e.g. partitioned Parquet table). + val baseStatuses = paths.distinct.map { p => + val qualified = fs.makeQualified(new Path(p)) + + if (!fs.exists(qualified) && maybeSchema.isDefined) { + fs.mkdirs(qualified) + prepareMetadata(qualified, maybeSchema.get, sparkContext.hadoopConfiguration) + } + + fs.getFileStatus(qualified) + }.toArray + assert(baseStatuses.forall(!_.isDir) || baseStatuses.forall(_.isDir)) + // Lists `FileStatus`es of all leaf nodes (files) under all base directories. val leaves = baseStatuses.flatMap { f => - val statuses = SparkHadoopUtil.get.listLeafStatuses(fs, f.getPath).filter { f => + SparkHadoopUtil.get.listLeafStatuses(fs, f.getPath).filter { f => isSummaryFile(f.getPath) || !(f.getPath.getName.startsWith("_") || f.getPath.getName.startsWith(".")) } - assert(statuses.nonEmpty, s"${f.getPath} is an empty folder.") - statuses } dataStatuses = leaves.filterNot(f => isSummaryFile(f.getPath)) @@ -225,13 +270,14 @@ case class ParquetRelation2 f -> new Footer(f.getPath, parquetMetadata) }.seq.toMap - partitionSpec = { - val partitionDirs = dataStatuses + partitionSpec = maybePartitionSpec.getOrElse { + val partitionDirs = leaves .filterNot(baseStatuses.contains) .map(_.getPath.getParent) .distinct if (partitionDirs.nonEmpty) { + // Parses names and values of partition columns, and infer their data types. ParquetRelation2.parsePartitions(partitionDirs, defaultPartitionName) } else { // No partition directories found, makes an empty specification @@ -241,20 +287,22 @@ case class ParquetRelation2 parquetSchema = maybeSchema.getOrElse(readSchema()) - dataSchemaIncludesPartitionKeys = + partitionKeysIncludedInParquetSchema = isPartitioned && - partitionColumns.forall(f => metadataCache.parquetSchema.fieldNames.contains(f.name)) + partitionColumns.forall(f => parquetSchema.fieldNames.contains(f.name)) schema = { - val fullParquetSchema = if (dataSchemaIncludesPartitionKeys) { - metadataCache.parquetSchema + val fullRelationSchema = if (partitionKeysIncludedInParquetSchema) { + parquetSchema } else { - StructType(metadataCache.parquetSchema.fields ++ partitionColumns.fields) + StructType(parquetSchema.fields ++ partitionColumns.fields) } + // If this Parquet relation is converted from a Hive Metastore table, must reconcile case + // insensitivity issue and possible schema mismatch. maybeMetastoreSchema - .map(ParquetRelation2.mergeMetastoreParquetSchema(_, fullParquetSchema)) - .getOrElse(fullParquetSchema) + .map(ParquetRelation2.mergeMetastoreParquetSchema(_, fullRelationSchema)) + .getOrElse(fullRelationSchema) } } @@ -303,13 +351,17 @@ case class ParquetRelation2 @transient private val metadataCache = new MetadataCache metadataCache.refresh() - private def partitionColumns = metadataCache.partitionSpec.partitionColumns + def partitionSpec = metadataCache.partitionSpec - private def partitions = metadataCache.partitionSpec.partitions + def partitionColumns = metadataCache.partitionSpec.partitionColumns - private def isPartitioned = partitionColumns.nonEmpty + def partitions = metadataCache.partitionSpec.partitions - private def dataSchemaIncludesPartitionKeys = metadataCache.dataSchemaIncludesPartitionKeys + def isPartitioned = partitionColumns.nonEmpty + + private def partitionKeysIncludedInDataSchema = metadataCache.partitionKeysIncludedInParquetSchema + + private def parquetSchema = metadataCache.parquetSchema override def schema = metadataCache.schema @@ -412,18 +464,21 @@ case class ParquetRelation2 // When the data does not include the key and the key is requested then we must fill it in // based on information from the input split. - if (!dataSchemaIncludesPartitionKeys && partitionKeyLocations.nonEmpty) { + if (!partitionKeysIncludedInDataSchema && partitionKeyLocations.nonEmpty) { baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { case p if split.getPath.getParent.toString == p.path => p.values }.get + val requiredPartOrdinal = partitionKeyLocations.keys.toSeq + iterator.map { pair => val row = pair._2.asInstanceOf[SpecificMutableRow] var i = 0 - while (i < partValues.size) { + while (i < requiredPartOrdinal.size) { // TODO Avoids boxing cost here! - row.update(partitionKeyLocations(i), partValues(i)) + val partOrdinal = requiredPartOrdinal(i) + row.update(partitionKeyLocations(partOrdinal), partValues(partOrdinal)) i += 1 } row @@ -457,6 +512,8 @@ case class ParquetRelation2 } override def insert(data: DataFrame, overwrite: Boolean): Unit = { + assert(paths.size == 1, s"Can't write to multiple destinations: ${paths.mkString(",")}") + // TODO: currently we do not check whether the "schema"s are compatible // That means if one first creates a table and then INSERTs data with // and incompatible schema the execution will fail. It would be nice @@ -464,7 +521,7 @@ case class ParquetRelation2 // before calling execute(). val job = new Job(sqlContext.sparkContext.hadoopConfiguration) - val writeSupport = if (schema.map(_.dataType).forall(_.isPrimitive)) { + val writeSupport = if (parquetSchema.map(_.dataType).forall(_.isPrimitive)) { log.debug("Initializing MutableRowWriteSupport") classOf[MutableRowWriteSupport] } else { @@ -474,7 +531,7 @@ case class ParquetRelation2 ParquetOutputFormat.setWriteSupportClass(job, writeSupport) val conf = ContextUtil.getConfiguration(job) - RowWriteSupport.setSchema(schema.toAttributes, conf) + RowWriteSupport.setSchema(data.schema.toAttributes, conf) val destinationPath = new Path(paths.head) @@ -544,14 +601,12 @@ object ParquetRelation2 { // Whether we should merge schemas collected from all Parquet part-files. val MERGE_SCHEMA = "mergeSchema" - // Hive Metastore schema, passed in when the Parquet relation is converted from Metastore - val METASTORE_SCHEMA = "metastoreSchema" - - // Default partition name to use when the partition column value is null or empty string + // Default partition name to use when the partition column value is null or empty string. val DEFAULT_PARTITION_NAME = "partition.defaultName" - // When true, the Parquet data source caches Parquet metadata for performance - val CACHE_METADATA = "cacheMetadata" + // Hive Metastore schema, used when converting Metastore Parquet tables. This option is only used + // internally. + private[sql] val METASTORE_SCHEMA = "metastoreSchema" private[parquet] def readSchema(footers: Seq[Footer], sqlContext: SQLContext): StructType = { footers.map { footer => @@ -579,6 +634,15 @@ object ParquetRelation2 { } } + /** + * Reconciles Hive Metastore case insensitivity issue and data type conflicts between Metastore + * schema and Parquet schema. + * + * Hive doesn't retain case information, while Parquet is case sensitive. On the other hand, the + * schema read from Parquet files may be incomplete (e.g. older versions of Parquet doesn't + * distinguish binary and string). This method generates a correct schema by merging Metastore + * schema data types and Parquet schema field names. + */ private[parquet] def mergeMetastoreParquetSchema( metastoreSchema: StructType, parquetSchema: StructType): StructType = { @@ -719,16 +783,15 @@ object ParquetRelation2 { * }}} */ private[parquet] def resolvePartitions(values: Seq[PartitionValues]): Seq[PartitionValues] = { - val distinctColNamesOfPartitions = values.map(_.columnNames).distinct - val columnCount = values.head.columnNames.size - // Column names of all partitions must match - assert(distinctColNamesOfPartitions.size == 1, { - val list = distinctColNamesOfPartitions.mkString("\t", "\n", "") + val distinctPartitionsColNames = values.map(_.columnNames).distinct + assert(distinctPartitionsColNames.size == 1, { + val list = distinctPartitionsColNames.mkString("\t", "\n", "") s"Conflicting partition column names detected:\n$list" }) // Resolves possible type conflicts for each column + val columnCount = values.head.columnNames.size val resolvedValues = (0 until columnCount).map { i => resolveTypeConflicts(values.map(_.literals(i))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index f8117c21773ae..eb2d5f25290b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.parquet +import org.scalatest.BeforeAndAfterAll import parquet.filter2.predicate.Operators._ import parquet.filter2.predicate.{FilterPredicate, Operators} @@ -40,7 +41,7 @@ import org.apache.spark.sql.{Column, DataFrame, QueryTest, SQLConf} * 2. `Tuple1(Option(x))` is used together with `AnyVal` types like `Int` to ensure the inferred * data type is nullable. */ -class ParquetFilterSuite extends QueryTest with ParquetTest { +class ParquetFilterSuiteBase extends QueryTest with ParquetTest { val sqlContext = TestSQLContext private def checkFilterPredicate( @@ -112,210 +113,224 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) } - def run(prefix: String): Unit = { - test(s"$prefix: filter pushdown - boolean") { - withParquetRDD((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false))) + test("filter pushdown - boolean") { + withParquetRDD((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false))) - checkFilterPredicate('_1 === true, classOf[Eq[_]], true) - checkFilterPredicate('_1 !== true, classOf[NotEq[_]], false) - } + checkFilterPredicate('_1 === true, classOf[Eq[_]], true) + checkFilterPredicate('_1 !== true, classOf[NotEq[_]], false) } + } - test(s"$prefix: filter pushdown - short") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit rdd => - checkFilterPredicate(Cast('_1, IntegerType) === 1, classOf[Eq[_]], 1) - checkFilterPredicate( - Cast('_1, IntegerType) !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate(Cast('_1, IntegerType) < 2, classOf[Lt[_]], 1) - checkFilterPredicate(Cast('_1, IntegerType) > 3, classOf[Gt[_]], 4) - checkFilterPredicate(Cast('_1, IntegerType) <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate(Cast('_1, IntegerType) >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === Cast('_1, IntegerType), classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > Cast('_1, IntegerType), classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < Cast('_1, IntegerType), classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= Cast('_1, IntegerType), classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= Cast('_1, IntegerType), classOf[GtEq[_]], 4) - - checkFilterPredicate(!(Cast('_1, IntegerType) < 4), classOf[GtEq[_]], 4) - checkFilterPredicate( - Cast('_1, IntegerType) > 2 && Cast('_1, IntegerType) < 4, classOf[Operators.And], 3) - checkFilterPredicate( - Cast('_1, IntegerType) < 2 || Cast('_1, IntegerType) > 3, - classOf[Operators.Or], - Seq(Row(1), Row(4))) - } + test("filter pushdown - short") { + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit rdd => + checkFilterPredicate(Cast('_1, IntegerType) === 1, classOf[Eq[_]], 1) + checkFilterPredicate( + Cast('_1, IntegerType) !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + + checkFilterPredicate(Cast('_1, IntegerType) < 2, classOf[Lt[_]], 1) + checkFilterPredicate(Cast('_1, IntegerType) > 3, classOf[Gt[_]], 4) + checkFilterPredicate(Cast('_1, IntegerType) <= 1, classOf[LtEq[_]], 1) + checkFilterPredicate(Cast('_1, IntegerType) >= 4, classOf[GtEq[_]], 4) + + checkFilterPredicate(Literal(1) === Cast('_1, IntegerType), classOf[Eq[_]], 1) + checkFilterPredicate(Literal(2) > Cast('_1, IntegerType), classOf[Lt[_]], 1) + checkFilterPredicate(Literal(3) < Cast('_1, IntegerType), classOf[Gt[_]], 4) + checkFilterPredicate(Literal(1) >= Cast('_1, IntegerType), classOf[LtEq[_]], 1) + checkFilterPredicate(Literal(4) <= Cast('_1, IntegerType), classOf[GtEq[_]], 4) + + checkFilterPredicate(!(Cast('_1, IntegerType) < 4), classOf[GtEq[_]], 4) + checkFilterPredicate( + Cast('_1, IntegerType) > 2 && Cast('_1, IntegerType) < 4, classOf[Operators.And], 3) + checkFilterPredicate( + Cast('_1, IntegerType) < 2 || Cast('_1, IntegerType) > 3, + classOf[Operators.Or], + Seq(Row(1), Row(4))) } + } - test(s"$prefix: filter pushdown - integer") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) + test("filter pushdown - integer") { + withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) + checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) + checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) + checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) + checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) + checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) + checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) + checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) + checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) - } + checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) + checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) + checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } + } - test(s"$prefix: filter pushdown - long") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) + test("filter pushdown - long") { + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) + checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) + checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) + checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) + checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) + checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) + checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) + checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) + checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) - } + checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) + checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) + checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } + } - test(s"$prefix: filter pushdown - float") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) + test("filter pushdown - float") { + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) + checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) + checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) + checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) + checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) + checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) + checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) + checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) + checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) - } + checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) + checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) + checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } + } - test(s"$prefix: filter pushdown - double") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) + test("filter pushdown - double") { + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) - checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) - checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) + checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) + checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) + checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1) + checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) - checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) + checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) + checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) + checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) + checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) - checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) - checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) - } + checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) + checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) + checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } + } - test(s"$prefix: filter pushdown - string") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { implicit rdd => - checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkFilterPredicate( - '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString))) - - checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1") - checkFilterPredicate( - '_1 !== "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString))) - - checkFilterPredicate('_1 < "2", classOf[Lt[_]], "1") - checkFilterPredicate('_1 > "3", classOf[Gt[_]], "4") - checkFilterPredicate('_1 <= "1", classOf[LtEq[_]], "1") - checkFilterPredicate('_1 >= "4", classOf[GtEq[_]], "4") - - checkFilterPredicate(Literal("1") === '_1, classOf[Eq[_]], "1") - checkFilterPredicate(Literal("2") > '_1, classOf[Lt[_]], "1") - checkFilterPredicate(Literal("3") < '_1, classOf[Gt[_]], "4") - checkFilterPredicate(Literal("1") >= '_1, classOf[LtEq[_]], "1") - checkFilterPredicate(Literal("4") <= '_1, classOf[GtEq[_]], "4") - - checkFilterPredicate(!('_1 < "4"), classOf[GtEq[_]], "4") - checkFilterPredicate('_1 > "2" && '_1 < "4", classOf[Operators.And], "3") - checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or], Seq(Row("1"), Row("4"))) - } + test("filter pushdown - string") { + withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { implicit rdd => + checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate( + '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString))) + + checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1") + checkFilterPredicate( + '_1 !== "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString))) + + checkFilterPredicate('_1 < "2", classOf[Lt[_]], "1") + checkFilterPredicate('_1 > "3", classOf[Gt[_]], "4") + checkFilterPredicate('_1 <= "1", classOf[LtEq[_]], "1") + checkFilterPredicate('_1 >= "4", classOf[GtEq[_]], "4") + + checkFilterPredicate(Literal("1") === '_1, classOf[Eq[_]], "1") + checkFilterPredicate(Literal("2") > '_1, classOf[Lt[_]], "1") + checkFilterPredicate(Literal("3") < '_1, classOf[Gt[_]], "4") + checkFilterPredicate(Literal("1") >= '_1, classOf[LtEq[_]], "1") + checkFilterPredicate(Literal("4") <= '_1, classOf[GtEq[_]], "4") + + checkFilterPredicate(!('_1 < "4"), classOf[GtEq[_]], "4") + checkFilterPredicate('_1 > "2" && '_1 < "4", classOf[Operators.And], "3") + checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or], Seq(Row("1"), Row("4"))) } + } - test(s"$prefix: filter pushdown - binary") { - implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes("UTF-8") - } + test("filter pushdown - binary") { + implicit class IntToBinary(int: Int) { + def b: Array[Byte] = int.toString.getBytes("UTF-8") + } - withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { implicit rdd => - checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) + withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { implicit rdd => + checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) - checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) - checkBinaryFilterPredicate( - '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.b)).toSeq) + checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkBinaryFilterPredicate( + '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.b)).toSeq) - checkBinaryFilterPredicate( - '_1 !== 1.b, classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.b)).toSeq) + checkBinaryFilterPredicate( + '_1 !== 1.b, classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.b)).toSeq) - checkBinaryFilterPredicate('_1 < 2.b, classOf[Lt[_]], 1.b) - checkBinaryFilterPredicate('_1 > 3.b, classOf[Gt[_]], 4.b) - checkBinaryFilterPredicate('_1 <= 1.b, classOf[LtEq[_]], 1.b) - checkBinaryFilterPredicate('_1 >= 4.b, classOf[GtEq[_]], 4.b) + checkBinaryFilterPredicate('_1 < 2.b, classOf[Lt[_]], 1.b) + checkBinaryFilterPredicate('_1 > 3.b, classOf[Gt[_]], 4.b) + checkBinaryFilterPredicate('_1 <= 1.b, classOf[LtEq[_]], 1.b) + checkBinaryFilterPredicate('_1 >= 4.b, classOf[GtEq[_]], 4.b) - checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq[_]], 1.b) - checkBinaryFilterPredicate(Literal(2.b) > '_1, classOf[Lt[_]], 1.b) - checkBinaryFilterPredicate(Literal(3.b) < '_1, classOf[Gt[_]], 4.b) - checkBinaryFilterPredicate(Literal(1.b) >= '_1, classOf[LtEq[_]], 1.b) - checkBinaryFilterPredicate(Literal(4.b) <= '_1, classOf[GtEq[_]], 4.b) + checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq[_]], 1.b) + checkBinaryFilterPredicate(Literal(2.b) > '_1, classOf[Lt[_]], 1.b) + checkBinaryFilterPredicate(Literal(3.b) < '_1, classOf[Gt[_]], 4.b) + checkBinaryFilterPredicate(Literal(1.b) >= '_1, classOf[LtEq[_]], 1.b) + checkBinaryFilterPredicate(Literal(4.b) <= '_1, classOf[GtEq[_]], 4.b) - checkBinaryFilterPredicate(!('_1 < 4.b), classOf[GtEq[_]], 4.b) - checkBinaryFilterPredicate('_1 > 2.b && '_1 < 4.b, classOf[Operators.And], 3.b) - checkBinaryFilterPredicate( - '_1 < 2.b || '_1 > 3.b, classOf[Operators.Or], Seq(Row(1.b), Row(4.b))) - } + checkBinaryFilterPredicate(!('_1 < 4.b), classOf[GtEq[_]], 4.b) + checkBinaryFilterPredicate('_1 > 2.b && '_1 < 4.b, classOf[Operators.And], 3.b) + checkBinaryFilterPredicate( + '_1 < 2.b || '_1 > 3.b, classOf[Operators.Or], Seq(Row(1.b), Row(4.b))) } } +} + +class ParquetDataSourceOnFilterSuite extends ParquetFilterSuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi + + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + } + + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + +class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { - run("Parquet data source enabled") + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") } - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { - run("Parquet data source disabled") + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index c306330818c0a..208f35761b807 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -21,6 +21,9 @@ import scala.collection.JavaConversions._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.scalatest.BeforeAndAfterAll import parquet.example.data.simple.SimpleGroup import parquet.example.data.{Group, GroupWriter} import parquet.hadoop.api.WriteSupport @@ -30,16 +33,13 @@ import parquet.hadoop.{ParquetFileWriter, ParquetWriter} import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf} -import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf, SaveMode} // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport // with an empty configuration (it is after all not intended to be used in this way?) @@ -64,10 +64,11 @@ private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteS /** * A test suite that tests basic Parquet I/O. */ -class ParquetIOSuite extends QueryTest with ParquetTest { - +class ParquetIOSuiteBase extends QueryTest with ParquetTest { val sqlContext = TestSQLContext + import sqlContext.implicits.localSeqToDataFrameHolder + /** * Writes `data` to a Parquet file, reads it back and check file contents. */ @@ -75,229 +76,281 @@ class ParquetIOSuite extends QueryTest with ParquetTest { withParquetRDD(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } - def run(prefix: String): Unit = { - test(s"$prefix: basic data types (without binary)") { - val data = (1 to 4).map { i => - (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) - } - checkParquetFile(data) + test("basic data types (without binary)") { + val data = (1 to 4).map { i => + (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) } + checkParquetFile(data) + } - test(s"$prefix: raw binary") { - val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) - withParquetRDD(data) { rdd => - assertResult(data.map(_._1.mkString(",")).sorted) { - rdd.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted - } + test("raw binary") { + val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) + withParquetRDD(data) { rdd => + assertResult(data.map(_._1.mkString(",")).sorted) { + rdd.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted } } + } - test(s"$prefix: string") { - val data = (1 to 4).map(i => Tuple1(i.toString)) - // Property spark.sql.parquet.binaryAsString shouldn't affect Parquet files written by Spark SQL - // as we store Spark SQL schema in the extra metadata. - withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "false")(checkParquetFile(data)) - withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "true")(checkParquetFile(data)) - } + test("string") { + val data = (1 to 4).map(i => Tuple1(i.toString)) + // Property spark.sql.parquet.binaryAsString shouldn't affect Parquet files written by Spark SQL + // as we store Spark SQL schema in the extra metadata. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "false")(checkParquetFile(data)) + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING -> "true")(checkParquetFile(data)) + } - test(s"$prefix: fixed-length decimals") { - - def makeDecimalRDD(decimal: DecimalType): DataFrame = - sparkContext - .parallelize(0 to 1000) - .map(i => Tuple1(i / 100.0)) - .toDF - // Parquet doesn't allow column names with spaces, have to add an alias here - .select($"_1" cast decimal as "dec") - - for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { - withTempPath { dir => - val data = makeDecimalRDD(DecimalType(precision, scale)) - data.saveAsParquetFile(dir.getCanonicalPath) - checkAnswer(parquetFile(dir.getCanonicalPath), data.collect().toSeq) - } + test("fixed-length decimals") { + + def makeDecimalRDD(decimal: DecimalType): DataFrame = + sparkContext + .parallelize(0 to 1000) + .map(i => Tuple1(i / 100.0)) + .toDF + // Parquet doesn't allow column names with spaces, have to add an alias here + .select($"_1" cast decimal as "dec") + + for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { + withTempPath { dir => + val data = makeDecimalRDD(DecimalType(precision, scale)) + data.saveAsParquetFile(dir.getCanonicalPath) + checkAnswer(parquetFile(dir.getCanonicalPath), data.collect().toSeq) } + } - // Decimals with precision above 18 are not yet supported - intercept[RuntimeException] { - withTempPath { dir => - makeDecimalRDD(DecimalType(19, 10)).saveAsParquetFile(dir.getCanonicalPath) - parquetFile(dir.getCanonicalPath).collect() - } + // Decimals with precision above 18 are not yet supported + intercept[RuntimeException] { + withTempPath { dir => + makeDecimalRDD(DecimalType(19, 10)).saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).collect() } + } - // Unlimited-length decimals are not yet supported - intercept[RuntimeException] { - withTempPath { dir => - makeDecimalRDD(DecimalType.Unlimited).saveAsParquetFile(dir.getCanonicalPath) - parquetFile(dir.getCanonicalPath).collect() - } + // Unlimited-length decimals are not yet supported + intercept[RuntimeException] { + withTempPath { dir => + makeDecimalRDD(DecimalType.Unlimited).saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).collect() } } + } + + test("map") { + val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) + checkParquetFile(data) + } + + test("array") { + val data = (1 to 4).map(i => Tuple1(Seq(i, i + 1))) + checkParquetFile(data) + } - test(s"$prefix: map") { - val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) - checkParquetFile(data) + test("struct") { + val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) + withParquetRDD(data) { rdd => + // Structs are converted to `Row`s + checkAnswer(rdd, data.map { case Tuple1(struct) => + Row(Row(struct.productIterator.toSeq: _*)) + }) } + } - test(s"$prefix: array") { - val data = (1 to 4).map(i => Tuple1(Seq(i, i + 1))) - checkParquetFile(data) + test("nested struct with array of array as field") { + val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) + withParquetRDD(data) { rdd => + // Structs are converted to `Row`s + checkAnswer(rdd, data.map { case Tuple1(struct) => + Row(Row(struct.productIterator.toSeq: _*)) + }) } + } - test(s"$prefix: struct") { - val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) - withParquetRDD(data) { rdd => - // Structs are converted to `Row`s - checkAnswer(rdd, data.map { case Tuple1(struct) => - Row(Row(struct.productIterator.toSeq: _*)) - }) - } + test("nested map with struct as value type") { + val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i")))) + withParquetRDD(data) { rdd => + checkAnswer(rdd, data.map { case Tuple1(m) => + Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) + }) } + } - test(s"$prefix: nested struct with array of array as field") { - val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) - withParquetRDD(data) { rdd => - // Structs are converted to `Row`s - checkAnswer(rdd, data.map { case Tuple1(struct) => - Row(Row(struct.productIterator.toSeq: _*)) - }) - } + test("nulls") { + val allNulls = ( + null.asInstanceOf[java.lang.Boolean], + null.asInstanceOf[Integer], + null.asInstanceOf[java.lang.Long], + null.asInstanceOf[java.lang.Float], + null.asInstanceOf[java.lang.Double]) + + withParquetRDD(allNulls :: Nil) { rdd => + val rows = rdd.collect() + assert(rows.size === 1) + assert(rows.head === Row(Seq.fill(5)(null): _*)) } + } - test(s"$prefix: nested map with struct as value type") { - val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i")))) - withParquetRDD(data) { rdd => - checkAnswer(rdd, data.map { case Tuple1(m) => - Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) - }) - } + test("nones") { + val allNones = ( + None.asInstanceOf[Option[Int]], + None.asInstanceOf[Option[Long]], + None.asInstanceOf[Option[String]]) + + withParquetRDD(allNones :: Nil) { rdd => + val rows = rdd.collect() + assert(rows.size === 1) + assert(rows.head === Row(Seq.fill(3)(null): _*)) } + } - test(s"$prefix: nulls") { - val allNulls = ( - null.asInstanceOf[java.lang.Boolean], - null.asInstanceOf[Integer], - null.asInstanceOf[java.lang.Long], - null.asInstanceOf[java.lang.Float], - null.asInstanceOf[java.lang.Double]) - - withParquetRDD(allNulls :: Nil) { rdd => - val rows = rdd.collect() - assert(rows.size === 1) - assert(rows.head === Row(Seq.fill(5)(null): _*)) - } + test("compression codec") { + def compressionCodecFor(path: String) = { + val codecs = ParquetTypesConverter + .readMetaData(new Path(path), Some(configuration)) + .getBlocks + .flatMap(_.getColumns) + .map(_.getCodec.name()) + .distinct + + assert(codecs.size === 1) + codecs.head } - test(s"$prefix: nones") { - val allNones = ( - None.asInstanceOf[Option[Int]], - None.asInstanceOf[Option[Long]], - None.asInstanceOf[Option[String]]) + val data = (0 until 10).map(i => (i, i.toString)) - withParquetRDD(allNones :: Nil) { rdd => - val rows = rdd.collect() - assert(rows.size === 1) - assert(rows.head === Row(Seq.fill(3)(null): _*)) + def checkCompressionCodec(codec: CompressionCodecName): Unit = { + withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { + withParquetFile(data) { path => + assertResult(conf.parquetCompressionCodec.toUpperCase) { + compressionCodecFor(path) + } + } } } - test(s"$prefix: compression codec") { - def compressionCodecFor(path: String) = { - val codecs = ParquetTypesConverter - .readMetaData(new Path(path), Some(configuration)) - .getBlocks - .flatMap(_.getColumns) - .map(_.getCodec.name()) - .distinct - - assert(codecs.size === 1) - codecs.head - } + // Checks default compression codec + checkCompressionCodec(CompressionCodecName.fromConf(conf.parquetCompressionCodec)) - val data = (0 until 10).map(i => (i, i.toString)) + checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) + checkCompressionCodec(CompressionCodecName.GZIP) + checkCompressionCodec(CompressionCodecName.SNAPPY) + } - def checkCompressionCodec(codec: CompressionCodecName): Unit = { - withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { - withParquetFile(data) { path => - assertResult(conf.parquetCompressionCodec.toUpperCase) { - compressionCodecFor(path) - } - } - } + test("read raw Parquet file") { + def makeRawParquetFile(path: Path): Unit = { + val schema = MessageTypeParser.parseMessageType( + """ + |message root { + | required boolean _1; + | required int32 _2; + | required int64 _3; + | required float _4; + | required double _5; + |} + """.stripMargin) + + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + + (0 until 10).foreach { i => + val record = new SimpleGroup(schema) + record.add(0, i % 2 == 0) + record.add(1, i) + record.add(2, i.toLong) + record.add(3, i.toFloat) + record.add(4, i.toDouble) + writer.write(record) } - // Checks default compression codec - checkCompressionCodec(CompressionCodecName.fromConf(conf.parquetCompressionCodec)) + writer.close() + } - checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) - checkCompressionCodec(CompressionCodecName.GZIP) - checkCompressionCodec(CompressionCodecName.SNAPPY) + withTempDir { dir => + val path = new Path(dir.toURI.toString, "part-r-0.parquet") + makeRawParquetFile(path) + checkAnswer(parquetFile(path.toString), (0 until 10).map { i => + Row(i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) + }) } + } - test(s"$prefix: read raw Parquet file") { - def makeRawParquetFile(path: Path): Unit = { - val schema = MessageTypeParser.parseMessageType( - """ - |message root { - | required boolean _1; - | required int32 _2; - | required int64 _3; - | required float _4; - | required double _5; - |} - """.stripMargin) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - (0 until 10).foreach { i => - val record = new SimpleGroup(schema) - record.add(0, i % 2 == 0) - record.add(1, i) - record.add(2, i.toLong) - record.add(3, i.toFloat) - record.add(4, i.toDouble) - writer.write(record) - } + test("write metadata") { + withTempPath { file => + val path = new Path(file.toURI.toString) + val fs = FileSystem.getLocal(configuration) + val attributes = ScalaReflection.attributesFor[(Int, String)] + ParquetTypesConverter.writeMetaData(attributes, path, configuration) - writer.close() - } + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) - withTempDir { dir => - val path = new Path(dir.toURI.toString, "part-r-0.parquet") - makeRawParquetFile(path) - checkAnswer(parquetFile(path.toString), (0 until 10).map { i => - Row(i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) - }) - } + val metaData = ParquetTypesConverter.readMetaData(path, Some(configuration)) + val actualSchema = metaData.getFileMetaData.getSchema + val expectedSchema = ParquetTypesConverter.convertFromAttributes(attributes) + + actualSchema.checkContains(expectedSchema) + expectedSchema.checkContains(actualSchema) } + } - test(s"$prefix: write metadata") { - withTempPath { file => - val path = new Path(file.toURI.toString) - val fs = FileSystem.getLocal(configuration) - val attributes = ScalaReflection.attributesFor[(Int, String)] - ParquetTypesConverter.writeMetaData(attributes, path, configuration) + test("save - overwrite") { + withParquetFile((1 to 10).map(i => (i, i.toString))) { file => + val newData = (11 to 20).map(i => (i, i.toString)) + newData.toDF().save("org.apache.spark.sql.parquet", SaveMode.Overwrite, Map("path" -> file)) + checkAnswer(parquetFile(file), newData.map(Row.fromTuple)) + } + } - assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) - assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) + test("save - ignore") { + val data = (1 to 10).map(i => (i, i.toString)) + withParquetFile(data) { file => + val newData = (11 to 20).map(i => (i, i.toString)) + newData.toDF().save("org.apache.spark.sql.parquet", SaveMode.Ignore, Map("path" -> file)) + checkAnswer(parquetFile(file), data.map(Row.fromTuple)) + } + } - val metaData = ParquetTypesConverter.readMetaData(path, Some(configuration)) - val actualSchema = metaData.getFileMetaData.getSchema - val expectedSchema = ParquetTypesConverter.convertFromAttributes(attributes) + test("save - throw") { + val data = (1 to 10).map(i => (i, i.toString)) + withParquetFile(data) { file => + val newData = (11 to 20).map(i => (i, i.toString)) + val errorMessage = intercept[Throwable] { + newData.toDF().save( + "org.apache.spark.sql.parquet", SaveMode.ErrorIfExists, Map("path" -> file)) + }.getMessage + assert(errorMessage.contains("already exists")) + } + } - actualSchema.checkContains(expectedSchema) - expectedSchema.checkContains(actualSchema) - } + test("save - append") { + val data = (1 to 10).map(i => (i, i.toString)) + withParquetFile(data) { file => + val newData = (11 to 20).map(i => (i, i.toString)) + newData.toDF().save("org.apache.spark.sql.parquet", SaveMode.Append, Map("path" -> file)) + checkAnswer(parquetFile(file), (data ++ newData).map(Row.fromTuple)) } } +} + +class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi + + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + } + + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + +class ParquetDataSourceOffIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { - run("Parquet data source enabled") + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") } - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { - run("Parquet data source disabled") + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index ae606d11a8f68..3bf0116c8f7e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -19,17 +19,24 @@ package org.apache.spark.sql.parquet import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path -import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.parquet.ParquetRelation2._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{QueryTest, Row, SQLContext} -class ParquetPartitionDiscoverySuite extends FunSuite with ParquetTest { +// The data where the partitioning key exists only in the directory structure. +case class ParquetData(intField: Int, stringField: String) + +// The data that also includes the partitioning key +case class ParquetDataWithKey(intField: Int, pi: Int, stringField: String, ps: String) + +class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { override val sqlContext: SQLContext = TestSQLContext + import sqlContext._ + val defaultPartitionName = "__NULL__" test("column type inference") { @@ -112,6 +119,17 @@ class ParquetPartitionDiscoverySuite extends FunSuite with ParquetTest { Partition(Row(10, "20"), "hdfs://host:9000/path/a=10/b=20"), Partition(Row(10.5, "hello"), "hdfs://host:9000/path/a=10.5/b=hello")))) + check(Seq( + s"hdfs://host:9000/path/a=10/b=20", + s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello"), + PartitionSpec( + StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType))), + Seq( + Partition(Row(10, "20"), s"hdfs://host:9000/path/a=10/b=20"), + Partition(Row(null, "hello"), s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello")))) + check(Seq( s"hdfs://host:9000/path/a=10/b=$defaultPartitionName", s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"), @@ -123,4 +141,182 @@ class ParquetPartitionDiscoverySuite extends FunSuite with ParquetTest { Partition(Row(10, null), s"hdfs://host:9000/path/a=10/b=$defaultPartitionName"), Partition(Row(10.5, null), s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName")))) } + + test("read partitioned table - normal case") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } { + makeParquetFile( + (1 to 10).map(i => ParquetData(i, i.toString)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + parquetFile(base.getCanonicalPath).registerTempTable("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( + sql("SELECT intField, pi FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + _ <- Seq("foo", "bar") + } yield Row(i, pi)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, 1, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, i.toString, pi, "foo")) + } + } + } + + test("read partitioned table - partition key included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + parquetFile(base.getCanonicalPath).registerTempTable("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, pi, i.toString, ps)) + + checkAnswer( + sql("SELECT intField, pi FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + _ <- Seq("foo", "bar") + } yield Row(i, pi)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, 1, i.toString, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, pi, i.toString, "foo")) + } + } + } + + test("read partitioned table - with nulls") { + withTempDir { base => + for { + // Must be `Integer` rather than `Int` here. `null.asInstanceOf[Int]` results in a zero... + pi <- Seq(1, null.asInstanceOf[Integer]) + ps <- Seq("foo", null.asInstanceOf[String]) + } { + makeParquetFile( + (1 to 10).map(i => ParquetData(i, i.toString)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + val parquetRelation = load( + "org.apache.spark.sql.parquet", + Map( + "path" -> base.getCanonicalPath, + ParquetRelation2.DEFAULT_PARTITION_NAME -> defaultPartitionName)) + + parquetRelation.registerTempTable("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, null.asInstanceOf[Integer]) + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi IS NULL"), + for { + i <- 1 to 10 + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, i.toString, null, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps IS NULL"), + for { + i <- 1 to 10 + pi <- Seq(1, null.asInstanceOf[Integer]) + } yield Row(i, i.toString, pi, null)) + } + } + } + + test("read partitioned table - with nulls and partition keys are included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + val parquetRelation = load( + "org.apache.spark.sql.parquet", + Map( + "path" -> base.getCanonicalPath, + ParquetRelation2.DEFAULT_PARTITION_NAME -> defaultPartitionName)) + + parquetRelation.registerTempTable("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, pi, i.toString, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps IS NULL"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, pi, i.toString, null)) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index cba06835f9a61..d0665450cd766 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,103 +17,120 @@ package org.apache.spark.sql.parquet +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ -import org.apache.spark.sql.{QueryTest, SQLConf} /** * A test suite that tests various Parquet queries. */ -class ParquetQuerySuite extends QueryTest with ParquetTest { +class ParquetQuerySuiteBase extends QueryTest with ParquetTest { val sqlContext = TestSQLContext - def run(prefix: String): Unit = { - test(s"$prefix: simple projection") { - withParquetTable((0 until 10).map(i => (i, i.toString)), "t") { - checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_))) - } + test("simple projection") { + withParquetTable((0 until 10).map(i => (i, i.toString)), "t") { + checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_))) } + } - test(s"$prefix: appending") { - val data = (0 until 10).map(i => (i, i.toString)) - withParquetTable(data, "t") { - sql("INSERT INTO TABLE t SELECT * FROM t") - checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) - } + test("appending") { + val data = (0 until 10).map(i => (i, i.toString)) + withParquetTable(data, "t") { + sql("INSERT INTO TABLE t SELECT * FROM t") + checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) } + } - // This test case will trigger the NPE mentioned in - // https://issues.apache.org/jira/browse/PARQUET-151. - ignore(s"$prefix: overwriting") { - val data = (0 until 10).map(i => (i, i.toString)) - withParquetTable(data, "t") { - sql("INSERT OVERWRITE TABLE t SELECT * FROM t") - checkAnswer(table("t"), data.map(Row.fromTuple)) - } + // This test case will trigger the NPE mentioned in + // https://issues.apache.org/jira/browse/PARQUET-151. + // Update: This also triggers SPARK-5746, should re enable it when we get both fixed. + ignore("overwriting") { + val data = (0 until 10).map(i => (i, i.toString)) + withParquetTable(data, "t") { + sql("INSERT OVERWRITE TABLE t SELECT * FROM t") + checkAnswer(table("t"), data.map(Row.fromTuple)) } + } - test(s"$prefix: self-join") { - // 4 rows, cells of column 1 of row 2 and row 4 are null - val data = (1 to 4).map { i => - val maybeInt = if (i % 2 == 0) None else Some(i) - (maybeInt, i.toString) - } - - withParquetTable(data, "t") { - val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") - val queryOutput = selfJoin.queryExecution.analyzed.output + test("self-join") { + // 4 rows, cells of column 1 of row 2 and row 4 are null + val data = (1 to 4).map { i => + val maybeInt = if (i % 2 == 0) None else Some(i) + (maybeInt, i.toString) + } - assertResult(4, s"Field count mismatches")(queryOutput.size) - assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { - queryOutput.filter(_.name == "_1").map(_.exprId).size - } + withParquetTable(data, "t") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") + val queryOutput = selfJoin.queryExecution.analyzed.output - checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + assertResult(4, "Field count mismatche")(queryOutput.size) + assertResult(2, "Duplicated expression ID in query plan:\n $selfJoin") { + queryOutput.filter(_.name == "_1").map(_.exprId).size } + + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) } + } - test(s"$prefix: nested data - struct with array field") { - val data = (1 to 10).map(i => Tuple1((i, Seq(s"val_$i")))) - withParquetTable(data, "t") { - checkAnswer(sql("SELECT _1._2[0] FROM t"), data.map { - case Tuple1((_, Seq(string))) => Row(string) - }) - } + test("nested data - struct with array field") { + val data = (1 to 10).map(i => Tuple1((i, Seq("val_$i")))) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT _1._2[0] FROM t"), data.map { + case Tuple1((_, Seq(string))) => Row(string) + }) } + } - test(s"$prefix: nested data - array of struct") { - val data = (1 to 10).map(i => Tuple1(Seq(i -> s"val_$i"))) - withParquetTable(data, "t") { - checkAnswer(sql("SELECT _1[0]._2 FROM t"), data.map { - case Tuple1(Seq((_, string))) => Row(string) - }) - } + test("nested data - array of struct") { + val data = (1 to 10).map(i => Tuple1(Seq(i -> "val_$i"))) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT _1[0]._2 FROM t"), data.map { + case Tuple1(Seq((_, string))) => Row(string) + }) } + } - test(s"$prefix: SPARK-1913 regression: columns only referenced by pushed down filters should remain") { - withParquetTable((1 to 10).map(Tuple1.apply), "t") { - checkAnswer(sql(s"SELECT _1 FROM t WHERE _1 < 10"), (1 to 9).map(Row.apply(_))) - } + test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { + withParquetTable((1 to 10).map(Tuple1.apply), "t") { + checkAnswer(sql("SELECT _1 FROM t WHERE _1 < 10"), (1 to 9).map(Row.apply(_))) } + } - test(s"$prefix: SPARK-5309 strings stored using dictionary compression in parquet") { - withParquetTable((0 until 1000).map(i => ("same", "run_" + i /100, 1)), "t") { + test("SPARK-5309 strings stored using dictionary compression in parquet") { + withParquetTable((0 until 1000).map(i => ("same", "run_" + i /100, 1)), "t") { - checkAnswer(sql(s"SELECT _1, _2, SUM(_3) FROM t GROUP BY _1, _2"), - (0 until 10).map(i => Row("same", "run_" + i, 100))) + checkAnswer(sql("SELECT _1, _2, SUM(_3) FROM t GROUP BY _1, _2"), + (0 until 10).map(i => Row("same", "run_" + i, 100))) - checkAnswer(sql(s"SELECT _1, _2, SUM(_3) FROM t WHERE _2 = 'run_5' GROUP BY _1, _2"), - List(Row("same", "run_5", 100))) - } + checkAnswer(sql("SELECT _1, _2, SUM(_3) FROM t WHERE _2 = 'run_5' GROUP BY _1, _2"), + List(Row("same", "run_5", 100))) } } +} + +class ParquetDataSourceOnQuerySuite extends ParquetQuerySuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi + + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + } + + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + +class ParquetDataSourceOffQuerySuite extends ParquetQuerySuiteBase with BeforeAndAfterAll { + val originalConf = sqlContext.conf.parquetUseDataSourceApi - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { - run("Parquet data source enabled") + override protected def beforeAll(): Unit = { + sqlContext.conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") } - withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { - run("Parquet data source disabled") + override protected def afterAll(): Unit = { + sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ddc7b181d4d46..87b380f950979 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -22,26 +22,24 @@ import java.sql.Timestamp import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Table -import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.parse.VariableSubstitution +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, SetCommand, QueryExecutionException} -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand} -import org.apache.spark.sql.sources.{CreateTableUsing, DataSourceStrategy} +import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, QueryExecutionException, SetCommand} +import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} +import org.apache.spark.sql.sources.DataSourceStrategy import org.apache.spark.sql.types._ /** @@ -244,6 +242,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override protected[sql] lazy val analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = false) { override val extendedRules = + catalog.ParquetConversions :: catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index eb1ee54247bea..6d794d0e11391 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -20,25 +20,25 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} -import com.google.common.cache.{LoadingCache, CacheLoader, CacheBuilder} - -import org.apache.hadoop.util.ReflectionUtils -import org.apache.hadoop.hive.metastore.{Warehouse, TableType} -import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition, FieldSchema} +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} +import org.apache.hadoop.hive.metastore.{TableType, Warehouse} import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} +import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.parquet.ParquetRelation2 +import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} import org.apache.spark.sql.sources.{DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -101,16 +101,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val caseSensitive: Boolean = false - /** * - * Creates a data source table (a table created with USING clause) in Hive's metastore. - * Returns true when the table has been created. Otherwise, false. - * @param tableName - * @param userSpecifiedSchema - * @param provider - * @param options - * @param isExternal - * @return - */ + /** + * Creates a data source table (a table created with USING clause) in Hive's metastore. + * Returns true when the table has been created. Otherwise, false. + */ def createDataSourceTable( tableName: String, userSpecifiedSchema: Option[StructType], @@ -141,7 +135,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } def hiveDefaultTableFilePath(tableName: String): String = { - val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase()) + val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase) hiveWarehouse.getTablePath(currentDatabase, tableName).toString } @@ -176,25 +170,41 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Nil } - val relation = MetastoreRelation( - databaseName, tblName, alias)( - table.getTTable, partitions.map(part => part.getTPartition))(hive) - - if (hive.convertMetastoreParquet && - hive.conf.parquetUseDataSourceApi && - relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet")) { - val metastoreSchema = StructType.fromAttributes(relation.output) - val paths = if (relation.hiveQlTable.isPartitioned) { - relation.hiveQlPartitions.map(p => p.getLocation) - } else { - Seq(relation.hiveQlTable.getDataLocation.toString) - } + MetastoreRelation(databaseName, tblName, alias)( + table.getTTable, partitions.map(part => part.getTPartition))(hive) + } + } - LogicalRelation(ParquetRelation2( - paths, Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json))(hive)) - } else { - relation + private def convertToParquetRelation(metastoreRelation: MetastoreRelation): LogicalRelation = { + val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) + + // NOTE: Instead of passing Metastore schema directly to `ParquetRelation2`, we have to + // serialize the Metastore schema to JSON and pass it as a data source option because of the + // evil case insensitivity issue, which is reconciled within `ParquetRelation2`. + if (metastoreRelation.hiveQlTable.isPartitioned) { + val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) + val partitionColumnDataTypes = partitionSchema.map(_.dataType) + val partitions = metastoreRelation.hiveQlPartitions.map { p => + val location = p.getLocation + val values = Row.fromSeq(p.getValues.zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + ParquetPartition(values, location) } + val partitionSpec = PartitionSpec(partitionSchema, partitions) + val paths = partitions.map(_.path) + LogicalRelation( + ParquetRelation2( + paths, + Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json), + None, + Some(partitionSpec))(hive)) + } else { + val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) + LogicalRelation( + ParquetRelation2( + paths, + Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json))(hive)) } } @@ -261,9 +271,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) - import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat import org.apache.hadoop.io.Text + import org.apache.hadoop.mapred.TextInputFormat tbl.setInputFormatClass(classOf[TextInputFormat]) tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) @@ -385,13 +395,56 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } + /** + * When scanning or writing to non-partitioned Metastore Parquet tables, convert them to Parquet + * data source relations for better performance. + * + * This rule can be considered as [[HiveStrategies.ParquetConversion]] done right. + */ + object ParquetConversions extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + // Collects all `MetastoreRelation`s which should be replaced + val toBeReplaced = plan.collect { + // Write path + case InsertIntoTable(relation: MetastoreRelation, _, _, _) + // Inserting into partitioned table is not supported in Parquet data source (yet). + if !relation.hiveQlTable.isPartitioned && + hive.convertMetastoreParquet && + hive.conf.parquetUseDataSourceApi && + relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => + relation + + // Read path + case p @ PhysicalOperation(_, _, relation: MetastoreRelation) + if hive.convertMetastoreParquet && + hive.conf.parquetUseDataSourceApi && + relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => + relation + } + + // Replaces all `MetastoreRelation`s with corresponding `ParquetRelation2`s, and fixes + // attribute IDs referenced in other nodes. + toBeReplaced.distinct.foldLeft(plan) { (lastPlan, relation) => + val parquetRelation = convertToParquetRelation(relation) + val attributedRewrites = AttributeMap(relation.output.zip(parquetRelation.output)) + + lastPlan.transformUp { + case r: MetastoreRelation if r == relation => parquetRelation + case other => other.transformExpressions { + case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) + } + } + } + } + } + /** * Creates any tables required for query execution. * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { import org.apache.hadoop.hive.ql.Context - import org.apache.hadoop.hive.ql.parse.{QB, ASTNode, SemanticAnalyzer} + import org.apache.hadoop.hive.ql.parse.{ASTNode, QB, SemanticAnalyzer} def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index cb138be90e2e1..965d159656d80 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -139,15 +139,19 @@ private[hive] trait HiveStrategies { val partitionLocations = partitions.map(_.getLocation) - hiveContext - .parquetFile(partitionLocations.head, partitionLocations.tail: _*) - .addPartitioningAttributes(relation.partitionKeys) - .lowerCase - .where(unresolvedOtherPredicates) - .select(unresolvedProjection: _*) - .queryExecution - .executedPlan - .fakeOutput(projectList.map(_.toAttribute)) :: Nil + if (partitionLocations.isEmpty) { + PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil + } else { + hiveContext + .parquetFile(partitionLocations.head, partitionLocations.tail: _*) + .addPartitioningAttributes(relation.partitionKeys) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection: _*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } } else { hiveContext .parquetFile(relation.hiveQlTable.getDataLocation.toString) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index e246cbb6d77f0..2acf1a7767c19 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -40,7 +40,7 @@ case class ParquetDataWithKey(p: Int, intField: Int, stringField: String) * A suite to test the automatic conversion of metastore tables with parquet data to use the * built in parquet support. */ -class ParquetMetastoreSuite extends ParquetPartitioningTest { +class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { override def beforeAll(): Unit = { super.beforeAll() @@ -97,6 +97,9 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { } override def afterAll(): Unit = { + sql("DROP TABLE partitioned_parquet") + sql("DROP TABLE partitioned_parquet_with_key") + sql("DROP TABLE normal_parquet") setConf("spark.sql.hive.convertMetastoreParquet", "false") } @@ -113,10 +116,38 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { } } +class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { + val originalConf = conf.parquetUseDataSourceApi + + override def beforeAll(): Unit = { + super.beforeAll() + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + } + + override def afterAll(): Unit = { + super.afterAll() + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + +class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { + val originalConf = conf.parquetUseDataSourceApi + + override def beforeAll(): Unit = { + super.beforeAll() + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") + } + + override def afterAll(): Unit = { + super.afterAll() + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + /** * A suite of tests for the Parquet support through the data sources API. */ -class ParquetSourceSuite extends ParquetPartitioningTest { +class ParquetSourceSuiteBase extends ParquetPartitioningTest { override def beforeAll(): Unit = { super.beforeAll() @@ -146,6 +177,34 @@ class ParquetSourceSuite extends ParquetPartitioningTest { } } +class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { + val originalConf = conf.parquetUseDataSourceApi + + override def beforeAll(): Unit = { + super.beforeAll() + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + } + + override def afterAll(): Unit = { + super.afterAll() + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + +class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { + val originalConf = conf.parquetUseDataSourceApi + + override def beforeAll(): Unit = { + super.beforeAll() + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") + } + + override def afterAll(): Unit = { + super.afterAll() + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) + } +} + /** * A collection of tests for parquet data with various forms of partitioning. */ @@ -191,107 +250,99 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll } } - def run(prefix: String): Unit = { - Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => - test(s"$prefix: ordering of the partitioning columns $table") { - checkAnswer( - sql(s"SELECT p, stringField FROM $table WHERE p = 1"), - Seq.fill(10)(Row(1, "part-1")) - ) - - checkAnswer( - sql(s"SELECT stringField, p FROM $table WHERE p = 1"), - Seq.fill(10)(Row("part-1", 1)) - ) - } - - test(s"$prefix: project the partitioning column $table") { - checkAnswer( - sql(s"SELECT p, count(*) FROM $table group by p"), - Row(1, 10) :: - Row(2, 10) :: - Row(3, 10) :: - Row(4, 10) :: - Row(5, 10) :: - Row(6, 10) :: - Row(7, 10) :: - Row(8, 10) :: - Row(9, 10) :: - Row(10, 10) :: Nil - ) - } - - test(s"$prefix: project partitioning and non-partitioning columns $table") { - checkAnswer( - sql(s"SELECT stringField, p, count(intField) FROM $table GROUP BY p, stringField"), - Row("part-1", 1, 10) :: - Row("part-2", 2, 10) :: - Row("part-3", 3, 10) :: - Row("part-4", 4, 10) :: - Row("part-5", 5, 10) :: - Row("part-6", 6, 10) :: - Row("part-7", 7, 10) :: - Row("part-8", 8, 10) :: - Row("part-9", 9, 10) :: - Row("part-10", 10, 10) :: Nil - ) - } - - test(s"$prefix: simple count $table") { - checkAnswer( - sql(s"SELECT COUNT(*) FROM $table"), - Row(100)) - } - - test(s"$prefix: pruned count $table") { - checkAnswer( - sql(s"SELECT COUNT(*) FROM $table WHERE p = 1"), - Row(10)) - } - - test(s"$prefix: non-existent partition $table") { - checkAnswer( - sql(s"SELECT COUNT(*) FROM $table WHERE p = 1000"), - Row(0)) - } - - test(s"$prefix: multi-partition pruned count $table") { - checkAnswer( - sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"), - Row(30)) - } - - test(s"$prefix: non-partition predicates $table") { - checkAnswer( - sql(s"SELECT COUNT(*) FROM $table WHERE intField IN (1,2,3)"), - Row(30)) - } - - test(s"$prefix: sum $table") { - checkAnswer( - sql(s"SELECT SUM(intField) FROM $table WHERE intField IN (1,2,3) AND p = 1"), - Row(1 + 2 + 3)) - } - - test(s"$prefix: hive udfs $table") { - checkAnswer( - sql(s"SELECT concat(stringField, stringField) FROM $table"), - sql(s"SELECT stringField FROM $table").map { - case Row(s: String) => Row(s + s) - }.collect().toSeq) - } + Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => + test(s"ordering of the partitioning columns $table") { + checkAnswer( + sql(s"SELECT p, stringField FROM $table WHERE p = 1"), + Seq.fill(10)(Row(1, "part-1")) + ) + + checkAnswer( + sql(s"SELECT stringField, p FROM $table WHERE p = 1"), + Seq.fill(10)(Row("part-1", 1)) + ) + } + + test(s"project the partitioning column $table") { + checkAnswer( + sql(s"SELECT p, count(*) FROM $table group by p"), + Row(1, 10) :: + Row(2, 10) :: + Row(3, 10) :: + Row(4, 10) :: + Row(5, 10) :: + Row(6, 10) :: + Row(7, 10) :: + Row(8, 10) :: + Row(9, 10) :: + Row(10, 10) :: Nil + ) + } + + test(s"project partitioning and non-partitioning columns $table") { + checkAnswer( + sql(s"SELECT stringField, p, count(intField) FROM $table GROUP BY p, stringField"), + Row("part-1", 1, 10) :: + Row("part-2", 2, 10) :: + Row("part-3", 3, 10) :: + Row("part-4", 4, 10) :: + Row("part-5", 5, 10) :: + Row("part-6", 6, 10) :: + Row("part-7", 7, 10) :: + Row("part-8", 8, 10) :: + Row("part-9", 9, 10) :: + Row("part-10", 10, 10) :: Nil + ) + } + + test(s"simple count $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table"), + Row(100)) } - test(s"$prefix: $prefix: non-part select(*)") { + test(s"pruned count $table") { checkAnswer( - sql("SELECT COUNT(*) FROM normal_parquet"), + sql(s"SELECT COUNT(*) FROM $table WHERE p = 1"), Row(10)) } - } - setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") - run("Parquet data source enabled") + test(s"non-existent partition $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p = 1000"), + Row(0)) + } + + test(s"multi-partition pruned count $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"), + Row(30)) + } + + test(s"non-partition predicates $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE intField IN (1,2,3)"), + Row(30)) + } - setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") - run("Parquet data source disabled") + test(s"sum $table") { + checkAnswer( + sql(s"SELECT SUM(intField) FROM $table WHERE intField IN (1,2,3) AND p = 1"), + Row(1 + 2 + 3)) + } + + test(s"hive udfs $table") { + checkAnswer( + sql(s"SELECT concat(stringField, stringField) FROM $table"), + sql(s"SELECT stringField FROM $table").map { + case Row(s: String) => Row(s + s) + }.collect().toSeq) + } + } + + test("non-part select(*)") { + checkAnswer( + sql("SELECT COUNT(*) FROM normal_parquet"), + Row(10)) + } } From 1115e8e739ec4d60604cd79bd452770f041510d4 Mon Sep 17 00:00:00 2001 From: Xutingjun <1039320815@qq.com> Date: Mon, 16 Feb 2015 14:54:23 +0000 Subject: [PATCH 127/817] [SPARK-5831][Streaming]When checkpoint file size is bigger than 10, then delete the old ones Author: Xutingjun <1039320815@qq.com> Closes #4621 from XuTingjun/checkpointFile and squashes the following commits: b5f2816 [Xutingjun] fix bug --- .../src/main/scala/org/apache/spark/streaming/Checkpoint.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b780282bdac37..f88a8a0151550 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -152,7 +152,7 @@ class CheckpointWriter( // Delete old checkpoint files val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs) - if (allCheckpointFiles.size > 4) { + if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach(file => { logInfo("Deleting " + file) fs.delete(file, true) From a3afa4a1bff88c4d8a5228fcf1e0cfc132541a22 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 Feb 2015 17:04:30 +0000 Subject: [PATCH 128/817] SPARK-5815 [MLLIB] Part 2. Deprecate SVDPlusPlus APIs that expose DoubleMatrix from JBLAS Now, deprecated runSVDPlusPlus and update run, for 1.4.0 / master only Author: Sean Owen Closes #4625 from srowen/SPARK-5815.2 and squashes the following commits: 6fd2ca5 [Sean Owen] Now, deprecated runSVDPlusPlus and update run, for 1.4.0 / master only --- .../apache/spark/graphx/lib/SVDPlusPlus.scala | 42 +++++++------------ 1 file changed, 15 insertions(+), 27 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index fc84cfbe64184..3e4157a63fd1c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -17,8 +17,6 @@ package org.apache.spark.graphx.lib -import org.apache.spark.annotation.Experimental - import scala.util.Random import org.jblas.DoubleMatrix import org.apache.spark.rdd._ @@ -40,8 +38,17 @@ object SVDPlusPlus { extends Serializable /** - * :: Experimental :: - * + * This method is now replaced by the updated version of `run()` and returns exactly + * the same result. + */ + @deprecated("Call run()", "1.4.0") + def runSVDPlusPlus(edges: RDD[Edge[Double]], conf: Conf) + : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = + { + run(edges, conf) + } + + /** * Implement SVD++ based on "Factorization Meets the Neighborhood: * a Multifaceted Collaborative Filtering Model", * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. @@ -49,35 +56,14 @@ object SVDPlusPlus { * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), * see the details on page 6. * - * This method temporarily replaces `run()`, and replaces `DoubleMatrix` in `run()`'s return - * value with `Array[Double]`. In 1.4.0, this method will be deprecated, but will be copied - * to replace `run()`, which will then be undeprecated. - * * @param edges edges for constructing the graph * * @param conf SVDPlusPlus parameters * * @return a graph with vertex attributes containing the trained model */ - @Experimental - def runSVDPlusPlus(edges: RDD[Edge[Double]], conf: Conf) - : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = - { - val (graph, u) = run(edges, conf) - // Convert DoubleMatrix to Array[Double]: - val newVertices = graph.vertices.mapValues(v => (v._1.toArray, v._2.toArray, v._3, v._4)) - (Graph(newVertices, graph.edges), u) - } - - /** - * This method is deprecated in favor of `runSVDPlusPlus()`, which replaces `DoubleMatrix` - * with `Array[Double]` in its return value. This method is deprecated. It will effectively - * be removed in 1.4.0 when `runSVDPlusPlus()` is copied to replace `run()`, and hence the - * return type of this method changes. - */ - @deprecated("Call runSVDPlusPlus", "1.3.0") def run(edges: RDD[Edge[Double]], conf: Conf) - : (Graph[(DoubleMatrix, DoubleMatrix, Double, Double), Double], Double) = + : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { // Generate default vertex attribute def defaultF(rank: Int): (DoubleMatrix, DoubleMatrix, Double, Double) = { @@ -194,7 +180,9 @@ object SVDPlusPlus { g.unpersist() g = gJoinT3 - (g, u) + // Convert DoubleMatrix to Array[Double]: + val newVertices = g.vertices.mapValues(v => (v._1.toArray, v._2.toArray, v._3, v._4)) + (Graph(newVertices, g.edges), u) } /** From 5c78be7a515fc2fc92cda0517318e7b5d85762f4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 16 Feb 2015 10:06:11 -0800 Subject: [PATCH 129/817] [SPARK-5799][SQL] Compute aggregation function on specified numeric columns Compute aggregation function on specified numeric columns. For example: val df = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")).toDataFrame("key", "value1", "value2", "rest") df.groupBy("key").min("value2") Author: Liang-Chi Hsieh Closes #4592 from viirya/specific_cols_agg and squashes the following commits: 9446896 [Liang-Chi Hsieh] For comments. 314c4cd [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg 353fad7 [Liang-Chi Hsieh] For python unit tests. 54ed0c4 [Liang-Chi Hsieh] Address comments. b079e6b [Liang-Chi Hsieh] Remove duplicate codes. 55100fb [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg 880c2ac [Liang-Chi Hsieh] Fix Python style checks. 4c63a01 [Liang-Chi Hsieh] Fix pyspark. b1a24fc [Liang-Chi Hsieh] Address comments. 2592f29 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into specific_cols_agg 27069c3 [Liang-Chi Hsieh] Combine functions and add varargs annotation. 371a3f7 [Liang-Chi Hsieh] Compute aggregation function on specified numeric columns. --- python/pyspark/sql/dataframe.py | 74 +++++++++++++++---- python/pyspark/sql/functions.py | 2 + .../org/apache/spark/sql/DataFrameImpl.scala | 4 +- .../org/apache/spark/sql/GroupedData.scala | 57 +++++++++++--- .../org/apache/spark/sql/DataFrameSuite.scala | 12 +++ 5 files changed, 123 insertions(+), 26 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 1438fe5285cc5..28a59e73a3410 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -664,6 +664,18 @@ def _api(self): return _api +def df_varargs_api(f): + def _api(self, *args): + jargs = ListConverter().convert(args, + self.sql_ctx._sc._gateway._gateway_client) + name = f.__name__ + jdf = getattr(self._jdf, name)(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jargs)) + return DataFrame(jdf, self.sql_ctx) + _api.__name__ = f.__name__ + _api.__doc__ = f.__doc__ + return _api + + class GroupedData(object): """ @@ -714,30 +726,60 @@ def count(self): [Row(age=2, count=1), Row(age=5, count=1)] """ - @dfapi - def mean(self): + @df_varargs_api + def mean(self, *cols): """Compute the average value for each numeric columns - for each group. This is an alias for `avg`.""" + for each group. This is an alias for `avg`. - @dfapi - def avg(self): + >>> df.groupBy().mean('age').collect() + [Row(AVG(age#0)=3.5)] + >>> df3.groupBy().mean('age', 'height').collect() + [Row(AVG(age#4)=3.5, AVG(height#5)=82.5)] + """ + + @df_varargs_api + def avg(self, *cols): """Compute the average value for each numeric columns - for each group.""" + for each group. - @dfapi - def max(self): + >>> df.groupBy().avg('age').collect() + [Row(AVG(age#0)=3.5)] + >>> df3.groupBy().avg('age', 'height').collect() + [Row(AVG(age#4)=3.5, AVG(height#5)=82.5)] + """ + + @df_varargs_api + def max(self, *cols): """Compute the max value for each numeric columns for - each group. """ + each group. - @dfapi - def min(self): + >>> df.groupBy().max('age').collect() + [Row(MAX(age#0)=5)] + >>> df3.groupBy().max('age', 'height').collect() + [Row(MAX(age#4)=5, MAX(height#5)=85)] + """ + + @df_varargs_api + def min(self, *cols): """Compute the min value for each numeric column for - each group.""" + each group. - @dfapi - def sum(self): + >>> df.groupBy().min('age').collect() + [Row(MIN(age#0)=2)] + >>> df3.groupBy().min('age', 'height').collect() + [Row(MIN(age#4)=2, MIN(height#5)=80)] + """ + + @df_varargs_api + def sum(self, *cols): """Compute the sum for each numeric columns for each - group.""" + group. + + >>> df.groupBy().sum('age').collect() + [Row(SUM(age#0)=7)] + >>> df3.groupBy().sum('age', 'height').collect() + [Row(SUM(age#4)=7, SUM(height#5)=165)] + """ def _create_column_from_literal(literal): @@ -945,6 +987,8 @@ def _test(): globs['sqlCtx'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() + globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), + Row(name='Bob', age=5, height=85)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 39aa550eeb5ad..d0e090607ff4f 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -158,6 +158,8 @@ def _test(): globs['sqlCtx'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() + globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), + Row(name='Bob', age=5, height=85)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 7b7efbe3477b6..9eb0c131405d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -88,12 +88,12 @@ private[sql] class DataFrameImpl protected[sql]( } } - protected[sql] def numericColumns: Seq[Expression] = { + protected[sql] def numericColumns(): Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get } } - + override def toDF(colNames: String*): DataFrame = { require(schema.size == colNames.size, "The number of columns doesn't match.\n" + 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 0868013fe7c96..a5a677b68863f 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,6 +23,8 @@ import scala.collection.JavaConversions._ 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.types.NumericType + /** @@ -39,13 +41,30 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) } - private[this] def aggregateNumericColumns(f: Expression => Expression): Seq[NamedExpression] = { - df.numericColumns.map { c => + private[this] def aggregateNumericColumns(colNames: String*)(f: Expression => Expression) + : Seq[NamedExpression] = { + + val columnExprs = if (colNames.isEmpty) { + // No columns specified. Use all numeric columns. + df.numericColumns + } else { + // Make sure all specified columns are numeric + colNames.map { colName => + val namedExpr = df.resolve(colName) + if (!namedExpr.dataType.isInstanceOf[NumericType]) { + throw new AnalysisException( + s""""$colName" is not a numeric column. """ + + "Aggregation function can only be performed on a numeric column.") + } + namedExpr + } + } + columnExprs.map { c => val a = f(c) Alias(a, a.toString)() } } - + private[this] def strToExpr(expr: String): (Expression => Expression) = { expr.toLowerCase match { case "avg" | "average" | "mean" => Average @@ -152,30 +171,50 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio /** * Compute the average value for each numeric columns for each group. This is an alias for `avg`. * The resulting [[DataFrame]] will also contain the grouping columns. + * When specified columns are given, only compute the average values for them. */ - def mean(): DataFrame = aggregateNumericColumns(Average) - + @scala.annotation.varargs + def mean(colNames: String*): DataFrame = { + aggregateNumericColumns(colNames:_*)(Average) + } + /** * Compute the max value for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. + * When specified columns are given, only compute the max values for them. */ - def max(): DataFrame = aggregateNumericColumns(Max) + @scala.annotation.varargs + def max(colNames: String*): DataFrame = { + aggregateNumericColumns(colNames:_*)(Max) + } /** * Compute the mean value for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. + * When specified columns are given, only compute the mean values for them. */ - def avg(): DataFrame = aggregateNumericColumns(Average) + @scala.annotation.varargs + def avg(colNames: String*): DataFrame = { + aggregateNumericColumns(colNames:_*)(Average) + } /** * Compute the min value for each numeric column for each group. * The resulting [[DataFrame]] will also contain the grouping columns. + * When specified columns are given, only compute the min values for them. */ - def min(): DataFrame = aggregateNumericColumns(Min) + @scala.annotation.varargs + def min(colNames: String*): DataFrame = { + aggregateNumericColumns(colNames:_*)(Min) + } /** * Compute the sum for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. + * When specified columns are given, only compute the sum for them. */ - def sum(): DataFrame = aggregateNumericColumns(Sum) + @scala.annotation.varargs + def sum(colNames: String*): DataFrame = { + aggregateNumericColumns(colNames:_*)(Sum) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f0cd43632ec3f..524571d9cc636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -162,6 +162,18 @@ class DataFrameSuite extends QueryTest { testData2.groupBy("a").agg(Map("b" -> "sum")), Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil ) + + val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) + .toDF("key", "value1", "value2", "rest") + + checkAnswer( + df1.groupBy("key").min(), + df1.groupBy("key").min("value1", "value2").collect() + ) + checkAnswer( + df1.groupBy("key").min("value2"), + Seq(Row("a", 0), Row("b", 4)) + ) } test("agg without groups") { From 9baac56ccd57d3890a9b6439d4e13bbe9381822b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 16 Feb 2015 10:09:55 -0800 Subject: [PATCH 130/817] Minor fixes for commit https://github.com/apache/spark/pull/4592. --- .../main/scala/org/apache/spark/sql/DataFrameImpl.scala | 6 +++--- .../main/scala/org/apache/spark/sql/GroupedData.scala | 9 ++++----- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 9eb0c131405d8..500e3c90fdbc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -83,17 +83,17 @@ private[sql] class DataFrameImpl protected[sql]( protected[sql] def resolve(colName: String): NamedExpression = { queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { - throw new RuntimeException( + throw new AnalysisException( s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") } } - protected[sql] def numericColumns(): Seq[Expression] = { + protected[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get } } - + override def toDF(colNames: String*): DataFrame = { require(schema.size == colNames.size, "The number of columns doesn't match.\n" + 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 a5a677b68863f..2ecf086de92f7 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 @@ -17,8 +17,8 @@ package org.apache.spark.sql -import scala.language.implicitConversions import scala.collection.JavaConversions._ +import scala.language.implicitConversions import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate import org.apache.spark.sql.types.NumericType - /** * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ @@ -48,13 +47,13 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio // No columns specified. Use all numeric columns. df.numericColumns } else { - // Make sure all specified columns are numeric + // Make sure all specified columns are numeric. colNames.map { colName => val namedExpr = df.resolve(colName) if (!namedExpr.dataType.isInstanceOf[NumericType]) { throw new AnalysisException( s""""$colName" is not a numeric column. """ + - "Aggregation function can only be performed on a numeric column.") + "Aggregation function can only be applied on a numeric column.") } namedExpr } @@ -64,7 +63,7 @@ class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expressio Alias(a, a.toString)() } } - + private[this] def strToExpr(expr: String): (Expression => Expression) = { expr.toLowerCase match { case "avg" | "average" | "mean" => Average From 8e25373ce72061d3b6a353259ec627606afa4a5f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 Feb 2015 19:32:31 +0000 Subject: [PATCH 131/817] SPARK-5795 [STREAMING] api.java.JavaPairDStream.saveAsNewAPIHadoopFiles may not friendly to java Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended. CC tdas for review Author: Sean Owen Closes #4608 from srowen/SPARK-5795 and squashes the following commits: 36f1ead [Sean Owen] Add code that shows compile problem and fix 036bd27 [Sean Owen] Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended. --- .../streaming/api/java/JavaPairDStream.scala | 20 +++++++++---------- .../apache/spark/streaming/JavaAPISuite.java | 18 +++++++++++++++++ 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index de124cf40eff1..bd01789b611a4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -726,7 +726,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsHadoopFiles[F <: OutputFormat[K, V]](prefix: String, suffix: String) { + def saveAsHadoopFiles(prefix: String, suffix: String) { dstream.saveAsHadoopFiles(prefix, suffix) } @@ -734,12 +734,12 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsHadoopFiles( + def saveAsHadoopFiles[F <: OutputFormat[_, _]]( prefix: String, suffix: String, keyClass: Class[_], valueClass: Class[_], - outputFormatClass: Class[_ <: OutputFormat[_, _]]) { + outputFormatClass: Class[F]) { dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass) } @@ -747,12 +747,12 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsHadoopFiles( + def saveAsHadoopFiles[F <: OutputFormat[_, _]]( prefix: String, suffix: String, keyClass: Class[_], valueClass: Class[_], - outputFormatClass: Class[_ <: OutputFormat[_, _]], + outputFormatClass: Class[F], conf: JobConf) { dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf) } @@ -761,7 +761,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](prefix: String, suffix: String) { + def saveAsNewAPIHadoopFiles(prefix: String, suffix: String) { dstream.saveAsNewAPIHadoopFiles(prefix, suffix) } @@ -769,12 +769,12 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsNewAPIHadoopFiles( + def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[_, _]]( prefix: String, suffix: String, keyClass: Class[_], valueClass: Class[_], - outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) { + outputFormatClass: Class[F]) { dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass) } @@ -782,12 +782,12 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". */ - def saveAsNewAPIHadoopFiles( + def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[_, _]]( prefix: String, suffix: String, keyClass: Class[_], valueClass: Class[_], - outputFormatClass: Class[_ <: NewOutputFormat[_, _]], + outputFormatClass: Class[F], conf: Configuration = new Configuration) { dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf) } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 2df8cf6a8a3df..57302ff407183 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1828,4 +1828,22 @@ private List> fileTestPrepare(File testDir) throws IOException { return expected; } + + // SPARK-5795: no logic assertions, just testing that intended API invocations compile + private void compileSaveAsJavaAPI(JavaPairDStream pds) { + pds.saveAsNewAPIHadoopFiles( + "", "", LongWritable.class, Text.class, + org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); + pds.saveAsHadoopFiles( + "", "", LongWritable.class, Text.class, + org.apache.hadoop.mapred.SequenceFileOutputFormat.class); + // Checks that a previous common workaround for this API still compiles + pds.saveAsNewAPIHadoopFiles( + "", "", LongWritable.class, Text.class, + (Class) org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); + pds.saveAsHadoopFiles( + "", "", LongWritable.class, Text.class, + (Class) org.apache.hadoop.mapred.SequenceFileOutputFormat.class); + } + } From cc552e042896350e21eec9b78593de25006ecc70 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 16 Feb 2015 12:21:08 -0800 Subject: [PATCH 132/817] [SQL] [Minor] Update the SpecificMutableRow.copy When profiling the Join / Aggregate queries via VisualVM, I noticed lots of `SpecificMutableRow` objects created, as well as the `MutableValue`, since the `SpecificMutableRow` are mostly used in data source implementation, but the `copy` method could be called multiple times in upper modules (e.g. in Join / aggregation etc.), duplicated instances created should be avoid. Author: Cheng Hao Closes #4619 from chenghao-intel/specific_mutable_row and squashes the following commits: 9300d23 [Cheng Hao] update the SpecificMutableRow.copy --- .../sql/catalyst/expressions/SpecificMutableRow.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 7434165f654f8..21d714c9a8c3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -220,13 +220,14 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR override def isNullAt(i: Int): Boolean = values(i).isNull override def copy(): Row = { - val newValues = new Array[MutableValue](values.length) + val newValues = new Array[Any](values.length) var i = 0 while (i < values.length) { - newValues(i) = values(i).copy() + newValues(i) = values(i).boxed i += 1 } - new SpecificMutableRow(newValues) + + new GenericRow(newValues) } override def update(ordinal: Int, value: Any): Unit = { From 275a0c08134dea1896eab73a8e017256900fb1db Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 16 Feb 2015 12:31:36 -0800 Subject: [PATCH 133/817] [SPARK-5824] [SQL] add null format in ctas and set default col comment to null Author: Daoyuan Wang Closes #4609 from adrian-wang/ctas and squashes the following commits: 0a75d5a [Daoyuan Wang] reorder import 93d1863 [Daoyuan Wang] add null format in ctas and set default col comment to null --- .../execution/HiveCompatibilitySuite.scala | 1 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 ++- ...ormatCTAS-0-36f9196395758cebfed837a1c391a1e | 0 ...rmatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b | 0 ...matCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++++++++++ ...matCTAS-11-4a4c16b53c612d00012d338c97bf5281 | 0 ...matCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b | 10 ++++++++++ ...matCTAS-13-2e59caa113585495d8684fee69d88bc0 | 0 ...matCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 | 0 ...rmatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d | 0 ...rmatCTAS-3-b0057150f237050f38c1efa1f2d6b273 | 6 ++++++ ...rmatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd | 0 ...rmatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 | 0 ...rmatCTAS-6-159fff36b548e00ee952d1df8ef19833 | 0 ...rmatCTAS-7-46900b082b02ce3e58087d1f41128f65 | 4 ++++ ...rmatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 | 18 ++++++++++++++++++ ...rmatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 | 1 + .../org/apache/spark/sql/hive/Shim12.scala | 2 ++ .../org/apache/spark/sql/hive/Shim13.scala | 7 +++++++ 19 files changed, 61 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-11-4a4c16b53c612d00012d338c97bf5281 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-13-2e59caa113585495d8684fee69d88bc0 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-6-159fff36b548e00ee952d1df8ef19833 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-7-46900b082b02ce3e58087d1f41128f65 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 create mode 100644 sql/hive/src/test/resources/golden/nullformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index e443e5bd5f54d..133f2d3c84a2e 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -640,6 +640,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "nonblock_op_deduplicate", "notable_alias1", "notable_alias2", + "nullformatCTAS", "nullgroup", "nullgroup2", "nullgroup3", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 6d794d0e11391..f82778c87672c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -240,7 +240,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { crtTbl.getCols } else { - schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) + schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), null)) } tbl.setFields(hiveSchema) @@ -314,6 +314,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with if (crtTbl != null && crtTbl.getLineDelim() != null) { tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) } + HiveShim.setTblNullFormat(crtTbl, tbl) if (crtTbl != null && crtTbl.getSerdeProps() != null) { val iter = crtTbl.getSerdeProps().entrySet().iterator() diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e b/sql/hive/src/test/resources/golden/nullformatCTAS-0-36f9196395758cebfed837a1c391a1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b b/sql/hive/src/test/resources/golden/nullformatCTAS-1-b5a31d4cb34218b8de1ac3fed59fa75b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b b/sql/hive/src/test/resources/golden/nullformatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b new file mode 100644 index 0000000000000..e74deff51c9ba --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-10-7f4f04b87c7ef9653b4646949b24cf0b @@ -0,0 +1,10 @@ +1.0 1 +1.0 1 +1.0 1 +1.0 1 +1.0 1 +NULL 1 +NULL NULL +1.0 NULL +1.0 1 +1.0 1 diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-11-4a4c16b53c612d00012d338c97bf5281 b/sql/hive/src/test/resources/golden/nullformatCTAS-11-4a4c16b53c612d00012d338c97bf5281 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b b/sql/hive/src/test/resources/golden/nullformatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b new file mode 100644 index 0000000000000..00ebb521970dd --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-12-7f4f04b87c7ef9653b4646949b24cf0b @@ -0,0 +1,10 @@ +1.0 1 +1.0 1 +1.0 1 +1.0 1 +1.0 1 +fooNull 1 +fooNull fooNull +1.0 fooNull +1.0 1 +1.0 1 diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-13-2e59caa113585495d8684fee69d88bc0 b/sql/hive/src/test/resources/golden/nullformatCTAS-13-2e59caa113585495d8684fee69d88bc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 b/sql/hive/src/test/resources/golden/nullformatCTAS-14-ad9fe9d68c2cf492259af4f6167c1b12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d b/sql/hive/src/test/resources/golden/nullformatCTAS-2-aa2bdbd93668dceae43d1a02f2ede68d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 b/sql/hive/src/test/resources/golden/nullformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 new file mode 100644 index 0000000000000..b00bcb3624532 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-3-b0057150f237050f38c1efa1f2d6b273 @@ -0,0 +1,6 @@ +a string +b string +c string +d string + +Detailed Table Information Table(tableName:base_tab, dbName:default, owner:animal, createTime:1423973915, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:c, type:string, comment:null), FieldSchema(name:d, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2573474017665704744/base_tab, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1423973915, COLUMN_STATS_ACCURATE=true, totalSize=130, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd b/sql/hive/src/test/resources/golden/nullformatCTAS-4-16c7086f39d6458b6c5cf2479f0473bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 b/sql/hive/src/test/resources/golden/nullformatCTAS-5-183d77b734ce6a373de5b3ebe1cd04c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-6-159fff36b548e00ee952d1df8ef19833 b/sql/hive/src/test/resources/golden/nullformatCTAS-6-159fff36b548e00ee952d1df8ef19833 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-7-46900b082b02ce3e58087d1f41128f65 b/sql/hive/src/test/resources/golden/nullformatCTAS-7-46900b082b02ce3e58087d1f41128f65 new file mode 100644 index 0000000000000..264c973ff7af1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-7-46900b082b02ce3e58087d1f41128f65 @@ -0,0 +1,4 @@ +a string +b string + +Detailed Table Information Table(tableName:null_tab3, dbName:default, owner:animal, createTime:1423973928, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2573474017665704744/null_tab3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.null.format=fooNull, serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1423973928, COLUMN_STATS_ACCURATE=true, totalSize=80, numRows=10, rawDataSize=70}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 b/sql/hive/src/test/resources/golden/nullformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 new file mode 100644 index 0000000000000..881917bcf1c69 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-8-7f26cbd6be5631a3acce26f667d1c5d8 @@ -0,0 +1,18 @@ +CREATE TABLE `null_tab3`( + `a` string, + `b` string) +ROW FORMAT DELIMITED + NULL DEFINED AS 'fooNull' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse2573474017665704744/null_tab3' +TBLPROPERTIES ( + 'numFiles'='1', + 'transient_lastDdlTime'='1423973928', + 'COLUMN_STATS_ACCURATE'='true', + 'totalSize'='80', + 'numRows'='10', + 'rawDataSize'='70') diff --git a/sql/hive/src/test/resources/golden/nullformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 b/sql/hive/src/test/resources/golden/nullformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullformatCTAS-9-22e1b3899de7087b39c24d9d8f643b47 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index b5a0754ff61f9..8534c7d7064e5 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -245,6 +245,8 @@ private[hive] object HiveShim { def prepareWritable(w: Writable): Writable = { w } + + def setTblNullFormat(crtTbl: CreateTableDesc, tbl: Table) = {} } class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index e4c1809c8bb21..72104f5b55761 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} @@ -410,6 +411,12 @@ private[hive] object HiveShim { } w } + + def setTblNullFormat(crtTbl: CreateTableDesc, tbl: Table) = { + if (crtTbl != null && crtTbl.getNullFormat() != null) { + tbl.setSerdeParam(serdeConstants.SERIALIZATION_NULL_FORMAT, crtTbl.getNullFormat()) + } + } } /* From 104b2c45805ce0a9c86e2823f402de6e9f0aee81 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 16 Feb 2015 12:32:56 -0800 Subject: [PATCH 134/817] [SQL] Initial support for reporting location of error in sql string Author: Michael Armbrust Closes #4587 from marmbrus/position and squashes the following commits: 0810052 [Michael Armbrust] fix tests 395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' into position e155dce [Michael Armbrust] more errors f3efa51 [Michael Armbrust] Update AnalysisException.scala d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of error in sql string --- .../apache/spark/sql/AnalysisException.scala | 17 +- .../sql/catalyst/analysis/Analyzer.scala | 19 +- .../spark/sql/catalyst/analysis/Catalog.scala | 6 + .../spark/sql/catalyst/analysis/package.scala | 10 ++ .../spark/sql/catalyst/trees/TreeNode.scala | 60 ++++++- .../sql/catalyst/trees/TreeNodeSuite.scala | 14 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 9 +- .../org/apache/spark/sql/hive/HiveQl.scala | 47 +++-- .../spark/sql/hive/CachedTableSuite.scala | 4 +- .../spark/sql/hive/ErrorPositionSuite.scala | 163 ++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 4 +- 11 files changed, 314 insertions(+), 39 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 871d560b9d54f..15add84878ecf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -17,7 +17,22 @@ package org.apache.spark.sql +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Thrown when a query fails to analyze, usually because the query itself is invalid. */ -class AnalysisException(message: String) extends Exception(message) with Serializable +@DeveloperApi +class AnalysisException protected[sql] ( + val message: String, + val line: Option[Int] = None, + val startPosition: Option[Int] = None) + extends Exception with Serializable { + + override def getMessage: String = { + val lineAnnotation = line.map(l => s" line $l").getOrElse("") + val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("") + s"$message;$lineAnnotation$positionAnnotation" + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 58a7003977c93..aa4320bd582cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -85,7 +85,7 @@ class Analyzer(catalog: Catalog, operator transformExpressionsUp { case a: Attribute if !a.resolved => val from = operator.inputSet.map(_.name).mkString(", ") - failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") + a.failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") case c: Cast if !c.resolved => failAnalysis( @@ -246,12 +246,21 @@ class Analyzer(catalog: Catalog, * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { + def getTable(u: UnresolvedRelation) = { + try { + catalog.lookupRelation(u.tableIdentifier, u.alias) + } catch { + case _: NoSuchTableException => + u.failAnalysis(s"no such table ${u.tableIdentifier}") + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case i @ InsertIntoTable(UnresolvedRelation(tableIdentifier, alias), _, _, _) => + case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _) => i.copy( - table = EliminateSubQueries(catalog.lookupRelation(tableIdentifier, alias))) - case UnresolvedRelation(tableIdentifier, alias) => - catalog.lookupRelation(tableIdentifier, alias) + table = EliminateSubQueries(getTable(u))) + case u: UnresolvedRelation => + getTable(u) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index f57eab24607f8..bf97215ee67da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -21,6 +21,12 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} +/** + * Thrown by a catalog when a table cannot be found. The analzyer will rethrow the exception + * as an AnalysisException with the correct position information. + */ +class NoSuchTableException extends Exception + /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 5dc9d0e566087..e95f19e69ed43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.trees.TreeNode + /** * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s @@ -32,4 +35,11 @@ package object analysis { val caseInsensitiveResolution = (a: String, b: String) => a.equalsIgnoreCase(b) val caseSensitiveResolution = (a: String, b: String) => a == b + + implicit class AnalysisErrorAt(t: TreeNode[_]) { + /** Fails the analysis at the point where a specific tree node was parsed. */ + def failAnalysis(msg: String) = { + throw new AnalysisException(msg, t.origin.line, t.origin.startPosition) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index e0930b056d5fa..109671bdca361 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -22,9 +22,42 @@ import org.apache.spark.sql.catalyst.errors._ /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) +case class Origin( + line: Option[Int] = None, + startPosition: Option[Int] = None) + +/** + * Provides a location for TreeNodes to ask about the context of their origin. For example, which + * line of code is currently being parsed. + */ +object CurrentOrigin { + private val value = new ThreadLocal[Origin]() { + override def initialValue: Origin = Origin() + } + + def get = value.get() + def set(o: Origin) = value.set(o) + + def reset() = value.set(Origin()) + + def setPosition(line: Int, start: Int) = { + value.set( + value.get.copy(line = Some(line), startPosition = Some(start))) + } + + def withOrigin[A](o: Origin)(f: => A): A = { + set(o) + val ret = try f finally { reset() } + reset() + ret + } +} + abstract class TreeNode[BaseType <: TreeNode[BaseType]] { self: BaseType with Product => + val origin = CurrentOrigin.get + /** Returns a Seq of the children of this node */ def children: Seq[BaseType] @@ -150,7 +183,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param rule the function used to transform this nodes children */ def transformDown(rule: PartialFunction[BaseType, BaseType]): BaseType = { - val afterRule = rule.applyOrElse(this, identity[BaseType]) + val afterRule = CurrentOrigin.withOrigin(origin) { + rule.applyOrElse(this, identity[BaseType]) + } + // Check if unchanged and then possibly return old copy to avoid gc churn. if (this fastEquals afterRule) { transformChildrenDown(rule) @@ -210,9 +246,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { def transformUp(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRuleOnChildren = transformChildrenUp(rule); if (this fastEquals afterRuleOnChildren) { - rule.applyOrElse(this, identity[BaseType]) + CurrentOrigin.withOrigin(origin) { + rule.applyOrElse(this, identity[BaseType]) + } } else { - rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) + CurrentOrigin.withOrigin(origin) { + rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) + } } } @@ -268,12 +308,14 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { try { - // Skip no-arg constructors that are just there for kryo. - val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head - if (otherCopyArgs.isEmpty) { - defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] - } else { - defaultCtor.newInstance((newArgs ++ otherCopyArgs).toArray: _*).asInstanceOf[this.type] + CurrentOrigin.withOrigin(origin) { + // Skip no-arg constructors that are just there for kryo. + val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head + if (otherCopyArgs.isEmpty) { + defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] + } else { + defaultCtor.newInstance((newArgs ++ otherCopyArgs).toArray: _*).asInstanceOf[this.type] + } } } catch { case e: java.lang.IllegalArgumentException => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index cdb843f959704..e7ce92a2160b6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -104,4 +104,18 @@ class TreeNodeSuite extends FunSuite { assert(actual === Dummy(None)) } + test("preserves origin") { + CurrentOrigin.setPosition(1,1) + val add = Add(Literal(1), Literal(1)) + CurrentOrigin.reset() + + val transformed = add transform { + case Literal(1, _) => Literal(2) + } + + assert(transformed.origin.line.isDefined) + assert(transformed.origin.startPosition.isDefined) + } + + } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f82778c87672c..12f86a04a37af 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -31,8 +31,8 @@ import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.analysis.{Catalog, OverrideCatalog} +import org.apache.spark.sql.{AnalysisException, SQLContext} +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical @@ -154,7 +154,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - val table = client.getTable(databaseName, tblName) + val table = try client.getTable(databaseName, tblName) catch { + case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => + throw new NoSuchTableException + } if (table.getProperty("spark.sql.sources.provider") != null) { cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5269460e5b6bc..5a1825a87dadb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.sql.Date + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf @@ -27,13 +28,14 @@ import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils -import org.apache.spark.sql.SparkSQLParser +import org.apache.spark.sql.{AnalysisException, SparkSQLParser} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.sources.DescribeCommand import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} @@ -211,12 +213,6 @@ private[hive] object HiveQl { } } - class ParseException(sql: String, cause: Throwable) - extends Exception(s"Failed to parse: $sql", cause) - - class SemanticException(msg: String) - extends Exception(s"Error in semantic analysis: $msg") - /** * Returns the AST for the given SQL string. */ @@ -236,8 +232,10 @@ private[hive] object HiveQl { /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = hqlParser(sql) + val errorRegEx = "line (\\d+):(\\d+) (.*)".r + /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String) = { + def createPlan(sql: String): LogicalPlan = { try { val tree = getAst(sql) if (nativeCommands contains tree.getText) { @@ -249,14 +247,23 @@ private[hive] object HiveQl { } } } catch { - case e: Exception => throw new ParseException(sql, e) - case e: NotImplementedError => sys.error( - s""" - |Unsupported language features in query: $sql - |${dumpTree(getAst(sql))} - |$e - |${e.getStackTrace.head} - """.stripMargin) + case pe: org.apache.hadoop.hive.ql.parse.ParseException => + pe.getMessage match { + case errorRegEx(line, start, message) => + throw new AnalysisException(message, Some(line.toInt), Some(start.toInt)) + case otherMessage => + throw new AnalysisException(otherMessage) + } + case e: Exception => + throw new AnalysisException(e.getMessage) + case e: NotImplementedError => + throw new AnalysisException( + s""" + |Unsupported language features in query: $sql + |${dumpTree(getAst(sql))} + |$e + |${e.getStackTrace.head} + """.stripMargin) } } @@ -292,6 +299,7 @@ private[hive] object HiveQl { /** @return matches of the form (tokenName, children). */ def unapply(t: Any): Option[(String, Seq[ASTNode])] = t match { case t: ASTNode => + CurrentOrigin.setPosition(t.getLine, t.getCharPositionInLine) Some((t.getText, Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) case _ => None @@ -1278,7 +1286,12 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) : StringBuilder = { node match { - case a: ASTNode => builder.append((" " * indent) + a.getText + "\n") + case a: ASTNode => builder.append( + (" " * indent) + a.getText + " " + + a.getLine + ", " + + a.getTokenStartIndex + "," + + a.getTokenStopIndex + ", " + + a.getCharPositionInLine + "\n") case other => sys.error(s"Non ASTNode encountered: $other") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 7c8b5205e239e..44d24273e722a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} import org.apache.spark.storage.RDDBlockId class CachedTableSuite extends QueryTest { @@ -96,7 +96,7 @@ class CachedTableSuite extends QueryTest { cacheTable("test") sql("SELECT * FROM test").collect() sql("DROP TABLE test") - intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + intercept[AnalysisException] { sql("SELECT * FROM test").collect() } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala new file mode 100644 index 0000000000000..f04437c595bf6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.{OutputStream, PrintStream} + +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.{AnalysisException, QueryTest} + +import scala.util.Try + +class ErrorPositionSuite extends QueryTest { + + positionTest("unresolved attribute 1", + "SELECT x FROM src", "x") + + positionTest("unresolved attribute 2", + "SELECT x FROM src", "x") + + positionTest("unresolved attribute 3", + "SELECT key, x FROM src", "x") + + positionTest("unresolved attribute 4", + """SELECT key, + |x FROM src + """.stripMargin, "x") + + positionTest("unresolved attribute 5", + """SELECT key, + | x FROM src + """.stripMargin, "x") + + positionTest("unresolved attribute 6", + """SELECT key, + | + | 1 + x FROM src + """.stripMargin, "x") + + positionTest("unresolved attribute 7", + """SELECT key, + | + | 1 + x + 1 FROM src + """.stripMargin, "x") + + positionTest("multi-char unresolved attribute", + """SELECT key, + | + | 1 + abcd + 1 FROM src + """.stripMargin, "abcd") + + positionTest("unresolved attribute group by", + """SELECT key FROM src GROUP BY + |x + """.stripMargin, "x") + + positionTest("unresolved attribute order by", + """SELECT key FROM src ORDER BY + |x + """.stripMargin, "x") + + positionTest("unresolved attribute where", + """SELECT key FROM src + |WHERE x = true + """.stripMargin, "x") + + positionTest("unresolved attribute backticks", + "SELECT `x` FROM src", "`x`") + + positionTest("parse error", + "SELECT WHERE", "WHERE") + + positionTest("bad relation", + "SELECT * FROM badTable", "badTable") + + ignore("other expressions") { + positionTest("bad addition", + "SELECT 1 + array(1)", "1 + array") + } + + /** Hive can be very noisy, messing up the output of our tests. */ + private def quietly[A](f: => A): A = { + val origErr = System.err + val origOut = System.out + try { + System.setErr(new PrintStream(new OutputStream { + def write(b: Int) = {} + })) + System.setOut(new PrintStream(new OutputStream { + def write(b: Int) = {} + })) + + f + } finally { + System.setErr(origErr) + System.setOut(origOut) + } + } + + /** + * Creates a test that checks to see if the error thrown when analyzing a given query includes + * the location of the given token in the query string. + * + * @param name the name of the test + * @param query the query to analyze + * @param token a unique token in the string that should be indicated by the exception + */ + def positionTest(name: String, query: String, token: String) = { + def parseTree = + Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") + + test(name) { + val error = intercept[AnalysisException] { + quietly(sql(query)) + } + val (line, expectedLineNum) = query.split("\n").zipWithIndex.collect { + case (l, i) if l.contains(token) => (l, i + 1) + }.headOption.getOrElse(sys.error(s"Invalid test. Token $token not in $query")) + val actualLine = error.line.getOrElse { + fail( + s"line not returned for error '${error.getMessage}' on token $token\n$parseTree" + ) + } + assert(actualLine === expectedLineNum, "wrong line") + + val expectedStart = line.indexOf(token) + val actualStart = error.startPosition.getOrElse { + fail( + s"start not returned for error on token $token\n" + + HiveQl.dumpTree(HiveQl.getAst(query)) + ) + } + assert(expectedStart === actualStart, + s"""Incorrect start position. + |== QUERY == + |$query + | + |== AST == + |$parseTree + | + |Actual: $actualStart, Expected: $expectedStart + |$line + |${" " * actualStart}^ + |0123456789 123456789 1234567890 + | 2 3 + """.stripMargin) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 978825938395f..e8d9eec3d88ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -22,7 +22,7 @@ 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._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.{QueryTest, Row, SQLConf} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -185,7 +185,7 @@ class SQLQuerySuite extends QueryTest { sql("SELECT * FROM test_ctas_1234"), sql("SELECT * FROM nested").collect().toSeq) - intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + intercept[AnalysisException] { sql("CREATE TABLE test_ctas_12345 AS SELECT * from notexists").collect() } } From b4d7c7032d755de42951f92d9535287ef6230b9b Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Mon, 16 Feb 2015 12:34:09 -0800 Subject: [PATCH 135/817] [SQL] Add fetched row count in SparkSQLCLIDriver before this change: ```scala Time taken: 0.619 seconds ``` after this change : ```scala Time taken: 0.619 seconds, Fetched: 4 row(s) ``` Author: OopsOutOfMemory Closes #4604 from OopsOutOfMemory/rowcount and squashes the following commits: 7252dea [OopsOutOfMemory] add fetched row count --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index bb19ac232fcbe..401e97b162dea 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -292,9 +292,13 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } } + var counter = 0 try { while (!out.checkError() && driver.getResults(res)) { - res.foreach(out.println) + res.foreach{ l => + counter += 1 + out.println(l) + } res.clear() } } catch { @@ -311,7 +315,11 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { ret = cret } - console.printInfo(s"Time taken: $timeTaken seconds", null) + var responseMsg = s"Time taken: $timeTaken seconds" + if (counter != 0) { + responseMsg += s", Fetched $counter row(s)" + } + console.printInfo(responseMsg , null) // Destroy the driver to release all the locks. driver.destroy() } else { From 6f54dee66100e5e58f6649158db257eb5009bd6a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 16 Feb 2015 12:48:55 -0800 Subject: [PATCH 136/817] [SPARK-5296] [SQL] Add more filter types for data sources API This PR adds the following filter types for data sources API: - `IsNull` - `IsNotNull` - `Not` - `And` - `Or` The code which converts Catalyst predicate expressions to data sources filters is very similar to filter conversion logics in `ParquetFilters` which converts Catalyst predicates to Parquet filter predicates. In this way we can support nested AND/OR/NOT predicates without changing current `BaseScan` type hierarchy. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4623) Author: Cheng Lian This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #4623 from liancheng/more-fiters and squashes the following commits: 1b296f4 [Cheng Lian] Add more filter types for data sources API --- .../org/apache/spark/sql/SQLContext.scala | 9 ++- .../apache/spark/sql/parquet/newParquet.scala | 5 +- .../sql/sources/DataSourceStrategy.scala | 81 +++++++++++++------ .../apache/spark/sql/sources/filters.scala | 5 ++ .../spark/sql/sources/FilteredScanSuite.scala | 34 +++++++- 5 files changed, 103 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index b42a52ebd2f16..1442250569416 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -28,16 +28,16 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, NoRelation} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.catalyst.{ScalaReflection, expressions} +import org.apache.spark.sql.execution.{Filter, _} import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.json._ -import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation, _} +import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.{Partition, SparkContext} @@ -867,7 +867,8 @@ class SQLContext(@transient val sparkContext: SparkContext) val projectSet = AttributeSet(projectList.flatMap(_.references)) val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) - val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) + val filterCondition = + prunePushedDownFilters(filterPredicates).reduceLeftOption(expressions.And) // Right now we still use a projection even if the only evaluation is applying an alias // to a column. Since this is a no-op, it could be avoided. However, using this diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 9279f5a903f55..9bb34e2df9a26 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} + import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.metadata.CompressionCodecName @@ -42,6 +43,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} +import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ @@ -497,7 +499,8 @@ case class ParquetRelation2( _.references.map(_.name).toSet.subsetOf(partitionColumnNames) } - val rawPredicate = partitionPruningPredicates.reduceOption(And).getOrElse(Literal(true)) + val rawPredicate = + partitionPruningPredicates.reduceOption(expressions.And).getOrElse(Literal(true)) val boundPredicate = InterpretedPredicate(rawPredicate transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) 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 624369afe87b5..a853385fdac68 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql.sources import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, AttributeSet, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, Strategy, execution} +import org.apache.spark.sql.{Row, Strategy, execution, sources} /** * A Strategy for planning scans over data sources defined using the sources API. @@ -88,7 +88,7 @@ private[sql] object DataSourceStrategy extends Strategy { val projectSet = AttributeSet(projectList.flatMap(_.references)) val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) - val filterCondition = filterPredicates.reduceLeftOption(And) + val filterCondition = filterPredicates.reduceLeftOption(expressions.And) val pushedFilters = filterPredicates.map { _ transform { case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes. @@ -118,27 +118,60 @@ private[sql] object DataSourceStrategy extends Strategy { } } - /** Turn Catalyst [[Expression]]s into data source [[Filter]]s. */ - protected[sql] def selectFilters(filters: Seq[Expression]): Seq[Filter] = filters.collect { - case expressions.EqualTo(a: Attribute, expressions.Literal(v, _)) => EqualTo(a.name, v) - case expressions.EqualTo(expressions.Literal(v, _), a: Attribute) => EqualTo(a.name, v) - - case expressions.GreaterThan(a: Attribute, expressions.Literal(v, _)) => GreaterThan(a.name, v) - case expressions.GreaterThan(expressions.Literal(v, _), a: Attribute) => LessThan(a.name, v) - - case expressions.LessThan(a: Attribute, expressions.Literal(v, _)) => LessThan(a.name, v) - case expressions.LessThan(expressions.Literal(v, _), a: Attribute) => GreaterThan(a.name, v) - - case expressions.GreaterThanOrEqual(a: Attribute, expressions.Literal(v, _)) => - GreaterThanOrEqual(a.name, v) - case expressions.GreaterThanOrEqual(expressions.Literal(v, _), a: Attribute) => - LessThanOrEqual(a.name, v) - - case expressions.LessThanOrEqual(a: Attribute, expressions.Literal(v, _)) => - LessThanOrEqual(a.name, v) - case expressions.LessThanOrEqual(expressions.Literal(v, _), a: Attribute) => - GreaterThanOrEqual(a.name, v) + /** + * Selects Catalyst predicate [[Expression]]s which are convertible into data source [[Filter]]s, + * and convert them. + */ + protected[sql] def selectFilters(filters: Seq[Expression]) = { + def translate(predicate: Expression): Option[Filter] = predicate match { + case expressions.EqualTo(a: Attribute, Literal(v, _)) => + Some(sources.EqualTo(a.name, v)) + case expressions.EqualTo(Literal(v, _), a: Attribute) => + Some(sources.EqualTo(a.name, v)) + + case expressions.GreaterThan(a: Attribute, Literal(v, _)) => + Some(sources.GreaterThan(a.name, v)) + case expressions.GreaterThan(Literal(v, _), a: Attribute) => + Some(sources.LessThan(a.name, v)) + + case expressions.LessThan(a: Attribute, Literal(v, _)) => + Some(sources.LessThan(a.name, v)) + case expressions.LessThan(Literal(v, _), a: Attribute) => + Some(sources.GreaterThan(a.name, v)) + + case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, _)) => + Some(sources.GreaterThanOrEqual(a.name, v)) + case expressions.GreaterThanOrEqual(Literal(v, _), a: Attribute) => + Some(sources.LessThanOrEqual(a.name, v)) + + case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) => + Some(sources.LessThanOrEqual(a.name, v)) + case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) => + Some(sources.GreaterThanOrEqual(a.name, v)) + + case expressions.InSet(a: Attribute, set) => + Some(sources.In(a.name, set.toArray)) + + case expressions.IsNull(a: Attribute) => + Some(sources.IsNull(a.name)) + case expressions.IsNotNull(a: Attribute) => + Some(sources.IsNotNull(a.name)) + + case expressions.And(left, right) => + (translate(left) ++ translate(right)).reduceOption(sources.And) + + case expressions.Or(left, right) => + for { + leftFilter <- translate(left) + rightFilter <- translate(right) + } yield sources.Or(leftFilter, rightFilter) + + case expressions.Not(child) => + translate(child).map(sources.Not) + + case _ => None + } - case expressions.InSet(a: Attribute, set) => In(a.name, set.toArray) + filters.flatMap(translate) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index 4a9fefc12b9ad..1e4505e36d2f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -25,3 +25,8 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter case class LessThan(attribute: String, value: Any) extends Filter case class LessThanOrEqual(attribute: String, value: Any) extends Filter case class In(attribute: String, values: Array[Any]) extends Filter +case class IsNull(attribute: String) extends Filter +case class IsNotNull(attribute: String) extends Filter +case class And(left: Filter, right: Filter) extends Filter +case class Or(left: Filter, right: Filter) extends Filter +case class Not(child: Filter) extends Filter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 390538d35a348..41cd35683c196 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -47,16 +47,22 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL FiltersPushed.list = filters - val filterFunctions = filters.collect { + def translateFilter(filter: Filter): Int => Boolean = filter match { case EqualTo("a", v) => (a: Int) => a == v case LessThan("a", v: Int) => (a: Int) => a < v case LessThanOrEqual("a", v: Int) => (a: Int) => a <= v case GreaterThan("a", v: Int) => (a: Int) => a > v case GreaterThanOrEqual("a", v: Int) => (a: Int) => a >= v case In("a", values) => (a: Int) => values.map(_.asInstanceOf[Int]).toSet.contains(a) + case IsNull("a") => (a: Int) => false // Int can't be null + case IsNotNull("a") => (a: Int) => true + case Not(pred) => (a: Int) => !translateFilter(pred)(a) + case And(left, right) => (a: Int) => translateFilter(left)(a) && translateFilter(right)(a) + case Or(left, right) => (a: Int) => translateFilter(left)(a) || translateFilter(right)(a) + case _ => (a: Int) => true } - def eval(a: Int) = !filterFunctions.map(_(a)).contains(false) + def eval(a: Int) = !filters.map(translateFilter(_)(a)).contains(false) sqlContext.sparkContext.parallelize(from to to).filter(eval).map(i => Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) @@ -136,6 +142,26 @@ class FilteredScanSuite extends DataSourceTest { "SELECT * FROM oneToTenFiltered WHERE b = 2", Seq(1).map(i => Row(i, i * 2)).toSeq) + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE a IS NULL", + Seq.empty[Row]) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE a IS NOT NULL", + (1 to 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE a < 5 AND a > 1", + (2 to 4).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", + Seq(1, 2, 9, 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", + (6 to 10).map(i => Row(i, i * 2)).toSeq) + testPushDown("SELECT * FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT a FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT b FROM oneToTenFiltered WHERE A = 1", 1) @@ -162,6 +188,10 @@ class FilteredScanSuite extends DataSourceTest { testPushDown("SELECT * FROM oneToTenFiltered WHERE a = 20", 0) testPushDown("SELECT * FROM oneToTenFiltered WHERE b = 1", 10) + testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 5 AND a > 1", 3) + testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", 4) + testPushDown("SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", 5) + def testPushDown(sqlString: String, expectedCount: Int): Unit = { test(s"PushDown Returns $expectedCount: $sqlString") { val queryExecution = sql(sqlString).queryExecution From c51ab37faddf4ede23243058dfb388e74a192552 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 16 Feb 2015 12:52:05 -0800 Subject: [PATCH 137/817] [SPARK-5833] [SQL] Adds REFRESH TABLE command Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` command to refresh (possibly cached) metadata in external data sources tables. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4624) Author: Cheng Lian Closes #4624 from liancheng/refresh-table and squashes the following commits: 8d1aa4c [Cheng Lian] Adds REFRESH TABLE command --- .../spark/sql/catalyst/analysis/Catalog.scala | 10 ++++ .../org/apache/spark/sql/sources/ddl.scala | 52 +++++++++++-------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- 4 files changed, 42 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index bf97215ee67da..9e6e2912e0622 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -46,6 +46,8 @@ trait Catalog { */ def getTables(databaseName: Option[String]): Seq[(String, Boolean)] + def refreshTable(databaseName: String, tableName: String): Unit + def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit def unregisterTable(tableIdentifier: Seq[String]): Unit @@ -119,6 +121,10 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { case (name, _) => (name, true) }.toSeq } + + override def refreshTable(databaseName: String, tableName: String): Unit = { + throw new UnsupportedOperationException + } } /** @@ -224,4 +230,8 @@ object EmptyCatalog extends Catalog { } override def unregisterAllTables(): Unit = {} + + override def refreshTable(databaseName: String, tableName: String): Unit = { + throw new UnsupportedOperationException + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 8cac9c0fdf7fa..1b5e8c280e2fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference} import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -66,6 +66,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected val EXTENDED = Keyword("EXTENDED") protected val AS = Keyword("AS") protected val COMMENT = Keyword("COMMENT") + protected val REFRESH = Keyword("REFRESH") // Data types. protected val STRING = Keyword("STRING") @@ -85,7 +86,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected val MAP = Keyword("MAP") protected val STRUCT = Keyword("STRUCT") - protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable + protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable | refreshTable protected def start: Parser[LogicalPlan] = ddl @@ -104,9 +105,8 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { * AS SELECT ... */ protected lazy val createTable: Parser[LogicalPlan] = - ( - (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident - ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ { + (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident ~ + tableCols.? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ { case temp ~ allowExisting ~ tableName ~ columns ~ provider ~ opts ~ query => if (temp.isDefined && allowExisting.isDefined) { throw new DDLException( @@ -145,8 +145,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { allowExisting.isDefined, managedIfNoPath = false) } - } - ) + } protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")" @@ -166,6 +165,12 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { DescribeCommand(UnresolvedRelation(tblIdentifier, None), e.isDefined) } + protected lazy val refreshTable: Parser[LogicalPlan] = + REFRESH ~> TABLE ~> (ident <~ ".").? ~ ident ^^ { + case maybeDatabaseName ~ tableName => + RefreshTable(maybeDatabaseName.getOrElse("default"), tableName) + } + protected lazy val options: Parser[Map[String, String]] = "(" ~> repsep(pair, ",") <~ ")" ^^ { case s: Seq[(String, String)] => s.toMap } @@ -177,10 +182,10 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { ident ~ dataType ~ (COMMENT ~> stringLit).? ^^ { case columnName ~ typ ~ cm => val meta = cm match { case Some(comment) => - new MetadataBuilder().putString(COMMENT.str.toLowerCase(), comment).build() + new MetadataBuilder().putString(COMMENT.str.toLowerCase, comment).build() case None => Metadata.empty } - StructField(columnName, typ, true, meta) + StructField(columnName, typ, nullable = true, meta) } protected lazy val primitiveType: Parser[DataType] = @@ -318,24 +323,18 @@ private[sql] case class DescribeCommand( isExtended: Boolean) extends Command { override val output = Seq( // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false, + AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("data_type", StringType, nullable = false, + AttributeReference("data_type", StringType, nullable = false, new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference("comment", StringType, nullable = false, + AttributeReference("comment", StringType, nullable = false, new MetadataBuilder().putString("comment", "comment of the column").build())()) } /** * Used to represent the operation of create table using a data source. - * @param tableName - * @param userSpecifiedSchema - * @param provider - * @param temporary - * @param options * @param allowExisting If it is true, we will do nothing when the table already exists. - * If it is false, an exception will be thrown - * @param managedIfNoPath + * If it is false, an exception will be thrown */ private[sql] case class CreateTableUsing( tableName: String, @@ -362,7 +361,7 @@ private[sql] case class CreateTableUsingAsLogicalPlan( options: Map[String, String], query: LogicalPlan) extends Command -private [sql] case class CreateTempTableUsing( +private[sql] case class CreateTempTableUsing( tableName: String, userSpecifiedSchema: Option[StructType], provider: String, @@ -376,7 +375,7 @@ private [sql] case class CreateTempTableUsing( } } -private [sql] case class CreateTempTableUsingAsSelect( +private[sql] case class CreateTempTableUsingAsSelect( tableName: String, provider: String, mode: SaveMode, @@ -393,6 +392,15 @@ private [sql] case class CreateTempTableUsingAsSelect( } } +private[sql] case class RefreshTable(databaseName: String, tableName: String) + extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.catalog.refreshTable(databaseName, tableName) + Seq.empty[Row] + } +} + /** * Builds a map in which keys are case insensitive */ @@ -408,7 +416,7 @@ protected[sql] class CaseInsensitiveMap(map: Map[String, String]) extends Map[St override def iterator: Iterator[(String, String)] = baseMap.iterator - override def -(key: String): Map[String, String] = baseMap - key.toLowerCase() + override def -(key: String): Map[String, String] = baseMap - key.toLowerCase } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 12f86a04a37af..580c5706dde67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -91,7 +91,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader) } - def refreshTable(databaseName: String, tableName: String): Unit = { + override def refreshTable(databaseName: String, tableName: String): Unit = { cachedDataSourceTables.refresh(QualifiedTableName(databaseName, tableName).toLowerCase) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index addf887ab9162..375aae5d51915 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -177,7 +177,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("SELECT * FROM jsonTable"), Row("a1", "b1")) - refreshTable("jsonTable") + sql("REFRESH TABLE jsonTable") // Check that the refresh worked checkAnswer( From bb05982dd25e008fb01684dff1f95d03e7271721 Mon Sep 17 00:00:00 2001 From: Matt Whelan Date: Mon, 16 Feb 2015 22:54:32 +0000 Subject: [PATCH 138/817] SPARK-5841: remove DiskBlockManager shutdown hook on stop After a call to stop, the shutdown hook is redundant, and causes a memory leak. Author: Matt Whelan Closes #4627 from MattWhelan/SPARK-5841 and squashes the following commits: d5f5c7f [Matt Whelan] SPARK-5841: remove DiskBlockManager shutdown hook on stop --- .../org/apache/spark/storage/DiskBlockManager.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 53eaedacbf291..ae9df8cbe9821 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -49,7 +49,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - addShutdownHook() + private val shutdownHook = addShutdownHook() /** Looks up a file by hashing it into one of our local subdirectories. */ // This method should be kept in sync with @@ -134,17 +134,22 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } } - private def addShutdownHook() { - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { + private def addShutdownHook(): Thread = { + val shutdownHook = new Thread("delete Spark local dirs") { override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") DiskBlockManager.this.stop() } - }) + } + Runtime.getRuntime.addShutdownHook(shutdownHook) + shutdownHook } /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { + // Remove the shutdown hook. It causes memory leaks if we leave it around. + Runtime.getRuntime.removeShutdownHook(shutdownHook) + // Only perform cleanup if an external service is not serving our shuffle files. if (!blockManager.externalShuffleServiceEnabled || blockManager.blockManagerId.isDriver) { localDirs.foreach { localDir => From c01c4ebcfe5c1a4a56a8987af596eca090c2cc2f Mon Sep 17 00:00:00 2001 From: Matt Whelan Date: Mon, 16 Feb 2015 23:05:34 +0000 Subject: [PATCH 139/817] SPARK-5357: Update commons-codec version to 1.10 (current) Resolves https://issues.apache.org/jira/browse/SPARK-5357 In commons-codec 1.5, Base64 instances are not thread safe. That was only true from 1.4-1.6. Author: Matt Whelan Closes #4153 from MattWhelan/depsUpdate and squashes the following commits: b4a91f4 [Matt Whelan] SPARK-5357: Update commons-codec version to 1.10 (current) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6810d71be4230..bb355bf735bee 100644 --- a/pom.xml +++ b/pom.xml @@ -404,7 +404,7 @@ commons-codec commons-codec - 1.5 + 1.10 org.apache.commons From 0cfda8461f173428f955aa9a7140b1356beea400 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 16 Feb 2015 15:25:11 -0800 Subject: [PATCH 140/817] [SPARK-2313] Use socket to communicate GatewayServer port back to Python driver This patch changes PySpark so that the GatewayServer's port is communicated back to the Python process that launches it over a local socket instead of a pipe. The old pipe-based approach was brittle and could fail if `spark-submit` printed unexpected to stdout. To accomplish this, I wrote a custom `PythonGatewayServer.main()` function to use in place of Py4J's `GatewayServer.main()`. Closes #3424. Author: Josh Rosen Closes #4603 from JoshRosen/SPARK-2313 and squashes the following commits: 6a7740b [Josh Rosen] Remove EchoOutputThread since it's no longer needed 0db501f [Josh Rosen] Use select() so that we don't block if GatewayServer dies. 9bdb4b6 [Josh Rosen] Handle case where getListeningPort returns -1 3fb7ed1 [Josh Rosen] Remove stdout=PIPE 2458934 [Josh Rosen] Use underscore to mark env var. as private d12c95d [Josh Rosen] Use Logging and Utils.tryOrExit() e5f9730 [Josh Rosen] Wrap everything in a giant try-block 2f70689 [Josh Rosen] Use stdin PIPE to share fate with driver 8bf956e [Josh Rosen] Initial cut at passing Py4J gateway port back to driver via socket --- .../api/python/PythonGatewayServer.scala | 64 +++++++++++++++++ .../org/apache/spark/deploy/SparkSubmit.scala | 4 +- python/pyspark/java_gateway.py | 72 +++++++++---------- 3 files changed, 97 insertions(+), 43 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala new file mode 100644 index 0000000000000..164e95081583f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import java.io.DataOutputStream +import java.net.Socket + +import py4j.GatewayServer + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * Process that starts a Py4J GatewayServer on an ephemeral port and communicates the bound port + * back to its caller via a callback port specified by the caller. + * + * This process is launched (via SparkSubmit) by the PySpark driver (see java_gateway.py). + */ +private[spark] object PythonGatewayServer extends Logging { + def main(args: Array[String]): Unit = Utils.tryOrExit { + // Start a GatewayServer on an ephemeral port + val gatewayServer: GatewayServer = new GatewayServer(null, 0) + gatewayServer.start() + val boundPort: Int = gatewayServer.getListeningPort + if (boundPort == -1) { + logError("GatewayServer failed to bind; exiting") + System.exit(1) + } else { + logDebug(s"Started PythonGatewayServer on port $boundPort") + } + + // Communicate the bound port back to the caller via the caller-specified callback port + val callbackHost = sys.env("_PYSPARK_DRIVER_CALLBACK_HOST") + val callbackPort = sys.env("_PYSPARK_DRIVER_CALLBACK_PORT").toInt + logDebug(s"Communicating GatewayServer port to Python driver at $callbackHost:$callbackPort") + val callbackSocket = new Socket(callbackHost, callbackPort) + val dos = new DataOutputStream(callbackSocket.getOutputStream) + dos.writeInt(boundPort) + dos.close() + callbackSocket.close() + + // Exit on EOF or broken pipe to ensure that this process dies when the Python driver dies: + while (System.in.read() != -1) { + // Do nothing + } + logDebug("Exiting due to broken pipe from Python driver") + System.exit(0) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 54399e99c98f0..012a89a31b046 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -39,7 +39,6 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver} import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ -import org.apache.spark.executor._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} /** @@ -284,8 +283,7 @@ object SparkSubmit { // If we're running a python app, set the main class to our specific python runner if (args.isPython && deployMode == CLIENT) { if (args.primaryResource == PYSPARK_SHELL) { - args.mainClass = "py4j.GatewayServer" - args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") + args.mainClass = "org.apache.spark.api.python.PythonGatewayServer" } else { // If a python file is provided, add it to the child arguments and list of files to deploy. // Usage: PythonAppRunner
    [app arguments] diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index a0a028446d5fd..936857e75c7e9 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -17,19 +17,20 @@ import atexit import os -import sys +import select import signal import shlex +import socket import platform from subprocess import Popen, PIPE -from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient +from pyspark.serializers import read_int + def launch_gateway(): SPARK_HOME = os.environ["SPARK_HOME"] - gateway_port = -1 if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: @@ -41,36 +42,42 @@ def launch_gateway(): submit_args = submit_args if submit_args is not None else "" submit_args = shlex.split(submit_args) command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"] + + # Start a socket that will be used by PythonGatewayServer to communicate its port to us + callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + callback_socket.bind(('127.0.0.1', 0)) + callback_socket.listen(1) + callback_host, callback_port = callback_socket.getsockname() + env = dict(os.environ) + env['_PYSPARK_DRIVER_CALLBACK_HOST'] = callback_host + env['_PYSPARK_DRIVER_CALLBACK_PORT'] = str(callback_port) + + # Launch the Java gateway. + # We open a pipe to stdin so that the Java gateway can die when the pipe is broken if not on_windows: # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): signal.signal(signal.SIGINT, signal.SIG_IGN) - env = dict(os.environ) env["IS_SUBPROCESS"] = "1" # tell JVM to exit after python exits - proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func, env=env) + proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env) else: # preexec_fn not supported on Windows - proc = Popen(command, stdout=PIPE, stdin=PIPE) + proc = Popen(command, stdin=PIPE, env=env) - try: - # Determine which ephemeral port the server started on: - gateway_port = proc.stdout.readline() - gateway_port = int(gateway_port) - except ValueError: - # Grab the remaining lines of stdout - (stdout, _) = proc.communicate() - exit_code = proc.poll() - error_msg = "Launching GatewayServer failed" - error_msg += " with exit code %d!\n" % exit_code if exit_code else "!\n" - error_msg += "Warning: Expected GatewayServer to output a port, but found " - if gateway_port == "" and stdout == "": - error_msg += "no output.\n" - else: - error_msg += "the following:\n\n" - error_msg += "--------------------------------------------------------------\n" - error_msg += gateway_port + stdout - error_msg += "--------------------------------------------------------------\n" - raise Exception(error_msg) + gateway_port = None + # We use select() here in order to avoid blocking indefinitely if the subprocess dies + # before connecting + while gateway_port is None and proc.poll() is None: + timeout = 1 # (seconds) + readable, _, _ = select.select([callback_socket], [], [], timeout) + if callback_socket in readable: + gateway_connection = callback_socket.accept()[0] + # Determine which ephemeral port the server started on: + gateway_port = read_int(gateway_connection.makefile()) + gateway_connection.close() + callback_socket.close() + if gateway_port is None: + raise Exception("Java gateway process exited before sending the driver its port number") # In Windows, ensure the Java child processes do not linger after Python has exited. # In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when @@ -88,21 +95,6 @@ def killChild(): Popen(["cmd", "/c", "taskkill", "/f", "/t", "/pid", str(proc.pid)]) atexit.register(killChild) - # Create a thread to echo output from the GatewayServer, which is required - # for Java log output to show up: - class EchoOutputThread(Thread): - - def __init__(self, stream): - Thread.__init__(self) - self.daemon = True - self.stream = stream - - def run(self): - while True: - line = self.stream.readline() - sys.stderr.write(line) - EchoOutputThread(proc.stdout).start() - # Connect to the gateway gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False) From 04b401da811e62a4365cf39ea95cadd0e737001c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 16 Feb 2015 15:43:56 -0800 Subject: [PATCH 141/817] HOTFIX: Break in Jekyll build from #4589 That patch had a line break in the middle of a {{ }} expression, which is not allowed. --- docs/streaming-programming-guide.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 997de9511ca3e..815c98713b738 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -908,8 +908,7 @@ JavaPairDStream runningCounts = pairs.updateStateByKey(updateFu The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Java code, take a look at the example -[JavaStatefulNetworkWordCount.java]({{site -.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming +[JavaStatefulNetworkWordCount.java]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming /JavaStatefulNetworkWordCount.java).
  • From 5b6cd65cd611b1a46a7d5eb33139c6224b96264e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Feb 2015 15:51:59 -0800 Subject: [PATCH 142/817] [SPARK-5746][SQL] Check invalid cases for the write path of data source API JIRA: https://issues.apache.org/jira/browse/SPARK-5746 liancheng marmbrus Author: Yin Huai Closes #4617 from yhuai/insertOverwrite and squashes the following commits: 8e3019d [Yin Huai] Fix compilation error. 499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite e76e85a [Yin Huai] Address comments. ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite f30bdad [Yin Huai] Use toDF. 99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite 6b7545c [Yin Huai] Add a pre write check to the data source API. a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS statements. --- .../sql/catalyst/analysis/Analyzer.scala | 13 +++- .../org/apache/spark/sql/DataFrameImpl.scala | 8 ++- .../org/apache/spark/sql/SQLContext.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 10 +-- .../sql/sources/DataSourceStrategy.scala | 5 +- .../org/apache/spark/sql/sources/ddl.scala | 29 ++++---- .../org/apache/spark/sql/sources/rules.scala | 72 +++++++++++++++++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 13 ++-- .../sources/CreateTableAsSelectSuite.scala | 28 ++++++++ .../spark/sql/sources/DataSourceTest.scala | 3 +- ...nsertIntoSuite.scala => InsertSuite.scala} | 46 +++++++++++- .../apache/spark/sql/hive/HiveContext.scala | 12 ++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 8 +-- 14 files changed, 197 insertions(+), 57 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/sources/{InsertIntoSuite.scala => InsertSuite.scala} (79%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index aa4320bd582cb..fc37b8cde0806 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -50,7 +50,13 @@ class Analyzer(catalog: Catalog, /** * Override to provide additional rules for the "Resolution" batch. */ - val extendedRules: Seq[Rule[LogicalPlan]] = Nil + val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil + + /** + * Override to provide additional rules for the "Check Analysis" batch. + * These rules will be evaluated after our built-in check rules. + */ + val extendedCheckRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( Batch("Resolution", fixedPoint, @@ -64,9 +70,10 @@ class Analyzer(catalog: Catalog, UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: typeCoercionRules ++ - extendedRules : _*), + extendedResolutionRules : _*), Batch("Check Analysis", Once, - CheckResolution), + CheckResolution +: + extendedCheckRules: _*), Batch("Remove SubQueries", fixedPoint, EliminateSubQueries) ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 500e3c90fdbc1..3c1cf8d5e3851 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -67,7 +67,11 @@ private[sql] class DataFrameImpl protected[sql]( @transient protected[sql] override val logicalPlan: LogicalPlan = queryExecution.logical match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile => + case _: Command | + _: InsertIntoTable | + _: CreateTableAsSelect[_] | + _: CreateTableUsingAsSelect | + _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => queryExecution.logical @@ -386,7 +390,7 @@ private[sql] class DataFrameImpl protected[sql]( mode: SaveMode, options: Map[String, String]): Unit = { val cmd = - CreateTableUsingAsLogicalPlan( + CreateTableUsingAsSelect( tableName, source, temporary = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 1442250569416..d08c2d1cfe030 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -92,7 +92,8 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = true) { - override val extendedRules = + override val extendedResolutionRules = + sources.PreWriteCheck(catalog) :: sources.PreInsertCastAndRename :: Nil } @@ -101,7 +102,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer @transient - protected[sql] val ddlParser = new DDLParser + protected[sql] val ddlParser = new DDLParser(sqlParser.apply(_)) @transient protected[sql] val sqlParser = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index e915e0e6a0ec1..5281c7502556a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -319,18 +319,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { sys.error("allowExisting should be set to false when creating a temporary table.") case CreateTableUsingAsSelect(tableName, provider, true, mode, opts, query) => - val logicalPlan = sqlContext.parseSql(query) - val cmd = - CreateTempTableUsingAsSelect(tableName, provider, mode, opts, logicalPlan) - ExecutedCommand(cmd) :: Nil - case c: CreateTableUsingAsSelect if !c.temporary => - sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") - - case CreateTableUsingAsLogicalPlan(tableName, provider, true, mode, opts, query) => val cmd = CreateTempTableUsingAsSelect(tableName, provider, mode, opts, query) ExecutedCommand(cmd) :: Nil - case c: CreateTableUsingAsLogicalPlan if !c.temporary => + case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") case LogicalDescribeCommand(table, isExtended) => 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 a853385fdac68..67f3507c61ab6 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 @@ -55,10 +55,7 @@ private[sql] object DataSourceStrategy extends Strategy { execution.PhysicalRDD(l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => - if (partition.nonEmpty) { - sys.error(s"Insert into a partition is not allowed because $l is not partitioned.") - } + l @ LogicalRelation(t: InsertableRelation), part, query, overwrite) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 1b5e8c280e2fe..dd8b3d211be64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.expressions.{Row, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -32,7 +32,8 @@ import org.apache.spark.util.Utils /** * A parser for foreign DDL commands. */ -private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { +private[sql] class DDLParser( + parseQuery: String => LogicalPlan) extends AbstractSparkSQLParser with Logging { def apply(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = { try { @@ -105,6 +106,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { * AS SELECT ... */ protected lazy val createTable: Parser[LogicalPlan] = + // TODO: Support database.table. (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident ~ tableCols.? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ { case temp ~ allowExisting ~ tableName ~ columns ~ provider ~ opts ~ query => @@ -128,12 +130,13 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { SaveMode.ErrorIfExists } + val queryPlan = parseQuery(query.get) CreateTableUsingAsSelect(tableName, provider, temp.isDefined, mode, options, - query.get) + queryPlan) } else { val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) CreateTableUsing( @@ -345,21 +348,23 @@ private[sql] case class CreateTableUsing( allowExisting: Boolean, managedIfNoPath: Boolean) extends Command +/** + * A node used to support CTAS statements and saveAsTable for the data source API. + * This node is a [[UnaryNode]] instead of a [[Command]] because we want the analyzer + * can analyze the logical plan that will be used to populate the table. + * So, [[PreWriteCheck]] can detect cases that are not allowed. + */ private[sql] case class CreateTableUsingAsSelect( tableName: String, provider: String, temporary: Boolean, mode: SaveMode, options: Map[String, String], - query: String) extends Command - -private[sql] case class CreateTableUsingAsLogicalPlan( - tableName: String, - provider: String, - temporary: Boolean, - mode: SaveMode, - options: Map[String, String], - query: LogicalPlan) extends Command + child: LogicalPlan) extends UnaryNode { + override def output = Seq.empty[Attribute] + // TODO: Override resolved after we support databaseName. + // override lazy val resolved = databaseName != None && childrenResolved +} private[sql] case class CreateTempTableUsing( tableName: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 4ed22d363da5b..36a9c0bdc41e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.sources +import org.apache.spark.sql.{SaveMode, AnalysisException} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, Catalog} import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Alias} +import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types.DataType @@ -26,11 +29,9 @@ import org.apache.spark.sql.types.DataType * A rule to do pre-insert data type casting and field renaming. Before we insert into * an [[InsertableRelation]], we will use this rule to make sure that * the columns to be inserted have the correct data type and fields have the correct names. - * @param resolver The resolver used by the Analyzer. */ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = { - plan.transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p @@ -46,7 +47,6 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { } castAndRenameChildOutput(i, l.output, child) } - } } /** If necessary, cast data types and rename fields to the expected types and names. */ @@ -74,3 +74,67 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { } } } + +/** + * A rule to do various checks before inserting into or writing to a data source table. + */ +private[sql] case class PreWriteCheck(catalog: Catalog) extends Rule[LogicalPlan] { + def failAnalysis(msg: String) = { throw new AnalysisException(msg) } + + def apply(plan: LogicalPlan): LogicalPlan = { + plan.foreach { + case i @ logical.InsertIntoTable( + l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => + // Right now, we do not support insert into a data source table with partition specs. + if (partition.nonEmpty) { + failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") + } else { + // Get all input data source relations of the query. + val srcRelations = query.collect { + case LogicalRelation(src: BaseRelation) => src + } + if (srcRelations.exists(src => src == t)) { + failAnalysis( + "Cannot insert overwrite into table that is also being read from.") + } else { + // OK + } + } + + case i @ logical.InsertIntoTable( + l: LogicalRelation, partition, query, overwrite) if !l.isInstanceOf[InsertableRelation] => + // The relation in l is not an InsertableRelation. + failAnalysis(s"$l does not allow insertion.") + + case CreateTableUsingAsSelect(tableName, _, _, SaveMode.Overwrite, _, query) => + // When the SaveMode is Overwrite, we need to check if the table is an input table of + // the query. If so, we will throw an AnalysisException to let users know it is not allowed. + if (catalog.tableExists(Seq(tableName))) { + // Need to remove SubQuery operator. + EliminateSubQueries(catalog.lookupRelation(Seq(tableName))) match { + // Only do the check if the table is a data source table + // (the relation is a BaseRelation). + case l @ LogicalRelation(dest: BaseRelation) => + // Get all input data source relations of the query. + val srcRelations = query.collect { + case LogicalRelation(src: BaseRelation) => src + } + if (srcRelations.exists(src => src == dest)) { + failAnalysis( + s"Cannot overwrite table $tableName that is also being read from.") + } else { + // OK + } + + case _ => // OK + } + } else { + // OK + } + + case _ => // OK + } + + plan + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d0665450cd766..9318c15520a10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -38,21 +38,22 @@ class ParquetQuerySuiteBase extends QueryTest with ParquetTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) + createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") withParquetTable(data, "t") { - sql("INSERT INTO TABLE t SELECT * FROM t") + sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) } + catalog.unregisterTable(Seq("tmp")) } - // This test case will trigger the NPE mentioned in - // https://issues.apache.org/jira/browse/PARQUET-151. - // Update: This also triggers SPARK-5746, should re enable it when we get both fixed. - ignore("overwriting") { + test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) + createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") withParquetTable(data, "t") { - sql("INSERT OVERWRITE TABLE t SELECT * FROM t") + sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(table("t"), data.map(Row.fromTuple)) } + catalog.unregisterTable(Seq("tmp")) } test("self-join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 29caed9337ff6..60355414a40fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import java.io.File +import org.apache.spark.sql.AnalysisException import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.util @@ -157,4 +158,31 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { """.stripMargin) } } + + test("it is not allowed to write to a table while querying it.") { + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + val message = intercept[AnalysisException] { + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jsonTable + """.stripMargin) + }.getMessage + assert( + message.contains("Cannot overwrite table "), + "Writing to a table while querying it should not be allowed.") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 53f5f7426e9e6..0ec6881d7afe6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -29,7 +29,8 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { @transient override protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = false) { - override val extendedRules = + override val extendedResolutionRules = + PreWriteCheck(catalog) :: PreInsertCastAndRename :: Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala similarity index 79% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/InsertIntoSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 36e504e759152..5682e5a2bcea9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -21,11 +21,11 @@ import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.Row +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.util import org.apache.spark.util.Utils -class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { +class InsertSuite extends DataSourceTest with BeforeAndAfterAll { import caseInsensisitiveContext._ @@ -129,6 +129,18 @@ class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { } } + test("it is not allowed to write to a table while querying it.") { + val message = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jsonTable + """.stripMargin) + }.getMessage + assert( + message.contains("Cannot insert overwrite into table that is also being read from."), + "INSERT OVERWRITE to a table while querying it should not be allowed.") + } + test("Caching") { // Cached Query Execution cacheTable("jsonTable") @@ -173,4 +185,34 @@ class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { uncacheTable("jsonTable") assertCached(sql("SELECT * FROM jsonTable"), 0) } + + test("it's not allowed to insert into a relation that is not an InsertableRelation") { + sql( + """ + |CREATE TEMPORARY TABLE oneToTen + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM oneToTen"), + (1 to 10).map(Row(_)).toSeq + ) + + val message = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE TABLE oneToTen SELECT a FROM jt + """.stripMargin) + }.getMessage + assert( + message.contains("does not allow insertion."), + "It is not allowed to insert into a table that is not an InsertableRelation." + ) + + dropTempTable("oneToTen") + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 87b380f950979..6c55bc6be17f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateSubQueries, Ov import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, QueryExecutionException, SetCommand} import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.sources.DataSourceStrategy +import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ /** @@ -64,14 +64,17 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) + @transient + protected[sql] val ddlParserWithHiveQL = new DDLParser(HiveQl.parseSql(_)) + override def sql(sqlText: String): DataFrame = { val substituted = new VariableSubstitution().substitute(hiveconf, sqlText) // TODO: Create a framework for registering parsers instead of just hardcoding if statements. if (conf.dialect == "sql") { super.sql(substituted) } else if (conf.dialect == "hiveql") { - DataFrame(this, - ddlParser(sqlText, exceptionOnError = false).getOrElse(HiveQl.parseSql(substituted))) + val ddlPlan = ddlParserWithHiveQL(sqlText, exceptionOnError = false) + DataFrame(this, ddlPlan.getOrElse(HiveQl.parseSql(substituted))) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") } @@ -241,12 +244,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient override protected[sql] lazy val analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = false) { - override val extendedRules = + override val extendedResolutionRules = catalog.ParquetConversions :: catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: ResolveUdtfsAlias :: + sources.PreWriteCheck(catalog) :: sources.PreInsertCastAndRename :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 580c5706dde67..72211fe2e46c6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -663,7 +663,7 @@ private[hive] case class MetastoreRelation } object HiveMetastoreTypes { - protected val ddlParser = new DDLParser + protected val ddlParser = new DDLParser(HiveQl.parseSql(_)) def toDataType(metastoreType: String): DataType = synchronized { ddlParser.parseType(metastoreType) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 965d159656d80..d2c39ab621713 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeComman import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{CreateTableUsingAsLogicalPlan, CreateTableUsingAsSelect, CreateTableUsing} +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, CreateTableUsing} import org.apache.spark.sql.types.StringType @@ -227,12 +227,6 @@ private[hive] trait HiveStrategies { tableName, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath)) :: Nil case CreateTableUsingAsSelect(tableName, provider, false, mode, opts, query) => - val logicalPlan = hiveContext.parseSql(query) - val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, logicalPlan) - ExecutedCommand(cmd) :: Nil - - case CreateTableUsingAsLogicalPlan(tableName, provider, false, mode, opts, query) => val cmd = CreateMetastoreDataSourceAsSelect(tableName, provider, mode, opts, query) ExecutedCommand(cmd) :: Nil From f3ff1eb2985ff3e1567645b898f6b42e4b01f237 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Feb 2015 15:54:01 -0800 Subject: [PATCH 143/817] [SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and aliases of data source tables. JIRA: https://issues.apache.org/jira/browse/SPARK-5839 Author: Yin Huai Closes #4626 from yhuai/SPARK-5839 and squashes the following commits: f779d85 [Yin Huai] Use subqeury to wrap replaced ParquetRelation. 2695f13 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-5839 f1ba6ca [Yin Huai] Address comment. 2c7fa08 [Yin Huai] Use Subqueries to wrap a data source table. --- .../spark/sql/parquet/ParquetQuerySuite.scala | 5 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++++++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 34 +++++++++++++++++++ 3 files changed, 53 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 9318c15520a10..8b4d05ec547c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -30,9 +30,10 @@ import org.apache.spark.sql.test.TestSQLContext._ class ParquetQuerySuiteBase extends QueryTest with ParquetTest { val sqlContext = TestSQLContext - test("simple projection") { + test("simple select queries") { withParquetTable((0 until 10).map(i => (i, i.toString)), "t") { - checkAnswer(sql("SELECT _1 FROM t"), (0 until 10).map(Row.apply(_))) + checkAnswer(sql("SELECT _1 FROM t where t._1 > 5"), (6 until 10).map(Row.apply(_))) + checkAnswer(sql("SELECT _1 FROM t as tmp where tmp._1 < 5"), (0 until 5).map(Row.apply(_))) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 72211fe2e46c6..87bc9fe4fe9ad 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -160,7 +160,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } if (table.getProperty("spark.sql.sources.provider") != null) { - cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) + val dataSourceTable = + cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) + // Then, if alias is specified, wrap the table with a Subquery using the alias. + // Othersie, wrap the table with a Subquery using the table name. + val withAlias = + alias.map(a => Subquery(a, dataSourceTable)).getOrElse( + Subquery(tableIdent.last, dataSourceTable)) + + withAlias } else if (table.isView) { // if the unresolved relation is from hive view // parse the text into logic node. @@ -433,7 +441,13 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val attributedRewrites = AttributeMap(relation.output.zip(parquetRelation.output)) lastPlan.transformUp { - case r: MetastoreRelation if r == relation => parquetRelation + case r: MetastoreRelation if r == relation => { + val withAlias = + r.alias.map(a => Subquery(a, parquetRelation)).getOrElse( + Subquery(r.tableName, parquetRelation)) + + withAlias + } case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 375aae5d51915..0263e3bb56617 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -401,6 +401,40 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("DROP TABLE jsonTable").collect().foreach(println) } + test("SPARK-5839 HiveMetastoreCatalog does not recognize table aliases of data source tables.") { + val originalDefaultSource = conf.defaultDataSourceName + + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + val df = jsonRDD(rdd) + + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + // Save the df as a managed table (by not specifiying the path). + df.saveAsTable("savedJsonTable") + + checkAnswer( + sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), + (1 to 4).map(i => Row(i, s"str${i}"))) + + checkAnswer( + sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), + (6 to 10).map(i => Row(i, s"str${i}"))) + + invalidateTable("savedJsonTable") + + checkAnswer( + sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), + (1 to 4).map(i => Row(i, s"str${i}"))) + + checkAnswer( + sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), + (6 to 10).map(i => Row(i, s"str${i}"))) + + // Drop table will also delete the data. + sql("DROP TABLE savedJsonTable") + + conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) + } + test("save table") { val originalDefaultSource = conf.defaultDataSourceName From cb6c48c874af2bd78ee73c1dc8a44fd28ecc0991 Mon Sep 17 00:00:00 2001 From: kai Date: Mon, 16 Feb 2015 15:58:05 -0800 Subject: [PATCH 144/817] [SQL] Optimize arithmetic and predicate operators Existing implementation of arithmetic operators and BinaryComparison operators have redundant type checking codes, e.g.: Expression.n2 is used by Add/Subtract/Multiply. (1) n2 always checks left.dataType == right.dataType. However, this checking should be done once when we resolve expression types; (2) n2 requires dataType is a NumericType. This can be done once. This PR optimizes arithmetic and predicate operators by removing such redundant type-checking codes. Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines shows that this PR can reduce the query time by 5.5% to 11%. The benchmark queries follow the template below, where OP is plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor. SELECT l_returnflag, l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP cnt2), ...., SUM(l_quantity OP cnt700) FROM ( SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 700 AS cnt700 FROM lineitem WHERE l_shipdate <= '1998-09-01' ) GROUP BY l_returnflag, l_linestatus; Author: kai Closes #4472 from kai-zeng/arithmetic-optimize and squashes the following commits: fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into arithmetic-optimize 4b3a1bb [kai] chmod a-x 5a41e49 [kai] chmod a-x Expression.scala cb37c94 [kai] rebase onto spark master 7f6e968 [kai] chmod 100755 -> 100644 6cddb46 [kai] format 7490dbc [kai] fix unresolved-expression exception for EqualTo 9c40bc0 [kai] fix bitwisenot 3cbd363 [kai] clean up test code ca47801 [kai] override evalInternal for bitwise ops 8fa84a1 [kai] add bitwise or and xor 6892fc4 [kai] revert override evalInternal f8eba24 [kai] override evalInternal 31ccdd4 [kai] rewrite all bitwise op and remove evalInternal 86297e2 [kai] generalized cb92ae1 [kai] bitwise-and: override eval 97a7d6c [kai] bitwise-and: override evalInternal using and func 0906c39 [kai] add bitwise test 62abbbc [kai] clean up predicate and arithmetic b34d58d [kai] add caching and benmark option 12c5b32 [kai] override eval 1cd7571 [kai] fix sqrt and maxof 03fd0c3 [kai] fix predicate 16fd84c [kai] optimize + - * / % -(unary) abs < > <= >= fd95823 [kai] remove unnecessary type checking 24d062f [kai] test suite --- .../apache/spark/sql/catalyst/SqlParser.scala | 0 .../spark/sql/catalyst/dsl/package.scala | 0 .../sql/catalyst/expressions/Expression.scala | 200 -------------- .../sql/catalyst/expressions/aggregates.scala | 0 .../sql/catalyst/expressions/arithmetic.scala | 243 ++++++++++++++---- .../sql/catalyst/expressions/predicates.scala | 107 +++++++- .../org/apache/spark/sql/types/Metadata.scala | 0 .../sql/catalyst/util/MetadataSuite.scala | 0 .../spark/sql/execution/Aggregate.scala | 0 .../hive/thriftserver/SparkSQLCLIDriver.scala | 0 10 files changed, 290 insertions(+), 260 deletions(-) mode change 100755 => 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala mode change 100755 => 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala mode change 100755 => 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala mode change 100755 => 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala mode change 100755 => 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala mode change 100755 => 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala mode change 100755 => 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala old mode 100755 new mode 100644 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala old mode 100755 new mode 100644 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index c32a4b886eb82..6ad39b8372cfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -77,206 +77,6 @@ abstract class Expression extends TreeNode[Expression] { case u: UnresolvedAttribute => PrettyAttribute(u.name) }.toString } - - /** - * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type - * and do any casting necessary of child evaluation. - */ - @inline - def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = { - val evalE = e.eval(i) - if (evalE == null) { - null - } else { - e.dataType match { - case n: NumericType => - val castedFunction = f.asInstanceOf[(Numeric[n.JvmType], n.JvmType) => n.JvmType] - castedFunction(n.numeric, evalE.asInstanceOf[n.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - - /** - * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed - * to be in the same data type, and also the return type. - * Either one of the expressions result is null, the evaluation result should be null. - */ - @inline - protected final def n2( - i: Row, - e1: Expression, - e2: Expression, - f: ((Numeric[Any], Any, Any) => Any)): Any = { - - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = e1.eval(i) - if(evalE1 == null) { - null - } else { - val evalE2 = e2.eval(i) - if (evalE2 == null) { - null - } else { - e1.dataType match { - case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => n.JvmType]( - n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - } - - /** - * Evaluation helper function for 2 Fractional children expressions. Those expressions are - * supposed to be in the same data type, and also the return type. - * Either one of the expressions result is null, the evaluation result should be null. - */ - @inline - protected final def f2( - i: Row, - e1: Expression, - e2: Expression, - f: ((Fractional[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = e1.eval(i: Row) - if(evalE1 == null) { - null - } else { - val evalE2 = e2.eval(i: Row) - if (evalE2 == null) { - null - } else { - e1.dataType match { - case ft: FractionalType => - f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType, ft.JvmType) => ft.JvmType]( - ft.fractional, evalE1.asInstanceOf[ft.JvmType], evalE2.asInstanceOf[ft.JvmType]) - case other => sys.error(s"Type $other does not support fractional operations") - } - } - } - } - - /** - * Evaluation helper function for 1 Fractional children expression. - * if the expression result is null, the evaluation result should be null. - */ - @inline - protected final def f1(i: Row, e1: Expression, f: ((Fractional[Any], Any) => Any)): Any = { - val evalE1 = e1.eval(i: Row) - if(evalE1 == null) { - null - } else { - e1.dataType match { - case ft: FractionalType => - f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType) => ft.JvmType]( - ft.fractional, evalE1.asInstanceOf[ft.JvmType]) - case other => sys.error(s"Type $other does not support fractional operations") - } - } - } - - /** - * Evaluation helper function for 2 Integral children expressions. Those expressions are - * supposed to be in the same data type, and also the return type. - * Either one of the expressions result is null, the evaluation result should be null. - */ - @inline - protected final def i2( - i: Row, - e1: Expression, - e2: Expression, - f: ((Integral[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = e1.eval(i) - if(evalE1 == null) { - null - } else { - val evalE2 = e2.eval(i) - if (evalE2 == null) { - null - } else { - e1.dataType match { - case i: IntegralType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( - i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) - case i: FractionalType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( - i.asIntegral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - } - - /** - * Evaluation helper function for 1 Integral children expression. - * if the expression result is null, the evaluation result should be null. - */ - @inline - protected final def i1(i: Row, e1: Expression, f: ((Integral[Any], Any) => Any)): Any = { - val evalE1 = e1.eval(i) - if(evalE1 == null) { - null - } else { - e1.dataType match { - case i: IntegralType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType) => i.JvmType]( - i.integral, evalE1.asInstanceOf[i.JvmType]) - case i: FractionalType => - f.asInstanceOf[(Integral[i.JvmType], i.JvmType) => i.JvmType]( - i.asIntegral, evalE1.asInstanceOf[i.JvmType]) - case other => sys.error(s"Type $other does not support numeric operations") - } - } - } - - /** - * Evaluation helper function for 2 Comparable children expressions. Those expressions are - * supposed to be in the same data type, and the return type should be Integer: - * Negative value: 1st argument less than 2nd argument - * Zero: 1st argument equals 2nd argument - * Positive value: 1st argument greater than 2nd argument - * - * Either one of the expressions result is null, the evaluation result should be null. - */ - @inline - protected final def c2( - i: Row, - e1: Expression, - e2: Expression, - f: ((Ordering[Any], Any, Any) => Any)): Any = { - if (e1.dataType != e2.dataType) { - throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") - } - - val evalE1 = e1.eval(i) - if(evalE1 == null) { - null - } else { - val evalE2 = e2.eval(i) - if (evalE2 == null) { - null - } else { - e1.dataType match { - case i: NativeType => - f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean]( - i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) - case other => sys.error(s"Type $other does not support ordered operations") - } - } - } - } } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala old mode 100755 new mode 100644 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 574907f566c0f..00b0d3c683fe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { @@ -28,8 +29,18 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"-$child" + lazy val numeric = dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + override def eval(input: Row): Any = { - n1(child, input, _.negate(_)) + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + numeric.negate(evalE) + } } } @@ -41,18 +52,19 @@ case class Sqrt(child: Expression) extends UnaryExpression { def nullable = true override def toString = s"SQRT($child)" + lazy val numeric = child.dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support non-negative numeric operations") + } + override def eval(input: Row): Any = { val evalE = child.eval(input) if (evalE == null) { null } else { - child.dataType match { - case n: NumericType => - val value = n.numeric.toDouble(evalE.asInstanceOf[n.JvmType]) - if (value < 0) null - else math.sqrt(value) - case other => sys.error(s"Type $other does not support non-negative numeric operations") - } + val value = numeric.toDouble(evalE) + if (value < 0) null + else math.sqrt(value) } } } @@ -98,19 +110,70 @@ abstract class BinaryArithmetic extends BinaryExpression { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "+" - override def eval(input: Row): Any = n2(input, left, right, _.plus(_, _)) + lazy val numeric = dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + numeric.plus(evalE1, evalE2) + } + } + } } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "-" - override def eval(input: Row): Any = n2(input, left, right, _.minus(_, _)) + lazy val numeric = dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + numeric.minus(evalE1, evalE2) + } + } + } } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "*" - override def eval(input: Row): Any = n2(input, left, right, _.times(_, _)) + lazy val numeric = dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + numeric.times(evalE1, evalE2) + } + } + } } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { @@ -118,16 +181,25 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic override def nullable = true + lazy val div: (Any, Any) => Any = dataType match { + case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div + case it: IntegralType => it.integral.asInstanceOf[Integral[Any]].quot + case other => sys.error(s"Type $other does not support numeric operations") + } + override def eval(input: Row): Any = { val evalE2 = right.eval(input) - dataType match { - case _ if evalE2 == null => null - case _ if evalE2 == 0 => null - case ft: FractionalType => f1(input, left, _.div(_, evalE2.asInstanceOf[ft.JvmType])) - case it: IntegralType => i1(input, left, _.quot(_, evalE2.asInstanceOf[it.JvmType])) + if (evalE2 == null || evalE2 == 0) { + null + } else { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + div(evalE1, evalE2) + } } } - } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { @@ -135,12 +207,23 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def nullable = true + lazy val integral = dataType match { + case i: IntegralType => i.integral.asInstanceOf[Integral[Any]] + case i: FractionalType => i.asIntegral.asInstanceOf[Integral[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + override def eval(input: Row): Any = { val evalE2 = right.eval(input) - dataType match { - case _ if evalE2 == null => null - case _ if evalE2 == 0 => null - case nt: NumericType => i1(input, left, _.rem(_, evalE2.asInstanceOf[nt.JvmType])) + if (evalE2 == null || evalE2 == 0) { + null + } else { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + integral.rem(evalE1, evalE2) + } } } } @@ -151,13 +234,19 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "&" - override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { - case ByteType => (evalE1.asInstanceOf[Byte] & evalE2.asInstanceOf[Byte]).toByte - case ShortType => (evalE1.asInstanceOf[Short] & evalE2.asInstanceOf[Short]).toShort - case IntegerType => evalE1.asInstanceOf[Int] & evalE2.asInstanceOf[Int] - case LongType => evalE1.asInstanceOf[Long] & evalE2.asInstanceOf[Long] + lazy val and: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 & evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 & evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any] case other => sys.error(s"Unsupported bitwise & operation on $other") } + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = and(evalE1, evalE2) } /** @@ -166,13 +255,19 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "|" - override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { - case ByteType => (evalE1.asInstanceOf[Byte] | evalE2.asInstanceOf[Byte]).toByte - case ShortType => (evalE1.asInstanceOf[Short] | evalE2.asInstanceOf[Short]).toShort - case IntegerType => evalE1.asInstanceOf[Int] | evalE2.asInstanceOf[Int] - case LongType => evalE1.asInstanceOf[Long] | evalE2.asInstanceOf[Long] + lazy val or: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 | evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 | evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any] case other => sys.error(s"Unsupported bitwise | operation on $other") } + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = or(evalE1, evalE2) } /** @@ -181,13 +276,19 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "^" - override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { - case ByteType => (evalE1.asInstanceOf[Byte] ^ evalE2.asInstanceOf[Byte]).toByte - case ShortType => (evalE1.asInstanceOf[Short] ^ evalE2.asInstanceOf[Short]).toShort - case IntegerType => evalE1.asInstanceOf[Int] ^ evalE2.asInstanceOf[Int] - case LongType => evalE1.asInstanceOf[Long] ^ evalE2.asInstanceOf[Long] + lazy val xor: (Any, Any) => Any = dataType match { + case ByteType => + ((evalE1: Byte, evalE2: Byte) => (evalE1 ^ evalE2).toByte).asInstanceOf[(Any, Any) => Any] + case ShortType => + ((evalE1: Short, evalE2: Short) => (evalE1 ^ evalE2).toShort).asInstanceOf[(Any, Any) => Any] + case IntegerType => + ((evalE1: Int, evalE2: Int) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] + case LongType => + ((evalE1: Long, evalE2: Long) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any] case other => sys.error(s"Unsupported bitwise ^ operation on $other") } + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = xor(evalE1, evalE2) } /** @@ -201,18 +302,24 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"~$child" + lazy val not: (Any) => Any = dataType match { + case ByteType => + ((evalE: Byte) => (~evalE).toByte).asInstanceOf[(Any) => Any] + case ShortType => + ((evalE: Short) => (~evalE).toShort).asInstanceOf[(Any) => Any] + case IntegerType => + ((evalE: Int) => ~evalE).asInstanceOf[(Any) => Any] + case LongType => + ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] + case other => sys.error(s"Unsupported bitwise ~ operation on $other") + } + override def eval(input: Row): Any = { val evalE = child.eval(input) if (evalE == null) { null } else { - dataType match { - case ByteType => (~evalE.asInstanceOf[Byte]).toByte - case ShortType => (~evalE.asInstanceOf[Short]).toShort - case IntegerType => ~evalE.asInstanceOf[Int] - case LongType => ~evalE.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise ~ operation on $other") - } + not(evalE) } } } @@ -226,21 +333,35 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def children = left :: right :: Nil - override def dataType = left.dataType + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType + + override def dataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + lazy val ordering = left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } override def eval(input: Row): Any = { - val leftEval = left.eval(input) - val rightEval = right.eval(input) - if (leftEval == null) { - rightEval - } else if (rightEval == null) { - leftEval + val evalE1 = left.eval(input) + val evalE2 = right.eval(input) + if (evalE1 == null) { + evalE2 + } else if (evalE2 == null) { + evalE1 } else { - val numeric = left.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]] - if (numeric.compare(leftEval, rightEval) < 0) { - rightEval + if (ordering.compare(evalE1, evalE2) < 0) { + evalE2 } else { - leftEval + evalE1 } } } @@ -259,5 +380,17 @@ case class Abs(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"Abs($child)" - override def eval(input: Row): Any = n1(child, input, _.abs(_)) + lazy val numeric = dataType match { + case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] + case other => sys.error(s"Type $other does not support numeric operations") + } + + override def eval(input: Row): Any = { + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + numeric.abs(evalE) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 365b1685a8e71..0024ef92c0452 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.{BinaryType, BooleanType} +import org.apache.spark.sql.types.{BinaryType, BooleanType, NativeType} object InterpretedPredicate { def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = @@ -201,22 +202,118 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" - override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) + + lazy val ordering = { + if (left.dataType != right.dataType) { + throw new TreeNodeException(this, + s"Types do not match ${left.dataType} != ${right.dataType}") + } + left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + ordering.lt(evalE1, evalE2) + } + } + } } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" - override def eval(input: Row): Any = c2(input, left, right, _.lteq(_, _)) + + lazy val ordering = { + if (left.dataType != right.dataType) { + throw new TreeNodeException(this, + s"Types do not match ${left.dataType} != ${right.dataType}") + } + left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + ordering.lteq(evalE1, evalE2) + } + } + } } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" - override def eval(input: Row): Any = c2(input, left, right, _.gt(_, _)) + + lazy val ordering = { + if (left.dataType != right.dataType) { + throw new TreeNodeException(this, + s"Types do not match ${left.dataType} != ${right.dataType}") + } + left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + ordering.gt(evalE1, evalE2) + } + } + } } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" - override def eval(input: Row): Any = c2(input, left, right, _.gteq(_, _)) + + lazy val ordering = { + if (left.dataType != right.dataType) { + throw new TreeNodeException(this, + s"Types do not match ${left.dataType} != ${right.dataType}") + } + left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + ordering.gteq(evalE1, evalE2) + } + } + } } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala old mode 100755 new mode 100644 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala old mode 100755 new mode 100644 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala old mode 100755 new mode 100644 diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala old mode 100755 new mode 100644 From e189cbb052d59eb499dd4312403925fdd72f5718 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Feb 2015 15:59:23 -0800 Subject: [PATCH 145/817] [SPARK-4865][SQL]Include temporary tables in SHOW TABLES This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` SQL command. The result of `SHOW TABLE` has two columns, `tableName` and `isTemporary`. For temporary tables, the value of `isTemporary` column will be `false`. JIRA: https://issues.apache.org/jira/browse/SPARK-4865 Author: Yin Huai Closes #4618 from yhuai/showTablesCommand and squashes the following commits: 0c09791 [Yin Huai] Use ShowTablesCommand. 85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and we will not see "OK" (originally generated by Hive's driver), use SHOW DATABASES in the test. 94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands. d71ed09 [Yin Huai] Fix test. a4a6ec3 [Yin Huai] Add SHOW TABLE command. --- .../org/apache/spark/sql/SQLContext.scala | 4 +- .../org/apache/spark/sql/SparkSQLParser.scala | 12 ++++- .../apache/spark/sql/execution/commands.scala | 34 +++++++++++++- .../apache/spark/sql/ListTablesSuite.scala | 37 +++++++++------ .../sql/hive/thriftserver/CliSuite.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 10 +++- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../spark/sql/hive/ListTablesSuite.scala | 46 ++++++++++--------- .../sql/hive/execution/HiveQuerySuite.scala | 14 +++--- 9 files changed, 111 insertions(+), 50 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d08c2d1cfe030..aa0fc3e359a9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -794,7 +794,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * indicating if a table is a temporary one or not). */ def tables(): DataFrame = { - createDataFrame(catalog.getTables(None)).toDF("tableName", "isTemporary") + DataFrame(this, ShowTablesCommand(None)) } /** @@ -803,7 +803,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * indicating if a table is a temporary one or not). */ def tables(databaseName: String): DataFrame = { - createDataFrame(catalog.getTables(Some(databaseName))).toDF("tableName", "isTemporary") + DataFrame(this, ShowTablesCommand(Some(databaseName))) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala index f1a4053b79113..00e19da4374a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala @@ -23,7 +23,7 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, SetCommand} +import org.apache.spark.sql.execution.{ShowTablesCommand, UncacheTableCommand, CacheTableCommand, SetCommand} import org.apache.spark.sql.types.StringType @@ -57,12 +57,15 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr protected val AS = Keyword("AS") protected val CACHE = Keyword("CACHE") + protected val IN = Keyword("IN") protected val LAZY = Keyword("LAZY") protected val SET = Keyword("SET") + protected val SHOW = Keyword("SHOW") protected val TABLE = Keyword("TABLE") + protected val TABLES = Keyword("TABLES") protected val UNCACHE = Keyword("UNCACHE") - override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | others + override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | show | others private lazy val cache: Parser[LogicalPlan] = CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ { @@ -80,6 +83,11 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr case input => SetCommandParser(input) } + private lazy val show: Parser[LogicalPlan] = + SHOW ~> TABLES ~ (IN ~> ident).? ^^ { + case _ ~ dbName => ShowTablesCommand(dbName) + } + private lazy val others: Parser[LogicalPlan] = wholeInput ^^ { case input => fallback(input) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 2b1726ad4e89f..c6cd6eb6a22b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType} import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} @@ -191,3 +191,35 @@ case class DescribeCommand( } } } + +/** + * A command for users to get tables in the given database. + * If a databaseName is not given, the current database will be used. + * The syntax of using this command in SQL is: + * {{{ + * SHOW TABLES [IN databaseName] + * }}} + * :: DeveloperApi :: + */ +@DeveloperApi +case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand { + + // The result of SHOW TABLES has two columns, tableName and isTemporary. + override val output = { + val schema = StructType( + StructField("tableName", StringType, false) :: + StructField("isTemporary", BooleanType, false) :: Nil) + + schema.toAttributes + } + + override def run(sqlContext: SQLContext) = { + // Since we need to return a Seq of rows, we will call getTables directly + // instead of calling tables in sqlContext. + val rows = sqlContext.catalog.getTables(databaseName).map { + case (tableName, isTemporary) => Row(tableName, isTemporary) + } + + rows + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala index 282b98a987dd4..f9f41eb358bd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala @@ -43,6 +43,10 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter { tables().filter("tableName = 'ListTablesSuiteTable'"), Row("ListTablesSuiteTable", true)) + checkAnswer( + sql("SHOW tables").filter("tableName = 'ListTablesSuiteTable'"), + Row("ListTablesSuiteTable", true)) + catalog.unregisterTable(Seq("ListTablesSuiteTable")) assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) } @@ -52,25 +56,32 @@ class ListTablesSuite extends QueryTest with BeforeAndAfter { tables("DB").filter("tableName = 'ListTablesSuiteTable'"), Row("ListTablesSuiteTable", true)) + checkAnswer( + sql("show TABLES in DB").filter("tableName = 'ListTablesSuiteTable'"), + Row("ListTablesSuiteTable", true)) + catalog.unregisterTable(Seq("ListTablesSuiteTable")) assert(tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) } test("query the returned DataFrame of tables") { - val tableDF = tables() - val schema = StructType( - StructField("tableName", StringType, true) :: + val expectedSchema = StructType( + StructField("tableName", StringType, false) :: StructField("isTemporary", BooleanType, false) :: Nil) - assert(schema === tableDF.schema) - tableDF.registerTempTable("tables") - checkAnswer( - sql("SELECT isTemporary, tableName from tables WHERE tableName = 'ListTablesSuiteTable'"), - Row(true, "ListTablesSuiteTable") - ) - checkAnswer( - tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), - Row("tables", true)) - dropTempTable("tables") + Seq(tables(), sql("SHOW TABLes")).foreach { + case tableDF => + assert(expectedSchema === tableDF.schema) + + tableDF.registerTempTable("tables") + checkAnswer( + sql("SELECT isTemporary, tableName from tables WHERE tableName = 'ListTablesSuiteTable'"), + Row(true, "ListTablesSuiteTable") + ) + checkAnswer( + tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), + Row("tables", true)) + dropTempTable("tables") + } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 60953576d0e37..8bca4b33b3ad1 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -121,6 +121,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { } test("Single command with -e") { - runCliWithin(1.minute, Seq("-e", "SHOW TABLES;"))("" -> "OK") + runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 87bc9fe4fe9ad..0e43faa8afdaf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -220,8 +220,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { - val dbName = databaseName.getOrElse(hive.sessionState.getCurrentDatabase) - client.getAllTables(dbName).map(tableName => (tableName, false)) + val dbName = if (!caseSensitive) { + if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None + } else { + databaseName + } + val db = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + + client.getAllTables(db).map(tableName => (tableName, false)) } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5a1825a87dadb..98263f602e9ec 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -64,7 +64,6 @@ private[hive] object HiveQl { "TOK_SHOWINDEXES", "TOK_SHOWINDEXES", "TOK_SHOWPARTITIONS", - "TOK_SHOWTABLES", "TOK_SHOW_TBLPROPERTIES", "TOK_LOCKTABLE", @@ -129,6 +128,7 @@ private[hive] object HiveQl { // Commands that we do not need to explain. protected val noExplainCommands = Seq( "TOK_DESCTABLE", + "TOK_SHOWTABLES", "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. ) ++ nativeCommands diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala index 321b784a3f842..e12a6c21ccac4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -49,29 +49,33 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { } test("get all tables of current database") { - val allTables = tables() - // We are using default DB. - checkAnswer( - allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0) - checkAnswer( - allTables.filter("tableName = 'hivelisttablessuitetable'"), - Row("hivelisttablessuitetable", false)) - assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0) + Seq(tables(), sql("SHOW TABLes")).foreach { + case allTables => + // We are using default DB. + checkAnswer( + allTables.filter("tableName = 'listtablessuitetable'"), + Row("listtablessuitetable", true)) + assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0) + checkAnswer( + allTables.filter("tableName = 'hivelisttablessuitetable'"), + Row("hivelisttablessuitetable", false)) + assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0) + } } test("getting all tables with a database name") { - val allTables = tables("ListTablesSuiteDB") - checkAnswer( - allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) - checkAnswer( - allTables.filter("tableName = 'indblisttablessuitetable'"), - Row("indblisttablessuitetable", true)) - assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) - checkAnswer( - allTables.filter("tableName = 'hiveindblisttablessuitetable'"), - Row("hiveindblisttablessuitetable", false)) + Seq(tables("listtablessuiteDb"), sql("SHOW TABLes in listTablesSuitedb")).foreach { + case allTables => + checkAnswer( + allTables.filter("tableName = 'listtablessuitetable'"), + Row("listtablessuitetable", true)) + checkAnswer( + allTables.filter("tableName = 'indblisttablessuitetable'"), + Row("indblisttablessuitetable", true)) + assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) + checkAnswer( + allTables.filter("tableName = 'hiveindblisttablessuitetable'"), + Row("hiveindblisttablessuitetable", false)) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index d01dbf80ef66d..955f3f51cfe9f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -62,7 +62,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-4908: concurrent hive native commands") { (1 to 100).par.map { _ => sql("USE default") - sql("SHOW TABLES") + sql("SHOW DATABASES") } } @@ -630,24 +630,24 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } test("Query Hive native command execution result") { - val tableName = "test_native_commands" + val databaseName = "test_native_commands" assertResult(0) { - sql(s"DROP TABLE IF EXISTS $tableName").count() + sql(s"DROP DATABASE IF EXISTS $databaseName").count() } assertResult(0) { - sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() + sql(s"CREATE DATABASE $databaseName").count() } assert( - sql("SHOW TABLES") + sql("SHOW DATABASES") .select('result) .collect() .map(_.getString(0)) - .contains(tableName)) + .contains(databaseName)) - assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) + assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM src GROUP BY key"))) TestHive.reset() } From 1294a6e01af0d4f6678ea8cb5d47dc97112608b5 Mon Sep 17 00:00:00 2001 From: Matt Whelan Date: Tue, 17 Feb 2015 00:59:49 +0000 Subject: [PATCH 146/817] SPARK-5848: tear down the ConsoleProgressBar timer The timer is a GC root, and failing to terminate it leaks SparkContext instances. Author: Matt Whelan Closes #4635 from MattWhelan/SPARK-5848 and squashes the following commits: 2a1e8a5 [Matt Whelan] SPARK-5848: teardown the ConsoleProgressBar timer --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../scala/org/apache/spark/ui/ConsoleProgressBar.scala | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 24a316e40e673..fd8fac6df0d0e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1363,6 +1363,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli cleaner.foreach(_.stop()) dagScheduler.stop() dagScheduler = null + progressBar.foreach(_.stop()) taskScheduler = null // TODO: Cache.stop()? env.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 27ba9e18237b5..67f572e79314d 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -28,7 +28,6 @@ import org.apache.spark._ * of them will be combined together, showed in one line. */ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { - // Carrige return val CR = '\r' // Update period of progress bar, in milliseconds @@ -121,4 +120,10 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { clear() lastFinishTime = System.currentTimeMillis() } + + /** + * Tear down the timer thread. The timer thread is a GC root, and it retains the entire + * SparkContext if it's not terminated. + */ + def stop(): Unit = timer.cancel() } From b1bd1dd3228ef50fa7310d466afd834b8cb1f22e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 16 Feb 2015 17:57:14 -0800 Subject: [PATCH 147/817] [SPARK-5788] [PySpark] capture the exception in python write thread The exception in Python writer thread will shutdown executor. Author: Davies Liu Closes #4577 from davies/exception and squashes the following commits: eb0ceff [Davies Liu] Update PythonRDD.scala 139b0db [Davies Liu] capture the exception in python write thread --- .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b89effc16d36d..252721192904f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -248,13 +248,13 @@ private[spark] class PythonRDD( } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) - worker.shutdownOutput() + Utils.tryLog(worker.shutdownOutput()) case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - worker.shutdownOutput() + Utils.tryLog(worker.shutdownOutput()) } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() From 16687651f05bde8ff2e2fcef100383168958bf7f Mon Sep 17 00:00:00 2001 From: azagrebin Date: Mon, 16 Feb 2015 18:06:19 -0800 Subject: [PATCH 148/817] [SPARK-3340] Deprecate ADD_JARS and ADD_FILES I created a patch that disables the environment variables. Thereby scala or python shell log a warning message to notify user about the deprecation with the following message: scala: "ADD_JARS environment variable is deprecated, use --jar spark submit argument instead" python: "Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead" Is it what is expected or the code associated with the variables should be just completely removed? Should it be somewhere documented? Author: azagrebin Closes #4616 from azagrebin/master and squashes the following commits: bab1aa9 [azagrebin] [SPARK-3340] Deprecate ADD_JARS and ADD_FILES: minor readability issue 0643895 [azagrebin] [SPARK-3340] Deprecate ADD_JARS and ADD_FILES: add warning messages 42f0107 [azagrebin] [SPARK-3340] Deprecate ADD_JARS and ADD_FILES --- python/pyspark/shell.py | 8 +++++--- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 7 ++++--- .../src/main/scala/org/apache/spark/repl/Main.scala | 3 +++ 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 89cf76920e353..4cf4b89ccfaa7 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,9 +35,10 @@ from pyspark.context import SparkContext from pyspark.storagelevel import StorageLevel -# this is the equivalent of ADD_JARS -add_files = (os.environ.get("ADD_FILES").split(',') - if os.environ.get("ADD_FILES") is not None else None) +# this is the deprecated equivalent of ADD_JARS +add_files = None +if os.environ.get("ADD_FILES") is not None: + add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) @@ -59,6 +60,7 @@ print("SparkContext available as sc.") if add_files is not None: + print("Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead") print("Adding files: [%s]" % ", ".join(add_files)) # The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index b4db3df795177..8dc0e0c965923 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1064,15 +1064,16 @@ class SparkILoop( private def main(settings: Settings): Unit = process(settings) } -object SparkILoop { +object SparkILoop extends Logging { implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp private def echo(msg: String) = Console println msg def getAddedJars: Array[String] = { val envJars = sys.env.get("ADD_JARS") - val propJars = sys.props.get("spark.jars").flatMap { p => - if (p == "") None else Some(p) + if (envJars.isDefined) { + logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") } + val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } val jars = propJars.orElse(envJars).getOrElse("") Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) } diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index dc25692749aad..2210fbaafeadb 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -51,6 +51,9 @@ object Main extends Logging { def getAddedJars: Array[String] = { val envJars = sys.env.get("ADD_JARS") + if (envJars.isDefined) { + logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") + } val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } val jars = propJars.orElse(envJars).getOrElse("") Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) From 58a82a7882d7a8a7e4064278c4bf28607d9a42ba Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 16 Feb 2015 18:08:02 -0800 Subject: [PATCH 149/817] [SPARK-5849] Handle more types of invalid JSON requests in SubmitRestProtocolMessage.parseAction This patch improves SubmitRestProtocol's handling of invalid JSON requests in cases where those requests were parsable as JSON but not as JSON objects (e.g. they could be parsed as ararys or strings). I replaced an unchecked cast with pattern-matching and added a new test case. Author: Josh Rosen Closes #4637 from JoshRosen/rest-protocol-cast and squashes the following commits: b3f282b [Josh Rosen] [SPARK-5849] Handle more types of invalid JSON in SubmitRestProtocolMessage.parseAction --- .../deploy/rest/SubmitRestProtocolMessage.scala | 16 ++++++++-------- .../deploy/rest/StandaloneRestSubmitSuite.scala | 4 ++++ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala index b877898231e3e..8f36635674a28 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy.rest -import scala.util.Try - import com.fasterxml.jackson.annotation._ import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility import com.fasterxml.jackson.annotation.JsonInclude.Include @@ -111,12 +109,14 @@ private[spark] object SubmitRestProtocolMessage { * If the action field is not found, throw a [[SubmitRestMissingFieldException]]. */ def parseAction(json: String): String = { - parse(json).asInstanceOf[JObject].obj - .find { case (f, _) => f == "action" } - .map { case (_, v) => v.asInstanceOf[JString].s } - .getOrElse { - throw new SubmitRestMissingFieldException(s"Action field not found in JSON:\n$json") - } + val value: Option[String] = parse(json) match { + case JObject(fields) => + fields.collectFirst { case ("action", v) => v }.collect { case JString(s) => s } + case _ => None + } + value.getOrElse { + throw new SubmitRestMissingFieldException(s"Action field not found in JSON:\n$json") + } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index a345e06ecb7d2..2fa90e3bd1c63 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -245,6 +245,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val goodJson = constructSubmitRequest(masterUrl).toJson val badJson1 = goodJson.replaceAll("action", "fraction") // invalid JSON val badJson2 = goodJson.substring(goodJson.size / 2) // malformed JSON + val notJson = "\"hello, world\"" val (response1, code1) = sendHttpRequestWithResponse(submitRequestPath, "POST") // missing JSON val (response2, code2) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson1) val (response3, code3) = sendHttpRequestWithResponse(submitRequestPath, "POST", badJson2) @@ -252,6 +253,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val (response5, code5) = sendHttpRequestWithResponse(s"$killRequestPath/", "POST") val (response6, code6) = sendHttpRequestWithResponse(statusRequestPath, "GET") // missing ID val (response7, code7) = sendHttpRequestWithResponse(s"$statusRequestPath/", "GET") + val (response8, code8) = sendHttpRequestWithResponse(submitRequestPath, "POST", notJson) // these should all fail as error responses getErrorResponse(response1) getErrorResponse(response2) @@ -260,6 +262,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { getErrorResponse(response5) getErrorResponse(response6) getErrorResponse(response7) + getErrorResponse(response8) assert(code1 === HttpServletResponse.SC_BAD_REQUEST) assert(code2 === HttpServletResponse.SC_BAD_REQUEST) assert(code3 === HttpServletResponse.SC_BAD_REQUEST) @@ -267,6 +270,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(code5 === HttpServletResponse.SC_BAD_REQUEST) assert(code6 === HttpServletResponse.SC_BAD_REQUEST) assert(code7 === HttpServletResponse.SC_BAD_REQUEST) + assert(code8 === HttpServletResponse.SC_BAD_REQUEST) } test("bad request paths") { From 0e180bfc3c7f18780d4fc4f42681609832418e43 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 16 Feb 2015 19:00:30 -0800 Subject: [PATCH 150/817] [SQL] Various DataFrame doc changes. Added a bunch of tags. Also changed parquetFile to take varargs rather than a string followed by varargs. Author: Reynold Xin Closes #4636 from rxin/df-doc and squashes the following commits: 651f80c [Reynold Xin] Fixed parquetFile in PySpark. 8dc3024 [Reynold Xin] [SQL] Various DataFrame doc changes. --- python/pyspark/sql/context.py | 7 +- .../scala/org/apache/spark/sql/Column.scala | 165 ++++++++++++- .../org/apache/spark/sql/DataFrame.scala | 14 +- .../org/apache/spark/sql/DataFrameImpl.scala | 4 +- .../org/apache/spark/sql/GroupedData.scala | 3 + .../org/apache/spark/sql/SQLContext.scala | 97 +++++--- .../apache/spark/sql/UDFRegistration.scala | 4 +- .../org/apache/spark/sql/functions.scala | 226 +++++++++++++++--- .../spark/sql/hive/HiveStrategies.scala | 3 +- 9 files changed, 436 insertions(+), 87 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 7683c1b4dfa4e..dd2cd5ee76f60 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -431,11 +431,10 @@ def parquetFile(self, *paths): True """ gateway = self._sc._gateway - jpath = paths[0] - jpaths = gateway.new_array(gateway.jvm.java.lang.String, len(paths) - 1) - for i in range(1, len(paths)): + jpaths = gateway.new_array(gateway.jvm.java.lang.String, len(paths)) + for i in range(0, len(paths)): jpaths[i] = paths[i] - jdf = self._ssql_ctx.parquetFile(jpath, jpaths) + jdf = self._ssql_ctx.parquetFile(jpaths) return DataFrame(jdf, self) def jsonFile(self, path, schema=None, samplingRatio=1.0): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index f6ecee1af8aad..8b6241c213c87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import scala.language.implicitConversions +import org.apache.spark.annotation.Experimental import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} @@ -41,21 +42,15 @@ private[sql] object Column { /** + * :: Experimental :: * A column in a [[DataFrame]]. * - * `Column` instances can be created by: - * {{{ - * // 1. Select a column out of a DataFrame - * df("colName") - * - * // 2. Create a literal expression - * Literal(1) - * - * // 3. Create new columns from - * }}} - * + * @groupname java_expr_ops Java-specific expression operators. + * @groupname expr_ops Expression operators. + * @groupname df_ops DataFrame functions. + * @groupname Ungrouped Support functions for DataFrames. */ -// TODO: Improve documentation. +@Experimental trait Column extends DataFrame { protected[sql] def expr: Expression @@ -129,6 +124,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.select( negate(col("amount") ); * }}} + * + * @group expr_ops */ def unary_- : Column = exprToColumn(UnaryMinus(expr)) @@ -142,6 +139,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.filter( not(df.col("isActive")) ); * }} + * + * @group expr_ops */ def unary_! : Column = exprToColumn(Not(expr)) @@ -155,6 +154,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").equalTo(col("colB")) ); * }}} + * + * @group expr_ops */ def === (other: Any): Column = constructColumn(other) { o => EqualTo(expr, o.expr) @@ -170,6 +171,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").equalTo(col("colB")) ); * }}} + * + * @group expr_ops */ def equalTo(other: Any): Column = this === other @@ -184,6 +187,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").notEqual(col("colB")) ); * }}} + * + * @group expr_ops */ def !== (other: Any): Column = constructColumn(other) { o => Not(EqualTo(expr, o.expr)) @@ -200,6 +205,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * df.filter( col("colA").notEqual(col("colB")) ); * }}} + * + * @group java_expr_ops */ def notEqual(other: Any): Column = constructColumn(other) { o => Not(EqualTo(expr, o.expr)) @@ -215,6 +222,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * people.select( people("age").gt(21) ); * }}} + * + * @group expr_ops */ def > (other: Any): Column = constructColumn(other) { o => GreaterThan(expr, o.expr) @@ -230,6 +239,8 @@ trait Column extends DataFrame { * import static org.apache.spark.sql.functions.*; * people.select( people("age").gt(21) ); * }}} + * + * @group java_expr_ops */ def gt(other: Any): Column = this > other @@ -242,6 +253,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").lt(21) ); * }}} + * + * @group expr_ops */ def < (other: Any): Column = constructColumn(other) { o => LessThan(expr, o.expr) @@ -256,6 +269,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").lt(21) ); * }}} + * + * @group java_expr_ops */ def lt(other: Any): Column = this < other @@ -268,6 +283,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").leq(21) ); * }}} + * + * @group expr_ops */ def <= (other: Any): Column = constructColumn(other) { o => LessThanOrEqual(expr, o.expr) @@ -282,6 +299,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").leq(21) ); * }}} + * + * @group java_expr_ops */ def leq(other: Any): Column = this <= other @@ -294,6 +313,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").geq(21) ) * }}} + * + * @group expr_ops */ def >= (other: Any): Column = constructColumn(other) { o => GreaterThanOrEqual(expr, o.expr) @@ -308,11 +329,15 @@ trait Column extends DataFrame { * // Java: * people.select( people("age").geq(21) ) * }}} + * + * @group java_expr_ops */ def geq(other: Any): Column = this >= other /** * Equality test that is safe for null values. + * + * @group expr_ops */ def <=> (other: Any): Column = constructColumn(other) { o => EqualNullSafe(expr, o.expr) @@ -320,16 +345,22 @@ trait Column extends DataFrame { /** * Equality test that is safe for null values. + * + * @group java_expr_ops */ def eqNullSafe(other: Any): Column = this <=> other /** * True if the current expression is null. + * + * @group expr_ops */ def isNull: Column = exprToColumn(IsNull(expr)) /** * True if the current expression is NOT null. + * + * @group expr_ops */ def isNotNull: Column = exprToColumn(IsNotNull(expr)) @@ -342,6 +373,8 @@ trait Column extends DataFrame { * // Java: * people.filter( people("inSchool").or(people("isEmployed")) ); * }}} + * + * @group expr_ops */ def || (other: Any): Column = constructColumn(other) { o => Or(expr, o.expr) @@ -356,6 +389,8 @@ trait Column extends DataFrame { * // Java: * people.filter( people("inSchool").or(people("isEmployed")) ); * }}} + * + * @group java_expr_ops */ def or(other: Column): Column = this || other @@ -368,6 +403,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("inSchool").and(people("isEmployed")) ); * }}} + * + * @group expr_ops */ def && (other: Any): Column = constructColumn(other) { o => And(expr, o.expr) @@ -382,6 +419,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("inSchool").and(people("isEmployed")) ); * }}} + * + * @group java_expr_ops */ def and(other: Column): Column = this && other @@ -394,6 +433,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").plus(people("weight")) ); * }}} + * + * @group expr_ops */ def + (other: Any): Column = constructColumn(other) { o => Add(expr, o.expr) @@ -408,6 +449,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").plus(people("weight")) ); * }}} + * + * @group java_expr_ops */ def plus(other: Any): Column = this + other @@ -420,6 +463,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").minus(people("weight")) ); * }}} + * + * @group expr_ops */ def - (other: Any): Column = constructColumn(other) { o => Subtract(expr, o.expr) @@ -434,6 +479,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").minus(people("weight")) ); * }}} + * + * @group java_expr_ops */ def minus(other: Any): Column = this - other @@ -446,6 +493,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").multiply(people("weight")) ); * }}} + * + * @group expr_ops */ def * (other: Any): Column = constructColumn(other) { o => Multiply(expr, o.expr) @@ -460,6 +509,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").multiply(people("weight")) ); * }}} + * + * @group java_expr_ops */ def multiply(other: Any): Column = this * other @@ -472,6 +523,8 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").divide(people("weight")) ); * }}} + * + * @group expr_ops */ def / (other: Any): Column = constructColumn(other) { o => Divide(expr, o.expr) @@ -486,11 +539,15 @@ trait Column extends DataFrame { * // Java: * people.select( people("height").divide(people("weight")) ); * }}} + * + * @group java_expr_ops */ def divide(other: Any): Column = this / other /** * Modulo (a.k.a. remainder) expression. + * + * @group expr_ops */ def % (other: Any): Column = constructColumn(other) { o => Remainder(expr, o.expr) @@ -498,29 +555,47 @@ trait Column extends DataFrame { /** * Modulo (a.k.a. remainder) expression. + * + * @group java_expr_ops */ def mod(other: Any): Column = this % other /** * A boolean expression that is evaluated to true if the value of this expression is contained * by the evaluated values of the arguments. + * + * @group expr_ops */ @scala.annotation.varargs def in(list: Column*): Column = { new IncomputableColumn(In(expr, list.map(_.expr))) } + /** + * SQL like expression. + * + * @group expr_ops + */ def like(literal: String): Column = exprToColumn(Like(expr, lit(literal).expr)) + /** + * SQL RLIKE expression (LIKE with Regex). + * + * @group expr_ops + */ def rlike(literal: String): Column = exprToColumn(RLike(expr, lit(literal).expr)) /** * An expression that gets an item at position `ordinal` out of an array. + * + * @group expr_ops */ def getItem(ordinal: Int): Column = exprToColumn(GetItem(expr, Literal(ordinal))) /** * An expression that gets a field by name in a [[StructField]]. + * + * @group expr_ops */ def getField(fieldName: String): Column = exprToColumn(UnresolvedGetField(expr, fieldName)) @@ -528,6 +603,8 @@ trait Column extends DataFrame { * An expression that returns a substring. * @param startPos expression for the starting position. * @param len expression for the length of the substring. + * + * @group expr_ops */ def substr(startPos: Column, len: Column): Column = exprToColumn(Substring(expr, startPos.expr, len.expr), computable = false) @@ -536,24 +613,51 @@ trait Column extends DataFrame { * An expression that returns a substring. * @param startPos starting position. * @param len length of the substring. + * + * @group expr_ops */ def substr(startPos: Int, len: Int): Column = exprToColumn(Substring(expr, lit(startPos).expr, lit(len).expr)) + /** + * Contains the other element. + * + * @group expr_ops + */ def contains(other: Any): Column = constructColumn(other) { o => Contains(expr, o.expr) } + /** + * String starts with. + * + * @group expr_ops + */ def startsWith(other: Column): Column = constructColumn(other) { o => StartsWith(expr, o.expr) } + /** + * String starts with another string literal. + * + * @group expr_ops + */ def startsWith(literal: String): Column = this.startsWith(lit(literal)) + /** + * String ends with. + * + * @group expr_ops + */ def endsWith(other: Column): Column = constructColumn(other) { o => EndsWith(expr, o.expr) } + /** + * String ends with another string literal. + * + * @group expr_ops + */ def endsWith(literal: String): Column = this.endsWith(lit(literal)) /** @@ -562,6 +666,8 @@ trait Column extends DataFrame { * // Renames colA to colB in select output. * df.select($"colA".as("colB")) * }}} + * + * @group expr_ops */ override def as(alias: String): Column = exprToColumn(Alias(expr, alias)()) @@ -571,6 +677,8 @@ trait Column extends DataFrame { * // Renames colA to colB in select output. * df.select($"colA".as('colB)) * }}} + * + * @group expr_ops */ override def as(alias: Symbol): Column = exprToColumn(Alias(expr, alias.name)()) @@ -584,6 +692,8 @@ trait Column extends DataFrame { * // equivalent to * df.select(df("colA").cast("int")) * }}} + * + * @group expr_ops */ def cast(to: DataType): Column = exprToColumn(Cast(expr, to)) @@ -595,6 +705,8 @@ trait Column extends DataFrame { * // Casts colA to integer. * df.select(df("colA").cast("int")) * }}} + * + * @group expr_ops */ def cast(to: String): Column = exprToColumn( Cast(expr, to.toLowerCase match { @@ -613,10 +725,39 @@ trait Column extends DataFrame { }) ) + /** + * Returns an ordering used in sorting. + * {{{ + * // Scala: sort a DataFrame by age column in descending order. + * df.sort(df("age").desc) + * + * // Java + * df.sort(df.col("age").desc()); + * }}} + * + * @group expr_ops + */ def desc: Column = exprToColumn(SortOrder(expr, Descending), computable = false) + /** + * Returns an ordering used in sorting. + * {{{ + * // Scala: sort a DataFrame by age column in ascending order. + * df.sort(df("age").asc) + * + * // Java + * df.sort(df.col("age").asc()); + * }}} + * + * @group expr_ops + */ def asc: Column = exprToColumn(SortOrder(expr, Ascending), computable = false) + /** + * Prints the plans (logical and physical) to the console for debugging purpose. + * + * @group df_ops + */ override def explain(extended: Boolean): Unit = { if (extended) { println(expr) 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 e21e989f36c65..c0c3cb40cf1da 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 @@ -41,19 +41,23 @@ private[sql] object DataFrame { * :: Experimental :: * A distributed collection of data organized into named columns. * - * A [[DataFrame]] is equivalent to a relational table in Spark SQL, and can be created using - * various functions in [[SQLContext]]. + * A [[DataFrame]] is equivalent to a relational table in Spark SQL. There are multiple ways + * to create a [[DataFrame]]: * {{{ + * // Create a DataFrame from Parquet files * val people = sqlContext.parquetFile("...") + * + * // Create a DataFrame from data sources + * val df = * }}} * * Once created, it can be manipulated using the various domain-specific-language (DSL) functions - * defined in: [[DataFrame]] (this class), [[Column]], [[functions]] for the DSL. + * defined in: [[DataFrame]] (this class), [[Column]], and [[functions]]. * - * To select a column from the data frame, use the apply method: + * To select a column from the data frame, use `apply` method in Scala and `col` in Java. * {{{ * val ageCol = people("age") // in Scala - * Column ageCol = people.apply("age") // in Java + * Column ageCol = people.col("age") // in Java * }}} * * Note that the [[Column]] type can also be manipulated through its various functions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 3c1cf8d5e3851..848ea2e05624f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -238,9 +238,10 @@ private[sql] class DataFrameImpl protected[sql]( } override def withColumnRenamed(existingName: String, newName: String): DataFrame = { + val resolver = sqlContext.analyzer.resolver val colNames = schema.map { field => val name = field.name - if (name == existingName) Column(name).as(newName) else Column(name) + if (resolver(name, existingName)) Column(name).as(newName) else Column(name) } select(colNames :_*) } @@ -368,6 +369,7 @@ private[sql] class DataFrameImpl protected[sql]( ///////////////////////////////////////////////////////////////////////////// override def rdd: RDD[Row] = { + // use a local variable to make sure the map closure doesn't capture the whole DataFrame val schema = this.schema queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) } 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 2ecf086de92f7..17158303b889a 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.collection.JavaConversions._ 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 @@ -27,8 +28,10 @@ import org.apache.spark.sql.types.NumericType /** + * :: Experimental :: * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ +@Experimental class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index aa0fc3e359a9b..0aae0942ca04f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -25,7 +25,7 @@ import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ @@ -43,14 +43,13 @@ import org.apache.spark.util.Utils import org.apache.spark.{Partition, SparkContext} /** - * :: AlphaComponent :: * The entry point for running relational queries using Spark. Allows the creation of [[DataFrame]] * objects and the execution of SQL queries. * + * @groupname ddl_ops Catalog DDL functions * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ -@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging with Serializable { @@ -127,13 +126,19 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] val cacheManager = new CacheManager(this) /** + * :: Experimental :: * A collection of methods that are considered experimental, but can be used to hook into * the query planner for advanced functionalities. */ + @Experimental @transient val experimental: ExperimentalMethods = new ExperimentalMethods(this) - /** Returns a [[DataFrame]] with no rows or columns. */ + /** + * :: Experimental :: + * Returns a [[DataFrame]] with no rows or columns. + */ + @Experimental @transient lazy val emptyDataFrame = DataFrame(this, NoRelation) @@ -178,9 +183,11 @@ class SQLContext(@transient val sparkContext: SparkContext) // scalastyle:off // Disable style checker so "implicits" object can start with lowercase i /** - * (Scala-specific) - * Implicit methods available in Scala for converting common Scala objects into [[DataFrame]]s. + * :: Experimental :: + * (Scala-specific) Implicit methods available in Scala for converting + * common Scala objects into [[DataFrame]]s. */ + @Experimental object implicits extends Serializable { // scalastyle:on @@ -250,10 +257,12 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** + * :: Experimental :: * Creates a DataFrame from an RDD of case classes. * * @group userf */ + @Experimental def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkPlan.currentContext.set(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -263,8 +272,10 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** + * :: Experimental :: * Creates a DataFrame from a local Seq of Product. */ + @Experimental def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { SparkPlan.currentContext.set(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -281,7 +292,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: DeveloperApi :: - * Creates a [[DataFrame]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD. + * Creates a [[DataFrame]] from an [[RDD]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided RDD matches * the provided schema. Otherwise, there will be runtime exception. * Example: @@ -316,6 +327,12 @@ class SQLContext(@transient val sparkContext: SparkContext) DataFrame(this, logicalPlan) } + /** + * :: DeveloperApi :: + * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s using the given schema. + * It is important to make sure that the structure of every [[Row]] of the provided RDD matches + * the provided schema. Otherwise, there will be runtime exception. + */ @DeveloperApi def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD.rdd, schema) @@ -402,13 +419,11 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - @DeveloperApi @deprecated("use createDataFrame", "1.3.0") def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD, schema) } - @DeveloperApi @deprecated("use createDataFrame", "1.3.0") def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD, schema) @@ -437,18 +452,22 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Loads a Parquet file, returning the result as a [[DataFrame]]. + * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty + * [[DataFrame]] if no paths are passed in. * * @group userf */ @scala.annotation.varargs - def parquetFile(path: String, paths: String*): DataFrame = - if (conf.parquetUseDataSourceApi) { - baseRelationToDataFrame(parquet.ParquetRelation2(path +: paths, Map.empty)(this)) + def parquetFile(paths: String*): DataFrame = { + if (paths.isEmpty) { + emptyDataFrame + } else if (conf.parquetUseDataSourceApi) { + baseRelationToDataFrame(parquet.ParquetRelation2(paths, Map.empty)(this)) } else { DataFrame(this, parquet.ParquetRelation( - (path +: paths).mkString(","), Some(sparkContext.hadoopConfiguration), this)) + paths.mkString(","), Some(sparkContext.hadoopConfiguration), this)) } + } /** * Loads a JSON file (one object per line), returning the result as a [[DataFrame]]. @@ -545,8 +564,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * Returns the dataset stored at path as a DataFrame, - * using the given data source. + * Returns the dataset stored at path as a DataFrame, using the given data source. */ @Experimental def load(path: String, source: String): DataFrame = { @@ -555,7 +573,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * Returns the dataset specified by the given data source and a set of options as a DataFrame. + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. */ @Experimental def load(source: String, options: java.util.Map[String, String]): DataFrame = { @@ -564,8 +583,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * (Scala-specific) - * Returns the dataset specified by the given data source and a set of options as a DataFrame. + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. */ @Experimental def load(source: String, options: Map[String, String]): DataFrame = { @@ -575,8 +594,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * Returns the dataset specified by the given data source and a set of options as a DataFrame, - * using the given schema as the schema of the DataFrame. + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. */ @Experimental def load( @@ -588,9 +607,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * (Scala-specific) - * Returns the dataset specified by the given data source and a set of options as a DataFrame, - * using the given schema as the schema of the DataFrame. + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. */ @Experimental def load( @@ -605,6 +623,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Creates an external table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. + * + * @group ddl_ops */ @Experimental def createExternalTable(tableName: String, path: String): DataFrame = { @@ -616,6 +636,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Creates an external table from the given path based on a data source * and returns the corresponding DataFrame. + * + * @group ddl_ops */ @Experimental def createExternalTable( @@ -629,6 +651,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Creates an external table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. + * + * @group ddl_ops */ @Experimental def createExternalTable( @@ -643,6 +667,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * (Scala-specific) * Creates an external table from the given path based on a data source and a set of options. * Then, returns the corresponding DataFrame. + * + * @group ddl_ops */ @Experimental def createExternalTable( @@ -666,6 +692,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Create an external table from the given path based on a data source, a schema and * a set of options. Then, returns the corresponding DataFrame. + * + * @group ddl_ops */ @Experimental def createExternalTable( @@ -681,6 +709,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * (Scala-specific) * Create an external table from the given path based on a data source, a schema and * a set of options. Then, returns the corresponding DataFrame. + * + * @group ddl_ops */ @Experimental def createExternalTable( @@ -734,26 +764,23 @@ class SQLContext(@transient val sparkContext: SparkContext) * of the RDD. */ @Experimental - def jdbcRDD(url: String, table: String, theParts: Array[String]): - DataFrame = { - val parts: Array[Partition] = theParts.zipWithIndex.map( - x => JDBCPartition(x._1, x._2).asInstanceOf[Partition]) + def jdbcRDD(url: String, table: String, theParts: Array[String]): DataFrame = { + val parts: Array[Partition] = theParts.zipWithIndex.map { case (part, i) => + JDBCPartition(part, i) : Partition + } jdbcRDD(url, table, parts) } - private def jdbcRDD(url: String, table: String, parts: Array[Partition]): - DataFrame = { + private def jdbcRDD(url: String, table: String, parts: Array[Partition]): DataFrame = { val relation = JDBCRelation(url, table, parts)(this) baseRelationToDataFrame(relation) } /** - * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only + * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. - * - * @group userf */ - def registerRDDAsTable(rdd: DataFrame, tableName: String): Unit = { + private[sql] def registerRDDAsTable(rdd: DataFrame, tableName: String): Unit = { catalog.registerTable(Seq(tableName), rdd.logicalPlan) } @@ -763,7 +790,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @param tableName the name of the table to be unregistered. * - * @group userf + * @group ddl_ops */ def dropTempTable(tableName: String): Unit = { cacheManager.tryUncacheQuery(table(tableName)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index d8b0a3b26dbab..8051df299252c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -32,9 +32,9 @@ import org.apache.spark.sql.types.DataType /** - * Functions for registering user-defined functions. + * Functions for registering user-defined functions. Use [[SQLContext.udf]] to access this. */ -class UDFRegistration(sqlContext: SQLContext) extends Logging { +class UDFRegistration private[sql] (sqlContext: SQLContext) extends Logging { private val functionRegistry = sqlContext.functionRegistry diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 4a0ec0b72ce81..2a1e086891423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} +import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions._ @@ -27,8 +28,15 @@ import org.apache.spark.sql.types._ /** - * Domain specific functions available for [[DataFrame]]. + * :: Experimental :: + * Functions available for [[DataFrame]]. + * + * @groupname udf_funcs UDF functions + * @groupname agg_funcs Aggregate functions + * @groupname normal_funcs Non-aggregate functions + * @groupname Ungrouped Support functions for DataFrames. */ +@Experimental // scalastyle:off object functions { // scalastyle:on @@ -37,11 +45,15 @@ object functions { /** * Returns a [[Column]] based on the given column name. + * + * @group normal_funcs */ def col(colName: String): Column = Column(colName) /** * Returns a [[Column]] based on the given column name. Alias of [[col]]. + * + * @group normal_funcs */ def column(colName: String): Column = Column(colName) @@ -51,6 +63,8 @@ object functions { * The passed in object is returned directly if it is already a [[Column]]. * If the object is a Scala Symbol, it is converted into a [[Column]] also. * Otherwise, a new [[Column]] is created to represent the literal value. + * + * @group normal_funcs */ def lit(literal: Any): Column = { literal match { @@ -84,80 +98,168 @@ object functions { ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// - /** Aggregate function: returns the sum of all values in the expression. */ + /** + * Aggregate function: returns the sum of all values in the expression. + * + * @group agg_funcs + */ def sum(e: Column): Column = Sum(e.expr) - /** Aggregate function: returns the sum of all values in the given column. */ + /** + * Aggregate function: returns the sum of all values in the given column. + * + * @group agg_funcs + */ def sum(columnName: String): Column = sum(Column(columnName)) - /** Aggregate function: returns the sum of distinct values in the expression. */ + /** + * Aggregate function: returns the sum of distinct values in the expression. + * + * @group agg_funcs + */ def sumDistinct(e: Column): Column = SumDistinct(e.expr) - /** Aggregate function: returns the sum of distinct values in the expression. */ + /** + * Aggregate function: returns the sum of distinct values in the expression. + * + * @group agg_funcs + */ def sumDistinct(columnName: String): Column = sumDistinct(Column(columnName)) - /** Aggregate function: returns the number of items in a group. */ + /** + * Aggregate function: returns the number of items in a group. + * + * @group agg_funcs + */ def count(e: Column): Column = e.expr match { // Turn count(*) into count(1) case s: Star => Count(Literal(1)) case _ => Count(e.expr) } - /** Aggregate function: returns the number of items in a group. */ + /** + * Aggregate function: returns the number of items in a group. + * + * @group agg_funcs + */ def count(columnName: String): Column = count(Column(columnName)) - /** Aggregate function: returns the number of distinct items in a group. */ + /** + * Aggregate function: returns the number of distinct items in a group. + * + * @group agg_funcs + */ @scala.annotation.varargs def countDistinct(expr: Column, exprs: Column*): Column = CountDistinct((expr +: exprs).map(_.expr)) - /** Aggregate function: returns the number of distinct items in a group. */ + /** + * Aggregate function: returns the number of distinct items in a group. + * + * @group agg_funcs + */ @scala.annotation.varargs def countDistinct(columnName: String, columnNames: String*): Column = countDistinct(Column(columnName), columnNames.map(Column.apply) :_*) - /** Aggregate function: returns the approximate number of distinct items in a group. */ + /** + * Aggregate function: returns the approximate number of distinct items in a group. + * + * @group agg_funcs + */ def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) - /** Aggregate function: returns the approximate number of distinct items in a group. */ + /** + * Aggregate function: returns the approximate number of distinct items in a group. + * + * @group agg_funcs + */ def approxCountDistinct(columnName: String): Column = approxCountDistinct(column(columnName)) - /** Aggregate function: returns the approximate number of distinct items in a group. */ + /** + * Aggregate function: returns the approximate number of distinct items in a group. + * + * @group agg_funcs + */ def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) - /** Aggregate function: returns the approximate number of distinct items in a group. */ + /** + * Aggregate function: returns the approximate number of distinct items in a group. + * + * @group agg_funcs + */ def approxCountDistinct(columnName: String, rsd: Double): Column = { approxCountDistinct(Column(columnName), rsd) } - /** Aggregate function: returns the average of the values in a group. */ + /** + * Aggregate function: returns the average of the values in a group. + * + * @group agg_funcs + */ def avg(e: Column): Column = Average(e.expr) - /** Aggregate function: returns the average of the values in a group. */ + /** + * Aggregate function: returns the average of the values in a group. + * + * @group agg_funcs + */ def avg(columnName: String): Column = avg(Column(columnName)) - /** Aggregate function: returns the first value in a group. */ + /** + * Aggregate function: returns the first value in a group. + * + * @group agg_funcs + */ def first(e: Column): Column = First(e.expr) - /** Aggregate function: returns the first value of a column in a group. */ + /** + * Aggregate function: returns the first value of a column in a group. + * + * @group agg_funcs + */ def first(columnName: String): Column = first(Column(columnName)) - /** Aggregate function: returns the last value in a group. */ + /** + * Aggregate function: returns the last value in a group. + * + * @group agg_funcs + */ def last(e: Column): Column = Last(e.expr) - /** Aggregate function: returns the last value of the column in a group. */ + /** + * Aggregate function: returns the last value of the column in a group. + * + * @group agg_funcs + */ def last(columnName: String): Column = last(Column(columnName)) - /** Aggregate function: returns the minimum value of the expression in a group. */ + /** + * Aggregate function: returns the minimum value of the expression in a group. + * + * @group agg_funcs + */ def min(e: Column): Column = Min(e.expr) - /** Aggregate function: returns the minimum value of the column in a group. */ + /** + * Aggregate function: returns the minimum value of the column in a group. + * + * @group agg_funcs + */ def min(columnName: String): Column = min(Column(columnName)) - /** Aggregate function: returns the maximum value of the expression in a group. */ + /** + * Aggregate function: returns the maximum value of the expression in a group. + * + * @group agg_funcs + */ def max(e: Column): Column = Max(e.expr) - /** Aggregate function: returns the maximum value of the column in a group. */ + /** + * Aggregate function: returns the maximum value of the column in a group. + * + * @group agg_funcs + */ def max(columnName: String): Column = max(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// @@ -168,6 +270,8 @@ object functions { * {{{ * df.select(coalesce(df("a"), df("b"))) * }}} + * + * @group normal_funcs */ @scala.annotation.varargs def coalesce(e: Column*): Column = Coalesce(e.map(_.expr)) @@ -182,6 +286,8 @@ object functions { * // Java: * df.select( negate(df.col("amount")) ); * }}} + * + * @group normal_funcs */ def negate(e: Column): Column = -e @@ -194,19 +300,37 @@ object functions { * // Java: * df.filter( not(df.col("isActive")) ); * }} + * + * @group normal_funcs */ def not(e: Column): Column = !e - /** Converts a string expression to upper case. */ + /** + * Converts a string expression to upper case. + * + * @group normal_funcs + */ def upper(e: Column): Column = Upper(e.expr) - /** Converts a string exprsesion to lower case. */ + /** + * Converts a string exprsesion to lower case. + * + * @group normal_funcs + */ def lower(e: Column): Column = Lower(e.expr) - /** Computes the square root of the specified float value. */ + /** + * Computes the square root of the specified float value. + * + * @group normal_funcs + */ def sqrt(e: Column): Column = Sqrt(e.expr) - /** Computes the absolutle value. */ + /** + * Computes the absolutle value. + * + * @group normal_funcs + */ def abs(e: Column): Column = Abs(e.expr) ////////////////////////////////////////////////////////////////////////////////////////////// @@ -222,6 +346,8 @@ object functions { /** * Defines a user-defined function of ${x} arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -236,6 +362,8 @@ object functions { /** * Call a Scala function of ${x} arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function$x[$fTypes], returnType: DataType${if (args.length > 0) ", " + args else ""}): Column = { ScalaUdf(f, returnType, Seq($argsInUdf)) @@ -246,6 +374,8 @@ object functions { /** * Defines a user-defined function of 0 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -254,6 +384,8 @@ object functions { /** * Defines a user-defined function of 1 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -262,6 +394,8 @@ object functions { /** * Defines a user-defined function of 2 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -270,6 +404,8 @@ object functions { /** * Defines a user-defined function of 3 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -278,6 +414,8 @@ object functions { /** * Defines a user-defined function of 4 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -286,6 +424,8 @@ object functions { /** * Defines a user-defined function of 5 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -294,6 +434,8 @@ object functions { /** * Defines a user-defined function of 6 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -302,6 +444,8 @@ object functions { /** * Defines a user-defined function of 7 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -310,6 +454,8 @@ object functions { /** * Defines a user-defined function of 8 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -318,6 +464,8 @@ object functions { /** * Defines a user-defined function of 9 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -326,6 +474,8 @@ object functions { /** * Defines a user-defined function of 10 arguments as user-defined function (UDF). * The data types are automatically inferred based on the function's signature. + * + * @group udf_funcs */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) @@ -336,6 +486,8 @@ object functions { /** * Call a Scala function of 0 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function0[_], returnType: DataType): Column = { ScalaUdf(f, returnType, Seq()) @@ -344,6 +496,8 @@ object functions { /** * Call a Scala function of 1 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function1[_, _], returnType: DataType, arg1: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr)) @@ -352,6 +506,8 @@ object functions { /** * Call a Scala function of 2 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function2[_, _, _], returnType: DataType, arg1: Column, arg2: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr)) @@ -360,6 +516,8 @@ object functions { /** * Call a Scala function of 3 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function3[_, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr)) @@ -368,6 +526,8 @@ object functions { /** * Call a Scala function of 4 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function4[_, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr)) @@ -376,6 +536,8 @@ object functions { /** * Call a Scala function of 5 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function5[_, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr)) @@ -384,6 +546,8 @@ object functions { /** * Call a Scala function of 6 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function6[_, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr)) @@ -392,6 +556,8 @@ object functions { /** * Call a Scala function of 7 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function7[_, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr)) @@ -400,6 +566,8 @@ object functions { /** * Call a Scala function of 8 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function8[_, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr)) @@ -408,6 +576,8 @@ object functions { /** * Call a Scala function of 9 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function9[_, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr)) @@ -416,6 +586,8 @@ object functions { /** * Call a Scala function of 10 arguments as user-defined function (UDF). This requires * you to specify the return data type. + * + * @group udf_funcs */ def callUDF(f: Function10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column): Column = { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index d2c39ab621713..e63cea60457d9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -143,7 +143,7 @@ private[hive] trait HiveStrategies { PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil } else { hiveContext - .parquetFile(partitionLocations.head, partitionLocations.tail: _*) + .parquetFile(partitionLocations: _*) .addPartitioningAttributes(relation.partitionKeys) .lowerCase .where(unresolvedOtherPredicates) @@ -152,6 +152,7 @@ private[hive] trait HiveStrategies { .executedPlan .fakeOutput(projectList.map(_.toAttribute)) :: Nil } + } else { hiveContext .parquetFile(relation.hiveQlTable.getDataLocation.toString) From ac6fe67e1d8bf01ee565f9cc09ad48d88a275829 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 16 Feb 2015 20:32:03 -0800 Subject: [PATCH 151/817] [SPARK-5363] [PySpark] check ending mark in non-block way There is chance of dead lock that the Python process is waiting for ending mark from JVM, but which is eaten by corrupted stream. This PR checks the ending mark from Python in non-block way, so it will not blocked by Python process. There is a small chance that the ending mark is sent by Python process but not available right now, then Python process will not be used. cc JoshRosen pwendell Author: Davies Liu Closes #4601 from davies/freeze and squashes the following commits: e15a8c3 [Davies Liu] update logging 890329c [Davies Liu] Merge branch 'freeze' of github.com:davies/spark into freeze 2bd2228 [Davies Liu] add more logging 656d544 [Davies Liu] Update PythonRDD.scala 05e1085 [Davies Liu] check ending mark in non-block way --- .../apache/spark/api/python/PythonRDD.scala | 21 +++++++++++++++---- python/pyspark/worker.py | 1 + 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 252721192904f..c3c8336a437a6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -144,11 +144,24 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } + // Check whether the worker is ready to be re-used. - if (stream.readInt() == SpecialLengths.END_OF_STREAM) { - if (reuse_worker) { - env.releasePythonWorker(pythonExec, envVars.toMap, worker) - released = true + if (reuse_worker) { + // It has a high possibility that the ending mark is already available, + // And current task should not be blocked by checking it + + if (stream.available() >= 4) { + val ending = stream.readInt() + if (ending == SpecialLengths.END_OF_STREAM) { + env.releasePythonWorker(pythonExec, envVars.toMap, worker) + released = true + logInfo(s"Communication with worker ended cleanly, re-use it: $worker") + } else { + logInfo(s"Communication with worker did not end cleanly (ending with $ending), " + + s"close it: $worker") + } + } else { + logInfo(s"The ending mark from worker is not available, close it: $worker") } } null diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8a93c320ec5d3..180bdbb4c2c4f 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -121,6 +121,7 @@ def process(): write_int(len(_accumulatorRegistry), outfile) for (aid, accum) in _accumulatorRegistry.items(): pickleSer._write_with_length((aid, accum._value), outfile) + outfile.flush() # check end of stream if read_int(infile) == SpecialLengths.END_OF_STREAM: From a51d51ffac00931c80ce93889a98c2f77aef8953 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 16 Feb 2015 20:33:33 -0800 Subject: [PATCH 152/817] SPARK-5850: Remove experimental label for Scala 2.11 and FlumePollingStream Author: Patrick Wendell Closes #4638 from pwendell/SPARK-5850 and squashes the following commits: 386126f [Patrick Wendell] SPARK-5850: Remove experimental label for Scala 2.11 and FlumePollingStream. --- docs/building-spark.md | 6 +++--- docs/streaming-flume-integration.md | 2 +- .../org/apache/spark/streaming/flume/FlumeUtils.scala | 8 -------- 3 files changed, 4 insertions(+), 12 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 088da7da4980e..4c3988e819ad8 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -111,9 +111,9 @@ To produce a Spark package compiled with Scala 2.11, use the `-Dscala-2.11` prop dev/change-version-to-2.11.sh mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package -Scala 2.11 support in Spark is experimental and does not support a few features. -Specifically, Spark's external Kafka library and JDBC component are not yet -supported in Scala 2.11 builds. +Scala 2.11 support in Spark does not support a few features due to dependencies +which are themselves not Scala 2.11 ready. Specifically, Spark's external +Kafka library and JDBC component are not yet supported in Scala 2.11 builds. # Spark Tests in Maven diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index ac01dd3d8019a..40e17246fea83 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -64,7 +64,7 @@ configuring Flume agents. 3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). -## Approach 2 (Experimental): Pull-based Approach using a Custom Sink +## Approach 2: Pull-based Approach using a Custom Sink Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following. - Flume pushes data into the sink, and the data stays buffered. diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 4b732c1592ab2..44dec45c227ca 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress -import org.apache.spark.annotation.Experimental import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} @@ -121,7 +120,6 @@ object FlumeUtils { * @param port Port of the host at which the Spark Sink is listening * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( ssc: StreamingContext, hostname: String, @@ -138,7 +136,6 @@ object FlumeUtils { * @param addresses List of InetSocketAddresses representing the hosts to connect to. * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( ssc: StreamingContext, addresses: Seq[InetSocketAddress], @@ -159,7 +156,6 @@ object FlumeUtils { * result in this stream using more threads * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( ssc: StreamingContext, addresses: Seq[InetSocketAddress], @@ -178,7 +174,6 @@ object FlumeUtils { * @param hostname Hostname of the host on which the Spark Sink is running * @param port Port of the host at which the Spark Sink is listening */ - @Experimental def createPollingStream( jssc: JavaStreamingContext, hostname: String, @@ -195,7 +190,6 @@ object FlumeUtils { * @param port Port of the host at which the Spark Sink is listening * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( jssc: JavaStreamingContext, hostname: String, @@ -212,7 +206,6 @@ object FlumeUtils { * @param addresses List of InetSocketAddresses on which the Spark Sink is running. * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( jssc: JavaStreamingContext, addresses: Array[InetSocketAddress], @@ -233,7 +226,6 @@ object FlumeUtils { * result in this stream using more threads * @param storageLevel Storage level to use for storing the received objects */ - @Experimental def createPollingStream( jssc: JavaStreamingContext, addresses: Array[InetSocketAddress], From d380f324c6d38ffacfda83a525a1a7e23347e5b8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 16 Feb 2015 20:42:57 -0800 Subject: [PATCH 153/817] [SPARK-5853][SQL] Schema support in Row. Author: Reynold Xin Closes #4640 from rxin/SPARK-5853 and squashes the following commits: 9c6f569 [Reynold Xin] [SPARK-5853][SQL] Schema support in Row. --- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 7 ++++++- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 6 +++--- .../org/apache/spark/sql/catalyst/expressions/rows.scala | 6 +++++- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 6 ++++++ 4 files changed, 20 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 3a70d25534968..d794f034f5578 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.DateUtils +import org.apache.spark.sql.types.{StructType, DateUtils} object Row { /** @@ -122,6 +122,11 @@ trait Row extends Serializable { /** Number of elements in the Row. */ def length: Int + /** + * Schema for the row. + */ + def schema: StructType = null + /** * Returns the value at position i. If the value is null, null is returned. The following * is a mapping between Spark SQL types and return types: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 11fd443733658..d6126c24fc50d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst import java.sql.Timestamp import org.apache.spark.util.Utils -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ @@ -91,9 +91,9 @@ trait ScalaReflection { def convertRowToScala(r: Row, schema: StructType): Row = { // TODO: This is very slow!!! - new GenericRow( + new GenericRowWithSchema( r.toSeq.zip(schema.fields.map(_.dataType)) - .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray) + .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray, schema) } /** Returns a Sequence of attributes for the given case class type. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 73ec7a6d114f5..faa366771824b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.{StructType, NativeType} /** @@ -149,6 +149,10 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { def copy() = this } +class GenericRowWithSchema(values: Array[Any], override val schema: StructType) + extends GenericRow(values) { +} + class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { /** No-arg constructor for serialization. */ def this() = this(null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 524571d9cc636..0da619def1d09 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -89,6 +89,12 @@ class DataFrameSuite extends QueryTest { testData.collect().toSeq) } + test("head and take") { + assert(testData.take(2) === testData.collect().take(2)) + assert(testData.head(2) === testData.collect().take(2)) + assert(testData.head(2).head.schema === testData.schema) + } + test("self join") { val df1 = testData.select(testData("key")).as('df1) val df2 = testData.select(testData("key")).as('df2) From fd84229e2aeb6a03760703c9dccd2db853779400 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 16 Feb 2015 22:09:04 -0800 Subject: [PATCH 154/817] [SPARK-5802][MLLIB] cache transformed data in glm If we need to transform the input data, we should cache the output to avoid re-computing feature vectors every iteration. dbtsai Author: Xiangrui Meng Closes #4593 from mengxr/SPARK-5802 and squashes the following commits: ae3be84 [Xiangrui Meng] cache transformed data in glm --- .../GeneralizedLinearAlgorithm.scala | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 17de215b97f9d..2b7145362ab0b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -205,7 +205,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] throw new SparkException("Input validation failed.") } - /** + /* * Scaling columns to unit variance as a heuristic to reduce the condition number: * * During the optimization process, the convergence (rate) depends on the condition number of @@ -225,26 +225,27 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Currently, it's only enabled in LogisticRegressionWithLBFGS */ val scaler = if (useFeatureScaling) { - (new StandardScaler(withStd = true, withMean = false)).fit(input.map(x => x.features)) + new StandardScaler(withStd = true, withMean = false).fit(input.map(_.features)) } else { null } // Prepend an extra variable consisting of all 1.0's for the intercept. - val data = if (addIntercept) { - if (useFeatureScaling) { - input.map(labeledPoint => - (labeledPoint.label, appendBias(scaler.transform(labeledPoint.features)))) - } else { - input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features))) - } - } else { - if (useFeatureScaling) { - input.map(labeledPoint => (labeledPoint.label, scaler.transform(labeledPoint.features))) + // TODO: Apply feature scaling to the weight vector instead of input data. + val data = + if (addIntercept) { + if (useFeatureScaling) { + input.map(lp => (lp.label, appendBias(scaler.transform(lp.features)))).cache() + } else { + input.map(lp => (lp.label, appendBias(lp.features))).cache() + } } else { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + if (useFeatureScaling) { + input.map(lp => (lp.label, scaler.transform(lp.features))).cache() + } else { + input.map(lp => (lp.label, lp.features)) + } } - } /** * TODO: For better convergence, in logistic regression, the intercepts should be computed From c06e42f2c1e5fcf123b466efd27ee4cb53bbed3f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 16 Feb 2015 22:10:39 -0800 Subject: [PATCH 155/817] HOTFIX: Style issue causing build break Caused by #4601 --- .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index c3c8336a437a6..e94c390df8224 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -157,8 +157,8 @@ private[spark] class PythonRDD( released = true logInfo(s"Communication with worker ended cleanly, re-use it: $worker") } else { - logInfo(s"Communication with worker did not end cleanly (ending with $ending), " + - s"close it: $worker") + logInfo(s"Communication with worker did not end cleanly " + + s"(ending with $ending), close it: $worker") } } else { logInfo(s"The ending mark from worker is not available, close it: $worker") From a65766bf0244a41b793b9dc5fbdd2882664ad00e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 17 Feb 2015 10:45:18 +0000 Subject: [PATCH 156/817] [SPARK-5826][Streaming] Fix Configuration not serializable problem Author: jerryshao Closes #4612 from jerryshao/SPARK-5826 and squashes the following commits: 7ec71db [jerryshao] Remove transient for conf statement 88d84e6 [jerryshao] Fix Configuration not serializable problem --- .../apache/spark/streaming/dstream/FileInputDStream.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 6379b88527ec8..4f7db41abe76f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming.dstream import java.io.{IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import scala.reflect.ClassTag @@ -27,6 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} +import org.apache.spark.SerializableWritable import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.streaming._ import org.apache.spark.util.{TimeStampedHashMap, Utils} @@ -78,6 +78,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) extends InputDStream[(K, V)](ssc_) { + private val serializableConfOpt = conf.map(new SerializableWritable(_)) + // This is a def so that it works during checkpoint recovery: private def clock = ssc.scheduler.clock @@ -240,7 +242,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( /** Generate one RDD from an array of files */ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = { val fileRDDs = files.map(file =>{ - val rdd = conf match { + val rdd = serializableConfOpt.map(_.value) match { case Some(config) => context.sparkContext.newAPIHadoopFile( file, fm.runtimeClass.asInstanceOf[Class[F]], From ee6e3eff02e9e08b1113ba6faf3397d7e7775087 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 17 Feb 2015 07:48:27 -0800 Subject: [PATCH 157/817] Revert "[SPARK-5363] [PySpark] check ending mark in non-block way" This reverts commits ac6fe67e1d8bf01ee565f9cc09ad48d88a275829 and c06e42f2c1e5fcf123b466efd27ee4cb53bbed3f. --- .../apache/spark/api/python/PythonRDD.scala | 21 ++++--------------- python/pyspark/worker.py | 1 - 2 files changed, 4 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e94c390df8224..252721192904f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -144,24 +144,11 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } - // Check whether the worker is ready to be re-used. - if (reuse_worker) { - // It has a high possibility that the ending mark is already available, - // And current task should not be blocked by checking it - - if (stream.available() >= 4) { - val ending = stream.readInt() - if (ending == SpecialLengths.END_OF_STREAM) { - env.releasePythonWorker(pythonExec, envVars.toMap, worker) - released = true - logInfo(s"Communication with worker ended cleanly, re-use it: $worker") - } else { - logInfo(s"Communication with worker did not end cleanly " + - s"(ending with $ending), close it: $worker") - } - } else { - logInfo(s"The ending mark from worker is not available, close it: $worker") + if (stream.readInt() == SpecialLengths.END_OF_STREAM) { + if (reuse_worker) { + env.releasePythonWorker(pythonExec, envVars.toMap, worker) + released = true } } null diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 180bdbb4c2c4f..8a93c320ec5d3 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -121,7 +121,6 @@ def process(): write_int(len(_accumulatorRegistry), outfile) for (aid, accum) in _accumulatorRegistry.items(): pickleSer._write_with_length((aid, accum._value), outfile) - outfile.flush() # check end of stream if read_int(infile) == SpecialLengths.END_OF_STREAM: From 3ce46e94fe77d15f18e916b76b37fa96356ace93 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 Feb 2015 10:10:01 -0800 Subject: [PATCH 158/817] SPARK-5856: In Maven build script, launch Zinc with more memory I've seen out of memory exceptions when trying to run many parallel builds against the same Zinc server during packaging. We should use the same increased memory settings we use for Maven itself. I tested this and confirmed that the Nailgun JVM launched with higher memory. Author: Patrick Wendell Closes #4643 from pwendell/zinc-memory and squashes the following commits: 717cfb0 [Patrick Wendell] SPARK-5856: Launch Zinc with larger memory options. --- build/mvn | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build/mvn b/build/mvn index 53babf54debb6..3561110a4c019 100755 --- a/build/mvn +++ b/build/mvn @@ -21,6 +21,8 @@ _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" # Preserve the calling directory _CALLING_DIR="$(pwd)" +# Options used during compilation +_COMPILE_JVM_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" # Installs any application tarball given a URL, the expected tarball name, # and, optionally, a checkable binary path to determine if the binary has @@ -136,6 +138,7 @@ cd "${_CALLING_DIR}" # Now that zinc is ensured to be installed, check its status and, if its # not running or just installed, start it if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`${ZINC_BIN} -status`" ]; then + export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} ${ZINC_BIN} -shutdown ${ZINC_BIN} -start -port ${ZINC_PORT} \ -scala-compiler "${SCALA_COMPILER}" \ @@ -143,7 +146,7 @@ if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`${ZINC_BIN} -status`" ]; then fi # Set any `mvn` options if not already present -export MAVEN_OPTS=${MAVEN_OPTS:-"-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"} +export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} # Last, call the `mvn` command as usual ${MVN_BIN} "$@" From c76da36c2163276b5c34e59fbb139eeb34ed0faa Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 17 Feb 2015 10:17:45 -0800 Subject: [PATCH 159/817] [SPARK-5858][MLLIB] Remove unnecessary first() call in GLM `numFeatures` is only used by multinomial logistic regression. Calling `.first()` for every GLM causes performance regression, especially in Python. Author: Xiangrui Meng Closes #4647 from mengxr/SPARK-5858 and squashes the following commits: 036dc7f [Xiangrui Meng] remove unnecessary first() call 12c5548 [Xiangrui Meng] check numFeatures only once --- .../spark/mllib/classification/LogisticRegression.scala | 6 +++++- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 7 ++++--- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 420d6e2861934..b787667b018e6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -355,6 +355,10 @@ class LogisticRegressionWithLBFGS } override protected def createModel(weights: Vector, intercept: Double) = { - new LogisticRegressionModel(weights, intercept, numFeatures, numOfLinearPredictor + 1) + if (numOfLinearPredictor == 1) { + new LogisticRegressionModel(weights, intercept) + } else { + new LogisticRegressionModel(weights, intercept, numFeatures, numOfLinearPredictor + 1) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 2b7145362ab0b..7c66e8cdebdbe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -126,7 +126,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * The dimension of training features. */ - protected var numFeatures: Int = 0 + protected var numFeatures: Int = -1 /** * Set if the algorithm should use feature scaling to improve the convergence during optimization. @@ -163,7 +163,9 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * RDD of LabeledPoint entries. */ def run(input: RDD[LabeledPoint]): M = { - numFeatures = input.first().features.size + if (numFeatures < 0) { + numFeatures = input.map(_.features.size).first() + } /** * When `numOfLinearPredictor > 1`, the intercepts are encapsulated into weights, @@ -193,7 +195,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * of LabeledPoint entries starting from the initial weights provided. */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { - numFeatures = input.first().features.size if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" From c74b07fa94a8da50437d952ae05cf6ac70fbb93e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Feb 2015 10:21:17 -0800 Subject: [PATCH 160/817] [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation Author: Michael Armbrust Closes #4642 from marmbrus/docs and squashes the following commits: d291c34 [Michael Armbrust] python tests 9be66e3 [Michael Armbrust] comments d56afc2 [Michael Armbrust] fix style f004747 [Michael Armbrust] fix build c4a907b [Michael Armbrust] fix tests 42e2b73 [Michael Armbrust] [SQL] Documentation / API Clean-up. --- project/SparkBuild.scala | 12 +- python/pyspark/sql/context.py | 28 +-- .../org/apache/spark/sql/jdbc/JDBCUtils.java | 59 ------ .../org/apache/spark/sql/DataFrame.scala | 153 +++++++++++++- .../org/apache/spark/sql/DataFrameImpl.scala | 33 ++- .../spark/sql/ExperimentalMethods.scala | 5 + .../apache/spark/sql/IncomputableColumn.scala | 4 + .../org/apache/spark/sql/SQLContext.scala | 200 ++++++++++++++---- .../spark/sql/UserDefinedFunction.scala | 3 +- .../org/apache/spark/sql/api/package.scala | 23 ++ .../apache/spark/sql/execution/commands.scala | 2 +- .../spark/sql/execution/debug/package.scala | 10 +- .../org/apache/spark/sql/jdbc/jdbc.scala | 74 ++----- .../sql/parquet/ParquetTableOperations.scala | 4 +- .../spark/sql/parquet/ParquetTest.scala | 4 +- .../apache/spark/sql/parquet/newParquet.scala | 6 +- .../sql/parquet/timestamp/NanoTime.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 4 +- .../apache/spark/sql/jdbc/JavaJDBCTest.java | 102 --------- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 7 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 20 +- .../spark/sql/jdbc/MySQLIntegration.scala | 14 +- .../spark/sql/jdbc/PostgresIntegration.scala | 4 +- .../execution/DescribeHiveTableCommand.scala | 4 +- .../spark/sql/hive/execution/commands.scala | 8 + .../spark/sql/hive/execution/package.scala} | 19 +- .../org/apache/spark/sql/hive/package.scala | 10 + .../sql/hive/parquet/FakeParquetSerDe.scala | 56 ----- .../org/apache/spark/sql/hive/Shim12.scala | 9 +- .../org/apache/spark/sql/hive/Shim13.scala | 9 +- 30 files changed, 483 insertions(+), 405 deletions(-) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/package.scala delete mode 100644 sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java rename sql/{core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala => hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala} (66%) delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8fb1239b4a96b..e4b1b96527fbd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -361,9 +361,16 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, catalyst, streamingFlumeSink, yarn), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, streamingFlumeSink, yarn), + + // Skip actual catalyst, but include the subproject. + // Catalyst is not public API and contains quasiquotes which break scaladoc. + unidocAllSources in (ScalaUnidoc, unidoc) := { + (unidocAllSources in (ScalaUnidoc, unidoc)).value + .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) + }, // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { @@ -376,6 +383,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("executor"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) }, // Javadoc options: create a window title, and group key packages on index page diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index dd2cd5ee76f60..2e2309f10375d 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -252,7 +252,7 @@ def applySchema(self, rdd, schema): >>> schema = StructType([StructField("field1", IntegerType(), False), ... StructField("field2", StringType(), False)]) >>> df = sqlCtx.applySchema(rdd2, schema) - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.sql("SELECT * from table1") >>> df2.collect() [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] @@ -405,17 +405,17 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): return self.applySchema(data, schema) - def registerRDDAsTable(self, rdd, tableName): + def registerDataFrameAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. Temporary tables exist only during the lifetime of this instance of SQLContext. - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") """ if (rdd.__class__ is DataFrame): df = rdd._jdf - self._ssql_ctx.registerRDDAsTable(df, tableName) + self._ssql_ctx.registerDataFrameAsTable(df, tableName) else: raise ValueError("Can only register DataFrame as table") @@ -456,7 +456,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): ... print>>ofn, json >>> ofn.close() >>> df1 = sqlCtx.jsonFile(jsonFile) - >>> sqlCtx.registerRDDAsTable(df1, "table1") + >>> sqlCtx.registerDataFrameAsTable(df1, "table1") >>> df2 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " ... "field6 as f4 from table1") @@ -467,7 +467,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema) - >>> sqlCtx.registerRDDAsTable(df3, "table2") + >>> sqlCtx.registerDataFrameAsTable(df3, "table2") >>> df4 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " ... "field6 as f4 from table2") @@ -485,7 +485,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): ... StructField("field5", ... ArrayType(IntegerType(), False), True)]), False)]) >>> df5 = sqlCtx.jsonFile(jsonFile, schema) - >>> sqlCtx.registerRDDAsTable(df5, "table3") + >>> sqlCtx.registerDataFrameAsTable(df5, "table3") >>> df6 = sqlCtx.sql( ... "SELECT field2 AS f1, field3.field5 as f2, " ... "field3.field5[0] as f3 from table3") @@ -509,7 +509,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): determine the schema. >>> df1 = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerRDDAsTable(df1, "table1") + >>> sqlCtx.registerDataFrameAsTable(df1, "table1") >>> df2 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " ... "field6 as f4 from table1") @@ -520,7 +520,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> df3 = sqlCtx.jsonRDD(json, df1.schema) - >>> sqlCtx.registerRDDAsTable(df3, "table2") + >>> sqlCtx.registerDataFrameAsTable(df3, "table2") >>> df4 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " ... "field6 as f4 from table2") @@ -538,7 +538,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): ... StructField("field5", ... ArrayType(IntegerType(), False), True)]), False)]) >>> df5 = sqlCtx.jsonRDD(json, schema) - >>> sqlCtx.registerRDDAsTable(df5, "table3") + >>> sqlCtx.registerDataFrameAsTable(df5, "table3") >>> df6 = sqlCtx.sql( ... "SELECT field2 AS f1, field3.field5 as f2, " ... "field3.field5[0] as f3 from table3") @@ -628,7 +628,7 @@ def createExternalTable(self, tableName, path=None, source=None, def sql(self, sqlQuery): """Return a L{DataFrame} representing the result of the given query. - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] @@ -638,7 +638,7 @@ def sql(self, sqlQuery): def table(self, tableName): """Returns the specified table as a L{DataFrame}. - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) True @@ -653,7 +653,7 @@ def tables(self, dbName=None): The returned DataFrame has two columns, tableName and isTemporary (a column with BooleanType indicating if a table is a temporary one or not). - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.tables() >>> df2.filter("tableName = 'table1'").first() Row(tableName=u'table1', isTemporary=True) @@ -668,7 +668,7 @@ def tableNames(self, dbName=None): If `dbName` is not specified, the current database will be used. - >>> sqlCtx.registerRDDAsTable(df, "table1") + >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> "table1" in sqlCtx.tableNames() True >>> "table1" in sqlCtx.tableNames("db") diff --git a/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java b/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java deleted file mode 100644 index aa441b2096f18..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.jdbc; - -import org.apache.spark.Partition; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.DataFrame; - -public class JDBCUtils { - /** - * Construct a DataFrame representing the JDBC table at the database - * specified by url with table name table. - */ - public static DataFrame jdbcRDD(SQLContext sql, String url, String table) { - Partition[] parts = new Partition[1]; - parts[0] = new JDBCPartition(null, 0); - return sql.baseRelationToDataFrame( - new JDBCRelation(url, table, parts, sql)); - } - - /** - * Construct a DataFrame representing the JDBC table at the database - * specified by url with table name table partitioned by parts. - * Here, parts is an array of expressions suitable for insertion into a WHERE - * clause; each one defines one partition. - */ - public static DataFrame jdbcRDD(SQLContext sql, String url, String table, String[] parts) { - Partition[] partitions = new Partition[parts.length]; - for (int i = 0; i < parts.length; i++) - partitions[i] = new JDBCPartition(parts[i], i); - return sql.baseRelationToDataFrame( - new JDBCRelation(url, table, partitions, sql)); - } - - private static JavaJDBCTrampoline trampoline = new JavaJDBCTrampoline(); - - public static void createJDBCTable(DataFrame rdd, String url, String table, boolean allowExisting) { - trampoline.createJDBCTable(rdd, url, table, allowExisting); - } - - public static void insertIntoJDBC(DataFrame rdd, String url, String table, boolean overwrite) { - trampoline.insertIntoJDBC(rdd, url, table, overwrite); - } -} 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 c0c3cb40cf1da..fa5fe84263ece 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 @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.sql.DriverManager + + import scala.collection.JavaConversions._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag @@ -27,6 +30,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -77,6 +81,12 @@ private[sql] object DataFrame { * .groupBy(department("name"), "gender") * .agg(avg(people("salary")), max(people("age"))) * }}} + * + * @groupname basic Basic DataFrame functions + * @groupname dfops Language Integrated Queries + * @groupname rdd RDD Operations + * @groupname output Output Operations + * @groupname action Actions */ // TODO: Improve documentation. @Experimental @@ -102,7 +112,8 @@ trait DataFrame extends RDDApi[Row] with Serializable { def toSchemaRDD: DataFrame = this /** - * Returns the object itself. Used to force an implicit conversion from RDD to DataFrame in Scala. + * Returns the object itself. + * @group basic */ // This is declared with parentheses to prevent the Scala compiler from treating // `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. @@ -116,31 +127,51 @@ trait DataFrame extends RDDApi[Row] with Serializable { * rdd.toDF // this implicit conversion creates a DataFrame with column name _1 and _2 * rdd.toDF("id", "name") // this creates a DataFrame with column name "id" and "name" * }}} + * @group basic */ @scala.annotation.varargs def toDF(colNames: String*): DataFrame - /** Returns the schema of this [[DataFrame]]. */ + /** + * Returns the schema of this [[DataFrame]]. + * @group basic + */ def schema: StructType - /** Returns all column names and their data types as an array. */ + /** + * Returns all column names and their data types as an array. + * @group basic + */ def dtypes: Array[(String, String)] - /** Returns all column names as an array. */ + /** + * Returns all column names as an array. + * @group basic + */ def columns: Array[String] = schema.fields.map(_.name) - /** Prints the schema to the console in a nice tree format. */ + /** + * Prints the schema to the console in a nice tree format. + * @group basic + */ def printSchema(): Unit - /** Prints the plans (logical and physical) to the console for debugging purpose. */ + /** + * Prints the plans (logical and physical) to the console for debugging purpose. + * @group basic + */ def explain(extended: Boolean): Unit - /** Only prints the physical plan to the console for debugging purpose. */ + /** + * Only prints the physical plan to the console for debugging purpose. + * @group basic + */ def explain(): Unit = explain(extended = false) /** * Returns true if the `collect` and `take` methods can be run locally * (without any Spark executors). + * @group basic */ def isLocal: Boolean @@ -154,6 +185,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * 1983 03 0.410516 0.442194 * 1984 04 0.450090 0.483521 * }}} + * @group basic */ def show(): Unit @@ -163,6 +195,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Note that cartesian joins are very expensive without an extra filter that can be pushed down. * * @param right Right side of the join operation. + * @group dfops */ def join(right: DataFrame): DataFrame @@ -174,6 +207,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df1.join(df2, $"df1Key" === $"df2Key") * df1.join(df2).where($"df1Key" === $"df2Key") * }}} + * @group dfops */ def join(right: DataFrame, joinExprs: Column): DataFrame @@ -194,6 +228,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @param right Right side of the join. * @param joinExprs Join expression. * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. + * @group dfops */ def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame @@ -205,6 +240,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df.sort($"sortcol") * df.sort($"sortcol".asc) * }}} + * @group dfops */ @scala.annotation.varargs def sort(sortCol: String, sortCols: String*): DataFrame @@ -214,6 +250,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * {{{ * df.sort($"col1", $"col2".desc) * }}} + * @group dfops */ @scala.annotation.varargs def sort(sortExprs: Column*): DataFrame @@ -221,6 +258,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. + * @group dfops */ @scala.annotation.varargs def orderBy(sortCol: String, sortCols: String*): DataFrame @@ -228,27 +266,32 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. + * @group dfops */ @scala.annotation.varargs def orderBy(sortExprs: Column*): DataFrame /** * Selects column based on the column name and return it as a [[Column]]. + * @group dfops */ def apply(colName: String): Column = col(colName) /** * Selects column based on the column name and return it as a [[Column]]. + * @group dfops */ def col(colName: String): Column /** * Returns a new [[DataFrame]] with an alias set. + * @group dfops */ def as(alias: String): DataFrame /** * (Scala-specific) Returns a new [[DataFrame]] with an alias set. + * @group dfops */ def as(alias: Symbol): DataFrame @@ -257,6 +300,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * {{{ * df.select($"colA", $"colB" + 1) * }}} + * @group dfops */ @scala.annotation.varargs def select(cols: Column*): DataFrame @@ -270,6 +314,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df.select("colA", "colB") * df.select($"colA", $"colB") * }}} + * @group dfops */ @scala.annotation.varargs def select(col: String, cols: String*): DataFrame @@ -281,6 +326,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * {{{ * df.selectExpr("colA", "colB as newName", "abs(colC)") * }}} + * @group dfops */ @scala.annotation.varargs def selectExpr(exprs: String*): DataFrame @@ -293,6 +339,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * peopleDf.where($"age" > 15) * peopleDf($"age" > 15) * }}} + * @group dfops */ def filter(condition: Column): DataFrame @@ -301,6 +348,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * {{{ * peopleDf.filter("age > 15") * }}} + * @group dfops */ def filter(conditionExpr: String): DataFrame @@ -312,6 +360,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * peopleDf.where($"age" > 15) * peopleDf($"age" > 15) * }}} + * @group dfops */ def where(condition: Column): DataFrame @@ -329,6 +378,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * "age" -> "max" * )) * }}} + * @group dfops */ @scala.annotation.varargs def groupBy(cols: Column*): GroupedData @@ -350,6 +400,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * "age" -> "max" * )) * }}} + * @group dfops */ @scala.annotation.varargs def groupBy(col1: String, cols: String*): GroupedData @@ -366,6 +417,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * "expense" -> "sum" * ) * }}} + * @group dfops */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { groupBy().agg(aggExpr, aggExprs :_*) @@ -378,6 +430,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} + * @group dfops */ def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) @@ -388,6 +441,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} + * @group dfops */ def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs) @@ -398,6 +452,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * df.agg(max($"age"), avg($"salary")) * df.groupBy().agg(max($"age"), avg($"salary")) * }} + * @group dfops */ @scala.annotation.varargs def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs :_*) @@ -405,24 +460,28 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function * and `head` is that `head` returns an array while `limit` returns a new [[DataFrame]]. + * @group dfops */ def limit(n: Int): DataFrame /** * Returns a new [[DataFrame]] containing union of rows in this frame and another frame. * This is equivalent to `UNION ALL` in SQL. + * @group dfops */ def unionAll(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] containing rows only in both this frame and another frame. * This is equivalent to `INTERSECT` in SQL. + * @group dfops */ def intersect(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] containing rows in this frame but not in another frame. * This is equivalent to `EXCEPT` in SQL. + * @group dfops */ def except(other: DataFrame): DataFrame @@ -432,6 +491,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. * @param seed Seed for sampling. + * @group dfops */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame @@ -440,6 +500,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. + * @group dfops */ def sample(withReplacement: Boolean, fraction: Double): DataFrame = { sample(withReplacement, fraction, Utils.random.nextLong) @@ -464,6 +525,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * * val bookCountPerWord = allWords.groupBy("word").agg(countDistinct("title")) * }}} + * @group dfops */ def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame @@ -476,6 +538,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * {{{ * df.explode("words", "word")(words: String => words.split(" ")) * }}} + * @group dfops */ def explode[A, B : TypeTag]( inputColumn: String, @@ -486,11 +549,13 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new [[DataFrame]] by adding a column. + * @group dfops */ def withColumn(colName: String, col: Column): DataFrame /** * Returns a new [[DataFrame]] with a column renamed. + * @group dfops */ def withColumnRenamed(existingName: String, newName: String): DataFrame @@ -511,62 +576,84 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns a new RDD by applying a function to all rows of this DataFrame. + * @group rdd */ override def map[R: ClassTag](f: Row => R): RDD[R] /** * Returns a new RDD by first applying a function to all rows of this [[DataFrame]], * and then flattening the results. + * @group rdd */ override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] /** * Returns a new RDD by applying a function to each partition of this DataFrame. + * @group rdd */ override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] /** * Applies a function `f` to all rows. + * @group rdd */ override def foreach(f: Row => Unit): Unit /** * Applies a function f to each partition of this [[DataFrame]]. + * @group rdd */ override def foreachPartition(f: Iterator[Row] => Unit): Unit /** * Returns the first `n` rows in the [[DataFrame]]. + * @group action */ override def take(n: Int): Array[Row] /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. + * @group action */ override def collect(): Array[Row] /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. + * @group action */ override def collectAsList(): java.util.List[Row] /** * Returns the number of rows in the [[DataFrame]]. + * @group action */ override def count(): Long /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. + * @group rdd */ override def repartition(numPartitions: Int): DataFrame - /** Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. */ + /** + * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. + * @group dfops + */ override def distinct: DataFrame + /** + * @group basic + */ override def persist(): this.type + /** + * @group basic + */ override def persist(newLevel: StorageLevel): this.type + /** + * @group basic + */ override def unpersist(blocking: Boolean): this.type ///////////////////////////////////////////////////////////////////////////// @@ -575,16 +662,19 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. + * @group rdd */ def rdd: RDD[Row] /** * Returns the content of the [[DataFrame]] as a [[JavaRDD]] of [[Row]]s. + * @group rdd */ def toJavaRDD: JavaRDD[Row] = rdd.toJavaRDD() /** * Returns the content of the [[DataFrame]] as a [[JavaRDD]] of [[Row]]s. + * @group rdd */ def javaRDD: JavaRDD[Row] = toJavaRDD @@ -592,7 +682,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Registers this RDD as a temporary table using the given name. The lifetime of this temporary * table is tied to the [[SQLContext]] that was used to create this DataFrame. * - * @group schema + * @group basic */ def registerTempTable(tableName: String): Unit @@ -600,6 +690,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Saves the contents of this [[DataFrame]] as a parquet file, preserving the schema. * Files that are written out using this method can be read back in as a [[DataFrame]] * using the `parquetFile` function in [[SQLContext]]. + * @group output */ def saveAsParquetFile(path: String): Unit @@ -613,6 +704,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable(tableName: String): Unit = { @@ -628,6 +720,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable(tableName: String, mode: SaveMode): Unit = { @@ -651,6 +744,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable( @@ -668,6 +762,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable( @@ -686,6 +781,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable( @@ -706,6 +802,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * there is no notion of a persisted catalog in a standard SQL context. Instead you can write * an RDD out to a parquet file, and then register that file as a table. This "table" can then * be the target of an `insertInto`. + * @group output */ @Experimental def saveAsTable( @@ -719,6 +816,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Saves the contents of this DataFrame to the given path, * using the default data source configured by spark.sql.sources.default and * [[SaveMode.ErrorIfExists]] as the save mode. + * @group output */ @Experimental def save(path: String): Unit = { @@ -729,6 +827,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * :: Experimental :: * Saves the contents of this DataFrame to the given path and [[SaveMode]] specified by mode, * using the default data source configured by spark.sql.sources.default. + * @group output */ @Experimental def save(path: String, mode: SaveMode): Unit = { @@ -740,6 +839,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * :: Experimental :: * Saves the contents of this DataFrame to the given path based on the given data source, * using [[SaveMode.ErrorIfExists]] as the save mode. + * @group output */ @Experimental def save(path: String, source: String): Unit = { @@ -750,6 +850,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * :: Experimental :: * Saves the contents of this DataFrame to the given path based on the given data source and * [[SaveMode]] specified by mode. + * @group output */ @Experimental def save(path: String, source: String, mode: SaveMode): Unit = { @@ -760,6 +861,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * :: Experimental :: * Saves the contents of this DataFrame based on the given data source, * [[SaveMode]] specified by mode, and a set of options. + * @group output */ @Experimental def save( @@ -774,6 +876,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * (Scala-specific) * Saves the contents of this DataFrame based on the given data source, * [[SaveMode]] specified by mode, and a set of options + * @group output */ @Experimental def save( @@ -784,6 +887,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. + * @group output */ @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit @@ -792,15 +896,46 @@ trait DataFrame extends RDDApi[Row] with Serializable { * :: Experimental :: * Adds the rows from this RDD to the specified table. * Throws an exception if the table already exists. + * @group output */ @Experimental def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. + * @group rdd */ def toJSON: RDD[String] + //////////////////////////////////////////////////////////////////////////// + // JDBC Write Support + //////////////////////////////////////////////////////////////////////////// + + /** + * Save this RDD to a JDBC database at `url` under the table name `table`. + * This will run a `CREATE TABLE` and a bunch of `INSERT INTO` statements. + * If you pass `true` for `allowExisting`, it will drop any table with the + * given name; if you pass `false`, it will throw if the table already + * exists. + * @group output + */ + def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit + + /** + * Save this RDD to a JDBC database at `url` under the table name `table`. + * Assumes the table already exists and has a compatible schema. If you + * pass `true` for `overwrite`, it will `TRUNCATE` the table before + * performing the `INSERT`s. + * + * The table must already exist on the database. It must have a schema + * that is compatible with the schema of this RDD; inserting the rows of + * the RDD in order via the simple statement + * `INSERT INTO table VALUES (?, ?, ..., ?)` should not fail. + * @group output + */ + def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit + + //////////////////////////////////////////////////////////////////////////// // for Python API //////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala index 848ea2e05624f..25bc9d929237d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.io.CharArrayWriter +import java.sql.DriverManager import scala.language.implicitConversions import scala.reflect.ClassTag @@ -36,6 +37,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExplainCommand, LogicalRDD, EvaluatePython} +import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{NumericType, StructType} @@ -375,7 +377,7 @@ private[sql] class DataFrameImpl protected[sql]( } override def registerTempTable(tableName: String): Unit = { - sqlContext.registerRDDAsTable(this, tableName) + sqlContext.registerDataFrameAsTable(this, tableName) } override def saveAsParquetFile(path: String): Unit = { @@ -441,6 +443,35 @@ private[sql] class DataFrameImpl protected[sql]( } } + def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit = { + val conn = DriverManager.getConnection(url) + try { + if (allowExisting) { + val sql = s"DROP TABLE IF EXISTS $table" + conn.prepareStatement(sql).executeUpdate() + } + val schema = JDBCWriteDetails.schemaString(this, url) + val sql = s"CREATE TABLE $table ($schema)" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + JDBCWriteDetails.saveTable(this, url, table) + } + + def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit = { + if (overwrite) { + val conn = DriverManager.getConnection(url) + try { + val sql = s"TRUNCATE TABLE $table" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + } + JDBCWriteDetails.saveTable(this, url, table) + } + //////////////////////////////////////////////////////////////////////////// // for Python API //////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index f0e6a8f332188..d5d7e35a6b35d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -20,8 +20,13 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * Holder for experimental methods for the bravest. We make NO guarantee about the stability * regarding binary compatibility and source compatibility of methods here. + * + * {{{ + * sqlContext.experimental.extraStrategies += ... + * }}} */ @Experimental class ExperimentalMethods protected[sql](sqlContext: SQLContext) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala index fc37cfa7a899f..b48b682b36e1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -173,6 +173,10 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def insertInto(tableName: String, overwrite: Boolean): Unit = err() + def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit = err() + + def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit = err() + override def toJSON: RDD[String] = err() protected[sql] override def javaToPython: JavaRDD[Array[Byte]] = err() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0aae0942ca04f..31afa0eb59a8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -43,11 +43,16 @@ import org.apache.spark.util.Utils import org.apache.spark.{Partition, SparkContext} /** - * The entry point for running relational queries using Spark. Allows the creation of [[DataFrame]] - * objects and the execution of SQL queries. + * The entry point for working with structured data (rows and columns) in Spark. Allows the + * creation of [[DataFrame]] objects as well as the execution of SQL queries. * - * @groupname ddl_ops Catalog DDL functions - * @groupname userf Spark SQL Functions + * @groupname basic Basic Operations + * @groupname ddl_ops Persistent Catalog DDL + * @groupname cachemgmt Cached Table Management + * @groupname genericdata Generic Data Sources + * @groupname specificdata Specific Data Sources + * @groupname config Configuration + * @groupname dataframes Custom DataFrame Creation * @groupname Ungrouped Support functions for language integrated queries. */ class SQLContext(@transient val sparkContext: SparkContext) @@ -61,24 +66,40 @@ class SQLContext(@transient val sparkContext: SparkContext) // Note that this is a lazy val so we can override the default value in subclasses. protected[sql] lazy val conf: SQLConf = new SQLConf - /** Set Spark SQL configuration properties. */ + /** + * Set Spark SQL configuration properties. + * + * @group config + */ def setConf(props: Properties): Unit = conf.setConf(props) - /** Set the given Spark SQL configuration property. */ + /** + * Set the given Spark SQL configuration property. + * + * @group config + */ def setConf(key: String, value: String): Unit = conf.setConf(key, value) - /** Return the value of Spark SQL configuration property for the given key. */ + /** + * Return the value of Spark SQL configuration property for the given key. + * + * @group config + */ def getConf(key: String): String = conf.getConf(key) /** * Return the value of Spark SQL configuration property for the given key. If the key is not set * yet, return `defaultValue`. + * + * @group config */ def getConf(key: String, defaultValue: String): String = conf.getConf(key, defaultValue) /** * Return all the configuration properties that have been set (i.e. not the default). * This creates a new copy of the config properties in the form of a Map. + * + * @group config */ def getAllConfs: immutable.Map[String, String] = conf.getAllConfs @@ -128,7 +149,9 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: * A collection of methods that are considered experimental, but can be used to hook into - * the query planner for advanced functionalities. + * the query planner for advanced functionality. + * + * @group basic */ @Experimental @transient @@ -137,6 +160,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: * Returns a [[DataFrame]] with no rows or columns. + * + * @group basic */ @Experimental @transient @@ -167,17 +192,28 @@ class SQLContext(@transient val sparkContext: SparkContext) * (Integer arg1, String arg2) -> arg2 + arg1), * DataTypes.StringType); * }}} + * + * @group basic */ @transient val udf: UDFRegistration = new UDFRegistration(this) - /** Returns true if the table is currently cached in-memory. */ + /** + * Returns true if the table is currently cached in-memory. + * @group cachemgmt + */ def isCached(tableName: String): Boolean = cacheManager.isCached(tableName) - /** Caches the specified table in-memory. */ + /** + * Caches the specified table in-memory. + * @group cachemgmt + */ def cacheTable(tableName: String): Unit = cacheManager.cacheTable(tableName) - /** Removes the specified table from the in-memory cache. */ + /** + * Removes the specified table from the in-memory cache. + * @group cachemgmt + */ def uncacheTable(tableName: String): Unit = cacheManager.uncacheTable(tableName) // scalastyle:off @@ -186,6 +222,13 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * (Scala-specific) Implicit methods available in Scala for converting * common Scala objects into [[DataFrame]]s. + * + * {{{ + * val sqlContext = new SQLContext + * import sqlContext._ + * }}} + * + * @group basic */ @Experimental object implicits extends Serializable { @@ -260,7 +303,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Creates a DataFrame from an RDD of case classes. * - * @group userf + * @group dataframes */ @Experimental def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { @@ -274,6 +317,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: * Creates a DataFrame from a local Seq of Product. + * + * @group dataframes */ @Experimental def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { @@ -285,6 +330,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Convert a [[BaseRelation]] created for external data sources into a [[DataFrame]]. + * + * @group dataframes */ def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = { DataFrame(this, LogicalRelation(baseRelation)) @@ -318,6 +365,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * dataFrame.registerTempTable("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} + * + * @group dataframes */ @DeveloperApi def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { @@ -332,6 +381,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s using the given schema. * It is important to make sure that the structure of every [[Row]] of the provided RDD matches * the provided schema. Otherwise, there will be runtime exception. + * + * @group dataframes */ @DeveloperApi def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { @@ -346,6 +397,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @param rowRDD an JavaRDD of Row * @param columns names for each column * @return DataFrame + * @group dataframes */ def createDataFrame(rowRDD: JavaRDD[Row], columns: java.util.List[String]): DataFrame = { createDataFrame(rowRDD.rdd, columns.toSeq) @@ -356,6 +408,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. + * @group dataframes */ def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = { val attributeSeq = getSchema(beanClass) @@ -383,6 +436,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, * SELECT * queries will return the columns in an undefined order. + * @group dataframes */ def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { createDataFrame(rdd.rdd, beanClass) @@ -416,8 +470,6 @@ class SQLContext(@transient val sparkContext: SparkContext) * dataFrame.registerTempTable("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} - * - * @group userf */ @deprecated("use createDataFrame", "1.3.0") def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { @@ -455,7 +507,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Loads a Parquet file, returning the result as a [[DataFrame]]. This function returns an empty * [[DataFrame]] if no paths are passed in. * - * @group userf + * @group specificdata */ @scala.annotation.varargs def parquetFile(paths: String*): DataFrame = { @@ -473,7 +525,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Loads a JSON file (one object per line), returning the result as a [[DataFrame]]. * It goes through the entire dataset once to determine the schema. * - * @group userf + * @group specificdata */ def jsonFile(path: String): DataFrame = jsonFile(path, 1.0) @@ -482,7 +534,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Loads a JSON file (one object per line) and applies the given schema, * returning the result as a [[DataFrame]]. * - * @group userf + * @group specificdata */ @Experimental def jsonFile(path: String, schema: StructType): DataFrame = { @@ -492,6 +544,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: + * @group specificdata */ @Experimental def jsonFile(path: String, samplingRatio: Double): DataFrame = { @@ -504,10 +557,18 @@ class SQLContext(@transient val sparkContext: SparkContext) * [[DataFrame]]. * It goes through the entire dataset once to determine the schema. * - * @group userf + * @group specificdata */ def jsonRDD(json: RDD[String]): DataFrame = jsonRDD(json, 1.0) + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * [[DataFrame]]. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + */ def jsonRDD(json: JavaRDD[String]): DataFrame = jsonRDD(json.rdd, 1.0) /** @@ -515,7 +576,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, * returning the result as a [[DataFrame]]. * - * @group userf + * @group specificdata */ @Experimental def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { @@ -528,6 +589,13 @@ class SQLContext(@transient val sparkContext: SparkContext) createDataFrame(rowRDD, appliedSchema) } + /** + * :: Experimental :: + * Loads an JavaRDD storing JSON objects (one object per record) and applies the given + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata + */ @Experimental def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { jsonRDD(json.rdd, schema) @@ -535,6 +603,10 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: + * Loads an RDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata */ @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { @@ -546,6 +618,13 @@ class SQLContext(@transient val sparkContext: SparkContext) createDataFrame(rowRDD, appliedSchema) } + /** + * :: Experimental :: + * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a [[DataFrame]]. + * + * @group specificdata + */ @Experimental def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { jsonRDD(json.rdd, samplingRatio); @@ -555,6 +634,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * Returns the dataset stored at path as a DataFrame, * using the default data source configured by spark.sql.sources.default. + * + * @group genericdata */ @Experimental def load(path: String): DataFrame = { @@ -565,6 +646,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: * Returns the dataset stored at path as a DataFrame, using the given data source. + * + * @group genericdata */ @Experimental def load(path: String, source: String): DataFrame = { @@ -575,6 +658,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * (Java-specific) Returns the dataset specified by the given data source and * a set of options as a DataFrame. + * + * @group genericdata */ @Experimental def load(source: String, options: java.util.Map[String, String]): DataFrame = { @@ -585,6 +670,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * (Scala-specific) Returns the dataset specified by the given data source and * a set of options as a DataFrame. + * + * @group genericdata */ @Experimental def load(source: String, options: Map[String, String]): DataFrame = { @@ -596,6 +683,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * (Java-specific) Returns the dataset specified by the given data source and * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata */ @Experimental def load( @@ -609,6 +698,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: * (Scala-specific) Returns the dataset specified by the given data source and * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * @group genericdata */ @Experimental def load( @@ -733,54 +823,70 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * Construct an RDD representing the database table accessible via JDBC URL + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL * url named table. + * + * @group specificdata */ @Experimental - def jdbcRDD(url: String, table: String): DataFrame = { - jdbcRDD(url, table, null.asInstanceOf[JDBCPartitioningInfo]) + def jdbc(url: String, table: String): DataFrame = { + jdbc(url, table, JDBCRelation.columnPartition(null)) } /** * :: Experimental :: - * Construct an RDD representing the database table accessible via JDBC URL - * url named table. The PartitioningInfo parameter - * gives the name of a column of integral type, a number of partitions, and - * advisory minimum and maximum values for the column. The RDD is - * partitioned according to said column. + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL + * url named table. Partitions of the table will be retrieved in parallel based on the parameters + * passed to this function. + * + * @param columnName the name of a column of integral type that will be used for partitioning. + * @param lowerBound the minimum value of `columnName` to retrieve + * @param upperBound the maximum value of `columnName` to retrieve + * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split + * evenly into this many partitions + * + * @group specificdata */ @Experimental - def jdbcRDD(url: String, table: String, partitioning: JDBCPartitioningInfo): - DataFrame = { + def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): DataFrame = { + val partitioning = JDBCPartitioningInfo(columnName, lowerBound, upperBound, numPartitions) val parts = JDBCRelation.columnPartition(partitioning) - jdbcRDD(url, table, parts) + jdbc(url, table, parts) } /** * :: Experimental :: - * Construct an RDD representing the database table accessible via JDBC URL + * Construct a [[DataFrame]] representing the database table accessible via JDBC URL * url named table. The theParts parameter gives a list expressions * suitable for inclusion in WHERE clauses; each one defines one partition - * of the RDD. + * of the [[DataFrame]]. + * + * @group specificdata */ @Experimental - def jdbcRDD(url: String, table: String, theParts: Array[String]): DataFrame = { + def jdbc(url: String, table: String, theParts: Array[String]): DataFrame = { val parts: Array[Partition] = theParts.zipWithIndex.map { case (part, i) => JDBCPartition(part, i) : Partition } - jdbcRDD(url, table, parts) + jdbc(url, table, parts) } - private def jdbcRDD(url: String, table: String, parts: Array[Partition]): DataFrame = { + private def jdbc(url: String, table: String, parts: Array[Partition]): DataFrame = { val relation = JDBCRelation(url, table, parts)(this) baseRelationToDataFrame(relation) } /** - * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only - * during the lifetime of this instance of SQLContext. + * Registers the given [[DataFrame]] as a temporary table in the catalog. Temporary tables exist + * only during the lifetime of this instance of SQLContext. */ - private[sql] def registerRDDAsTable(rdd: DataFrame, tableName: String): Unit = { + private[sql] def registerDataFrameAsTable(rdd: DataFrame, tableName: String): Unit = { catalog.registerTable(Seq(tableName), rdd.logicalPlan) } @@ -790,7 +896,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @param tableName the name of the table to be unregistered. * - * @group ddl_ops + * @group basic */ def dropTempTable(tableName: String): Unit = { cacheManager.tryUncacheQuery(table(tableName)) @@ -801,7 +907,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Executes a SQL query using Spark, returning the result as a [[DataFrame]]. The dialect that is * used for SQL parsing can be configured with 'spark.sql.dialect'. * - * @group userf + * @group basic */ def sql(sqlText: String): DataFrame = { if (conf.dialect == "sql") { @@ -811,7 +917,11 @@ class SQLContext(@transient val sparkContext: SparkContext) } } - /** Returns the specified table as a [[DataFrame]]. */ + /** + * Returns the specified table as a [[DataFrame]]. + * + * @group ddl_ops + */ def table(tableName: String): DataFrame = DataFrame(this, catalog.lookupRelation(Seq(tableName))) @@ -819,6 +929,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * Returns a [[DataFrame]] containing names of existing tables in the current database. * The returned DataFrame has two columns, tableName and isTemporary (a Boolean * indicating if a table is a temporary one or not). + * + * @group ddl_ops */ def tables(): DataFrame = { DataFrame(this, ShowTablesCommand(None)) @@ -828,6 +940,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * Returns a [[DataFrame]] containing names of existing tables in the given database. * The returned DataFrame has two columns, tableName and isTemporary (a Boolean * indicating if a table is a temporary one or not). + * + * @group ddl_ops */ def tables(databaseName: String): DataFrame = { DataFrame(this, ShowTablesCommand(Some(databaseName))) @@ -835,6 +949,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Returns the names of tables in the current database as an array. + * + * @group ddl_ops */ def tableNames(): Array[String] = { catalog.getTables(None).map { @@ -844,6 +960,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Returns the names of tables in the given database as an array. + * + * @group ddl_ops */ def tableNames(databaseName: String): Array[String] = { catalog.getTables(Some(databaseName)).map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala index ee94a5fdbe376..295db539adfc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.types.DataType * df.select( predict(df("score")) ) * }}} */ -case class UserDefinedFunction(f: AnyRef, dataType: DataType) { +case class UserDefinedFunction protected[sql] (f: AnyRef, dataType: DataType) { def apply(exprs: Column*): Column = { Column(ScalaUdf(f, dataType, exprs.map(_.expr))) @@ -58,6 +58,7 @@ private[sql] case class UserDefinedPythonFunction( accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType) { + /** Returns a [[Column]] that will evaluate to calling this UDF with the given input. */ def apply(exprs: Column*): Column = { val udf = PythonUDF(name, command, envVars, pythonIncludes, pythonExec, broadcastVars, accumulator, dataType, exprs.map(_.expr)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/package.scala new file mode 100644 index 0000000000000..cbbd005228d44 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/package.scala @@ -0,0 +1,23 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +/** + * Contains API classes that are specific to a single language (i.e. Java). + */ +package object api diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index c6cd6eb6a22b6..7c92e9fc88168 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -144,7 +144,7 @@ case class CacheTableCommand( override def run(sqlContext: SQLContext) = { plan.foreach { logicalPlan => - sqlContext.registerRDDAsTable(DataFrame(sqlContext, logicalPlan), tableName) + sqlContext.registerDataFrameAsTable(DataFrame(sqlContext, logicalPlan), tableName) } sqlContext.cacheTable(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index acef49aabfe70..73162b22fa9cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -32,7 +32,9 @@ import org.apache.spark.sql.types._ * * Usage: * {{{ - * sql("SELECT key FROM src").debug + * import org.apache.spark.sql.execution.debug._ + * sql("SELECT key FROM src").debug() + * dataFrame.typeCheck() * }}} */ package object debug { @@ -144,11 +146,9 @@ package object debug { } /** - * :: DeveloperApi :: * Helper functions for checking that runtime types match a given schema. */ - @DeveloperApi - object TypeCheck { + private[sql] object TypeCheck { def typeCheck(data: Any, schema: DataType): Unit = (data, schema) match { case (null, _) => @@ -174,10 +174,8 @@ package object debug { } /** - * :: DeveloperApi :: * Augments [[DataFrame]]s with debug methods. */ - @DeveloperApi private[sql] case class TypeCheck(child: SparkPlan) extends SparkPlan { import TypeCheck._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index 34a83f0a5dad8..34f864f5fda7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartit import org.apache.spark.sql.types._ package object jdbc { - object JDBCWriteDetails extends Logging { + private[sql] object JDBCWriteDetails extends Logging { /** * Returns a PreparedStatement that inserts a row into table via conn. */ - private def insertStatement(conn: Connection, table: String, rddSchema: StructType): + def insertStatement(conn: Connection, table: String, rddSchema: StructType): PreparedStatement = { val sql = new StringBuilder(s"INSERT INTO $table VALUES (") var fieldsLeft = rddSchema.fields.length @@ -56,7 +56,7 @@ package object jdbc { * non-Serializable. Instead, we explicitly close over all variables that * are used. */ - private[jdbc] def savePartition(url: String, table: String, iterator: Iterator[Row], + def savePartition(url: String, table: String, iterator: Iterator[Row], rddSchema: StructType, nullTypes: Array[Int]): Iterator[Byte] = { val conn = DriverManager.getConnection(url) var committed = false @@ -117,19 +117,14 @@ package object jdbc { } Array[Byte]().iterator } - } - /** - * Make it so that you can call createJDBCTable and insertIntoJDBC on a DataFrame. - */ - implicit class JDBCDataFrame(rdd: DataFrame) { /** * Compute the schema string for this RDD. */ - private def schemaString(url: String): String = { + def schemaString(df: DataFrame, url: String): String = { val sb = new StringBuilder() val quirks = DriverQuirks.get(url) - rdd.schema.fields foreach { field => { + df.schema.fields foreach { field => { val name = field.name var typ: String = quirks.getJDBCType(field.dataType)._1 if (typ == null) typ = field.dataType match { @@ -156,9 +151,9 @@ package object jdbc { /** * Saves the RDD to the database in a single transaction. */ - private def saveTable(url: String, table: String) { + def saveTable(df: DataFrame, url: String, table: String) { val quirks = DriverQuirks.get(url) - var nullTypes: Array[Int] = rdd.schema.fields.map(field => { + var nullTypes: Array[Int] = df.schema.fields.map(field => { var nullType: Option[Int] = quirks.getJDBCType(field.dataType)._2 if (nullType.isEmpty) { field.dataType match { @@ -175,61 +170,16 @@ package object jdbc { case DateType => java.sql.Types.DATE case DecimalType.Unlimited => java.sql.Types.DECIMAL case _ => throw new IllegalArgumentException( - s"Can't translate null value for field $field") + s"Can't translate null value for field $field") } } else nullType.get }).toArray - val rddSchema = rdd.schema - rdd.mapPartitions(iterator => JDBCWriteDetails.savePartition( - url, table, iterator, rddSchema, nullTypes)).collect() - } - - /** - * Save this RDD to a JDBC database at `url` under the table name `table`. - * This will run a `CREATE TABLE` and a bunch of `INSERT INTO` statements. - * If you pass `true` for `allowExisting`, it will drop any table with the - * given name; if you pass `false`, it will throw if the table already - * exists. - */ - def createJDBCTable(url: String, table: String, allowExisting: Boolean) { - val conn = DriverManager.getConnection(url) - try { - if (allowExisting) { - val sql = s"DROP TABLE IF EXISTS $table" - conn.prepareStatement(sql).executeUpdate() - } - val schema = schemaString(url) - val sql = s"CREATE TABLE $table ($schema)" - conn.prepareStatement(sql).executeUpdate() - } finally { - conn.close() + val rddSchema = df.schema + df.foreachPartition { iterator => + JDBCWriteDetails.savePartition(url, table, iterator, rddSchema, nullTypes) } - saveTable(url, table) } - /** - * Save this RDD to a JDBC database at `url` under the table name `table`. - * Assumes the table already exists and has a compatible schema. If you - * pass `true` for `overwrite`, it will `TRUNCATE` the table before - * performing the `INSERT`s. - * - * The table must already exist on the database. It must have a schema - * that is compatible with the schema of this RDD; inserting the rows of - * the RDD in order via the simple statement - * `INSERT INTO table VALUES (?, ?, ..., ?)` should not fail. - */ - def insertIntoJDBC(url: String, table: String, overwrite: Boolean) { - if (overwrite) { - val conn = DriverManager.getConnection(url) - try { - val sql = s"TRUNCATE TABLE $table" - conn.prepareStatement(sql).executeUpdate() - } finally { - conn.close() - } - } - saveTable(url, table) - } - } // implicit class JDBCDataFrame + } } // package object jdbc diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 7dd8bea49b8a5..65966458eb670 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -55,7 +55,7 @@ import org.apache.spark.{Logging, SerializableWritable, TaskContext} * Parquet table scan operator. Imports the file that backs the given * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``. */ -case class ParquetTableScan( +private[sql] case class ParquetTableScan( attributes: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Seq[Expression]) @@ -210,7 +210,7 @@ case class ParquetTableScan( * (only detected via filename pattern so will not catch all cases). */ @DeveloperApi -case class InsertIntoParquetTable( +private[sql] case class InsertIntoParquetTable( relation: ParquetRelation, child: SparkPlan, overwrite: Boolean = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index d0856df8d4f43..052728c5d5ceb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils * convenient to use tuples rather than special case classes when writing test cases/suites. * Especially, `Tuple1.apply` can be used to easily wrap a single type/value. */ -trait ParquetTest { +private[sql] trait ParquetTest { val sqlContext: SQLContext import sqlContext.implicits.{localSeqToDataFrameHolder, rddToDataFrameHolder} @@ -121,7 +121,7 @@ trait ParquetTest { (data: Seq[T], tableName: String) (f: => Unit): Unit = { withParquetRDD(data) { rdd => - sqlContext.registerRDDAsTable(rdd, tableName) + sqlContext.registerDataFrameAsTable(rdd, tableName) withTempTable(tableName)(f) } } 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 9bb34e2df9a26..95bea9201163d 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 @@ -72,7 +72,7 @@ import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWrita * null or empty string. This is similar to the `hive.exec.default.partition.name` configuration * in Hive. */ -class DefaultSource +private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { @@ -147,7 +147,7 @@ private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: * discovery. */ @DeveloperApi -case class ParquetRelation2( +private[sql] case class ParquetRelation2( paths: Seq[String], parameters: Map[String, String], maybeSchema: Option[StructType] = None, @@ -600,7 +600,7 @@ case class ParquetRelation2( } } -object ParquetRelation2 { +private[sql] object ParquetRelation2 { // Whether we should merge schemas collected from all Parquet part-files. val MERGE_SCHEMA = "mergeSchema" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala index 887161684429f..e24475292ceaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala @@ -53,7 +53,7 @@ private[parquet] class NanoTime extends Serializable { "NanoTime{julianDay=" + julianDay + ", timeOfDayNanos=" + timeOfDayNanos + "}" } -object NanoTime { +private[sql] object NanoTime { def fromBinary(bytes: Binary): NanoTime = { Preconditions.checkArgument(bytes.length() == 12, "Must be 12 bytes") val buf = bytes.toByteBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index dd8b3d211be64..5020689f7a105 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -374,7 +374,7 @@ private[sql] case class CreateTempTableUsing( def run(sqlContext: SQLContext) = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) - sqlContext.registerRDDAsTable( + sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty } @@ -390,7 +390,7 @@ private[sql] case class CreateTempTableUsingAsSelect( def run(sqlContext: SQLContext) = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) - sqlContext.registerRDDAsTable( + sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty diff --git a/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java b/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java deleted file mode 100644 index 80bd74f5b5525..0000000000000 --- a/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java +++ /dev/null @@ -1,102 +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.jdbc; - -import org.junit.*; -import static org.junit.Assert.*; -import java.sql.Connection; -import java.sql.DriverManager; - -import org.apache.spark.SparkEnv; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.api.java.*; -import org.apache.spark.sql.test.TestSQLContext$; - -public class JavaJDBCTest { - static String url = "jdbc:h2:mem:testdb1"; - - static Connection conn = null; - - // This variable will always be null if TestSQLContext is intact when running - // these tests. Some Java tests do not play nicely with others, however; - // they create a SparkContext of their own at startup and stop it at exit. - // This renders TestSQLContext inoperable, meaning we have to do the same - // thing. If this variable is nonnull, that means we allocated a - // SparkContext of our own and that we need to stop it at teardown. - static JavaSparkContext localSparkContext = null; - - static SQLContext sql = TestSQLContext$.MODULE$; - - @Before - public void beforeTest() throws Exception { - if (SparkEnv.get() == null) { // A previous test destroyed TestSQLContext. - localSparkContext = new JavaSparkContext("local", "JavaAPISuite"); - sql = new SQLContext(localSparkContext); - } - Class.forName("org.h2.Driver"); - conn = DriverManager.getConnection(url); - conn.prepareStatement("create schema test").executeUpdate(); - conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate(); - conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate(); - conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate(); - conn.prepareStatement("insert into test.people values ('joe', 3)").executeUpdate(); - conn.commit(); - } - - @After - public void afterTest() throws Exception { - if (localSparkContext != null) { - localSparkContext.stop(); - localSparkContext = null; - } - try { - conn.close(); - } finally { - conn = null; - } - } - - @Test - public void basicTest() throws Exception { - DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE"); - Row[] rows = rdd.collect(); - assertEquals(rows.length, 3); - } - - @Test - public void partitioningTest() throws Exception { - String[] parts = new String[2]; - parts[0] = "THEID < 2"; - parts[1] = "THEID = 2"; // Deliberately forget about one of them. - DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE", parts); - Row[] rows = rdd.collect(); - assertEquals(rows.length, 2); - } - - @Test - public void writeTest() throws Exception { - DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE"); - JDBCUtils.createJDBCTable(rdd, url, "TEST.PEOPLECOPY", false); - DataFrame rdd2 = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLECOPY"); - Row[] rows = rdd2.collect(); - assertEquals(rows.length, 3); - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index d25c1390db15c..07db672217bc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -164,17 +164,16 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("Basic API") { - assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE").collect.size == 3) + assert(TestSQLContext.jdbc(url, "TEST.PEOPLE").collect.size == 3) } test("Partitioning via JDBCPartitioningInfo API") { - val parts = JDBCPartitioningInfo("THEID", 0, 4, 3) - assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", "THEID", 0, 4, 3).collect.size == 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", parts).collect.size == 3) } test("H2 integral types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 21e70936102fd..ad2fbc3f04a9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -57,8 +57,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) srdd.createJDBCTable(url, "TEST.BASICCREATETEST", false) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.BASICCREATETEST").count) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.BASICCREATETEST").collect()(0).length) + assert(2 == TestSQLContext.jdbc(url, "TEST.BASICCREATETEST").count) + assert(2 == TestSQLContext.jdbc(url, "TEST.BASICCREATETEST").collect()(0).length) } test("CREATE with overwrite") { @@ -66,12 +66,12 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) srdd.createJDBCTable(url, "TEST.DROPTEST", false) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) - assert(3 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").collect()(0).length) + assert(2 == TestSQLContext.jdbc(url, "TEST.DROPTEST").count) + assert(3 == TestSQLContext.jdbc(url, "TEST.DROPTEST").collect()(0).length) srdd2.createJDBCTable(url, "TEST.DROPTEST", true) - assert(1 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").collect()(0).length) + assert(1 == TestSQLContext.jdbc(url, "TEST.DROPTEST").count) + assert(2 == TestSQLContext.jdbc(url, "TEST.DROPTEST").collect()(0).length) } test("CREATE then INSERT to append") { @@ -80,8 +80,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { srdd.createJDBCTable(url, "TEST.APPENDTEST", false) srdd2.insertIntoJDBC(url, "TEST.APPENDTEST", false) - assert(3 == TestSQLContext.jdbcRDD(url, "TEST.APPENDTEST").count) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.APPENDTEST").collect()(0).length) + assert(3 == TestSQLContext.jdbc(url, "TEST.APPENDTEST").count) + assert(2 == TestSQLContext.jdbc(url, "TEST.APPENDTEST").collect()(0).length) } test("CREATE then INSERT to truncate") { @@ -90,8 +90,8 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { srdd.createJDBCTable(url, "TEST.TRUNCATETEST", false) srdd2.insertIntoJDBC(url, "TEST.TRUNCATETEST", true) - assert(1 == TestSQLContext.jdbcRDD(url, "TEST.TRUNCATETEST").count) - assert(2 == TestSQLContext.jdbcRDD(url, "TEST.TRUNCATETEST").collect()(0).length) + assert(1 == TestSQLContext.jdbc(url, "TEST.TRUNCATETEST").count) + assert(2 == TestSQLContext.jdbc(url, "TEST.TRUNCATETEST").collect()(0).length) } test("Incompatible INSERT to append") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala index 89920f2650c3a..4f38110c80cc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala @@ -143,7 +143,7 @@ class MySQLDatabase { } test("Basic test") { - val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "tbl") + val rdd = TestSQLContext.jdbc(url(ip, "foo"), "tbl") val rows = rdd.collect assert(rows.length == 2) val types = rows(0).toSeq.map(x => x.getClass.toString) @@ -153,7 +153,7 @@ class MySQLDatabase { } test("Numeric types") { - val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") + val rdd = TestSQLContext.jdbc(url(ip, "foo"), "numbers") val rows = rdd.collect assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) @@ -181,7 +181,7 @@ class MySQLDatabase { } test("Date types") { - val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") + val rdd = TestSQLContext.jdbc(url(ip, "foo"), "dates") val rows = rdd.collect assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) @@ -199,7 +199,7 @@ class MySQLDatabase { } test("String types") { - val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings") + val rdd = TestSQLContext.jdbc(url(ip, "foo"), "strings") val rows = rdd.collect assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) @@ -225,9 +225,9 @@ class MySQLDatabase { } test("Basic write test") { - val rdd1 = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") - val rdd2 = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") - val rdd3 = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings") + val rdd1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers") + val rdd2 = TestSQLContext.jdbc(url(ip, "foo"), "dates") + val rdd3 = TestSQLContext.jdbc(url(ip, "foo"), "strings") rdd1.createJDBCTable(url(ip, "foo"), "numberscopy", false) rdd2.createJDBCTable(url(ip, "foo"), "datescopy", false) rdd3.createJDBCTable(url(ip, "foo"), "stringscopy", false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala index c174d7adb7204..7b47feeb7887e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala @@ -113,7 +113,7 @@ class PostgresDatabase { } test("Type mapping for various types") { - val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar") val rows = rdd.collect assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) @@ -142,7 +142,7 @@ class PostgresDatabase { } test("Basic write test") { - val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar") rdd.createJDBCTable(url(db.ip), "public.barcopy", false) // Test only that it doesn't bomb out. } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index bfacc51ef57ab..07b5a84fb6602 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -29,9 +29,9 @@ import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.SQLContext /** - * Implementation for "describe [extended] table". - * * :: DeveloperApi :: + * + * Implementation for "describe [extended] table". */ @DeveloperApi case class DescribeHiveTableCommand( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 0aa5f7f7b88bd..6afd8eea05418 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -102,6 +102,10 @@ case class AddFile(path: String) extends RunnableCommand { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class CreateMetastoreDataSource( tableName: String, userSpecifiedSchema: Option[StructType], @@ -141,6 +145,10 @@ case class CreateMetastoreDataSource( } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class CreateMetastoreDataSourceAsSelect( tableName: String, provider: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala similarity index 66% rename from sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala index 86bb67ec74256..4989c42e964ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala @@ -15,16 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.jdbc +package org.apache.spark.sql.hive -import org.apache.spark.sql.DataFrame - -private[jdbc] class JavaJDBCTrampoline { - def createJDBCTable(rdd: DataFrame, url: String, table: String, allowExisting: Boolean) { - rdd.createJDBCTable(url, table, allowExisting); - } - - def insertIntoJDBC(rdd: DataFrame, url: String, table: String, overwrite: Boolean) { - rdd.insertIntoJDBC(url, table, overwrite); - } -} +/** + * Physical execution operators used for running queries against data stored in Hive. These + * are not intended for use by users, but are documents so that it is easier to understand + * the output of EXPLAIN queries. + */ +package object execution diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala index a6c8ed4f7e866..db074361ef03c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala @@ -17,4 +17,14 @@ package org.apache.spark.sql +/** + * Support for running Spark SQL queries using functionality from Apache Hive (does not require an + * existing Hive installation). Supported Hive features include: + * - Using HiveQL to express queries. + * - Reading metadata from the Hive Metastore using HiveSerDes. + * - Hive UDFs, UDAs, UDTs + * + * Users that would like access to this functionality should create a + * [[hive.HiveContext HiveContext]] instead of a [[SQLContext]]. + */ package object hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala deleted file mode 100644 index 2a16c9d1a27c9..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala +++ /dev/null @@ -1,56 +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.parquet - -import java.util.Properties - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category -import org.apache.hadoop.hive.serde2.{SerDeStats, SerDe} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.hadoop.io.Writable - -/** - * A placeholder that allows Spark SQL users to create metastore tables that are stored as - * parquet files. It is only intended to pass the checks that the serde is valid and exists - * when a CREATE TABLE is run. The actual work of decoding will be done by ParquetTableScan - * when "spark.sql.hive.convertMetastoreParquet" is set to true. - */ -@deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " + - "placeholder in the Hive MetaStore", "1.2.0") -class FakeParquetSerDe extends SerDe { - override def getObjectInspector: ObjectInspector = new ObjectInspector { - override def getCategory: Category = Category.PRIMITIVE - - override def getTypeName: String = "string" - } - - override def deserialize(p1: Writable): AnyRef = throwError - - override def initialize(p1: Configuration, p2: Properties): Unit = {} - - override def getSerializedClass: Class[_ <: Writable] = throwError - - override def getSerDeStats: SerDeStats = throwError - - override def serialize(p1: scala.Any, p2: ObjectInspector): Writable = throwError - - private def throwError = - sys.error( - "spark.sql.hive.convertMetastoreParquet must be set to true to use FakeParquetSerDe") -} diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 8534c7d7064e5..30646ddbc29d8 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -43,7 +43,9 @@ import org.apache.hadoop.mapred.InputFormat import org.apache.spark.sql.types.{Decimal, DecimalType} -case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { +private[hive] case class HiveFunctionWrapper(functionClassName: String) + extends java.io.Serializable { + // for Serialization def this() = this(null) @@ -249,6 +251,9 @@ private[hive] object HiveShim { def setTblNullFormat(crtTbl: CreateTableDesc, tbl: Table) = {} } -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) +private[hive] class ShimFileSinkDesc( + var dir: String, + var tableInfo: TableDesc, + var compressed: Boolean) extends FileSinkDesc(dir, tableInfo, compressed) { } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 72104f5b55761..f9fcbdae15745 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -56,7 +56,9 @@ import org.apache.spark.sql.types.{Decimal, DecimalType} * * @param functionClassName UDF class name */ -case class HiveFunctionWrapper(var functionClassName: String) extends java.io.Externalizable { +private[hive] case class HiveFunctionWrapper(var functionClassName: String) + extends java.io.Externalizable { + // for Serialization def this() = this(null) @@ -423,7 +425,10 @@ private[hive] object HiveShim { * Bug introduced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. * Fix it through wrapper. */ -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) +private[hive] class ShimFileSinkDesc( + var dir: String, + var tableInfo: TableDesc, + var compressed: Boolean) extends Serializable with Logging { var compressCodec: String = _ var compressType: String = _ From d8adefefcc2a4af32295440ed1d4917a6968f017 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 10:22:48 -0800 Subject: [PATCH 161/817] [SPARK-5859] [PySpark] [SQL] fix DataFrame Python API 1. added explain() 2. add isLocal() 3. do not call show() in __repl__ 4. add foreach() and foreachPartition() 5. add distinct() 6. fix functions.col()/column()/lit() 7. fix unit tests in sql/functions.py 8. fix unicode in showString() Author: Davies Liu Closes #4645 from davies/df6 and squashes the following commits: 6b46a2c [Davies Liu] fix DataFrame Python API --- python/pyspark/sql/dataframe.py | 65 +++++++++++++++++++++++++++------ python/pyspark/sql/functions.py | 12 +++--- 2 files changed, 59 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 28a59e73a3410..841724095f693 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -238,6 +238,22 @@ def printSchema(self): """ print (self._jdf.schema().treeString()) + def explain(self, extended=False): + """ + Prints the plans (logical and physical) to the console for + debugging purpose. + + If extended is False, only prints the physical plan. + """ + self._jdf.explain(extended) + + def isLocal(self): + """ + Returns True if the `collect` and `take` methods can be run locally + (without any Spark executors). + """ + return self._jdf.isLocal() + def show(self): """ Print the first 20 rows. @@ -247,14 +263,12 @@ def show(self): 2 Alice 5 Bob >>> df - age name - 2 Alice - 5 Bob + DataFrame[age: int, name: string] """ - print (self) + print self._jdf.showString().encode('utf8', 'ignore') def __repr__(self): - return self._jdf.showString() + return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) def count(self): """Return the number of elements in this RDD. @@ -336,6 +350,8 @@ def mapPartitions(self, f, preservesPartitioning=False): """ Return a new RDD by applying a function to each partition. + It's a shorthand for df.rdd.mapPartitions() + >>> rdd = sc.parallelize([1, 2, 3, 4], 4) >>> def f(iterator): yield 1 >>> rdd.mapPartitions(f).sum() @@ -343,6 +359,31 @@ def mapPartitions(self, f, preservesPartitioning=False): """ return self.rdd.mapPartitions(f, preservesPartitioning) + def foreach(self, f): + """ + Applies a function to all rows of this DataFrame. + + It's a shorthand for df.rdd.foreach() + + >>> def f(person): + ... print person.name + >>> df.foreach(f) + """ + return self.rdd.foreach(f) + + def foreachPartition(self, f): + """ + Applies a function to each partition of this DataFrame. + + It's a shorthand for df.rdd.foreachPartition() + + >>> def f(people): + ... for person in people: + ... print person.name + >>> df.foreachPartition(f) + """ + return self.rdd.foreachPartition(f) + def cache(self): """ Persist with the default storage level (C{MEMORY_ONLY_SER}). """ @@ -377,8 +418,13 @@ def repartition(self, numPartitions): """ Return a new :class:`DataFrame` that has exactly `numPartitions` partitions. """ - rdd = self._jdf.repartition(numPartitions, None) - return DataFrame(rdd, self.sql_ctx) + return DataFrame(self._jdf.repartition(numPartitions, None), self.sql_ctx) + + def distinct(self): + """ + Return a new :class:`DataFrame` containing the distinct rows in this DataFrame. + """ + return DataFrame(self._jdf.distinct(), self.sql_ctx) def sample(self, withReplacement, fraction, seed=None): """ @@ -957,10 +1003,7 @@ def cast(self, dataType): return Column(jc, self.sql_ctx) def __repr__(self): - if self._jdf.isComputable(): - return self._jdf.samples() - else: - return 'Column<%s>' % self._jdf.toString() + return 'Column<%s>' % self._jdf.toString().encode('utf8') def toPandas(self): """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index d0e090607ff4f..fc61162f0b827 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -37,7 +37,7 @@ def _create_function(name, doc=""): """ Create a function for aggregator by name""" def _(col): sc = SparkContext._active_spark_context - jc = getattr(sc._jvm.functions, name)(_to_java_column(col)) + jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) return Column(jc) _.__name__ = name _.__doc__ = doc @@ -140,6 +140,7 @@ def __call__(self, *cols): def udf(f, returnType=StringType()): """Create a user defined function (UDF) + >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) >>> df.select(slen(df.name).alias('slen')).collect() [Row(slen=5), Row(slen=3)] @@ -151,17 +152,14 @@ def _test(): import doctest from pyspark.context import SparkContext from pyspark.sql import Row, SQLContext - import pyspark.sql.dataframe - globs = pyspark.sql.dataframe.__dict__.copy() + import pyspark.sql.functions + globs = pyspark.sql.functions.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() - globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() - globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), - Row(name='Bob', age=5, height=85)]).toDF() (failure_count, test_count) = doctest.testmod( - pyspark.sql.dataframe, globs=globs, + pyspark.sql.functions, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) globs['sc'].stop() if failure_count: From d8f69cf78862d13a48392a0b94388b8d403523da Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Tue, 17 Feb 2015 10:57:16 -0800 Subject: [PATCH 162/817] [SPARK-5778] throw if nonexistent metrics config file provided previous behavior was to log an error; this is fine in the general case where no `spark.metrics.conf` parameter was specified, in which case a default `metrics.properties` is looked for, and the execption logged and suppressed if it doesn't exist. if the user has purposefully specified a metrics.conf file, however, it makes more sense to show them an error when said file doesn't exist. Author: Ryan Williams Closes #4571 from ryan-williams/metrics and squashes the following commits: 5bccb14 [Ryan Williams] private-ize some MetricsConfig members 08ff998 [Ryan Williams] rename METRICS_CONF: DEFAULT_METRICS_CONF_FILENAME f4d7fab [Ryan Williams] fix tests ad24b0e [Ryan Williams] add "metrics.properties" to .rat-excludes 94e810b [Ryan Williams] throw if nonexistent Sink class is specified 31d2c30 [Ryan Williams] metrics code review feedback 56287db [Ryan Williams] throw if nonexistent metrics config file provided --- .rat-excludes | 1 + .../apache/spark/metrics/MetricsConfig.scala | 32 ++++++++++--------- .../apache/spark/metrics/MetricsSystem.scala | 5 ++- .../resources/test_metrics_system.properties | 2 -- .../spark/metrics/MetricsConfigSuite.scala | 2 +- 5 files changed, 23 insertions(+), 19 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index a788e8273d8a2..8c61e67a0c7d1 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -19,6 +19,7 @@ fairscheduler.xml.template spark-defaults.conf.template log4j.properties log4j.properties.template +metrics.properties metrics.properties.template slaves slaves.template diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 1b7a5d1f1980a..8edf493780687 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -28,12 +28,12 @@ import org.apache.spark.util.Utils private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { - val DEFAULT_PREFIX = "*" - val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - val METRICS_CONF = "metrics.properties" + private val DEFAULT_PREFIX = "*" + private val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + private val DEFAULT_METRICS_CONF_FILENAME = "metrics.properties" - val properties = new Properties() - var propertyCategories: mutable.HashMap[String, Properties] = null + private[metrics] val properties = new Properties() + private[metrics] var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet") @@ -47,20 +47,22 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi setDefaultProperties(properties) // If spark.metrics.conf is not set, try to get file in class path - var is: InputStream = null - try { - is = configFile match { - case Some(f) => new FileInputStream(f) - case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF) + val isOpt: Option[InputStream] = configFile.map(new FileInputStream(_)).orElse { + try { + Option(Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_METRICS_CONF_FILENAME)) + } catch { + case e: Exception => + logError("Error loading default configuration file", e) + None } + } - if (is != null) { + isOpt.foreach { is => + try { properties.load(is) + } finally { + is.close() } - } catch { - case e: Exception => logError("Error loading configure file", e) - } finally { - if (is != null) is.close() } propertyCategories = subProperties(properties, INSTANCE_REGEX) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 83e8eb71260eb..345db36630fd5 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -191,7 +191,10 @@ private[spark] class MetricsSystem private ( sinks += sink.asInstanceOf[Sink] } } catch { - case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) + case e: Exception => { + logError("Sink class " + classPath + " cannot be instantialized") + throw e + } } } } diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 35d0bd3b8d0b8..4e8b8465696e5 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -18,7 +18,5 @@ *.sink.console.period = 10 *.sink.console.unit = seconds test.sink.console.class = org.apache.spark.metrics.sink.ConsoleSink -test.sink.dummy.class = org.apache.spark.metrics.sink.DummySink -test.source.dummy.class = org.apache.spark.metrics.source.DummySource test.sink.console.period = 20 test.sink.console.unit = minutes diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index 1a9ce8c607dcd..37e528435aa5d 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -27,7 +27,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig(Option("dummy-file")) + val conf = new MetricsConfig(None) conf.initialize() assert(conf.properties.size() === 4) From b271c265b742fa6947522eda4592e9e6a7fd1f3a Mon Sep 17 00:00:00 2001 From: xukun 00228947 Date: Tue, 17 Feb 2015 18:59:41 +0000 Subject: [PATCH 163/817] [SPARK-5661]function hasShutdownDeleteTachyonDir should use shutdownDeleteTachyonPaths to determine whether contains file hasShutdownDeleteTachyonDir(file: TachyonFile) should use shutdownDeleteTachyonPaths(not shutdownDeletePaths) to determine Whether contain file. To solve it ,delete two unused function. Author: xukun 00228947 Author: viper-kun Closes #4418 from viper-kun/deleteunusedfun and squashes the following commits: 87340eb [viper-kun] fix style 3d6c69e [xukun 00228947] fix bug 2bc397e [xukun 00228947] deleteunusedfun --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c06bd6fab0cc9..df21ed37e76b1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -213,8 +213,8 @@ private[spark] object Utils extends Logging { // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() - shutdownDeletePaths.synchronized { - shutdownDeletePaths.contains(absolutePath) + shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths.contains(absolutePath) } } From 9b746f380869b54d673e3758ca5e4475f76c864a Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 17 Feb 2015 11:19:23 -0800 Subject: [PATCH 164/817] [SPARK-3381] [MLlib] Eliminate bins for unordered features in DecisionTrees For unordered features, it is sufficient to use splits since the threshold of the split corresponds the threshold of the HighSplit of the bin and there is no use of the LowSplit. Author: MechCoder Closes #4231 from MechCoder/spark-3381 and squashes the following commits: 58c19a5 [MechCoder] COSMIT c274b74 [MechCoder] Remove unordered feature calculation in labeledPointToTreePoint b2b9b89 [MechCoder] COSMIT d3ee042 [MechCoder] [SPARK-3381] [MLlib] Eliminate bins for unordered features --- .../spark/mllib/tree/DecisionTree.scala | 37 +++++-------------- .../spark/mllib/tree/impl/TreePoint.scala | 14 ++----- .../spark/mllib/tree/DecisionTreeSuite.scala | 37 +------------------ 3 files changed, 15 insertions(+), 73 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index f1f85994e61b2..b9d0c56dd1ea3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -327,14 +327,14 @@ object DecisionTree extends Serializable with Logging { * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (feature, bin). * @param treePoint Data point being aggregated. - * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param splits possible splits indexed (numFeatures)(numSplits) * @param unorderedFeatures Set of indices of unordered features. * @param instanceWeight Weight (importance) of instance in dataset. */ private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - bins: Array[Array[Bin]], + splits: Array[Array[Split]], unorderedFeatures: Set[Int], instanceWeight: Double, featuresForNode: Option[Array[Int]]): Unit = { @@ -362,7 +362,7 @@ object DecisionTree extends Serializable with Logging { val numSplits = agg.metadata.numSplits(featureIndex) var splitIndex = 0 while (splitIndex < numSplits) { - if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { + if (splits(featureIndex)(splitIndex).categories.contains(featureValue)) { agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } else { @@ -506,8 +506,8 @@ object DecisionTree extends Serializable with Logging { if (metadata.unorderedFeatures.isEmpty) { orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) } else { - mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, - instanceWeight, featuresForNode) + mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, splits, + metadata.unorderedFeatures, instanceWeight, featuresForNode) } } } @@ -1024,35 +1024,15 @@ object DecisionTree extends Serializable with Logging { // Categorical feature val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { - // TODO: The second half of the bins are unused. Actually, we could just use - // splits and not build bins for unordered features. That should be part of - // a later PR since it will require changing other code (using splits instead - // of bins in a few places). // Unordered features - // 2^(maxFeatureValue - 1) - 1 combinations + // 2^(maxFeatureValue - 1) - 1 combinations splits(featureIndex) = new Array[Split](numSplits) - bins(featureIndex) = new Array[Bin](numBins) var splitIndex = 0 while (splitIndex < numSplits) { val categories: List[Double] = extractMultiClassCategories(splitIndex + 1, featureArity) splits(featureIndex)(splitIndex) = new Split(featureIndex, Double.MinValue, Categorical, categories) - bins(featureIndex)(splitIndex) = { - if (splitIndex == 0) { - new Bin( - new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), - Categorical, - Double.MinValue) - } else { - new Bin( - splits(featureIndex)(splitIndex - 1), - splits(featureIndex)(splitIndex), - Categorical, - Double.MinValue) - } - } splitIndex += 1 } } else { @@ -1060,8 +1040,11 @@ object DecisionTree extends Serializable with Logging { // Bins correspond to feature values, so we do not need to compute splits or bins // beforehand. Splits are constructed as needed during training. splits(featureIndex) = new Array[Split](0) - bins(featureIndex) = new Array[Bin](0) } + // For ordered features, bins correspond to feature values. + // For unordered categorical features, there is no need to construct the bins. + // since there is a one-to-one correspondence between the splits and the bins. + bins(featureIndex) = new Array[Bin](0) } featureIndex += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index 35e361ae309cc..50b292e71b067 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -55,17 +55,15 @@ private[tree] object TreePoint { input: RDD[LabeledPoint], bins: Array[Array[Bin]], metadata: DecisionTreeMetadata): RDD[TreePoint] = { - // Construct arrays for featureArity and isUnordered for efficiency in the inner loop. + // Construct arrays for featureArity for efficiency in the inner loop. val featureArity: Array[Int] = new Array[Int](metadata.numFeatures) - val isUnordered: Array[Boolean] = new Array[Boolean](metadata.numFeatures) var featureIndex = 0 while (featureIndex < metadata.numFeatures) { featureArity(featureIndex) = metadata.featureArity.getOrElse(featureIndex, 0) - isUnordered(featureIndex) = metadata.isUnordered(featureIndex) featureIndex += 1 } input.map { x => - TreePoint.labeledPointToTreePoint(x, bins, featureArity, isUnordered) + TreePoint.labeledPointToTreePoint(x, bins, featureArity) } } @@ -74,19 +72,17 @@ private[tree] object TreePoint { * @param bins Bins for features, of size (numFeatures, numBins). * @param featureArity Array indexed by feature, with value 0 for continuous and numCategories * for categorical features. - * @param isUnordered Array index by feature, with value true for unordered categorical features. */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, bins: Array[Array[Bin]], - featureArity: Array[Int], - isUnordered: Array[Boolean]): TreePoint = { + featureArity: Array[Int]): TreePoint = { val numFeatures = labeledPoint.features.size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { arr(featureIndex) = findBin(featureIndex, labeledPoint, featureArity(featureIndex), - isUnordered(featureIndex), bins) + bins) featureIndex += 1 } new TreePoint(labeledPoint.label, arr) @@ -96,14 +92,12 @@ private[tree] object TreePoint { * Find bin for one (labeledPoint, feature). * * @param featureArity 0 for continuous features; number of categories for categorical features. - * @param isUnorderedFeature (only applies if feature is categorical) * @param bins Bins for features, of size (numFeatures, numBins). */ private def findBin( featureIndex: Int, labeledPoint: LabeledPoint, featureArity: Int, - isUnorderedFeature: Boolean, bins: Array[Array[Bin]]): Int = { /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 7b1aed5ffeb3e..4c162df810bb2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -190,7 +190,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 3) - assert(bins(0).length === 6) + assert(bins(0).length === 0) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -228,41 +228,6 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(splits(1)(2).categories.contains(0.0)) assert(splits(1)(2).categories.contains(1.0)) - // Check bins. - - assert(bins(0)(0).category === Double.MinValue) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(0.0)) - assert(bins(1)(0).category === Double.MinValue) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(0)(1).category === Double.MinValue) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(0.0)) - assert(bins(0)(1).highSplit.categories.length === 1) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(1)(1).category === Double.MinValue) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 1) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(0)(2).category === Double.MinValue) - assert(bins(0)(2).lowSplit.categories.length === 1) - assert(bins(0)(2).lowSplit.categories.contains(1.0)) - assert(bins(0)(2).highSplit.categories.length === 2) - assert(bins(0)(2).highSplit.categories.contains(1.0)) - assert(bins(0)(2).highSplit.categories.contains(0.0)) - assert(bins(1)(2).category === Double.MinValue) - assert(bins(1)(2).lowSplit.categories.length === 1) - assert(bins(1)(2).lowSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.length === 2) - assert(bins(1)(2).highSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.contains(0.0)) - } test("Multiclass classification with ordered categorical features: split and bin calculations") { From 24f358b9d6bc7a72a4fb493b7f845a40ed941a5d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 17 Feb 2015 11:35:26 -0800 Subject: [PATCH 165/817] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3297 (close requested by 'andrewor14') Closes #3345 (close requested by 'pwendell') Closes #2729 (close requested by 'srowen') Closes #2320 (close requested by 'pwendell') Closes #4529 (close requested by 'andrewor14') Closes #2098 (close requested by 'srowen') Closes #4120 (close requested by 'andrewor14') From 49c19fdbad57f0609bbcc9278f9eaa8115a73604 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 Feb 2015 19:40:06 +0000 Subject: [PATCH 166/817] SPARK-5841 [CORE] [HOTFIX] Memory leak in DiskBlockManager Avoid call to remove shutdown hook being called from shutdown hook CC pwendell JoshRosen MattWhelan Author: Sean Owen Closes #4648 from srowen/SPARK-5841.2 and squashes the following commits: 51548db [Sean Owen] Avoid call to remove shutdown hook being called from shutdown hook --- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index ae9df8cbe9821..b297f3fd9dd1e 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -138,7 +138,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon val shutdownHook = new Thread("delete Spark local dirs") { override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") - DiskBlockManager.this.stop() + DiskBlockManager.this.doStop() } } Runtime.getRuntime.addShutdownHook(shutdownHook) @@ -149,7 +149,10 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon private[spark] def stop() { // Remove the shutdown hook. It causes memory leaks if we leave it around. Runtime.getRuntime.removeShutdownHook(shutdownHook) + doStop() + } + private def doStop(): Unit = { // Only perform cleanup if an external service is not serving our shuffle files. if (!blockManager.externalShuffleServiceEnabled || blockManager.blockManagerId.isDriver) { localDirs.foreach { localDir => From fc4eb9505adda192eb38cb4454d532027690bfa3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 12:05:06 -0800 Subject: [PATCH 167/817] [SPARK-5864] [PySpark] support .jar as python package A jar file containing Python sources in it could be used as a Python package, just like zip file. spark-submit already put the jar file into PYTHONPATH, this patch also put it in the sys.path, then it could be used in Python worker. Author: Davies Liu Closes #4652 from davies/jar and squashes the following commits: 17d3f76 [Davies Liu] support .jar as python package --- python/pyspark/context.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index bf1f61c8504ed..40b3152b23843 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -64,6 +64,8 @@ class SparkContext(object): _lock = Lock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH + PACKAGE_EXTENSIONS = ('.zip', '.egg', '.jar') + def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, environment=None, batchSize=0, serializer=PickleSerializer(), conf=None, gateway=None, jsc=None, profiler_cls=BasicProfiler): @@ -185,7 +187,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, for path in self._conf.get("spark.submit.pyFiles", "").split(","): if path != "": (dirname, filename) = os.path.split(path) - if filename.lower().endswith("zip") or filename.lower().endswith("egg"): + if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) @@ -705,7 +707,7 @@ def addPyFile(self, path): self.addFile(path) (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix - if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): + if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) From 31efb39c1deb253032b38e8fbafde4b2b1dde1f6 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 17 Feb 2015 12:16:52 -0800 Subject: [PATCH 168/817] [Minor] fix typo in SQL document Author: CodingCat Closes #4656 from CodingCat/fix_typo and squashes the following commits: b41d15c [CodingCat] recover 689fe46 [CodingCat] fix typo --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8022c5ecc2430..0146a4ed1b745 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -159,7 +159,7 @@ you to construct DataFrames when the columns and their types are not known until
    -The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes +The Scala interface for Spark SQL supports automatically converting an RDD containing case classes to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex From 4611de1cef7363bc71ec608560dfd866ae477747 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Feb 2015 12:23:18 -0800 Subject: [PATCH 169/817] [SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog Current `ParquetConversions` in `HiveMetastoreCatalog` will transformUp the given plan multiple times if there are many Metastore Parquet tables. Since the transformUp operation is recursive, it should be better to only perform it once. Author: Liang-Chi Hsieh Closes #4651 from viirya/parquet_atonce and squashes the following commits: c1ed29d [Liang-Chi Hsieh] Fix bug. e0f919b [Liang-Chi Hsieh] Only transformUp the given plan once. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 37 ++++++++++--------- 1 file changed, 20 insertions(+), 17 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 0e43faa8afdaf..cfd6f27371d0d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -430,33 +430,36 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with hive.convertMetastoreParquet && hive.conf.parquetUseDataSourceApi && relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => - relation + val parquetRelation = convertToParquetRelation(relation) + val attributedRewrites = relation.output.zip(parquetRelation.output) + (relation, parquetRelation, attributedRewrites) // Read path case p @ PhysicalOperation(_, _, relation: MetastoreRelation) if hive.convertMetastoreParquet && hive.conf.parquetUseDataSourceApi && relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => - relation + val parquetRelation = convertToParquetRelation(relation) + val attributedRewrites = relation.output.zip(parquetRelation.output) + (relation, parquetRelation, attributedRewrites) } + val relationMap = toBeReplaced.map(r => (r._1, r._2)).toMap + val attributedRewrites = AttributeMap(toBeReplaced.map(_._3).fold(Nil)(_ ++: _)) + // Replaces all `MetastoreRelation`s with corresponding `ParquetRelation2`s, and fixes // attribute IDs referenced in other nodes. - toBeReplaced.distinct.foldLeft(plan) { (lastPlan, relation) => - val parquetRelation = convertToParquetRelation(relation) - val attributedRewrites = AttributeMap(relation.output.zip(parquetRelation.output)) - - lastPlan.transformUp { - case r: MetastoreRelation if r == relation => { - val withAlias = - r.alias.map(a => Subquery(a, parquetRelation)).getOrElse( - Subquery(r.tableName, parquetRelation)) - - withAlias - } - case other => other.transformExpressions { - case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) - } + plan.transformUp { + case r: MetastoreRelation if relationMap.contains(r) => { + val parquetRelation = relationMap(r) + val withAlias = + r.alias.map(a => Subquery(a, parquetRelation)).getOrElse( + Subquery(r.tableName, parquetRelation)) + + withAlias + } + case other => other.transformExpressions { + case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) } } } From ac506b7c2846f656e03839bbd0e93827c7cc613e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Feb 2015 12:24:13 -0800 Subject: [PATCH 170/817] [Minor][SQL] Use same function to check path parameter in JSONRelation Author: Liang-Chi Hsieh Closes #4649 from viirya/use_checkpath and squashes the following commits: 0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter. --- .../main/scala/org/apache/spark/sql/json/JSONRelation.scala | 4 ++-- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index 24848634de9cf..3b68b7c275016 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -37,7 +37,7 @@ private[sql] class DefaultSource override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val path = checkPath(parameters) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) JSONRelation(path, samplingRatio, None)(sqlContext) @@ -48,7 +48,7 @@ private[sql] class DefaultSource sqlContext: SQLContext, parameters: Map[String, String], schema: StructType): BaseRelation = { - val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val path = checkPath(parameters) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) JSONRelation(path, samplingRatio, Some(schema))(sqlContext) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0263e3bb56617..485d5c95bfc44 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -547,7 +547,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Map.empty[String, String]) }.getMessage assert( - message.contains("Option 'path' not specified"), + message.contains("'path' must be specified for json data."), "We should complain that path is not specified.") sql("DROP TABLE savedJsonTable") From 9d281fa56022800dc008a3de233fec44379a2bd7 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 17 Feb 2015 12:25:35 -0800 Subject: [PATCH 171/817] [SQL] [Minor] Update the HiveContext Unittest In unit test, the table src(key INT, value STRING) is not the same as HIVE src(key STRING, value STRING) https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql And in the reflect.q, test failed for expression `reflect("java.lang.Integer", "valueOf", key, 16)`, which expect the argument `key` as STRING not INT. This PR doesn't aim to change the `src` schema, we can do that after 1.3 released, however, we probably need to re-generate all the golden files. Author: Cheng Hao Closes #4584 from chenghao-intel/reflect and squashes the following commits: e5bdc3a [Cheng Hao] Move the test case reflect into blacklist 184abfd [Cheng Hao] revert the change to table src1 d9bcf92 [Cheng Hao] Update the HiveContext Unittest --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 6 ++++++ .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 1 + .../golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + .../golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 + .../golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 | 3 +++ .../golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca | 0 .../golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 | 5 +++++ 7 files changed, 17 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 133f2d3c84a2e..c6ead4562d51e 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -225,6 +225,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Needs constant object inspectors "udf_round", + // the table src(key INT, value STRING) is not the same as HIVE unittest. In Hive + // is src(key STRING, value STRING), and in the reflect.q, it failed in + // Integer.valueOf, which expect the first argument passed as STRING type not INT. + "udf_reflect", + // Sort with Limit clause causes failure. "ctas", "ctas_hadoop20", @@ -886,6 +891,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_power", "udf_radians", "udf_rand", + "udf_reflect2", "udf_regexp", "udf_regexp_extract", "udf_regexp_replace", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 840fbc197259a..a2d99f1f4b28d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -196,6 +196,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java + // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql val hiveQTestUtilTables = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect2-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 b/sql/hive/src/test/resources/golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 new file mode 100644 index 0000000000000..cd35e5b290db5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect2-1-7bec330c7bc6f71cbaf9bf1883d1b184 @@ -0,0 +1 @@ +reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 b/sql/hive/src/test/resources/golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 new file mode 100644 index 0000000000000..48ef97292ab62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect2-2-c5a05379f482215a5a484bed0299bf19 @@ -0,0 +1,3 @@ +reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection +Use this UDF to call Java methods by matching the argument signature + diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca b/sql/hive/src/test/resources/golden/udf_reflect2-3-effc057c78c00b0af26a4ac0f5f116ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 b/sql/hive/src/test/resources/golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 new file mode 100644 index 0000000000000..176ea0358d7ea --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect2-4-73d466e70e96e9e5f0cd373b37d4e1f4 @@ -0,0 +1,5 @@ +238 -18 238 238 238 238.0 238.0 238 val_238 val_238_concat false true false false false val_238 -1 -1 VALUE_238 al_238 al_2 VAL_238 val_238 2013-02-15 19:41:20 113 1 5 19 41 20 1360986080000 +86 86 86 86 86 86.0 86.0 86 val_86 val_86_concat true true true true true val_86 -1 -1 VALUE_86 al_86 al_8 VAL_86 val_86 2013-02-15 19:41:20 113 1 5 19 41 20 1360986080000 +311 55 311 311 311 311.0 311.0 311 val_311 val_311_concat false true false false false val_311 5 6 VALUE_311 al_311 al_3 VAL_311 val_311 2013-02-15 19:41:20 113 1 5 19 41 20 1360986080000 +27 27 27 27 27 27.0 27.0 27 val_27 val_27_concat false true false false false val_27 -1 -1 VALUE_27 al_27 al_2 VAL_27 val_27 2013-02-15 19:41:20 113 1 5 19 41 20 1360986080000 +165 -91 165 165 165 165.0 165.0 165 val_165 val_165_concat false true false false false val_165 4 4 VALUE_165 al_165 al_1 VAL_165 val_165 2013-02-15 19:41:20 113 1 5 19 41 20 1360986080000 From de4836f8f12c36c1b350cef288a75b5e59155735 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Feb 2015 13:23:45 -0800 Subject: [PATCH 172/817] [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext Author: Michael Armbrust Closes #4657 from marmbrus/pythonUdfs and squashes the following commits: a7823a8 [Michael Armbrust] [SPARK-5868][SQL] Fix python UDFs in HiveContext and checks in SQLContext --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 3 ++- .../main/scala/org/apache/spark/sql/execution/pythonUdfs.scala | 3 +++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 31afa0eb59a8e..709b350144c75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -113,6 +113,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = true) { override val extendedResolutionRules = + ExtractPythonUdfs :: sources.PreWriteCheck(catalog) :: sources.PreInsertCastAndRename :: Nil @@ -1059,7 +1060,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @DeveloperApi protected[sql] class QueryExecution(val logical: LogicalPlan) { - lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical)) + lazy val analyzed: LogicalPlan = analyzer(logical) lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed) lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 3a2f8d75dac5e..69de4d168a372 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -205,6 +205,9 @@ case class EvaluatePython( extends logical.UnaryNode { def output = child.output :+ resultAttribute + + // References should not include the produced attribute. + override def references = udf.references } /** From 445a755b884885b88c1778fd56a3151045b0b0ed Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 13:36:43 -0800 Subject: [PATCH 173/817] [SPARK-4172] [PySpark] Progress API in Python This patch bring the pull based progress API into Python, also a example in Python. Author: Davies Liu Closes #3027 from davies/progress_api and squashes the following commits: b1ba984 [Davies Liu] fix style d3b9253 [Davies Liu] add tests, mute the exception after stop 4297327 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api 969fa9d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api 25590c9 [Davies Liu] update with Java API 360de2d [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api c0f1021 [Davies Liu] Merge branch 'master' of github.com:apache/spark into progress_api 023afb3 [Davies Liu] add Python API and example for progress API --- .../spark/scheduler/TaskResultGetter.scala | 40 ++++---- examples/src/main/python/status_api_demo.py | 67 +++++++++++++ python/pyspark/__init__.py | 15 +-- python/pyspark/context.py | 7 ++ python/pyspark/status.py | 96 +++++++++++++++++++ python/pyspark/tests.py | 31 ++++++ 6 files changed, 232 insertions(+), 24 deletions(-) create mode 100644 examples/src/main/python/status_api_demo.py create mode 100644 python/pyspark/status.py diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 774f3d8cdb275..3938580aeea59 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import java.util.concurrent.RejectedExecutionException import scala.language.existentials import scala.util.control.NonFatal @@ -95,25 +96,30 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason - getTaskResultExecutor.execute(new Runnable { - override def run(): Unit = Utils.logUncaughtExceptions { - try { - if (serializedData != null && serializedData.limit() > 0) { - reason = serializer.get().deserialize[TaskEndReason]( - serializedData, Utils.getSparkClassLoader) + try { + getTaskResultExecutor.execute(new Runnable { + override def run(): Unit = Utils.logUncaughtExceptions { + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = serializer.get().deserialize[TaskEndReason]( + serializedData, Utils.getSparkClassLoader) + } + } catch { + case cnd: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastrophic + // if we can't deserialize the reason. + val loader = Utils.getContextOrSparkClassLoader + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case ex: Exception => {} } - } catch { - case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastrophic if we can't - // deserialize the reason. - val loader = Utils.getContextOrSparkClassLoader - logError( - "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Exception => {} + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } - scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) - } - }) + }) + } catch { + case e: RejectedExecutionException if sparkEnv.isStopped => + // ignore it + } } def stop() { diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py new file mode 100644 index 0000000000000..a33bdc475a06d --- /dev/null +++ b/examples/src/main/python/status_api_demo.py @@ -0,0 +1,67 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import time +import threading +import Queue + +from pyspark import SparkConf, SparkContext + + +def delayed(seconds): + def f(x): + time.sleep(seconds) + return x + return f + + +def call_in_background(f, *args): + result = Queue.Queue(1) + t = threading.Thread(target=lambda: result.put(f(*args))) + t.daemon = True + t.start() + return result + + +def main(): + conf = SparkConf().set("spark.ui.showConsoleProgress", "false") + sc = SparkContext(appName="PythonStatusAPIDemo", conf=conf) + + def run(): + rdd = sc.parallelize(range(10), 10).map(delayed(2)) + reduced = rdd.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y) + return reduced.map(delayed(2)).collect() + + result = call_in_background(run) + status = sc.statusTracker() + while result.empty(): + ids = status.getJobIdsForGroup() + for id in ids: + job = status.getJobInfo(id) + print "Job", id, "status: ", job.status + for sid in job.stageIds: + info = status.getStageInfo(sid) + if info: + print "Stage %d: %d tasks total (%d active, %d complete)" % \ + (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks) + time.sleep(1) + + print "Job results are:", result.get() + sc.stop() + +if __name__ == "__main__": + main() diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index d3efcdf221d82..5f70ac6ed8fe6 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -22,17 +22,17 @@ - :class:`SparkContext`: Main entry point for Spark functionality. - - L{RDD} + - :class:`RDD`: A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. - - L{Broadcast} + - :class:`Broadcast`: A broadcast variable that gets reused across tasks. - - L{Accumulator} + - :class:`Accumulator`: An "add-only" shared variable that tasks can only add values to. - - L{SparkConf} + - :class:`SparkConf`: For configuring Spark. - - L{SparkFiles} + - :class:`SparkFiles`: Access files shipped with jobs. - - L{StorageLevel} + - :class:`StorageLevel`: Finer-grained cache persistence levels. """ @@ -45,6 +45,7 @@ from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast from pyspark.serializers import MarshalSerializer, PickleSerializer +from pyspark.status import * from pyspark.profiler import Profiler, BasicProfiler # for back compatibility @@ -53,5 +54,5 @@ __all__ = [ "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", - "Profiler", "BasicProfiler", + "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", ] diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 40b3152b23843..6011caf9f1c5a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -32,6 +32,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call +from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler from py4j.java_collections import ListConverter @@ -810,6 +811,12 @@ def cancelAllJobs(self): """ self._jsc.sc().cancelAllJobs() + def statusTracker(self): + """ + Return :class:`StatusTracker` object + """ + return StatusTracker(self._jsc.statusTracker()) + def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): """ Executes the given partitionFunc on the specified set of partitions, diff --git a/python/pyspark/status.py b/python/pyspark/status.py new file mode 100644 index 0000000000000..a6fa7dd3144d4 --- /dev/null +++ b/python/pyspark/status.py @@ -0,0 +1,96 @@ +# +# 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. +# + +from collections import namedtuple + +__all__ = ["SparkJobInfo", "SparkStageInfo", "StatusTracker"] + + +class SparkJobInfo(namedtuple("SparkJobInfo", "jobId stageIds status")): + """ + Exposes information about Spark Jobs. + """ + + +class SparkStageInfo(namedtuple("SparkStageInfo", + "stageId currentAttemptId name numTasks numActiveTasks " + "numCompletedTasks numFailedTasks")): + """ + Exposes information about Spark Stages. + """ + + +class StatusTracker(object): + """ + Low-level status reporting APIs for monitoring job and stage progress. + + These APIs intentionally provide very weak consistency semantics; + consumers of these APIs should be prepared to handle empty / missing + information. For example, a job's stage ids may be known but the status + API may not have any information about the details of those stages, so + `getStageInfo` could potentially return `None` for a valid stage id. + + To limit memory usage, these APIs only provide information on recent + jobs / stages. These APIs will provide information for the last + `spark.ui.retainedStages` stages and `spark.ui.retainedJobs` jobs. + """ + def __init__(self, jtracker): + self._jtracker = jtracker + + def getJobIdsForGroup(self, jobGroup=None): + """ + Return a list of all known jobs in a particular job group. If + `jobGroup` is None, then returns all known jobs that are not + associated with a job group. + + The returned list may contain running, failed, and completed jobs, + and may vary across invocations of this method. This method does + not guarantee the order of the elements in its result. + """ + return list(self._jtracker.getJobIdsForGroup(jobGroup)) + + def getActiveStageIds(self): + """ + Returns an array containing the ids of all active stages. + """ + return sorted(list(self._jtracker.getActiveStageIds())) + + def getActiveJobsIds(self): + """ + Returns an array containing the ids of all active jobs. + """ + return sorted((list(self._jtracker.getActiveJobIds()))) + + def getJobInfo(self, jobId): + """ + Returns a :class:`SparkJobInfo` object, or None if the job info + could not be found or was garbage collected. + """ + job = self._jtracker.getJobInfo(jobId) + if job is not None: + return SparkJobInfo(jobId, job.stageIds(), str(job.status())) + + def getStageInfo(self, stageId): + """ + Returns a :class:`SparkStageInfo` object, or None if the stage + info could not be found or was garbage collected. + """ + stage = self._jtracker.getStageInfo(stageId) + if stage is not None: + # TODO: fetch them in batch for better performance + attrs = [getattr(stage, f)() for f in SparkStageInfo._fields[1:]] + return SparkStageInfo(stageId, *attrs) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b5e28c498040b..d6afc1cdaa4dc 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1550,6 +1550,37 @@ def test_with_stop(self): sc.stop() self.assertEqual(SparkContext._active_spark_context, None) + def test_progress_api(self): + with SparkContext() as sc: + sc.setJobGroup('test_progress_api', '', True) + + rdd = sc.parallelize(range(10)).map(lambda x: time.sleep(100)) + t = threading.Thread(target=rdd.collect) + t.daemon = True + t.start() + # wait for scheduler to start + time.sleep(1) + + tracker = sc.statusTracker() + jobIds = tracker.getJobIdsForGroup('test_progress_api') + self.assertEqual(1, len(jobIds)) + job = tracker.getJobInfo(jobIds[0]) + self.assertEqual(1, len(job.stageIds)) + stage = tracker.getStageInfo(job.stageIds[0]) + self.assertEqual(rdd.getNumPartitions(), stage.numTasks) + + sc.cancelAllJobs() + t.join() + # wait for event listener to update the status + time.sleep(1) + + job = tracker.getJobInfo(jobIds[0]) + self.assertEqual('FAILED', job.status) + self.assertEqual([], tracker.getActiveJobsIds()) + self.assertEqual([], tracker.getActiveStageIds()) + + sc.stop() + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From 3df85dccbc8fd1ba19bbcdb8d359c073b1494d98 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 13:48:38 -0800 Subject: [PATCH 174/817] [SPARK-5871] output explain in Python Author: Davies Liu Closes #4658 from davies/explain and squashes the following commits: db87ea2 [Davies Liu] output explain in Python --- python/pyspark/sql/dataframe.py | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 841724095f693..388033d385e13 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -244,8 +244,25 @@ def explain(self, extended=False): debugging purpose. If extended is False, only prints the physical plan. - """ - self._jdf.explain(extended) + + >>> df.explain() + PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at SQLContext.scala:... + + >>> df.explain(True) + == Parsed Logical Plan == + ... + == Analyzed Logical Plan == + ... + == Optimized Logical Plan == + ... + == Physical Plan == + ... + == RDD == + """ + if extended: + print self._jdf.queryExecution().toString() + else: + print self._jdf.queryExecution().executedPlan().toString() def isLocal(self): """ @@ -1034,7 +1051,7 @@ def _test(): Row(name='Bob', age=5, height=85)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, - optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) globs['sc'].stop() if failure_count: exit(-1) From 4d4cc760fa9687ce563320094557ef9144488676 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 15:44:37 -0800 Subject: [PATCH 175/817] [SPARK-5872] [SQL] create a sqlCtx in pyspark shell The sqlCtx will be HiveContext if hive is built in assembly jar, or SQLContext if not. It also skip the Hive tests in pyspark.sql.tests if no hive is available. Author: Davies Liu Closes #4659 from davies/sqlctx and squashes the following commits: 0e6629a [Davies Liu] sqlCtx in pyspark --- python/pyspark/shell.py | 13 ++++++++++++- python/pyspark/sql/tests.py | 12 ++++++++++-- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 4cf4b89ccfaa7..1a02fece9c5a5 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -31,8 +31,12 @@ import atexit import os import platform + +import py4j + import pyspark from pyspark.context import SparkContext +from pyspark.sql import SQLContext, HiveContext from pyspark.storagelevel import StorageLevel # this is the deprecated equivalent of ADD_JARS @@ -46,6 +50,13 @@ sc = SparkContext(appName="PySparkShell", pyFiles=add_files) atexit.register(lambda: sc.stop()) +try: + # Try to access HiveConf, it will raise exception if Hive is not added + sc._jvm.org.apache.hadoop.hive.conf.HiveConf() + sqlCtx = HiveContext(sc) +except py4j.protocol.Py4JError: + sqlCtx = SQLContext(sc) + print("""Welcome to ____ __ / __/__ ___ _____/ /__ @@ -57,7 +68,7 @@ platform.python_version(), platform.python_build()[0], platform.python_build()[1])) -print("SparkContext available as sc.") +print("SparkContext available as sc, %s available as sqlCtx." % sqlCtx.__class__.__name__) if add_files is not None: print("Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead") diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index aa80bca34655d..52f7e65d9ca78 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -25,6 +25,8 @@ import shutil import tempfile +import py4j + if sys.version_info[:2] <= (2, 6): try: import unittest2 as unittest @@ -329,9 +331,12 @@ class HiveContextSQLTests(ReusedPySparkTestCase): def setUpClass(cls): ReusedPySparkTestCase.setUpClass() cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + try: + cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf() + except py4j.protocol.Py4JError: + cls.sqlCtx = None + return os.unlink(cls.tempdir.name) - print "type", type(cls.sc) - print "type", type(cls.sc._jsc) _scala_HiveContext =\ cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) cls.sqlCtx = HiveContext(cls.sc, _scala_HiveContext) @@ -344,6 +349,9 @@ def tearDownClass(cls): shutil.rmtree(cls.tempdir.name, ignore_errors=True) def test_save_and_load_table(self): + if self.sqlCtx is None: + return # no hive available, skipped + df = self.df tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) From 117121a4ecaadda156a82255333670775e7727db Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 Feb 2015 15:47:59 -0800 Subject: [PATCH 176/817] [SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table to a data source parquet table. The problem is that after we create an empty hive metastore parquet table (e.g. `CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir for us, which cause our data source `ParquetRelation2` fail to get the schema of the table. See JIRA for the case to reproduce the bug and the exception. This PR is based on #4562 from chenghao-intel. JIRA: https://issues.apache.org/jira/browse/SPARK-5852 Author: Yin Huai Author: Cheng Hao Closes #4655 from yhuai/CTASParquet and squashes the following commits: b8b3450 [Yin Huai] Update tests. 2ac94f7 [Yin Huai] Update tests. 3db3d20 [Yin Huai] Minor update. d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala. 36978d1 [Cheng Hao] Update the code as feedback a04930b [Cheng Hao] fix bug of scan an empty parquet based table 442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext --- .../apache/spark/sql/parquet/newParquet.scala | 18 ++- .../sql/hive/MetastoreDataSourcesSuite.scala | 38 ++++++ .../spark/sql/parquet/parquetSuites.scala | 114 +++++++++++++++++- 3 files changed, 164 insertions(+), 6 deletions(-) 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 95bea9201163d..16b771344bfcd 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 @@ -287,7 +287,16 @@ private[sql] case class ParquetRelation2( } } - parquetSchema = maybeSchema.getOrElse(readSchema()) + // To get the schema. We first try to get the schema defined in maybeSchema. + // If maybeSchema is not defined, we will try to get the schema from existing parquet data + // (through readSchema). If data does not exist, we will try to get the schema defined in + // maybeMetastoreSchema (defined in the options of the data source). + // Finally, if we still could not get the schema. We throw an error. + parquetSchema = + maybeSchema + .orElse(readSchema()) + .orElse(maybeMetastoreSchema) + .getOrElse(sys.error("Failed to get the schema.")) partitionKeysIncludedInParquetSchema = isPartitioned && @@ -308,7 +317,7 @@ private[sql] case class ParquetRelation2( } } - private def readSchema(): StructType = { + private def readSchema(): Option[StructType] = { // Sees which file(s) we need to touch in order to figure out the schema. val filesToTouch = // Always tries the summary files first if users don't require a merged schema. In this case, @@ -611,7 +620,8 @@ private[sql] object ParquetRelation2 { // internally. private[sql] val METASTORE_SCHEMA = "metastoreSchema" - private[parquet] def readSchema(footers: Seq[Footer], sqlContext: SQLContext): StructType = { + private[parquet] def readSchema( + footers: Seq[Footer], sqlContext: SQLContext): Option[StructType] = { footers.map { footer => val metadata = footer.getParquetMetadata.getFileMetaData val parquetSchema = metadata.getSchema @@ -630,7 +640,7 @@ private[sql] object ParquetRelation2 { sqlContext.conf.isParquetBinaryAsString, sqlContext.conf.isParquetINT96AsTimestamp)) } - }.reduce { (left, right) => + }.reduceOption { (left, right) => try left.merge(right) catch { case e: Throwable => throw new SparkException(s"Failed to merge incompatible schemas $left and $right", e) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 485d5c95bfc44..c30090fabbc8d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.io.File + import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils @@ -30,6 +31,8 @@ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.sql.parquet.ParquetRelation2 +import org.apache.spark.sql.sources.LogicalRelation /** * Tests for persisting tables created though the data sources API into the metastore. @@ -553,4 +556,39 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("DROP TABLE savedJsonTable") conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) } + + if (HiveShim.version == "0.13.1") { + test("scan a parquet table created through a CTAS statement") { + val originalConvertMetastore = getConf("spark.sql.hive.convertMetastoreParquet", "true") + val originalUseDataSource = getConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + setConf("spark.sql.hive.convertMetastoreParquet", "true") + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + jsonRDD(rdd).registerTempTable("jt") + sql( + """ + |create table test_parquet_ctas STORED AS parquET + |AS select tmp.a from jt tmp where tmp.a < 5 + """.stripMargin) + + checkAnswer( + sql(s"SELECT a FROM test_parquet_ctas WHERE a > 2 "), + Row(3) :: Row(4) :: Nil + ) + + table("test_parquet_ctas").queryExecution.analyzed match { + case LogicalRelation(p: ParquetRelation2) => // OK + case _ => + fail( + s"test_parquet_ctas should be converted to ${classOf[ParquetRelation2].getCanonicalName}") + } + + // Clenup and reset confs. + sql("DROP TABLE IF EXISTS jt") + sql("DROP TABLE IF EXISTS test_parquet_ctas") + setConf("spark.sql.hive.convertMetastoreParquet", originalConvertMetastore) + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalUseDataSource) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 2acf1a7767c19..653f4b47367c4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -20,15 +20,15 @@ package org.apache.spark.sql.parquet import java.io.File -import org.apache.spark.sql.catalyst.expressions.Row import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.PhysicalRDD import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ - +import org.apache.spark.sql.sources.LogicalRelation // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -121,13 +121,123 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { override def beforeAll(): Unit = { super.beforeAll() + + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + jsonRDD(rdd).registerTempTable("jt") + + sql( + """ + |create table test_parquet + |( + | intField INT, + | stringField STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") } override def afterAll(): Unit = { super.afterAll() + sql("DROP TABLE IF EXISTS jt") + sql("DROP TABLE IF EXISTS test_parquet") + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("scan an empty parquet table") { + checkAnswer(sql("SELECT count(*) FROM test_parquet"), Row(0)) + } + + test("scan an empty parquet table with upper case") { + checkAnswer(sql("SELECT count(INTFIELD) FROM TEST_parquet"), Row(0)) + } + + test("insert into an empty parquet table") { + sql( + """ + |create table test_insert_parquet + |( + | intField INT, + | stringField STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + // Insert into am empty table. + sql("insert into table test_insert_parquet select a, b from jt where jt.a > 5") + checkAnswer( + sql(s"SELECT intField, stringField FROM test_insert_parquet WHERE intField < 8"), + Row(6, "str6") :: Row(7, "str7") :: Nil + ) + // Insert overwrite. + sql("insert overwrite table test_insert_parquet select a, b from jt where jt.a < 5") + checkAnswer( + sql(s"SELECT intField, stringField FROM test_insert_parquet WHERE intField > 2"), + Row(3, "str3") :: Row(4, "str4") :: Nil + ) + sql("DROP TABLE IF EXISTS test_insert_parquet") + + // Create it again. + sql( + """ + |create table test_insert_parquet + |( + | intField INT, + | stringField STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + // Insert overwrite an empty table. + sql("insert overwrite table test_insert_parquet select a, b from jt where jt.a < 5") + checkAnswer( + sql(s"SELECT intField, stringField FROM test_insert_parquet WHERE intField > 2"), + Row(3, "str3") :: Row(4, "str4") :: Nil + ) + // Insert into the table. + sql("insert into table test_insert_parquet select a, b from jt") + checkAnswer( + sql(s"SELECT intField, stringField FROM test_insert_parquet"), + (1 to 10).map(i => Row(i, s"str$i")) ++ (1 to 4).map(i => Row(i, s"str$i")) + ) + sql("DROP TABLE IF EXISTS test_insert_parquet") + } + + test("scan a parquet table created through a CTAS statement") { + sql( + """ + |create table test_parquet_ctas ROW FORMAT + |SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + |AS select * from jt + """.stripMargin) + + checkAnswer( + sql(s"SELECT a, b FROM test_parquet_ctas WHERE a = 1"), + Seq(Row(1, "str1")) + ) + + table("test_parquet_ctas").queryExecution.analyzed match { + case LogicalRelation(p: ParquetRelation2) => // OK + case _ => + fail( + s"test_parquet_ctas should be converted to ${classOf[ParquetRelation2].getCanonicalName}") + } + + sql("DROP TABLE IF EXISTS test_parquet_ctas") + } } class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { From c3d2b90bde2e11823909605d518167548df66bd8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 17 Feb 2015 16:54:57 -0800 Subject: [PATCH 177/817] [SPARK-5785] [PySpark] narrow dependency for cogroup/join in PySpark Currently, PySpark does not support narrow dependency during cogroup/join when the two RDDs have the partitioner, another unnecessary shuffle stage will come in. The Python implementation of cogroup/join is different than Scala one, it depends on union() and partitionBy(). This patch will try to use PartitionerAwareUnionRDD() in union(), when all the RDDs have the same partitioner. It also fix `reservePartitioner` in all the map() or mapPartitions(), then partitionBy() can skip the unnecessary shuffle stage. Author: Davies Liu Closes #4629 from davies/narrow and squashes the following commits: dffe34e [Davies Liu] improve test, check number of stages for join/cogroup 1ed3ba2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into narrow 4d29932 [Davies Liu] address comment cc28d97 [Davies Liu] add unit tests 940245e [Davies Liu] address comments ff5a0a6 [Davies Liu] skip the partitionBy() on Python side eb26c62 [Davies Liu] narrow dependency in PySpark --- .../scala/org/apache/spark/SparkContext.scala | 11 ++++- .../apache/spark/api/python/PythonRDD.scala | 10 ++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 8 ++- python/pyspark/join.py | 8 +-- python/pyspark/rdd.py | 49 +++++++++++++------ python/pyspark/streaming/dstream.py | 2 +- python/pyspark/tests.py | 38 +++++++++++++- 7 files changed, 101 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fd8fac6df0d0e..d59b466830fdc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -961,11 +961,18 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** Build the union of a list of RDDs. */ - def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds) + def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = { + val partitioners = rdds.flatMap(_.partitioner).toSet + if (partitioners.size == 1) { + new PartitionerAwareUnionRDD(this, rdds) + } else { + new UnionRDD(this, rdds) + } + } /** Build the union of a list of RDDs passed as variable-length arguments. */ def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = - new UnionRDD(this, Seq(first) ++ rest) + union(Seq(first) ++ rest) /** Get an RDD that has no partitions or elements. */ def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 252721192904f..dcb6e6313a1d2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -303,6 +303,7 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Long, Array[Byte])](prev) { override def getPartitions = prev.partitions + override val partitioner = prev.partitioner override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (Utils.deserializeLongValue(a), b) @@ -329,6 +330,15 @@ private[spark] object PythonRDD extends Logging { } } + /** + * Return an RDD of values from an RDD of (Long, Array[Byte]), with preservePartitions=true + * + * This is useful for PySpark to have the partitioner after partitionBy() + */ + def valueOfPair(pair: JavaPairRDD[Long, Array[Byte]]): JavaRDD[Array[Byte]] = { + pair.rdd.mapPartitions(it => it.map(_._2), true) + } + /** * Adapter for calling SparkContext#runJob from Python. * diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index fe55a5124f3b6..3ab9e54f0ec56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -462,7 +462,13 @@ abstract class RDD[T: ClassTag]( * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). */ - def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other)) + def union(other: RDD[T]): RDD[T] = { + if (partitioner.isDefined && other.partitioner == partitioner) { + new PartitionerAwareUnionRDD(sc, Array(this, other)) + } else { + new UnionRDD(sc, Array(this, other)) + } + } /** * Return the union of this RDD and another one. Any identical elements will appear multiple diff --git a/python/pyspark/join.py b/python/pyspark/join.py index b4a844713745a..efc1ef9396412 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -35,8 +35,8 @@ def _do_python_join(rdd, other, numPartitions, dispatch): - vs = rdd.map(lambda (k, v): (k, (1, v))) - ws = other.map(lambda (k, v): (k, (2, v))) + vs = rdd.mapValues(lambda v: (1, v)) + ws = other.mapValues(lambda v: (2, v)) return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x: dispatch(x.__iter__())) @@ -98,8 +98,8 @@ def dispatch(seq): def python_cogroup(rdds, numPartitions): def make_mapper(i): - return lambda (k, v): (k, (i, v)) - vrdds = [rdd.map(make_mapper(i)) for i, rdd in enumerate(rdds)] + return lambda v: (i, v) + vrdds = [rdd.mapValues(make_mapper(i)) for i, rdd in enumerate(rdds)] union_vrdds = reduce(lambda acc, other: acc.union(other), vrdds) rdd_len = len(vrdds) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index bd4f16e058045..ba2347ae76844 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -111,6 +111,19 @@ def _parse_memory(s): return int(float(s[:-1]) * units[s[-1].lower()]) +class Partitioner(object): + def __init__(self, numPartitions, partitionFunc): + self.numPartitions = numPartitions + self.partitionFunc = partitionFunc + + def __eq__(self, other): + return (isinstance(other, Partitioner) and self.numPartitions == other.numPartitions + and self.partitionFunc == other.partitionFunc) + + def __call__(self, k): + return self.partitionFunc(k) % self.numPartitions + + class RDD(object): """ @@ -126,7 +139,7 @@ def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSeri self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer self._id = jrdd.id() - self._partitionFunc = None + self.partitioner = None def _pickled(self): return self._reserialize(AutoBatchedSerializer(PickleSerializer())) @@ -450,14 +463,17 @@ def union(self, other): if self._jrdd_deserializer == other._jrdd_deserializer: rdd = RDD(self._jrdd.union(other._jrdd), self.ctx, self._jrdd_deserializer) - return rdd else: # These RDDs contain data in different serialized formats, so we # must normalize them to the default serializer. self_copy = self._reserialize() other_copy = other._reserialize() - return RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx, - self.ctx.serializer) + rdd = RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx, + self.ctx.serializer) + if (self.partitioner == other.partitioner and + self.getNumPartitions() == rdd.getNumPartitions()): + rdd.partitioner = self.partitioner + return rdd def intersection(self, other): """ @@ -1588,6 +1604,9 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): """ if numPartitions is None: numPartitions = self._defaultReducePartitions() + partitioner = Partitioner(numPartitions, partitionFunc) + if self.partitioner == partitioner: + return self # Transferring O(n) objects to Java is too expensive. # Instead, we'll form the hash buckets in Python, @@ -1632,18 +1651,16 @@ def add_shuffle_key(split, iterator): yield pack_long(split) yield outputSerializer.dumps(items) - keyed = self.mapPartitionsWithIndex(add_shuffle_key) + keyed = self.mapPartitionsWithIndex(add_shuffle_key, preservesPartitioning=True) keyed._bypass_serializer = True with SCCallSiteSync(self.context) as css: pairRDD = self.ctx._jvm.PairwiseRDD( keyed._jrdd.rdd()).asJavaPairRDD() - partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, - id(partitionFunc)) - jrdd = pairRDD.partitionBy(partitioner).values() + jpartitioner = self.ctx._jvm.PythonPartitioner(numPartitions, + id(partitionFunc)) + jrdd = self.ctx._jvm.PythonRDD.valueOfPair(pairRDD.partitionBy(jpartitioner)) rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer)) - # This is required so that id(partitionFunc) remains unique, - # even if partitionFunc is a lambda: - rdd._partitionFunc = partitionFunc + rdd.partitioner = partitioner return rdd # TODO: add control over map-side aggregation @@ -1689,7 +1706,7 @@ def combineLocally(iterator): merger.mergeValues(iterator) return merger.iteritems() - locally_combined = self.mapPartitions(combineLocally) + locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) def _mergeCombiners(iterator): @@ -1698,7 +1715,7 @@ def _mergeCombiners(iterator): merger.mergeCombiners(iterator) return merger.iteritems() - return shuffled.mapPartitions(_mergeCombiners, True) + return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True) def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None): """ @@ -2077,8 +2094,8 @@ def lookup(self, key): """ values = self.filter(lambda (k, v): k == key).values() - if self._partitionFunc is not None: - return self.ctx.runJob(values, lambda x: x, [self._partitionFunc(key)], False) + if self.partitioner is not None: + return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)], False) return values.collect() @@ -2243,7 +2260,7 @@ def pipeline_func(split, iterator): self._id = None self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False - self._partitionFunc = prev._partitionFunc if self.preservesPartitioning else None + self.partitioner = prev.partitioner if self.preservesPartitioning else None self._broadcast = None def __del__(self): diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 2fe39392ff081..3fa42444239f7 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -578,7 +578,7 @@ def reduceFunc(t, a, b): if a is None: g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) else: - g = a.cogroup(b, numPartitions) + g = a.cogroup(b.partitionBy(numPartitions), numPartitions) g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) return state.filter(lambda (k, v): v is not None) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index d6afc1cdaa4dc..f64e25c60740a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -727,7 +727,6 @@ def test_multiple_python_java_RDD_conversions(self): (u'1', {u'director': u'David Lean'}), (u'2', {u'director': u'Andrew Dominik'}) ] - from pyspark.rdd import RDD data_rdd = self.sc.parallelize(data) data_java_rdd = data_rdd._to_java_object_rdd() data_python_rdd = self.sc._jvm.SerDe.javaToPython(data_java_rdd) @@ -740,6 +739,43 @@ def test_multiple_python_java_RDD_conversions(self): converted_rdd = RDD(data_python_rdd, self.sc) self.assertEqual(2, converted_rdd.count()) + def test_narrow_dependency_in_join(self): + rdd = self.sc.parallelize(range(10)).map(lambda x: (x, x)) + parted = rdd.partitionBy(2) + self.assertEqual(2, parted.union(parted).getNumPartitions()) + self.assertEqual(rdd.getNumPartitions() + 2, parted.union(rdd).getNumPartitions()) + self.assertEqual(rdd.getNumPartitions() + 2, rdd.union(parted).getNumPartitions()) + + self.sc.setJobGroup("test1", "test", True) + tracker = self.sc.statusTracker() + + d = sorted(parted.join(parted).collect()) + self.assertEqual(10, len(d)) + self.assertEqual((0, (0, 0)), d[0]) + jobId = tracker.getJobIdsForGroup("test1")[0] + self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) + + self.sc.setJobGroup("test2", "test", True) + d = sorted(parted.join(rdd).collect()) + self.assertEqual(10, len(d)) + self.assertEqual((0, (0, 0)), d[0]) + jobId = tracker.getJobIdsForGroup("test2")[0] + self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) + + self.sc.setJobGroup("test3", "test", True) + d = sorted(parted.cogroup(parted).collect()) + self.assertEqual(10, len(d)) + self.assertEqual([[0], [0]], map(list, d[0][1])) + jobId = tracker.getJobIdsForGroup("test3")[0] + self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) + + self.sc.setJobGroup("test4", "test", True) + d = sorted(parted.cogroup(rdd).collect()) + self.assertEqual(10, len(d)) + self.assertEqual([[0], [0]], map(list, d[0][1])) + jobId = tracker.getJobIdsForGroup("test4")[0] + self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) + class ProfilerTests(PySparkTestCase): From ae6cfb3acdbc2721d25793698a4a440f0519dbec Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 17 Feb 2015 17:15:43 -0800 Subject: [PATCH 178/817] [SPARK-5811] Added documentation for maven coordinates and added Spark Packages support Documentation for maven coordinates + Spark Package support. Added pyspark tests for `--packages` Author: Burak Yavuz Author: Davies Liu Closes #4662 from brkyvz/SPARK-5811 and squashes the following commits: 56ccccd [Burak Yavuz] fixed broken test 64cb8ee [Burak Yavuz] passed pep8 on local c07b81e [Burak Yavuz] fixed pep8 a8bd6b7 [Burak Yavuz] submit PR 4ef4046 [Burak Yavuz] ready for PR 8fb02e5 [Burak Yavuz] merged master 25c9b9f [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into python-jar 560d13b [Burak Yavuz] before PR 17d3f76 [Davies Liu] support .jar as python package a3eb717 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into SPARK-5811 c60156d [Burak Yavuz] [SPARK-5811] Added documentation for maven coordinates --- .../org/apache/spark/deploy/SparkSubmit.scala | 52 ++++++++++---- .../spark/deploy/SparkSubmitUtilsSuite.scala | 13 ++-- docs/programming-guide.md | 19 ++++- docs/submitting-applications.md | 5 ++ python/pyspark/tests.py | 69 +++++++++++++++++-- 5 files changed, 131 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 012a89a31b046..4c4110812e0a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -252,6 +252,26 @@ object SparkSubmit { val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files + // too for packages that include Python code + val resolvedMavenCoordinates = + SparkSubmitUtils.resolveMavenCoordinates( + args.packages, Option(args.repositories), Option(args.ivyRepoPath)) + if (!resolvedMavenCoordinates.trim.isEmpty) { + if (args.jars == null || args.jars.trim.isEmpty) { + args.jars = resolvedMavenCoordinates + } else { + args.jars += s",$resolvedMavenCoordinates" + } + if (args.isPython) { + if (args.pyFiles == null || args.pyFiles.trim.isEmpty) { + args.pyFiles = resolvedMavenCoordinates + } else { + args.pyFiles += s",$resolvedMavenCoordinates" + } + } + } + // Require all python files to be local, so we can add them to the PYTHONPATH // In YARN cluster mode, python files are distributed as regular files, which can be non-local if (args.isPython && !isYarnCluster) { @@ -307,18 +327,6 @@ object SparkSubmit { // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" - // Resolve maven dependencies if there are any and add classpath to jars - val resolvedMavenCoordinates = - SparkSubmitUtils.resolveMavenCoordinates( - args.packages, Option(args.repositories), Option(args.ivyRepoPath)) - if (!resolvedMavenCoordinates.trim.isEmpty) { - if (args.jars == null || args.jars.trim.isEmpty) { - args.jars = resolvedMavenCoordinates - } else { - args.jars += s",$resolvedMavenCoordinates" - } - } - // A list of rules to map each argument to system properties or command-line options in // each deploy mode; we iterate through these below val options = List[OptionAssigner]( @@ -646,13 +654,15 @@ private[spark] object SparkSubmitUtils { private[spark] case class MavenCoordinate(groupId: String, artifactId: String, version: String) /** - * Extracts maven coordinates from a comma-delimited string + * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided + * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. The latter provides + * simplicity for Spark Package users. * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ private[spark] def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { coordinates.split(",").map { p => - val splits = p.split(":") + val splits = p.replace("/", ":").split(":") require(splits.length == 3, s"Provided Maven Coordinates must be in the form " + s"'groupId:artifactId:version'. The coordinate provided is: $p") require(splits(0) != null && splits(0).trim.nonEmpty, s"The groupId cannot be null or " + @@ -682,6 +692,13 @@ private[spark] object SparkSubmitUtils { br.setName("central") cr.add(br) + val sp: IBiblioResolver = new IBiblioResolver + sp.setM2compatible(true) + sp.setUsepoms(true) + sp.setRoot("http://dl.bintray.com/spark-packages/maven") + sp.setName("spark-packages") + cr.add(sp) + val repositoryList = remoteRepos.getOrElse("") // add any other remote repositories other than maven central if (repositoryList.trim.nonEmpty) { @@ -794,14 +811,19 @@ private[spark] object SparkSubmitUtils { val md = getModuleDescriptor md.setDefaultConf(ivyConfName) - // Add an exclusion rule for Spark + // Add an exclusion rule for Spark and Scala Library val sparkArtifacts = new ArtifactId(new ModuleId("org.apache.spark", "*"), "*", "*", "*") val sparkDependencyExcludeRule = new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null) sparkDependencyExcludeRule.addConfiguration(ivyConfName) + val scalaArtifacts = new ArtifactId(new ModuleId("*", "scala-library"), "*", "*", "*") + val scalaDependencyExcludeRule = + new DefaultExcludeRule(scalaArtifacts, ivySettings.getMatcher("glob"), null) + scalaDependencyExcludeRule.addConfiguration(ivyConfName) // Exclude any Spark dependencies, and add all supplied maven artifacts as dependencies md.addExcludeRule(sparkDependencyExcludeRule) + md.addExcludeRule(scalaDependencyExcludeRule) addDependenciesToIvy(md, artifacts, ivyConfName) // resolve dependencies diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 53665350013cd..ad62b35f624f6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -57,20 +57,23 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { test("create repo resolvers") { val resolver1 = SparkSubmitUtils.createRepoResolvers(None) - // should have central by default - assert(resolver1.getResolvers.size() === 1) + // should have central and spark-packages by default + assert(resolver1.getResolvers.size() === 2) assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central") + assert(resolver1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "spark-packages") val repos = "a/1,b/2,c/3" val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos)) - assert(resolver2.getResolvers.size() === 4) + assert(resolver2.getResolvers.size() === 5) val expected = repos.split(",").map(r => s"$r/") resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) => if (i == 0) { assert(resolver.getName === "central") + } else if (i == 1) { + assert(resolver.getName === "spark-packages") } else { - assert(resolver.getName === s"repo-$i") - assert(resolver.getRoot === expected(i - 1)) + assert(resolver.getName === s"repo-${i - 1}") + assert(resolver.getRoot === expected(i - 2)) } } } diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 118701549a759..4e4af76316863 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -173,8 +173,11 @@ in-process. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `--master` argument, and you can add JARs to the classpath -by passing a comma-separated list to the `--jars` argument. -For example, to run `bin/spark-shell` on exactly four cores, use: +by passing a comma-separated list to the `--jars` argument. You can also add dependencies +(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) +can be passed to the `--repositories` argument. For example, to run `bin/spark-shell` on exactly +four cores, use: {% highlight bash %} $ ./bin/spark-shell --master local[4] @@ -186,6 +189,12 @@ Or, to also add `code.jar` to its classpath, use: $ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} +To include a dependency using maven coordinates: + +{% highlight bash %} +$ ./bin/spark-shell --master local[4] --packages "org.example:example:0.1" +{% endhighlight %} + For a complete list of options, run `spark-shell --help`. Behind the scenes, `spark-shell` invokes the more general [`spark-submit` script](submitting-applications.html). @@ -196,7 +205,11 @@ For a complete list of options, run `spark-shell --help`. Behind the scenes, In the PySpark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `--master` argument, and you can add Python .zip, .egg or .py files -to the runtime path by passing a comma-separated list to `--py-files`. +to the runtime path by passing a comma-separated list to `--py-files`. You can also add dependencies +(e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) +can be passed to the `--repositories` argument. Any python dependencies a Spark Package has (listed in +the requirements.txt of that package) must be manually installed using pip when necessary. For example, to run `bin/pyspark` on exactly four cores, use: {% highlight bash %} diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 14a87f8436984..57b074778f2b0 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -174,6 +174,11 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. +Users may also include any other dependencies by supplying a comma-delimited list of maven coordinates +with `--packages`. All transitive dependencies will be handled when using this command. Additional +repositories (or resolvers in SBT) can be added in a comma-delimited fashion with the flag `--repositories`. +These commands can be used with `pyspark`, `spark-shell`, and `spark-submit` to include Spark Packages. + For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries to executors. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f64e25c60740a..52e82091c9f81 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1440,31 +1440,59 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.programDir) - def createTempFile(self, name, content): + def createTempFile(self, name, content, dir=None): """ Create a temp file with the given name and content and return its path. Strips leading spaces from content up to the first '|' in each line. """ pattern = re.compile(r'^ *\|', re.MULTILINE) content = re.sub(pattern, '', content.strip()) - path = os.path.join(self.programDir, name) + if dir is None: + path = os.path.join(self.programDir, name) + else: + os.makedirs(os.path.join(self.programDir, dir)) + path = os.path.join(self.programDir, dir, name) with open(path, "w") as f: f.write(content) return path - def createFileInZip(self, name, content): + def createFileInZip(self, name, content, ext=".zip", dir=None, zip_name=None): """ Create a zip archive containing a file with the given content and return its path. Strips leading spaces from content up to the first '|' in each line. """ pattern = re.compile(r'^ *\|', re.MULTILINE) content = re.sub(pattern, '', content.strip()) - path = os.path.join(self.programDir, name + ".zip") + if dir is None: + path = os.path.join(self.programDir, name + ext) + else: + path = os.path.join(self.programDir, dir, zip_name + ext) zip = zipfile.ZipFile(path, 'w') zip.writestr(name, content) zip.close() return path + def create_spark_package(self, artifact_name): + group_id, artifact_id, version = artifact_name.split(":") + self.createTempFile("%s-%s.pom" % (artifact_id, version), (""" + | + | + | 4.0.0 + | %s + | %s + | %s + | + """ % (group_id, artifact_id, version)).lstrip(), + os.path.join(group_id, artifact_id, version)) + self.createFileInZip("%s.py" % artifact_id, """ + |def myfunc(x): + | return x + 1 + """, ".jar", os.path.join(group_id, artifact_id, version), + "%s-%s" % (artifact_id, version)) + def test_single_script(self): """Submit and test a single script file""" script = self.createTempFile("test.py", """ @@ -1533,6 +1561,39 @@ def test_module_dependency_on_cluster(self): self.assertEqual(0, proc.returncode) self.assertIn("[2, 3, 4]", out) + def test_package_dependency(self): + """Submit and test a script with a dependency on a Spark Package""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + self.create_spark_package("a:mylib:0.1") + proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", + "file:" + self.programDir, script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + + def test_package_dependency_on_cluster(self): + """Submit and test a script with a dependency on a Spark Package on a cluster""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + self.create_spark_package("a:mylib:0.1") + proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", + "file:" + self.programDir, "--master", + "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + def test_single_script_on_cluster(self): """Submit and test a single script on a cluster""" script = self.createTempFile("test.py", """ From d46d6246d225ff3af09ebae1a09d4de2430c502d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 17 Feb 2015 17:39:58 -0800 Subject: [PATCH 179/817] [SPARK-4454] Properly synchronize accesses to DAGScheduler cacheLocs map This patch addresses a race condition in DAGScheduler by properly synchronizing accesses to its `cacheLocs` map. This map is accessed by the `getCacheLocs` and `clearCacheLocs()` methods, which can be called by separate threads, since DAGScheduler's `getPreferredLocs()` method is called by SparkContext and indirectly calls `getCacheLocs()`. If this map is cleared by the DAGScheduler event processing thread while a user thread is submitting a job and computing preferred locations, then this can cause the user thread to throw "NoSuchElementException: key not found" errors. Most accesses to DAGScheduler's internal state do not need synchronization because that state is only accessed from the event processing loop's thread. An alternative approach to fixing this bug would be to refactor this code so that SparkContext sends the DAGScheduler a message in order to get the list of preferred locations. However, this would involve more extensive changes to this code and would be significantly harder to backport to maintenance branches since some of the related code has undergone significant refactoring (e.g. the introduction of EventLoop). Since `cacheLocs` is the only state that's accessed in this way, adding simple synchronization seems like a better short-term fix. See #3345 for additional context. Author: Josh Rosen Closes #4660 from JoshRosen/SPARK-4454 and squashes the following commits: 12d64ba [Josh Rosen] Properly synchronize accesses to DAGScheduler cacheLocs map. --- .../apache/spark/scheduler/DAGScheduler.scala | 34 +++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 79035571adb05..9c355d7c3e632 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -98,7 +98,13 @@ class DAGScheduler( private[scheduler] val activeJobs = new HashSet[ActiveJob] - // Contains the locations that each RDD's partitions are cached on + /** + * Contains the locations that each RDD's partitions are cached on. This map's keys are RDD ids + * and its values are arrays indexed by partition numbers. Each array value is the set of + * locations where that RDD partition is cached. + * + * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454). + */ private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with @@ -183,18 +189,17 @@ class DAGScheduler( eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } - private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { - if (!cacheLocs.contains(rdd.id)) { + private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized { + cacheLocs.getOrElseUpdate(rdd.id, { val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) - cacheLocs(rdd.id) = blockIds.map { id => + blockIds.map { id => locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) } - } - cacheLocs(rdd.id) + }) } - private def clearCacheLocs() { + private def clearCacheLocs(): Unit = cacheLocs.synchronized { cacheLocs.clear() } @@ -1276,17 +1281,26 @@ class DAGScheduler( } /** - * Synchronized method that might be called from other threads. + * Gets the locality information associated with a partition of a particular RDD. + * + * This method is thread-safe and is called from both DAGScheduler and SparkContext. + * * @param rdd whose partitions are to be looked at * @param partition to lookup locality information for * @return list of machines that are preferred by the partition */ private[spark] - def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized { + def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = { getPreferredLocsInternal(rdd, partition, new HashSet) } - /** Recursive implementation for getPreferredLocs. */ + /** + * Recursive implementation for getPreferredLocs. + * + * This method is thread-safe because it only accesses DAGScheduler state through thread-safe + * methods (getCacheLocs()); please be careful when modifying this method, because any new + * DAGScheduler state accessed by it may require additional synchronization. + */ private def getPreferredLocsInternal( rdd: RDD[_], partition: Int, From a51fc7ef9adb6a41c4857918217f800858fced2c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 17 Feb 2015 17:45:16 -0800 Subject: [PATCH 180/817] [SPARK-4454] Revert getOrElse() cleanup in DAGScheduler.getCacheLocs() This method is performance-sensitive and this change wasn't necessary. --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9c355d7c3e632..8b62d2405ecb7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -190,13 +190,15 @@ class DAGScheduler( } private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized { - cacheLocs.getOrElseUpdate(rdd.id, { + // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times + if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) - blockIds.map { id => + cacheLocs(rdd.id) = blockIds.map { id => locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) } - }) + } + cacheLocs(rdd.id) } private def clearCacheLocs(): Unit = cacheLocs.synchronized { From d5f12bfe8f0a98d6fee114bb24376668ebe2898e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 Feb 2015 17:50:39 -0800 Subject: [PATCH 181/817] [SPARK-5875][SQL]logical.Project should not be resolved if it contains aggregates or generators https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the bug and explain the root cause. Author: Yin Huai Closes #4663 from yhuai/projectResolved and squashes the following commits: 472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or Generator, it's resolved field should be false. --- .../plans/logical/basicOperators.scala | 10 ++++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 13 +++++++- .../sql/hive/execution/SQLQuerySuite.scala | 32 ++++++++++++++++++- 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 9628e93274a11..89544add74430 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -23,6 +23,16 @@ import org.apache.spark.sql.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) + + override lazy val resolved: Boolean = { + val containsAggregatesOrGenerators = projectList.exists ( _.collect { + case agg: AggregateExpression => agg + case generator: Generator => generator + }.nonEmpty + ) + + !expressions.exists(!_.resolved) && childrenResolved && !containsAggregatesOrGenerators + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e70c651e1486e..aec7847356cd4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Literal, Alias, AttributeReference} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ @@ -58,6 +58,17 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(caseInsensitiveAnalyze(plan).resolved) } + test("check project's resolved") { + assert(Project(testRelation.output, testRelation).resolved) + + assert(!Project(Seq(UnresolvedAttribute("a")), testRelation).resolved) + + val explode = Explode(Nil, AttributeReference("a", IntegerType, nullable = true)()) + assert(!Project(Seq(Alias(explode, "explode")()), testRelation).resolved) + + assert(!Project(Seq(Alias(Count(Literal(1)), "count")()), testRelation).resolved) + } + test("analyze project") { assert( caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index e8d9eec3d88ff..ae03bc5e9953f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.{MetastoreRelation, HiveShim} 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._ @@ -316,4 +316,34 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") } + + test("logical.Project should not be resolved if it contains aggregates or generators") { + // This test is used to test the fix of SPARK-5875. + // The original issue was that Project's resolved will be true when it contains + // AggregateExpressions or Generators. However, in this case, the Project + // is not in a valid state (cannot be executed). Because of this bug, the analysis rule of + // PreInsertionCasts will actually start to work before ImplicitGenerate and then + // generates an invalid query plan. + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + jsonRDD(rdd).registerTempTable("data") + val originalConf = getConf("spark.sql.hive.convertCTAS", "false") + setConf("spark.sql.hive.convertCTAS", "false") + + sql("CREATE TABLE explodeTest (key bigInt)") + table("explodeTest").queryExecution.analyzed match { + case metastoreRelation: MetastoreRelation => // OK + case _ => + fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation") + } + + sql(s"INSERT OVERWRITE TABLE explodeTest SELECT explode(a) AS val FROM data") + checkAnswer( + sql("SELECT key from explodeTest"), + (1 to 5).flatMap(i => Row(i) :: Row(i + 1) :: Nil) + ) + + sql("DROP TABLE explodeTest") + dropTempTable("data") + setConf("spark.sql.hive.convertCTAS", originalConf) + } } From e50934f11e1e3ded21a631e5ab69db3c79467137 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 Feb 2015 18:14:33 -0800 Subject: [PATCH 182/817] [SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements. JIRA: https://issues.apache.org/jira/browse/SPARK-5723 Author: Yin Huai This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #4639 from yhuai/defaultCTASFileFormat and squashes the following commits: a568137 [Yin Huai] Merge remote-tracking branch 'upstream/master' into defaultCTASFileFormat ad2b07d [Yin Huai] Update tests and error messages. 8af5b2a [Yin Huai] Update conf key and unit test. 5a67903 [Yin Huai] Use data source write path for Hive's CTAS statements when no storage format/handler is specified. --- .../apache/spark/sql/hive/HiveContext.scala | 15 ++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 75 +++++++++++++++---- .../spark/sql/hive/execution/commands.scala | 17 +++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 6 +- .../sql/hive/execution/SQLQuerySuite.scala | 70 +++++++++++++++++ 5 files changed, 158 insertions(+), 25 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 6c55bc6be17f9..d3365b1e8f44c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -61,6 +61,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertMetastoreParquet: Boolean = getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true" + /** + * When true, a table created by a Hive CTAS statement (no USING clause) will be + * converted to a data source table, using the data source set by spark.sql.sources.default. + * The table in CTAS statement will be converted when it meets any of the following conditions: + * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or + * a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml + * is either TextFile or SequenceFile. + * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe + * is specified (no ROW FORMAT SERDE clause). + * - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format + * and no SerDe is specified (no ROW FORMAT SERDE clause). + */ + protected[sql] def convertCTAS: Boolean = + getConf("spark.sql.hive.convertCTAS", "false").toBoolean + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index cfd6f27371d0d..f7ad2efc9544e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, SQLContext} +import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} -import org.apache.spark.sql.sources.{DDLParser, LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -502,24 +502,69 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Some(sa.getQB().getTableDesc) } - execution.CreateTableAsSelect( - databaseName, - tableName, - child, - allowExisting, - desc) + // Check if the query specifies file format or storage handler. + val hasStorageSpec = desc match { + case Some(crtTbl) => + crtTbl != null && (crtTbl.getSerName != null || crtTbl.getStorageHandler != null) + case None => false + } + + if (hive.convertCTAS && !hasStorageSpec) { + // Do the conversion when spark.sql.hive.convertCTAS is true and the query + // does not specify any storage format (file format and storage handler). + if (dbName.isDefined) { + throw new AnalysisException( + "Cannot specify database name in a CTAS statement " + + "when spark.sql.hive.convertCTAS is set to true.") + } + + val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists + CreateTableUsingAsSelect( + tblName, + hive.conf.defaultDataSourceName, + temporary = false, + mode, + options = Map.empty[String, String], + child + ) + } else { + execution.CreateTableAsSelect( + databaseName, + tableName, + child, + allowExisting, + desc) + } case p: LogicalPlan if p.resolved => p case p @ CreateTableAsSelect(db, tableName, child, allowExisting, None) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - execution.CreateTableAsSelect( - databaseName, - tableName, - child, - allowExisting, - None) + if (hive.convertCTAS) { + if (dbName.isDefined) { + throw new AnalysisException( + "Cannot specify database name in a CTAS statement " + + "when spark.sql.hive.convertCTAS is set to true.") + } + + val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists + CreateTableUsingAsSelect( + tblName, + hive.conf.defaultDataSourceName, + temporary = false, + mode, + options = Map.empty[String, String], + child + ) + } else { + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + execution.CreateTableAsSelect( + databaseName, + tableName, + child, + allowExisting, + None) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 6afd8eea05418..c88d0e6b79491 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ @@ -121,7 +122,7 @@ case class CreateMetastoreDataSource( if (allowExisting) { return Seq.empty[Row] } else { - sys.error(s"Table $tableName already exists.") + throw new AnalysisException(s"Table $tableName already exists.") } } @@ -172,9 +173,11 @@ case class CreateMetastoreDataSourceAsSelect( // Check if we need to throw an exception or just return. mode match { case SaveMode.ErrorIfExists => - sys.error(s"Table $tableName already exists. " + - s"If you want to append into it, please set mode to SaveMode.Append. " + - s"Or, if you want to overwrite it, please set mode to SaveMode.Overwrite.") + throw new AnalysisException(s"Table $tableName already exists. " + + s"If you are using saveAsTable, you can set SaveMode to SaveMode.Append to " + + s"insert data into the table or set SaveMode to SaveMode.Overwrite to overwrite" + + s"the existing data. " + + s"Or, if you are using SQL CREATE TABLE, you need to drop $tableName first.") case SaveMode.Ignore => // Since the table already exists and the save mode is Ignore, we will just return. return Seq.empty[Row] @@ -199,7 +202,7 @@ case class CreateMetastoreDataSourceAsSelect( s"== Actual Schema ==" +: createdRelation.schema.treeString.split("\\\n")).mkString("\n")} """.stripMargin - sys.error(errorMessage) + throw new AnalysisException(errorMessage) } else if (i != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + @@ -216,10 +219,10 @@ case class CreateMetastoreDataSourceAsSelect( s"== Actual Relation ==" :: createdRelation.toString :: Nil).mkString("\n")} """.stripMargin - sys.error(errorMessage) + throw new AnalysisException(errorMessage) } case o => - sys.error(s"Saving data in ${o.toString} is not supported.") + throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } case SaveMode.Overwrite => hiveContext.sql(s"DROP TABLE IF EXISTS $tableName") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index c30090fabbc8d..e5156ae821bf4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -306,8 +306,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { |SELECT * FROM jsonTable """.stripMargin) - // Create the table again should trigger a AlreadyExistsException. - val message = intercept[RuntimeException] { + // Create the table again should trigger a AnalysisException. + val message = intercept[AnalysisException] { sql( s""" |CREATE TABLE ctasJsonTable @@ -516,7 +516,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("SELECT * FROM createdJsonTable"), df.collect()) - var message = intercept[RuntimeException] { + var message = intercept[AnalysisException] { createExternalTable("createdJsonTable", filePath.toString) }.getMessage assert(message.contains("Table createdJsonTable already exists."), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index ae03bc5e9953f..f2bc73bf3bdf9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.hive.{MetastoreRelation, HiveShim} 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._ +import org.apache.spark.sql.parquet.ParquetRelation2 +import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} @@ -42,6 +45,73 @@ class SQLQuerySuite extends QueryTest { ) } + test("CTAS without serde") { + def checkRelation(tableName: String, isDataSourceParquet: Boolean): Unit = { + val relation = EliminateSubQueries(catalog.lookupRelation(Seq(tableName))) + relation match { + case LogicalRelation(r: ParquetRelation2) => + if (!isDataSourceParquet) { + fail( + s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + + s"${ParquetRelation2.getClass.getCanonicalName}.") + } + + case r: MetastoreRelation => + if (isDataSourceParquet) { + fail( + s"${ParquetRelation2.getClass.getCanonicalName} is expected, but found " + + s"${classOf[MetastoreRelation].getCanonicalName}.") + } + } + } + + val originalConf = getConf("spark.sql.hive.convertCTAS", "false") + + setConf("spark.sql.hive.convertCTAS", "true") + + sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + var message = intercept[AnalysisException] { + sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + }.getMessage + assert(message.contains("Table ctas1 already exists")) + checkRelation("ctas1", true) + sql("DROP TABLE ctas1") + + // Specifying database name for query can be converted to data source write path + // is not allowed right now. + message = intercept[AnalysisException] { + sql("CREATE TABLE default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + }.getMessage + assert( + message.contains("Cannot specify database name in a CTAS statement"), + "When spark.sql.hive.convertCTAS is true, we should not allow " + + "database name specified.") + + sql("CREATE TABLE ctas1 stored as textfile AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true) + sql("DROP TABLE ctas1") + + sql( + "CREATE TABLE ctas1 stored as sequencefile AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true) + sql("DROP TABLE ctas1") + + sql("CREATE TABLE ctas1 stored as rcfile AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false) + sql("DROP TABLE ctas1") + + sql("CREATE TABLE ctas1 stored as orc AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false) + sql("DROP TABLE ctas1") + + sql("CREATE TABLE ctas1 stored as parquet AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false) + sql("DROP TABLE ctas1") + + setConf("spark.sql.hive.convertCTAS", originalConf) + } + test("CTAS with serde") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect() sql( From 3912d332464dcd124c60b734724c34d9742466a4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 17 Feb 2015 22:44:16 -0800 Subject: [PATCH 183/817] [SPARK-5731][Streaming][Test] Fix incorrect test in DirectKafkaStreamSuite The test was incorrect. Instead of counting the number of records, it counted the number of partitions of RDD generated by DStream. Which is not its intention. I will be testing this patch multiple times to understand its flakiness. PS: This was caused by my refactoring in https://github.com/apache/spark/pull/4384/ koeninger check it out. Author: Tathagata Das Closes #4597 from tdas/kafka-flaky-test and squashes the following commits: d236235 [Tathagata Das] Unignored last test. e9a1820 [Tathagata Das] fix test --- .../kafka/DirectKafkaStreamSuite.scala | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 926094449e7fc..17ca9d145d665 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -20,20 +20,21 @@ package org.apache.spark.streaming.kafka import java.io.File import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata import kafka.serializer.StringDecoder import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.scalatest.concurrent.{Eventually, Timeouts} +import org.scalatest.concurrent.Eventually -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} -import org.apache.spark.streaming.dstream.{DStream, InputDStream} +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata class DirectKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with BeforeAndAfterAll with Eventually { @@ -67,13 +68,14 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } - ignore("basic stream receiving with multiple topics and smallest starting offset") { + test("basic stream receiving with multiple topics and smallest starting offset") { val topics = Set("basic1", "basic2", "basic3") val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => createTopic(t) sendMessages(t, data) } + val totalSent = data.values.sum * topics.size val kafkaParams = Map( "metadata.broker.list" -> s"$brokerAddress", "auto.offset.reset" -> "smallest" @@ -84,7 +86,8 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics) } - var total = 0L + + val allReceived = new ArrayBuffer[(String, String)] stream.foreachRDD { rdd => // Get the offset ranges in the RDD @@ -104,16 +107,17 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase collected.foreach { case (partSize, rangeSize) => assert(partSize === rangeSize, "offset ranges are wrong") } - total += collected.size // Add up all the collected items } + stream.foreachRDD { rdd => allReceived ++= rdd.collect() } ssc.start() eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(total === data.values.sum * topics.size, "didn't get all messages") + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) } ssc.stop() } - ignore("receiving from largest starting offset") { + test("receiving from largest starting offset") { val topic = "largest" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) @@ -158,7 +162,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } - ignore("creating stream by offset") { + test("creating stream by offset") { val topic = "offset" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) @@ -204,7 +208,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Test to verify the offset ranges can be recovered from the checkpoints - ignore("offset recovery") { + test("offset recovery") { val topic = "recovery" createTopic(topic) testDir = Utils.createTempDir() From 61ab08549cb6fceb6de1b5c490c55a89d4bd28fa Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 17 Feb 2015 23:36:20 -0800 Subject: [PATCH 184/817] [Minor] [SQL] Cleans up DataFrame variable names and toDF() calls Although we've migrated to the DataFrame API, lots of code still uses `rdd` or `srdd` as local variable names. This PR tries to address these naming inconsistencies and some other minor DataFrame related style issues. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4670) Author: Cheng Lian Closes #4670 from liancheng/df-cleanup and squashes the following commits: 3e14448 [Cheng Lian] Cleans up DataFrame variable names and toDF() calls --- .../examples/ml/CrossValidatorExample.scala | 2 +- .../examples/ml/DeveloperApiExample.scala | 4 +- .../spark/examples/ml/MovieLensALS.scala | 6 +-- .../examples/ml/SimpleParamsExample.scala | 6 +-- .../ml/SimpleTextClassificationPipeline.scala | 4 +- .../spark/examples/mllib/DatasetExample.scala | 2 +- .../spark/examples/sql/RDDRelation.scala | 2 +- .../examples/sql/hive/HiveFromSpark.scala | 2 +- .../mllib/classification/NaiveBayes.scala | 2 +- .../impl/GLMClassificationModel.scala | 2 +- .../regression/impl/GLMRegressionModel.scala | 2 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../mllib/tree/model/treeEnsembleModels.scala | 2 +- .../spark/ml/recommendation/ALSSuite.scala | 4 +- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 4 +- .../spark/sql/parquet/ParquetTest.scala | 6 +-- .../apache/spark/sql/CachedTableSuite.scala | 14 +++-- .../org/apache/spark/sql/DataFrameSuite.scala | 26 +++++---- .../org/apache/spark/sql/JoinSuite.scala | 8 +-- .../org/apache/spark/sql/QueryTest.scala | 46 ++++++++-------- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +-- .../sql/ScalaReflectionRelationSuite.scala | 10 ++-- .../scala/org/apache/spark/sql/TestData.scala | 48 +++++++++-------- .../spark/sql/jdbc/JDBCWriteSuite.scala | 46 ++++++++-------- .../spark/sql/jdbc/MySQLIntegration.scala | 53 ++++++++----------- .../spark/sql/jdbc/PostgresIntegration.scala | 30 +++++------ .../sql/parquet/ParquetFilterSuite.scala | 40 +++++++------- .../spark/sql/parquet/ParquetIOSuite.scala | 28 +++++----- .../spark/sql/parquet/ParquetQuerySuite.scala | 2 +- .../sql/parquet/ParquetSchemaSuite.scala | 4 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 4 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 8 +-- .../spark/sql/hive/StatisticsSuite.scala | 38 ++++++------- .../sql/hive/execution/HiveQuerySuite.scala | 20 +++---- .../hive/execution/HiveResolutionSuite.scala | 6 +-- .../sql/hive/execution/HiveUdfSuite.scala | 18 +++---- 37 files changed, 250 insertions(+), 259 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index f0241943ef410..7ab892cd7560c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -90,7 +90,7 @@ object CrossValidatorExample { crossval.setNumFolds(2) // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. - val cvModel = crossval.fit(training.toDF) + val cvModel = crossval.fit(training.toDF()) // Prepare test documents, which are unlabeled. val test = sc.parallelize(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 54aadd2288817..df26798e41b7b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -58,7 +58,7 @@ object DeveloperApiExample { lr.setMaxIter(10) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model = lr.fit(training.toDF) + val model = lr.fit(training.toDF()) // Prepare test data. val test = sc.parallelize(Seq( @@ -67,7 +67,7 @@ object DeveloperApiExample { LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test.toDF) + val sumPredictions: Double = model.transform(test.toDF()) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala index adaf796dc1896..96b2dd463e253 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala @@ -137,9 +137,9 @@ object MovieLensALS { .setRegParam(params.regParam) .setNumBlocks(params.numBlocks) - val model = als.fit(training.toDF) + val model = als.fit(training.toDF()) - val predictions = model.transform(test.toDF).cache() + val predictions = model.transform(test.toDF()).cache() // Evaluate the model. // TODO: Create an evaluator to compute RMSE. @@ -158,7 +158,7 @@ object MovieLensALS { // Inspect false positives. predictions.registerTempTable("prediction") - sc.textFile(params.movies).map(Movie.parseMovie).toDF.registerTempTable("movie") + sc.textFile(params.movies).map(Movie.parseMovie).toDF().registerTempTable("movie") sqlContext.sql( """ |SELECT userId, prediction.movieId, title, rating, prediction diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index c5bb5515b1930..e8af5c162586a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -58,7 +58,7 @@ object SimpleParamsExample { .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. - val model1 = lr.fit(training.toDF) + val model1 = lr.fit(training.toDF()) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -77,7 +77,7 @@ object SimpleParamsExample { // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. - val model2 = lr.fit(training.toDF, paramMapCombined) + val model2 = lr.fit(training.toDF(), paramMapCombined) println("Model 2 was fit using parameters: " + model2.fittingParamMap) // Prepare test data. @@ -90,7 +90,7 @@ object SimpleParamsExample { // LogisticRegression.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test.toDF) + model2.transform(test.toDF()) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 8b47f88e48df1..a11db6fd5c382 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -69,7 +69,7 @@ object SimpleTextClassificationPipeline { .setStages(Array(tokenizer, hashingTF, lr)) // Fit the pipeline to training documents. - val model = pipeline.fit(training.toDF) + val model = pipeline.fit(training.toDF()) // Prepare test documents, which are unlabeled. val test = sc.parallelize(Seq( @@ -79,7 +79,7 @@ object SimpleTextClassificationPipeline { Document(7L, "apache hadoop"))) // Make predictions on test documents. - model.transform(test.toDF) + model.transform(test.toDF()) .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala index c98c68a02f2be..e943d6c889fab 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala @@ -81,7 +81,7 @@ object DatasetExample { println(s"Loaded ${origData.count()} instances from file: ${params.input}") // Convert input data to DataFrame explicitly. - val df: DataFrame = origData.toDF + val df: DataFrame = origData.toDF() println(s"Inferred schema:\n${df.schema.prettyJson}") println(s"Converted to DataFrame with ${df.count()} records") diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 79d3d5a24ceaf..6331d1c0060f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -34,7 +34,7 @@ object RDDRelation { // Importing the SQL context gives access to all the SQL functions and implicit conversions. import sqlContext.implicits._ - val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF + val df = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))).toDF() // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. df.registerTempTable("records") diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 7128deba54da7..b7ba60ec28155 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -68,7 +68,7 @@ object HiveFromSpark { // You can also register RDDs as temporary tables within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.toDF.registerTempTable("records") + rdd.toDF().registerTempTable("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") 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 dd7a9469d5041..b11fd4f128c56 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 @@ -102,7 +102,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) // Create Parquet data. - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() dataRDD.saveAsParquetFile(dataPath(path)) } 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 0a358f2e4f39c..8956189ff1158 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 @@ -62,7 +62,7 @@ private[classification] object GLMClassificationModel { // Create Parquet data. val data = Data(weights, intercept, threshold) - sc.parallelize(Seq(data), 1).toDF.saveAsParquetFile(Loader.dataPath(path)) + sc.parallelize(Seq(data), 1).toDF().saveAsParquetFile(Loader.dataPath(path)) } /** 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 7b27aaa322b00..bd7e340ca2d8e 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 @@ -58,7 +58,7 @@ private[regression] object GLMRegressionModel { // Create Parquet data. val data = Data(weights, intercept) - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() // TODO: repartition with 1 partition after SPARK-5532 gets fixed dataRDD.saveAsParquetFile(Loader.dataPath(path)) } 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 5dac62b0c42f0..060fd5b859a51 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 @@ -197,7 +197,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] { val nodes = model.topNode.subtreeIterator.toSeq val dataRDD: DataFrame = sc.parallelize(nodes) .map(NodeData.apply(0, _)) - .toDF + .toDF() dataRDD.saveAsParquetFile(Loader.dataPath(path)) } 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 e507f247cca76..4897906aea5b3 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 @@ -289,7 +289,7 @@ private[tree] object TreeEnsembleModel { // Create Parquet data. val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => tree.topNode.subtreeIterator.toSeq.map(node => NodeData(treeId, node)) - }.toDF + }.toDF() dataRDD.saveAsParquetFile(Loader.dataPath(path)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index b118a8dcf1363..376c3626f9bbb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -358,8 +358,8 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { .setNumUserBlocks(numUserBlocks) .setNumItemBlocks(numItemBlocks) val alpha = als.getAlpha - val model = als.fit(training.toDF) - val predictions = model.transform(test.toDF) + val model = als.fit(training.toDF()) + val predictions = model.transform(test.toDF()) .select("rating", "prediction") .map { case Row(rating: Float, prediction: Float) => (rating.toDouble, prediction.toDouble) 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 fa5fe84263ece..5007a5a34de1a 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 @@ -124,7 +124,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * from a RDD of tuples into a [[DataFrame]] with meaningful names. For example: * {{{ * val rdd: RDD[(Int, String)] = ... - * rdd.toDF // this implicit conversion creates a DataFrame with column name _1 and _2 + * rdd.toDF() // this implicit conversion creates a DataFrame with column name _1 and _2 * rdd.toDF("id", "name") // this creates a DataFrame with column name "id" and "name" * }}} * @group basic diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 709b350144c75..db32fa80dd3e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -887,8 +887,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * Registers the given [[DataFrame]] as a temporary table in the catalog. Temporary tables exist * only during the lifetime of this instance of SQLContext. */ - private[sql] def registerDataFrameAsTable(rdd: DataFrame, tableName: String): Unit = { - catalog.registerTable(Seq(tableName), rdd.logicalPlan) + private[sql] def registerDataFrameAsTable(df: DataFrame, tableName: String): Unit = { + catalog.registerTable(Seq(tableName), df.logicalPlan) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index 052728c5d5ceb..0fa2fe90f9674 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -99,7 +99,7 @@ private[sql] trait ParquetTest { * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ - protected def withParquetRDD[T <: Product: ClassTag: TypeTag] + protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] (data: Seq[T]) (f: DataFrame => Unit): Unit = { withParquetFile(data)(path => f(sqlContext.parquetFile(path))) @@ -120,8 +120,8 @@ private[sql] trait ParquetTest { protected def withParquetTable[T <: Product: ClassTag: TypeTag] (data: Seq[T], tableName: String) (f: => Unit): Unit = { - withParquetRDD(data) { rdd => - sqlContext.registerDataFrameAsTable(rdd, tableName) + withParquetDataFrame(data) { df => + sqlContext.registerDataFrameAsTable(df, tableName) withTempTable(tableName)(f) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 691dae0a0561b..e70e866fdbf14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -18,17 +18,15 @@ package org.apache.spark.sql import scala.concurrent.duration._ -import scala.language.implicitConversions -import scala.language.postfixOps +import scala.language.{implicitConversions, postfixOps} import org.scalatest.concurrent.Eventually._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.storage.{StorageLevel, RDDBlockId} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} case class BigData(s: String) @@ -59,15 +57,15 @@ class CachedTableSuite extends QueryTest { test("unpersist an uncached table will not raise exception") { assert(None == cacheManager.lookupCachedData(testData)) - testData.unpersist(true) + testData.unpersist(blocking = true) assert(None == cacheManager.lookupCachedData(testData)) - testData.unpersist(false) + testData.unpersist(blocking = false) assert(None == cacheManager.lookupCachedData(testData)) testData.persist() assert(None != cacheManager.lookupCachedData(testData)) - testData.unpersist(true) + testData.unpersist(blocking = true) assert(None == cacheManager.lookupCachedData(testData)) - testData.unpersist(false) + testData.unpersist(blocking = false) assert(None == cacheManager.lookupCachedData(testData)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 0da619def1d09..f31bc38922d4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql -import org.apache.spark.sql.TestData._ - import scala.language.postfixOps import org.apache.spark.sql.functions._ @@ -251,20 +249,20 @@ class DataFrameSuite extends QueryTest { Seq(Row(3,1), Row(3,2), Row(2,1), Row(2,2), Row(1,1), Row(1,2))) checkAnswer( - arrayData.toDF.orderBy('data.getItem(0).asc), - arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq) + arrayData.toDF().orderBy('data.getItem(0).asc), + arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq) checkAnswer( - arrayData.toDF.orderBy('data.getItem(0).desc), - arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq) + arrayData.toDF().orderBy('data.getItem(0).desc), + arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq) checkAnswer( - arrayData.toDF.orderBy('data.getItem(1).asc), - arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq) + arrayData.toDF().orderBy('data.getItem(1).asc), + arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq) checkAnswer( - arrayData.toDF.orderBy('data.getItem(1).desc), - arrayData.toDF.collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq) + arrayData.toDF().orderBy('data.getItem(1).desc), + arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq) } test("limit") { @@ -273,11 +271,11 @@ class DataFrameSuite extends QueryTest { testData.take(10).toSeq) checkAnswer( - arrayData.toDF.limit(1), + arrayData.toDF().limit(1), arrayData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) checkAnswer( - mapData.toDF.limit(1), + mapData.toDF().limit(1), mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) } @@ -411,7 +409,7 @@ class DataFrameSuite extends QueryTest { } test("addColumn") { - val df = testData.toDF.withColumn("newCol", col("key") + 1) + val df = testData.toDF().withColumn("newCol", col("key") + 1) checkAnswer( df, testData.collect().map { case Row(key: Int, value: String) => @@ -421,7 +419,7 @@ class DataFrameSuite extends QueryTest { } test("renameColumn") { - val df = testData.toDF.withColumn("newCol", col("key") + 1) + val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") checkAnswer( df, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index fd73065c4ada3..dd0948ad824be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -40,8 +40,8 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { } def assertJoin(sqlString: String, c: Class[_]): Any = { - val rdd = sql(sqlString) - val physical = rdd.queryExecution.sparkPlan + val df = sql(sqlString) + val physical = df.queryExecution.sparkPlan val operators = physical.collect { case j: ShuffledHashJoin => j case j: HashOuterJoin => j @@ -410,8 +410,8 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { } test("left semi join") { - val rdd = sql("SELECT * FROM testData2 LEFT SEMI JOIN testData ON key = a") - checkAnswer(rdd, + val df = sql("SELECT * FROM testData2 LEFT SEMI JOIN testData ON key = a") + checkAnswer(df, Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index dfb6858957fb9..9b4dd6c620fec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -35,36 +35,36 @@ class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer contains all of the keywords, or the * none of keywords are listed in the answer - * @param rdd the [[DataFrame]] to be executed + * @param df the [[DataFrame]] to be executed * @param exists true for make sure the keywords are listed in the output, otherwise * to make sure none of the keyword are not listed in the output * @param keywords keyword in string array */ - def checkExistence(rdd: DataFrame, exists: Boolean, keywords: String*) { - val outputs = rdd.collect().map(_.mkString).mkString + def checkExistence(df: DataFrame, exists: Boolean, keywords: String*) { + val outputs = df.collect().map(_.mkString).mkString for (key <- keywords) { if (exists) { - assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") + assert(outputs.contains(key), s"Failed for $df ($key doesn't exist in result)") } else { - assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") + assert(!outputs.contains(key), s"Failed for $df ($key existed in the result)") } } } /** * Runs the plan and makes sure the answer matches the expected result. - * @param rdd the [[DataFrame]] to be executed + * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { - QueryTest.checkAnswer(rdd, expectedAnswer) match { + protected def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]): Unit = { + QueryTest.checkAnswer(df, expectedAnswer) match { case Some(errorMessage) => fail(errorMessage) case None => } } - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { - checkAnswer(rdd, Seq(expectedAnswer)) + protected def checkAnswer(df: DataFrame, expectedAnswer: Row): Unit = { + checkAnswer(df, Seq(expectedAnswer)) } def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { @@ -95,11 +95,11 @@ object QueryTest { * If there was exception during the execution or the contents of the DataFrame does not * match the expected result, an error message will be returned. Otherwise, a [[None]] will * be returned. - * @param rdd the [[DataFrame]] to be executed + * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ - def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { - val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty + def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { + val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty def prepareAnswer(answer: Seq[Row]): Seq[Row] = { // Converts data to types that we can do equality comparison using Scala collections. // For BigDecimal type, the Scala type has a better definition of equality test (similar to @@ -110,14 +110,14 @@ object QueryTest { case o => o }) } - if (!isSorted) converted.sortBy(_.toString) else converted + if (!isSorted) converted.sortBy(_.toString()) else converted } - val sparkAnswer = try rdd.collect().toSeq catch { + val sparkAnswer = try df.collect().toSeq catch { case e: Exception => val errorMessage = s""" |Exception thrown while executing query: - |${rdd.queryExecution} + |${df.queryExecution} |== Exception == |$e |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} @@ -129,17 +129,17 @@ object QueryTest { val errorMessage = s""" |Results do not match for query: - |${rdd.logicalPlan} + |${df.logicalPlan} |== Analyzed Plan == - |${rdd.queryExecution.analyzed} + |${df.queryExecution.analyzed} |== Physical Plan == - |${rdd.queryExecution.executedPlan} + |${df.queryExecution.executedPlan} |== Results == |${sideBySide( s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString), + prepareAnswer(expectedAnswer).map(_.toString()), s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} + prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")} """.stripMargin return Some(errorMessage) } @@ -147,8 +147,8 @@ object QueryTest { return None } - def checkAnswer(rdd: DataFrame, expectedAnswer: java.util.List[Row]): String = { - checkAnswer(rdd, expectedAnswer.toSeq) match { + def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): String = { + checkAnswer(df, expectedAnswer.toSeq) match { case Some(errorMessage) => errorMessage case None => null } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 97684f75e79fe..097bf0dd23c89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1034,10 +1034,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) - rdd1.toDF.registerTempTable("nulldata1") + rdd1.toDF().registerTempTable("nulldata1") val nullCheckData2 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd2 = sparkContext.parallelize((0 to 1).map(i => nullCheckData2(i))) - rdd2.toDF.registerTempTable("nulldata2") + rdd2.toDF().registerTempTable("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1046,7 +1046,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("Multi-column COUNT(DISTINCT ...)") { val data = TestData(1,"val_1") :: TestData(2,"val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) - rdd.toDF.registerTempTable("distinctData") + rdd.toDF().registerTempTable("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 9a48f8d0634cb..23df6e7eac043 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -82,7 +82,7 @@ class ScalaReflectionRelationSuite extends FunSuite { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) - rdd.toDF.registerTempTable("reflectData") + rdd.toDF().registerTempTable("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -93,7 +93,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) val rdd = sparkContext.parallelize(data :: Nil) - rdd.toDF.registerTempTable("reflectNullData") + rdd.toDF().registerTempTable("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) } @@ -101,7 +101,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) - rdd.toDF.registerTempTable("reflectOptionalData") + rdd.toDF().registerTempTable("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) } @@ -109,7 +109,7 @@ class ScalaReflectionRelationSuite extends FunSuite { // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) - rdd.toDF.registerTempTable("reflectBinary") + rdd.toDF().registerTempTable("reflectBinary") val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) @@ -128,7 +128,7 @@ class ScalaReflectionRelationSuite extends FunSuite { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) val rdd = sparkContext.parallelize(data :: Nil) - rdd.toDF.registerTempTable("reflectComplexData") + rdd.toDF().registerTempTable("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === new GenericRow(Array[Any]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index c511eb1469167..637f59b2e68ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -29,11 +29,11 @@ case class TestData(key: Int, value: String) object TestData { val testData = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))).toDF + (1 to 100).map(i => TestData(i, i.toString))).toDF() testData.registerTempTable("testData") val negativeData = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(-i, (-i).toString))).toDF + (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() negativeData.registerTempTable("negativeData") case class LargeAndSmallInts(a: Int, b: Int) @@ -44,7 +44,7 @@ object TestData { LargeAndSmallInts(2147483645, 1) :: LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: - LargeAndSmallInts(3, 2) :: Nil).toDF + LargeAndSmallInts(3, 2) :: Nil).toDF() largeAndSmallInts.registerTempTable("largeAndSmallInts") case class TestData2(a: Int, b: Int) @@ -55,7 +55,7 @@ object TestData { TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil, 2).toDF + TestData2(3, 2) :: Nil, 2).toDF() testData2.registerTempTable("testData2") case class DecimalData(a: BigDecimal, b: BigDecimal) @@ -67,7 +67,7 @@ object TestData { DecimalData(2, 1) :: DecimalData(2, 2) :: DecimalData(3, 1) :: - DecimalData(3, 2) :: Nil).toDF + DecimalData(3, 2) :: Nil).toDF() decimalData.registerTempTable("decimalData") case class BinaryData(a: Array[Byte], b: Int) @@ -77,14 +77,14 @@ object TestData { BinaryData("22".getBytes(), 5) :: BinaryData("122".getBytes(), 3) :: BinaryData("121".getBytes(), 2) :: - BinaryData("123".getBytes(), 4) :: Nil).toDF + BinaryData("123".getBytes(), 4) :: Nil).toDF() binaryData.registerTempTable("binaryData") case class TestData3(a: Int, b: Option[Int]) val testData3 = TestSQLContext.sparkContext.parallelize( TestData3(1, None) :: - TestData3(2, Some(2)) :: Nil).toDF + TestData3(2, Some(2)) :: Nil).toDF() testData3.registerTempTable("testData3") val emptyTableData = logical.LocalRelation($"a".int, $"b".int) @@ -97,7 +97,7 @@ object TestData { UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: - UpperCaseData(6, "F") :: Nil).toDF + UpperCaseData(6, "F") :: Nil).toDF() upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) @@ -106,7 +106,7 @@ object TestData { LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: - LowerCaseData(4, "d") :: Nil).toDF + LowerCaseData(4, "d") :: Nil).toDF() lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) @@ -114,7 +114,7 @@ object TestData { TestSQLContext.sparkContext.parallelize( ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) - arrayData.toDF.registerTempTable("arrayData") + arrayData.toDF().registerTempTable("arrayData") case class MapData(data: scala.collection.Map[Int, String]) val mapData = @@ -124,18 +124,18 @@ object TestData { MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - mapData.toDF.registerTempTable("mapData") + mapData.toDF().registerTempTable("mapData") case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) - repeatedData.toDF.registerTempTable("repeatedData") + repeatedData.toDF().registerTempTable("repeatedData") val nullableRepeatedData = TestSQLContext.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - nullableRepeatedData.toDF.registerTempTable("nullableRepeatedData") + nullableRepeatedData.toDF().registerTempTable("nullableRepeatedData") case class NullInts(a: Integer) val nullInts = @@ -144,7 +144,7 @@ object TestData { NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil - ).toDF + ).toDF() nullInts.registerTempTable("nullInts") val allNulls = @@ -152,7 +152,7 @@ object TestData { NullInts(null) :: NullInts(null) :: NullInts(null) :: - NullInts(null) :: Nil).toDF + NullInts(null) :: Nil).toDF() allNulls.registerTempTable("allNulls") case class NullStrings(n: Int, s: String) @@ -160,11 +160,15 @@ object TestData { TestSQLContext.sparkContext.parallelize( NullStrings(1, "abc") :: NullStrings(2, "ABC") :: - NullStrings(3, null) :: Nil).toDF + NullStrings(3, null) :: Nil).toDF() nullStrings.registerTempTable("nullStrings") case class TableName(tableName: String) - TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).toDF.registerTempTable("tableName") + TestSQLContext + .sparkContext + .parallelize(TableName("test") :: Nil) + .toDF() + .registerTempTable("tableName") val unparsedStrings = TestSQLContext.sparkContext.parallelize( @@ -177,22 +181,22 @@ object TestData { val timestamps = TestSQLContext.sparkContext.parallelize((1 to 3).map { i => TimestampField(new Timestamp(i)) }) - timestamps.toDF.registerTempTable("timestamps") + timestamps.toDF().registerTempTable("timestamps") case class IntField(i: Int) // An RDD with 4 elements and 8 partitions val withEmptyParts = TestSQLContext.sparkContext.parallelize((1 to 4).map(IntField), 8) - withEmptyParts.toDF.registerTempTable("withEmptyParts") + withEmptyParts.toDF().registerTempTable("withEmptyParts") case class Person(id: Int, name: String, age: Int) case class Salary(personId: Int, salary: Double) val person = TestSQLContext.sparkContext.parallelize( Person(0, "mike", 30) :: - Person(1, "jim", 20) :: Nil).toDF + Person(1, "jim", 20) :: Nil).toDF() person.registerTempTable("person") val salary = TestSQLContext.sparkContext.parallelize( Salary(0, 2000.0) :: - Salary(1, 1000.0) :: Nil).toDF + Salary(1, 1000.0) :: Nil).toDF() salary.registerTempTable("salary") case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) @@ -200,6 +204,6 @@ object TestData { TestSQLContext.sparkContext.parallelize( ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) - :: Nil).toDF + :: Nil).toDF() complexData.registerTempTable("complexData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index ad2fbc3f04a9c..ee5c7620d1a22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.jdbc -import java.math.BigDecimal +import java.sql.DriverManager + +import org.scalatest.{BeforeAndAfter, FunSuite} + import org.apache.spark.sql.Row -import org.apache.spark.sql.types._ import org.apache.spark.sql.test._ -import org.scalatest.{FunSuite, BeforeAndAfter} -import java.sql.DriverManager -import TestSQLContext._ +import org.apache.spark.sql.types._ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { val url = "jdbc:h2:mem:testdb2" @@ -54,53 +54,53 @@ class JDBCWriteSuite extends FunSuite with BeforeAndAfter { StructField("seq", IntegerType) :: Nil) test("Basic CREATE") { - val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val df = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) - srdd.createJDBCTable(url, "TEST.BASICCREATETEST", false) + df.createJDBCTable(url, "TEST.BASICCREATETEST", false) assert(2 == TestSQLContext.jdbc(url, "TEST.BASICCREATETEST").count) assert(2 == TestSQLContext.jdbc(url, "TEST.BASICCREATETEST").collect()(0).length) } test("CREATE with overwrite") { - val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) - val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) + val df = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) + val df2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) - srdd.createJDBCTable(url, "TEST.DROPTEST", false) + df.createJDBCTable(url, "TEST.DROPTEST", false) assert(2 == TestSQLContext.jdbc(url, "TEST.DROPTEST").count) assert(3 == TestSQLContext.jdbc(url, "TEST.DROPTEST").collect()(0).length) - srdd2.createJDBCTable(url, "TEST.DROPTEST", true) + df2.createJDBCTable(url, "TEST.DROPTEST", true) assert(1 == TestSQLContext.jdbc(url, "TEST.DROPTEST").count) assert(2 == TestSQLContext.jdbc(url, "TEST.DROPTEST").collect()(0).length) } test("CREATE then INSERT to append") { - val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) + val df = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val df2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) - srdd.createJDBCTable(url, "TEST.APPENDTEST", false) - srdd2.insertIntoJDBC(url, "TEST.APPENDTEST", false) + df.createJDBCTable(url, "TEST.APPENDTEST", false) + df2.insertIntoJDBC(url, "TEST.APPENDTEST", false) assert(3 == TestSQLContext.jdbc(url, "TEST.APPENDTEST").count) assert(2 == TestSQLContext.jdbc(url, "TEST.APPENDTEST").collect()(0).length) } test("CREATE then INSERT to truncate") { - val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) + val df = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val df2 = TestSQLContext.createDataFrame(sc.parallelize(arr1x2), schema2) - srdd.createJDBCTable(url, "TEST.TRUNCATETEST", false) - srdd2.insertIntoJDBC(url, "TEST.TRUNCATETEST", true) + df.createJDBCTable(url, "TEST.TRUNCATETEST", false) + df2.insertIntoJDBC(url, "TEST.TRUNCATETEST", true) assert(1 == TestSQLContext.jdbc(url, "TEST.TRUNCATETEST").count) assert(2 == TestSQLContext.jdbc(url, "TEST.TRUNCATETEST").collect()(0).length) } test("Incompatible INSERT to append") { - val srdd = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) - val srdd2 = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) + val df = TestSQLContext.createDataFrame(sc.parallelize(arr2x2), schema2) + val df2 = TestSQLContext.createDataFrame(sc.parallelize(arr2x3), schema3) - srdd.createJDBCTable(url, "TEST.INCOMPATIBLETEST", false) + df.createJDBCTable(url, "TEST.INCOMPATIBLETEST", false) intercept[org.apache.spark.SparkException] { - srdd2.insertIntoJDBC(url, "TEST.INCOMPATIBLETEST", true) + df2.insertIntoJDBC(url, "TEST.INCOMPATIBLETEST", true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala index 4f38110c80cc6..5b8a76f461faf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala @@ -18,18 +18,13 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal -import java.sql.{Date, DriverManager, Timestamp} -import com.spotify.docker.client.{DefaultDockerClient, DockerClient} +import java.sql.{Date, Timestamp} + +import com.spotify.docker.client.DockerClient import com.spotify.docker.client.messages.ContainerConfig -import org.scalatest.{FunSuite, BeforeAndAfterAll, Ignore} +import org.scalatest.{BeforeAndAfterAll, FunSuite, Ignore} -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ -import org.apache.spark.sql._ import org.apache.spark.sql.test._ -import TestSQLContext._ - -import org.apache.spark.sql.jdbc._ class MySQLDatabase { val docker: DockerClient = DockerClientFactory.get() @@ -37,9 +32,9 @@ class MySQLDatabase { println("Pulling mysql") docker.pull("mysql") println("Configuring container") - val config = (ContainerConfig.builder().image("mysql") - .env("MYSQL_ROOT_PASSWORD=rootpass") - .build()) + val config = ContainerConfig.builder().image("mysql") + .env("MYSQL_ROOT_PASSWORD=rootpass") + .build() println("Creating container") val id = docker.createContainer(config).id println("Starting container " + id) @@ -57,11 +52,10 @@ class MySQLDatabase { println("Closing docker client") DockerClientFactory.close(docker) } catch { - case e: Exception => { + case e: Exception => println(e) println("You may need to clean this up manually.") throw e - } } } } @@ -86,10 +80,9 @@ class MySQLDatabase { println("Database is up.") return; } catch { - case e: java.sql.SQLException => { + case e: java.sql.SQLException => lastException = e java.lang.Thread.sleep(250) - } } } } @@ -143,8 +136,8 @@ class MySQLDatabase { } test("Basic test") { - val rdd = TestSQLContext.jdbc(url(ip, "foo"), "tbl") - val rows = rdd.collect + val df = TestSQLContext.jdbc(url(ip, "foo"), "tbl") + val rows = df.collect() assert(rows.length == 2) val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types.length == 2) @@ -153,8 +146,8 @@ class MySQLDatabase { } test("Numeric types") { - val rdd = TestSQLContext.jdbc(url(ip, "foo"), "numbers") - val rows = rdd.collect + val df = TestSQLContext.jdbc(url(ip, "foo"), "numbers") + val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types.length == 9) @@ -181,8 +174,8 @@ class MySQLDatabase { } test("Date types") { - val rdd = TestSQLContext.jdbc(url(ip, "foo"), "dates") - val rows = rdd.collect + val df = TestSQLContext.jdbc(url(ip, "foo"), "dates") + val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types.length == 5) @@ -199,8 +192,8 @@ class MySQLDatabase { } test("String types") { - val rdd = TestSQLContext.jdbc(url(ip, "foo"), "strings") - val rows = rdd.collect + val df = TestSQLContext.jdbc(url(ip, "foo"), "strings") + val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types.length == 9) @@ -225,11 +218,11 @@ class MySQLDatabase { } test("Basic write test") { - val rdd1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers") - val rdd2 = TestSQLContext.jdbc(url(ip, "foo"), "dates") - val rdd3 = TestSQLContext.jdbc(url(ip, "foo"), "strings") - rdd1.createJDBCTable(url(ip, "foo"), "numberscopy", false) - rdd2.createJDBCTable(url(ip, "foo"), "datescopy", false) - rdd3.createJDBCTable(url(ip, "foo"), "stringscopy", false) + val df1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers") + val df2 = TestSQLContext.jdbc(url(ip, "foo"), "dates") + val df3 = TestSQLContext.jdbc(url(ip, "foo"), "strings") + df1.createJDBCTable(url(ip, "foo"), "numberscopy", false) + df2.createJDBCTable(url(ip, "foo"), "datescopy", false) + df3.createJDBCTable(url(ip, "foo"), "stringscopy", false) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala index 7b47feeb7887e..e17be99ac31d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.jdbc -import java.math.BigDecimal -import org.apache.spark.sql.test._ -import org.scalatest.{FunSuite, BeforeAndAfterAll, Ignore} import java.sql.DriverManager -import TestSQLContext._ -import com.spotify.docker.client.{DefaultDockerClient, DockerClient} + +import com.spotify.docker.client.DockerClient import com.spotify.docker.client.messages.ContainerConfig +import org.scalatest.{BeforeAndAfterAll, FunSuite, Ignore} + +import org.apache.spark.sql.test._ class PostgresDatabase { val docker: DockerClient = DockerClientFactory.get() @@ -31,9 +31,9 @@ class PostgresDatabase { println("Pulling postgres") docker.pull("postgres") println("Configuring container") - val config = (ContainerConfig.builder().image("postgres") - .env("POSTGRES_PASSWORD=rootpass") - .build()) + val config = ContainerConfig.builder().image("postgres") + .env("POSTGRES_PASSWORD=rootpass") + .build() println("Creating container") val id = docker.createContainer(config).id println("Starting container " + id) @@ -51,11 +51,10 @@ class PostgresDatabase { println("Closing docker client") DockerClientFactory.close(docker) } catch { - case e: Exception => { + case e: Exception => println(e) println("You may need to clean this up manually.") throw e - } } } } @@ -79,10 +78,9 @@ class PostgresDatabase { println("Database is up.") return; } catch { - case e: java.sql.SQLException => { + case e: java.sql.SQLException => lastException = e java.lang.Thread.sleep(250) - } } } } @@ -113,8 +111,8 @@ class PostgresDatabase { } test("Type mapping for various types") { - val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar") - val rows = rdd.collect + val df = TestSQLContext.jdbc(url(db.ip), "public.bar") + val rows = df.collect() assert(rows.length == 1) val types = rows(0).toSeq.map(x => x.getClass.toString) assert(types.length == 10) @@ -142,8 +140,8 @@ class PostgresDatabase { } test("Basic write test") { - val rdd = TestSQLContext.jdbc(url(db.ip), "public.bar") - rdd.createJDBCTable(url(db.ip), "public.barcopy", false) + val df = TestSQLContext.jdbc(url(db.ip), "public.bar") + df.createJDBCTable(url(db.ip), "public.barcopy", false) // Test only that it doesn't bomb out. } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index eb2d5f25290b1..4d32e84fc1115 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -45,7 +45,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { val sqlContext = TestSQLContext private def checkFilterPredicate( - rdd: DataFrame, + df: DataFrame, predicate: Predicate, filterClass: Class[_ <: FilterPredicate], checker: (DataFrame, Seq[Row]) => Unit, @@ -53,7 +53,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { val output = predicate.collect { case a: Attribute => a }.distinct withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { - val query = rdd + val query = df .select(output.map(e => Column(e)): _*) .where(Column(predicate)) @@ -85,36 +85,36 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { private def checkFilterPredicate (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) - (implicit rdd: DataFrame): Unit = { - checkFilterPredicate(rdd, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected) + (implicit df: DataFrame): Unit = { + checkFilterPredicate(df, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected) } private def checkFilterPredicate[T] (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: T) - (implicit rdd: DataFrame): Unit = { - checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) + (implicit df: DataFrame): Unit = { + checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) } private def checkBinaryFilterPredicate (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) - (implicit rdd: DataFrame): Unit = { - def checkBinaryAnswer(rdd: DataFrame, expected: Seq[Row]) = { + (implicit df: DataFrame): Unit = { + def checkBinaryAnswer(df: DataFrame, expected: Seq[Row]) = { assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).toSeq.sorted) { - rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted + df.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted } } - checkFilterPredicate(rdd, predicate, filterClass, checkBinaryAnswer _, expected) + checkFilterPredicate(df, predicate, filterClass, checkBinaryAnswer _, expected) } private def checkBinaryFilterPredicate (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Array[Byte]) - (implicit rdd: DataFrame): Unit = { - checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) + (implicit df: DataFrame): Unit = { + checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) } test("filter pushdown - boolean") { - withParquetRDD((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit rdd => + withParquetDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false))) @@ -124,7 +124,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - short") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit df => checkFilterPredicate(Cast('_1, IntegerType) === 1, classOf[Eq[_]], 1) checkFilterPredicate( Cast('_1, IntegerType) !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) @@ -151,7 +151,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - integer") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) @@ -176,7 +176,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - long") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) @@ -201,7 +201,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - float") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) @@ -226,7 +226,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - double") { - withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) @@ -251,7 +251,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { } test("filter pushdown - string") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate( '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString))) @@ -282,7 +282,7 @@ class ParquetFilterSuiteBase extends QueryTest with ParquetTest { def b: Array[Byte] = int.toString.getBytes("UTF-8") } - withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { implicit rdd => + withParquetDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 208f35761b807..36f3406a7825f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -73,7 +73,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { * Writes `data` to a Parquet file, reads it back and check file contents. */ protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { - withParquetRDD(data)(r => checkAnswer(r, data.map(Row.fromTuple))) + withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } test("basic data types (without binary)") { @@ -85,9 +85,9 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { test("raw binary") { val data = (1 to 4).map(i => Tuple1(Array.fill(3)(i.toByte))) - withParquetRDD(data) { rdd => + withParquetDataFrame(data) { df => assertResult(data.map(_._1.mkString(",")).sorted) { - rdd.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted + df.collect().map(_.getAs[Array[Byte]](0).mkString(",")).sorted } } } @@ -106,7 +106,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { sparkContext .parallelize(0 to 1000) .map(i => Tuple1(i / 100.0)) - .toDF + .toDF() // Parquet doesn't allow column names with spaces, have to add an alias here .select($"_1" cast decimal as "dec") @@ -147,9 +147,9 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { test("struct") { val data = (1 to 4).map(i => Tuple1((i, s"val_$i"))) - withParquetRDD(data) { rdd => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s - checkAnswer(rdd, data.map { case Tuple1(struct) => + checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) }) } @@ -157,9 +157,9 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { test("nested struct with array of array as field") { val data = (1 to 4).map(i => Tuple1((i, Seq(Seq(s"val_$i"))))) - withParquetRDD(data) { rdd => + withParquetDataFrame(data) { df => // Structs are converted to `Row`s - checkAnswer(rdd, data.map { case Tuple1(struct) => + checkAnswer(df, data.map { case Tuple1(struct) => Row(Row(struct.productIterator.toSeq: _*)) }) } @@ -167,8 +167,8 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { test("nested map with struct as value type") { val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i")))) - withParquetRDD(data) { rdd => - checkAnswer(rdd, data.map { case Tuple1(m) => + withParquetDataFrame(data) { df => + checkAnswer(df, data.map { case Tuple1(m) => Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) }) } @@ -182,8 +182,8 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { null.asInstanceOf[java.lang.Float], null.asInstanceOf[java.lang.Double]) - withParquetRDD(allNulls :: Nil) { rdd => - val rows = rdd.collect() + withParquetDataFrame(allNulls :: Nil) { df => + val rows = df.collect() assert(rows.size === 1) assert(rows.head === Row(Seq.fill(5)(null): _*)) } @@ -195,8 +195,8 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { None.asInstanceOf[Option[Long]], None.asInstanceOf[Option[String]]) - withParquetRDD(allNones :: Nil) { rdd => - val rows = rdd.collect() + withParquetDataFrame(allNones :: Nil) { df => + val rows = df.collect() assert(rows.size === 1) assert(rows.head === Row(Seq.fill(3)(null): _*)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 8b4d05ec547c6..b98ba09ccfc2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -68,7 +68,7 @@ class ParquetQuerySuiteBase extends QueryTest with ParquetTest { val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") val queryOutput = selfJoin.queryExecution.analyzed.output - assertResult(4, "Field count mismatche")(queryOutput.size) + assertResult(4, "Field count mismatches")(queryOutput.size) assertResult(2, "Duplicated expression ID in query plan:\n $selfJoin") { queryOutput.filter(_.name == "_1").map(_.exprId).size } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 2e6c2d5f9ab55..ad880e2bc3679 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -36,8 +36,8 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { private def testSchema[T <: Product: ClassTag: TypeTag]( testName: String, messageType: String, isThriftDerived: Boolean = false): Unit = { test(testName) { - val actual = ParquetTypesConverter.convertFromAttributes(ScalaReflection.attributesFor[T], - isThriftDerived) + val actual = ParquetTypesConverter.convertFromAttributes( + ScalaReflection.attributesFor[T], isThriftDerived) val expected = MessageTypeParser.parseMessageType(messageType) actual.checkContains(expected) expected.checkContains(actual) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 9fcb04ca23590..d4b175fa443a4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -37,7 +37,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ val testData = TestHive.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))).toDF + (1 to 100).map(i => TestData(i, i.toString))).toDF() before { // Since every we are doing tests for DDL statements, @@ -65,7 +65,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { // Make sure the table has been updated. checkAnswer( sql("SELECT * FROM createAndInsertTest"), - testData.toDF.collect().toSeq ++ testData.toDF.collect().toSeq + testData.toDF().collect().toSeq ++ testData.toDF().collect().toSeq ) // Now overwrite. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index e5156ae821bf4..0bd82773f3a55 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -154,7 +154,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("check change without refresh") { val tempDir = File.createTempFile("sparksql", "json") tempDir.delete() - sparkContext.parallelize(("a", "b") :: Nil).toDF + sparkContext.parallelize(("a", "b") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql( @@ -171,7 +171,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row("a", "b")) FileUtils.deleteDirectory(tempDir) - sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF + sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) // Schema is cached so the new column does not show. The updated values in existing columns @@ -192,7 +192,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { test("drop, change, recreate") { val tempDir = File.createTempFile("sparksql", "json") tempDir.delete() - sparkContext.parallelize(("a", "b") :: Nil).toDF + sparkContext.parallelize(("a", "b") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql( @@ -209,7 +209,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row("a", "b")) FileUtils.deleteDirectory(tempDir) - sparkContext.parallelize(("a", "b", "c") :: Nil).toDF + sparkContext.parallelize(("a", "b", "c") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) sql("DROP TABLE jsonTable") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 6f07fd5a879c0..1e05a024b8807 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -127,11 +127,11 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { } test("estimates the size of a test MetastoreRelation") { - val rdd = sql("""SELECT * FROM src""") - val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => + val df = sql("""SELECT * FROM src""") + val sizes = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") + assert(sizes.size === 1, s"Size wrong for:\n ${df.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } @@ -145,10 +145,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { ct: ClassTag[_]) = { before() - var rdd = sql(query) + var df = sql(query) // Assert src has a size smaller than the threshold. - val sizes = rdd.queryExecution.analyzed.collect { + val sizes = df.queryExecution.analyzed.collect { case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } assert(sizes.size === 2 && sizes(0) <= conf.autoBroadcastJoinThreshold @@ -157,21 +157,21 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. - var bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } assert(bhj.size === 1, - s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + s"actual query plans do not contain broadcast join: ${df.queryExecution}") - checkAnswer(rdd, expectedAnswer) // check correctness of output + checkAnswer(df, expectedAnswer) // check correctness of output TestHive.conf.settings.synchronized { val tmp = conf.autoBroadcastJoinThreshold sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") - rdd = sql(query) - bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + df = sql(query) + bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") - val shj = rdd.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } + val shj = df.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } assert(shj.size === 1, "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") @@ -199,10 +199,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { |left semi JOIN src b ON a.key=86 and a.key = b.key""".stripMargin val answer = Row(86, "val_86") - var rdd = sql(leftSemiJoinQuery) + var df = sql(leftSemiJoinQuery) // Assert src has a size smaller than the threshold. - val sizes = rdd.queryExecution.analyzed.collect { + val sizes = df.queryExecution.analyzed.collect { case r if implicitly[ClassTag[MetastoreRelation]].runtimeClass .isAssignableFrom(r.getClass) => r.statistics.sizeInBytes @@ -213,25 +213,25 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. - var bhj = rdd.queryExecution.sparkPlan.collect { + var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastLeftSemiJoinHash => j } assert(bhj.size === 1, - s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + s"actual query plans do not contain broadcast join: ${df.queryExecution}") - checkAnswer(rdd, answer) // check correctness of output + checkAnswer(df, answer) // check correctness of output TestHive.conf.settings.synchronized { val tmp = conf.autoBroadcastJoinThreshold sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") - rdd = sql(leftSemiJoinQuery) - bhj = rdd.queryExecution.sparkPlan.collect { + df = sql(leftSemiJoinQuery) + bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastLeftSemiJoinHash => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") - val shj = rdd.queryExecution.sparkPlan.collect { + val shj = df.queryExecution.sparkPlan.collect { case j: LeftSemiJoinHash => j } assert(shj.size === 1, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 955f3f51cfe9f..bb0a67dc03e1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -429,7 +429,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES |('serialization.last.column.takes.rest'='true') FROM src; """.stripMargin.replaceAll("\n", " ")) - + createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") @@ -567,7 +567,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.toDF.registerTempTable("REGisteredTABle") + testData.toDF().registerTempTable("REGisteredTABle") assertResult(Array(Row(2, "str2"))) { sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -583,8 +583,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-1704: Explain commands as a DataFrame") { sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - val rdd = sql("explain select key, count(value) from src group by key") - assert(isExplanation(rdd)) + val df = sql("explain select key, count(value) from src group by key") + assert(isExplanation(df)) TestHive.reset() } @@ -592,7 +592,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).toDF.registerTempTable("having_test") + TestHive.sparkContext.parallelize(fixture).toDF().registerTempTable("having_test") val results = sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() @@ -740,7 +740,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.toDF.registerTempTable("test_describe_commands2") + testData.toDF().registerTempTable("test_describe_commands2") assertResult( Array( @@ -900,8 +900,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { - sparkContext.makeRDD(Seq.empty[LogEntry]).toDF.registerTempTable("rawLogs") - sparkContext.makeRDD(Seq.empty[LogFile]).toDF.registerTempTable("logFiles") + sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().registerTempTable("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).toDF().registerTempTable("logFiles") sql( """ @@ -979,8 +979,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { val testVal = "test.val.0" val nonexistentKey = "nonexistent" val KV = "([^=]+)=([^=]*)".r - def collectResults(rdd: DataFrame): Set[(String, String)] = - rdd.collect().map { + def collectResults(df: DataFrame): Set[(String, String)] = + df.collect().map { case Row(key: String, value: String) => key -> value case Row(KV(key, value)) => key -> value }.toSet diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 6fc4cc14265ec..f4440e5b7846a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -77,7 +77,7 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .toDF.registerTempTable("caseSensitivityTest") + .toDF().registerTempTable("caseSensitivityTest") val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), @@ -88,14 +88,14 @@ class HiveResolutionSuite extends HiveComparisonTest { ignore("case insensitivity with scala reflection joins") { // Test resolution with Scala Reflection sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .toDF.registerTempTable("caseSensitivityTest") + .toDF().registerTempTable("caseSensitivityTest") sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .toDF.registerTempTable("nestedRepeatedTest") + .toDF().registerTempTable("nestedRepeatedTest") assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 245161d2ebbca..cb405f56bf53d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -62,7 +62,7 @@ class HiveUdfSuite extends QueryTest { | getStruct(1).f5 FROM src LIMIT 1 """.stripMargin).head() === Row(1, 2, 3, 4, 5)) } - + test("SPARK-4785 When called with arguments referring column fields, PMOD throws NPE") { checkAnswer( sql("SELECT PMOD(CAST(key as INT), 10) FROM src LIMIT 1"), @@ -96,7 +96,7 @@ class HiveUdfSuite extends QueryTest { test("SPARK-2693 udaf aggregates test") { checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"), sql("SELECT max(key) FROM src").collect().toSeq) - + checkAnswer(sql("SELECT percentile(key, array(1, 1)) FROM src LIMIT 1"), sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq) } @@ -104,14 +104,14 @@ class HiveUdfSuite extends QueryTest { test("Generic UDAF aggregates") { checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999)) FROM src LIMIT 1"), sql("SELECT max(key) FROM src LIMIT 1").collect().toSeq) - + checkAnswer(sql("SELECT percentile_approx(100.0, array(0.9, 0.9)) FROM src LIMIT 1"), sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq) } - + test("UDFIntegerToString") { val testData = TestHive.sparkContext.parallelize( - IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF + IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() testData.registerTempTable("integerTable") sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") @@ -127,7 +127,7 @@ class HiveUdfSuite extends QueryTest { val testData = TestHive.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: - ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF + ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() testData.registerTempTable("listListIntTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") @@ -142,7 +142,7 @@ class HiveUdfSuite extends QueryTest { test("UDFListString") { val testData = TestHive.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: - ListStringCaseClass(Seq("d", "e")) :: Nil).toDF + ListStringCaseClass(Seq("d", "e")) :: Nil).toDF() testData.registerTempTable("listStringTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") @@ -156,7 +156,7 @@ class HiveUdfSuite extends QueryTest { test("UDFStringString") { val testData = TestHive.sparkContext.parallelize( - StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF + StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF() testData.registerTempTable("stringTable") sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") @@ -173,7 +173,7 @@ class HiveUdfSuite extends QueryTest { ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: - Nil).toDF + Nil).toDF() testData.registerTempTable("TwoListTable") sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") From de0dd6de2476c22be3f41f1bf0b3ef7ffeb60001 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Wed, 18 Feb 2015 01:00:13 -0800 Subject: [PATCH 185/817] Avoid deprecation warnings in JDBCSuite. This pull request replaces calls to deprecated methods from `java.util.Date` with near-equivalents in `java.util.Calendar`. Author: Tor Myklebust Closes #4668 from tmyklebu/master and squashes the following commits: 66215b1 [Tor Myklebust] Use GregorianCalendar instead of Timestamp get methods. --- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 33 +++++++++++-------- 1 file changed, 20 insertions(+), 13 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 07db672217bc1..cd737c0b62767 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal +import java.sql.DriverManager +import java.util.{Calendar, GregorianCalendar} + import org.apache.spark.sql.test._ import org.scalatest.{FunSuite, BeforeAndAfter} -import java.sql.DriverManager import TestSQLContext._ class JDBCSuite extends FunSuite with BeforeAndAfter { @@ -206,20 +208,25 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(rows(0).getString(5).equals("I am a clob!")) } + test("H2 time types") { val rows = sql("SELECT * FROM timetypes").collect() - assert(rows(0).getAs[java.sql.Timestamp](0).getHours == 12) - assert(rows(0).getAs[java.sql.Timestamp](0).getMinutes == 34) - assert(rows(0).getAs[java.sql.Timestamp](0).getSeconds == 56) - assert(rows(0).getAs[java.sql.Date](1).getYear == 96) - assert(rows(0).getAs[java.sql.Date](1).getMonth == 0) - assert(rows(0).getAs[java.sql.Date](1).getDate == 1) - assert(rows(0).getAs[java.sql.Timestamp](2).getYear == 102) - assert(rows(0).getAs[java.sql.Timestamp](2).getMonth == 1) - assert(rows(0).getAs[java.sql.Timestamp](2).getDate == 20) - assert(rows(0).getAs[java.sql.Timestamp](2).getHours == 11) - assert(rows(0).getAs[java.sql.Timestamp](2).getMinutes == 22) - assert(rows(0).getAs[java.sql.Timestamp](2).getSeconds == 33) + val cal = new GregorianCalendar(java.util.Locale.ROOT) + cal.setTime(rows(0).getAs[java.sql.Timestamp](0)) + assert(cal.get(Calendar.HOUR_OF_DAY) == 12) + assert(cal.get(Calendar.MINUTE) == 34) + assert(cal.get(Calendar.SECOND) == 56) + cal.setTime(rows(0).getAs[java.sql.Timestamp](1)) + assert(cal.get(Calendar.YEAR) == 1996) + assert(cal.get(Calendar.MONTH) == 0) + assert(cal.get(Calendar.DAY_OF_MONTH) == 1) + cal.setTime(rows(0).getAs[java.sql.Timestamp](2)) + assert(cal.get(Calendar.YEAR) == 2002) + assert(cal.get(Calendar.MONTH) == 1) + assert(cal.get(Calendar.DAY_OF_MONTH) == 20) + assert(cal.get(Calendar.HOUR) == 11) + assert(cal.get(Calendar.MINUTE) == 22) + assert(cal.get(Calendar.SECOND) == 33) assert(rows(0).getAs[java.sql.Timestamp](2).getNanos == 543543543) } From c1b6fa9838f9d26d60fab3b05a96649882e3dd5b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 Feb 2015 01:00:54 -0800 Subject: [PATCH 186/817] [SPARK-5878] fix DataFrame.repartition() in Python Also add tests for distinct() Author: Davies Liu Closes #4667 from davies/repartition and squashes the following commits: 79059fd [Davies Liu] add test cb4915e [Davies Liu] fix repartition --- python/pyspark/sql/dataframe.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 388033d385e13..52bd75bf8a369 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -434,12 +434,18 @@ def unpersist(self, blocking=True): def repartition(self, numPartitions): """ Return a new :class:`DataFrame` that has exactly `numPartitions` partitions. + + >>> df.repartition(10).rdd.getNumPartitions() + 10 """ - return DataFrame(self._jdf.repartition(numPartitions, None), self.sql_ctx) + return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx) def distinct(self): """ Return a new :class:`DataFrame` containing the distinct rows in this DataFrame. + + >>> df.distinct().count() + 2L """ return DataFrame(self._jdf.distinct(), self.sql_ctx) From e79a7a626d9ac2e2474b9d5008c6b5d07df5c6f1 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Wed, 18 Feb 2015 10:13:28 +0000 Subject: [PATCH 187/817] SPARK-4610 addendum: [Minor] [MLlib] Minor doc fix in GBT classification example numClassesForClassification has been renamed to numClasses. Author: MechCoder Closes #4672 from MechCoder/minor-doc and squashes the following commits: d2ddb7f [MechCoder] Minor doc fix in GBT classification example --- docs/mllib-ensembles.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index 23ede04b62d5b..fb90b7039971c 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -458,7 +458,7 @@ val (trainingData, testData) = (splits(0), splits(1)) // The defaultParams for Classification use LogLoss by default. val boostingStrategy = BoostingStrategy.defaultParams("Classification") boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. -boostingStrategy.treeStrategy.numClassesForClassification = 2 +boostingStrategy.treeStrategy.numClasses = 2 boostingStrategy.treeStrategy.maxDepth = 5 // Empty categoricalFeaturesInfo indicates all features are continuous. boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() From 82197ed3bd4b8c29b0c4b183994753f0e02b6903 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 18 Feb 2015 12:20:11 +0000 Subject: [PATCH 188/817] [SPARK-4949]shutdownCallback in SparkDeploySchedulerBackend should be enclosed by synchronized block. A variable `shutdownCallback` in SparkDeploySchedulerBackend can be accessed from multiple threads so it should be enclosed by synchronized block. Author: Kousuke Saruta Closes #3781 from sarutak/SPARK-4949 and squashes the following commits: c146c93 [Kousuke Saruta] Removed "setShutdownCallback" method c7265dc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4949 42ca528 [Kousuke Saruta] Changed the declaration of the variable "shutdownCallback" as a volatile reference instead of AtomicReference 552df7c [Kousuke Saruta] Changed the declaration of the variable "shutdownCallback" as a volatile reference instead of AtomicReference f556819 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4949 1b60fd1 [Kousuke Saruta] Improved the locking logics 5942765 [Kousuke Saruta] Enclosed shutdownCallback in SparkDeploySchedulerBackend by synchronized block --- .../cluster/SparkDeploySchedulerBackend.scala | 35 +++++++++---------- 1 file changed, 16 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 40fc6b59cdf7b..a0aa555f6244f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler.cluster +import java.util.concurrent.Semaphore + import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} @@ -31,16 +33,16 @@ private[spark] class SparkDeploySchedulerBackend( with AppClientListener with Logging { - var client: AppClient = null - var stopping = false - var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - @volatile var appId: String = _ + private var client: AppClient = null + private var stopping = false + + @volatile var shutdownCallback: SparkDeploySchedulerBackend => Unit = _ + @volatile private var appId: String = _ - val registrationLock = new Object() - var registrationDone = false + private val registrationBarrier = new Semaphore(0) - val maxCores = conf.getOption("spark.cores.max").map(_.toInt) - val totalExpectedCores = maxCores.getOrElse(0) + private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + private val totalExpectedCores = maxCores.getOrElse(0) override def start() { super.start() @@ -95,8 +97,10 @@ private[spark] class SparkDeploySchedulerBackend( stopping = true super.stop() client.stop() - if (shutdownCallback != null) { - shutdownCallback(this) + + val callback = shutdownCallback + if (callback != null) { + callback(this) } } @@ -149,18 +153,11 @@ private[spark] class SparkDeploySchedulerBackend( } private def waitForRegistration() = { - registrationLock.synchronized { - while (!registrationDone) { - registrationLock.wait() - } - } + registrationBarrier.acquire() } private def notifyContext() = { - registrationLock.synchronized { - registrationDone = true - registrationLock.notifyAll() - } + registrationBarrier.release() } } From 5aecdcf1f23a826f6236096001de1dd811dbc443 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 18 Feb 2015 14:41:44 +0000 Subject: [PATCH 189/817] SPARK-5669 [BUILD] [HOTFIX] Spark assembly includes incompatibly licensed libgfortran, libgcc code via JBLAS Correct exclusion path for JBLAS native libs. (More explanation coming soon on the mailing list re: 1.3.0 RC1) Author: Sean Owen Closes #4673 from srowen/SPARK-5669.2 and squashes the following commits: e29693c [Sean Owen] Correct exclusion path for JBLAS native libs --- assembly/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index fbb6e94839d42..3d1ed0dd8a7bd 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -119,9 +119,9 @@ org.jblas:jblas - lib/Linux/i386/** - lib/Mac OS X/** - lib/Windows/** + lib/static/Linux/i386/** + lib/static/Mac OS X/** + lib/static/Windows/** From 85e9d091d5d785d412e91038c2490131e64f5634 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 18 Feb 2015 10:09:56 -0800 Subject: [PATCH 190/817] [SPARK-5519][MLLIB] add user guide with example code for fp-growth The API is still not very Java-friendly because `Array[Item]` in `freqItemsets` is recognized as `Object` in Java. We might want to define a case class to wrap the return pair to make it Java friendly. Author: Xiangrui Meng Closes #4661 from mengxr/SPARK-5519 and squashes the following commits: 58ccc25 [Xiangrui Meng] add user guide with example code for fp-growth --- docs/mllib-frequent-pattern-mining.md | 100 ++++++++++++++++++ docs/mllib-guide.md | 2 + .../examples/mllib/JavaFPGrowthExample.java | 63 +++++++++++ .../examples/mllib/FPGrowthExample.scala | 51 +++++++++ 4 files changed, 216 insertions(+) create mode 100644 docs/mllib-frequent-pattern-mining.md create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md new file mode 100644 index 0000000000000..0ff9738768aca --- /dev/null +++ b/docs/mllib-frequent-pattern-mining.md @@ -0,0 +1,100 @@ +--- +layout: global +title: Frequent Pattern Mining - MLlib +displayTitle: MLlib - Frequent Pattern Mining +--- + +Mining frequent items, itemsets, subsequences, or other substructures is usually among the +first steps to analyze a large-scale dataset, which has been an active research topic in +data mining for years. +We refer users to Wikipedia's [association rule learning](http://en.wikipedia.org/wiki/Association_rule_learning) +for more information. +MLlib provides a parallel implementation of FP-growth, +a popular algorithm to mining frequent itemsets. + +## FP-growth + +The FP-growth algorithm is described in the paper +[Han et al., Mining frequent patterns without candidate generation](http://dx.doi.org/10.1145/335191.335372), +where "FP" stands for frequent pattern. +Given a dataset of transactions, the first step of FP-growth is to calculate item frequencies and identify frequent items. +Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) algorithms designed for the same purpose, +the second step of FP-growth uses a suffix tree (FP-tree) structure to encode transactions without generating candidate sets +explicitly, which are usually expensive to generate. +After the second step, the frequent itemsets can be extracted from the FP-tree. +In MLlib, we implemented a parallel version of FP-growth called PFP, +as described in [Li et al., PFP: Parallel FP-growth for query recommendation](http://dx.doi.org/10.1145/1454008.1454027). +PFP distributes the work of growing FP-trees based on the suffices of transactions, +and hence more scalable than a single-machine implementation. +We refer users to the papers for more details. + +MLlib's FP-growth implementation takes the following (hyper-)parameters: + +* `minSupport`: the minimum support for an itemset to be identified as frequent. + For example, if an item appears 3 out of 5 transactions, it has a support of 3/5=0.6. +* `numPartitions`: the number of partitions used to distribute the work. + +**Examples** + +
    +
    + +[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the +FP-growth algorithm. +It take a `JavaRDD` of transactions, where each transaction is an `Iterable` of items of a generic type. +Calling `FPGrowth.run` with transactions returns an +[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html) +that stores the frequent itemsets with their frequencies. + +{% highlight scala %} +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} + +val transactions: RDD[Array[String]] = ... + +val fpg = new FPGrowth() + .setMinSupport(0.2) + .setNumPartitions(10) +val model = fpg.run(transactions) + +model.freqItemsets.collect().foreach { case (itemset, freq) => + println(itemset.mkString("[", ",", "]") + ", " + freq) +} +{% endhighlight %} + +
    + +
    + +[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the +FP-growth algorithm. +It take an `RDD` of transactions, where each transaction is an `Array` of items of a generic type. +Calling `FPGrowth.run` with transactions returns an +[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html) +that stores the frequent itemsets with their frequencies. + +{% highlight java %} +import java.util.Arrays; +import java.util.List; + +import scala.Tuple2; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.fpm.FPGrowth; +import org.apache.spark.mllib.fpm.FPGrowthModel; + +JavaRDD> transactions = ... + +FPGrowth fpg = new FPGrowth() + .setMinSupport(0.2) + .setNumPartitions(10); + +FPGrowthModel model = fpg.run(transactions); + +for (Tuple2 s: model.javaFreqItemsets().collect()) { + System.out.println("(" + Arrays.toString((Object[]) s._1()) + "): " + s._2()); +} +{% endhighlight %} + +
    +
    diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index fbe809b3478e5..0ca51f92d7a61 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -34,6 +34,8 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * singular value decomposition (SVD) * principal component analysis (PCA) * [Feature extraction and transformation](mllib-feature-extraction.html) +* [Frequent pattern mining](mllib-frequent-pattern-mining.html) + * FP-growth * [Optimization (developer)](mllib-optimization.html) * stochastic gradient descent * limited-memory BFGS (L-BFGS) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java new file mode 100644 index 0000000000000..0db572d7607a9 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import java.util.ArrayList; +import java.util.Arrays; + +import scala.Tuple2; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.fpm.FPGrowth; +import org.apache.spark.mllib.fpm.FPGrowthModel; + +/** + * Java example for mining frequent itemsets using FP-growth. + */ +public class JavaFPGrowthExample { + + public static void main(String[] args) { + SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + + // TODO: Read a user-specified input file. + @SuppressWarnings("unchecked") + JavaRDD> transactions = sc.parallelize(Lists.newArrayList( + Lists.newArrayList("r z h k p".split(" ")), + Lists.newArrayList("z y x w v u t s".split(" ")), + Lists.newArrayList("s x o n r".split(" ")), + Lists.newArrayList("x z y m t s q e".split(" ")), + Lists.newArrayList("z".split(" ")), + Lists.newArrayList("x z y r q t p".split(" "))), 2); + + FPGrowth fpg = new FPGrowth() + .setMinSupport(0.3); + FPGrowthModel model = fpg.run(transactions); + + for (Tuple2 s: model.javaFreqItemsets().collect()) { + System.out.println(Arrays.toString((Object[]) s._1()) + ", " + s._2()); + } + + sc.stop(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala new file mode 100644 index 0000000000000..ae66107d7015b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.fpm.FPGrowth +import org.apache.spark.{SparkContext, SparkConf} + +/** + * Example for mining frequent itemsets using FP-growth. + */ +object FPGrowthExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("FPGrowthExample") + val sc = new SparkContext(conf) + + // TODO: Read a user-specified input file. + val transactions = sc.parallelize(Seq( + "r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p").map(_.split(" ")), numSlices = 2) + + val fpg = new FPGrowth() + .setMinSupport(0.3) + val model = fpg.run(transactions) + + model.freqItemsets.collect().foreach { case (itemset, freq) => + println(itemset.mkString("[", ",", "]") + ", " + freq) + } + + sc.stop() + } +} From a8eb92dcb9ab1e6d8a34eed9a8fddeda645b5094 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 18 Feb 2015 10:11:08 -0800 Subject: [PATCH 191/817] [SPARK-5507] Added documentation for BlockMatrix Docs for BlockMatrix. mengxr Author: Burak Yavuz Closes #4664 from brkyvz/SPARK-5507PR and squashes the following commits: 4db30b0 [Burak Yavuz] [SPARK-5507] Added documentation for BlockMatrix --- docs/mllib-data-types.md | 75 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 101dc2f8695f3..24d22b9bcdfa4 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -296,6 +296,81 @@ backed by an RDD of its entries. The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. In general the use of non-deterministic RDDs can lead to errors. +### BlockMatrix + +A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock`s, where `MatrixBlock` is +a tuple of `((Int, Int), Matrix)`, where the `(Int, Int)` is the index of the block, and `Matrix` is +the sub-matrix at the given index with size `rowsPerBlock` x `colsPerBlock`. +`BlockMatrix` supports methods such as `.add` and `.multiply` with another `BlockMatrix`. +`BlockMatrix` also has a helper function `.validate` which can be used to debug whether the +`BlockMatrix` is set up properly. + +
    +
    + +A [`BlockMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.BlockMatrix) can be +most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` using `.toBlockMatrix()`. +`.toBlockMatrix()` will create blocks of size 1024 x 1024. Users may change the sizes of their blocks +by supplying the values through `.toBlockMatrix(rowsPerBlock, colsPerBlock)`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.SingularValueDecomposition +import org.apache.spark.mllib.linalg.distributed.{BlockMatrix, CoordinateMatrix, MatrixEntry} + +val entries: RDD[MatrixEntry] = ... // an RDD of (i, j, v) matrix entries +// Create a CoordinateMatrix from an RDD[MatrixEntry]. +val coordMat: CoordinateMatrix = new CoordinateMatrix(entries) +// Transform the CoordinateMatrix to a BlockMatrix +val matA: BlockMatrix = coordMat.toBlockMatrix().cache() + +// validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. +// Nothing happens if it is valid. +matA.validate + +// Calculate A^T A. +val AtransposeA = matA.transpose.multiply(matA) + +// get SVD of 2 * A +val A2 = matA.add(matA) +val svd = A2.toIndexedRowMatrix().computeSVD(20, false, 1e-9) +{% endhighlight %} +
    + +
    + +A [`BlockMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.BlockMatrix) can be +most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` using `.toBlockMatrix()`. +`.toBlockMatrix()` will create blocks of size 1024 x 1024. Users may change the sizes of their blocks +by supplying the values through `.toBlockMatrix(rowsPerBlock, colsPerBlock)`. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.SingularValueDecomposition; +import org.apache.spark.mllib.linalg.distributed.BlockMatrix; +import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; + +JavaRDD entries = ... // a JavaRDD of (i, j, v) Matrix Entries +// Create a CoordinateMatrix from a JavaRDD. +CoordinateMatrix coordMat = new CoordinateMatrix(entries.rdd()); +// Transform the CoordinateMatrix to a BlockMatrix +BlockMatrix matA = coordMat.toBlockMatrix().cache(); + +// validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. +// Nothing happens if it is valid. +matA.validate(); + +// Calculate A^T A. +BlockMatrix AtransposeA = matA.transpose().multiply(matA); + +// get SVD of 2 * A +BlockMatrix A2 = matA.add(matA); +SingularValueDecomposition svd = + A2.toIndexedRowMatrix().computeSVD(20, false, 1e-9); +{% endhighlight %} +
    +
    + ### RowMatrix A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD From f0e3b71077a6c28aba29a7a75e901a9e0911b9f0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 18 Feb 2015 14:02:32 -0800 Subject: [PATCH 192/817] [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction Also added test cases for checking the serializability of HiveContext and SQLContext. Author: Reynold Xin Closes #4628 from rxin/SPARK-5840 and squashes the following commits: ecb3bcd [Reynold Xin] test cases and reviews. 55eb822 [Reynold Xin] [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction. --- .../apache/spark/sql/SerializationSuite.scala | 32 +++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 35 ++++++++++--------- .../spark/sql/hive/SerializationSuite.scala | 33 +++++++++++++++++ 3 files changed, 84 insertions(+), 16 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala new file mode 100644 index 0000000000000..6f6d3c9c243d4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.sql.test.TestSQLContext + +class SerializationSuite extends FunSuite { + + test("[SPARK-5235] SQLContext should be serializable") { + val sqlContext = new SQLContext(TestSQLContext.sparkContext) + new JavaSerializer(new SparkConf()).newInstance().serialize(sqlContext) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index d3365b1e8f44c..2e205e67c0fdd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -222,22 +222,25 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be * set in the SQLConf *as well as* in the HiveConf. */ - @transient protected[hive] lazy val (hiveconf, sessionState) = - Option(SessionState.get()) - .orElse { - val newState = new SessionState(new HiveConf(classOf[SessionState])) - // Only starts newly created `SessionState` instance. Any existing `SessionState` instance - // returned by `SessionState.get()` must be the most recently started one. - SessionState.start(newState) - Some(newState) - } - .map { state => - setConf(state.getConf.getAllProperties) - if (state.out == null) state.out = new PrintStream(outputBuffer, true, "UTF-8") - if (state.err == null) state.err = new PrintStream(outputBuffer, true, "UTF-8") - (state.getConf, state) - } - .get + @transient protected[hive] lazy val sessionState: SessionState = { + var state = SessionState.get() + if (state == null) { + state = new SessionState(new HiveConf(classOf[SessionState])) + SessionState.start(state) + } + if (state.out == null) { + state.out = new PrintStream(outputBuffer, true, "UTF-8") + } + if (state.err == null) { + state.err = new PrintStream(outputBuffer, true, "UTF-8") + } + state + } + + @transient protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf + } override def setConf(key: String, value: String): Unit = { super.setConf(key, value) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala new file mode 100644 index 0000000000000..d6ddd539d159d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.sql.hive.test.TestHive + +class SerializationSuite extends FunSuite { + + test("[SPARK-5840] HiveContext should be serializable") { + val hiveContext = new HiveContext(TestHive.sparkContext) + hiveContext.hiveconf + new JavaSerializer(new SparkConf()).newInstance().serialize(hiveContext) + } +} From aa8f10e82a743d59ce87348af19c0177eb618a66 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 18 Feb 2015 14:17:04 -0800 Subject: [PATCH 193/817] [SPARK-5722] [SQL] [PySpark] infer int as LongType The `int` is 64-bit on 64-bit machine (very common now), we should infer it as LongType for it in Spark SQL. Also, LongType in SQL will come back as `int`. Author: Davies Liu Closes #4666 from davies/long and squashes the following commits: 6bc6cc4 [Davies Liu] infer int as LongType --- python/pyspark/sql/dataframe.py | 14 +++++++----- python/pyspark/sql/tests.py | 22 ++++++++++++++++++- python/pyspark/sql/types.py | 8 +++---- .../org/apache/spark/sql/SQLContext.scala | 1 + .../spark/sql/execution/pythonUdfs.scala | 1 + 5 files changed, 35 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 52bd75bf8a369..c68c97e9260e2 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -803,7 +803,7 @@ def mean(self, *cols): >>> df.groupBy().mean('age').collect() [Row(AVG(age#0)=3.5)] >>> df3.groupBy().mean('age', 'height').collect() - [Row(AVG(age#4)=3.5, AVG(height#5)=82.5)] + [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] """ @df_varargs_api @@ -814,7 +814,7 @@ def avg(self, *cols): >>> df.groupBy().avg('age').collect() [Row(AVG(age#0)=3.5)] >>> df3.groupBy().avg('age', 'height').collect() - [Row(AVG(age#4)=3.5, AVG(height#5)=82.5)] + [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] """ @df_varargs_api @@ -825,7 +825,7 @@ def max(self, *cols): >>> df.groupBy().max('age').collect() [Row(MAX(age#0)=5)] >>> df3.groupBy().max('age', 'height').collect() - [Row(MAX(age#4)=5, MAX(height#5)=85)] + [Row(MAX(age#4L)=5, MAX(height#5L)=85)] """ @df_varargs_api @@ -836,7 +836,7 @@ def min(self, *cols): >>> df.groupBy().min('age').collect() [Row(MIN(age#0)=2)] >>> df3.groupBy().min('age', 'height').collect() - [Row(MIN(age#4)=2, MIN(height#5)=80)] + [Row(MIN(age#4L)=2, MIN(height#5L)=80)] """ @df_varargs_api @@ -847,7 +847,7 @@ def sum(self, *cols): >>> df.groupBy().sum('age').collect() [Row(SUM(age#0)=7)] >>> df3.groupBy().sum('age', 'height').collect() - [Row(SUM(age#4)=7, SUM(height#5)=165)] + [Row(SUM(age#4L)=7, SUM(height#5L)=165)] """ @@ -1051,7 +1051,9 @@ def _test(): sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) - globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() + globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\ + .toDF(StructType([StructField('age', IntegerType()), + StructField('name', StringType())])) globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), Row(name='Bob', age=5, height=85)]).toDF() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 52f7e65d9ca78..8e1bb36598727 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -38,7 +38,7 @@ from pyspark.sql import SQLContext, HiveContext, Column from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, StructField, \ - UserDefinedType, DoubleType, LongType, StringType + UserDefinedType, DoubleType, LongType, StringType, _infer_type from pyspark.tests import ReusedPySparkTestCase @@ -324,6 +324,26 @@ def test_help_command(self): pydoc.render_doc(df.foo) pydoc.render_doc(df.take(1)) + def test_infer_long_type(self): + longrow = [Row(f1='a', f2=100000000000000)] + df = self.sc.parallelize(longrow).toDF() + self.assertEqual(df.schema.fields[1].dataType, LongType()) + + # this saving as Parquet caused issues as well. + output_dir = os.path.join(self.tempdir.name, "infer_long_type") + df.saveAsParquetFile(output_dir) + df1 = self.sqlCtx.parquetFile(output_dir) + self.assertEquals('a', df1.first().f1) + self.assertEquals(100000000000000, df1.first().f2) + + self.assertEqual(_infer_type(1), LongType()) + self.assertEqual(_infer_type(2**10), LongType()) + self.assertEqual(_infer_type(2**20), LongType()) + self.assertEqual(_infer_type(2**31 - 1), LongType()) + self.assertEqual(_infer_type(2**31), LongType()) + self.assertEqual(_infer_type(2**61), LongType()) + self.assertEqual(_infer_type(2**71), LongType()) + class HiveContextSQLTests(ReusedPySparkTestCase): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 40bd7e54a9d7b..9409c6f9f6556 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -583,7 +583,7 @@ def _parse_datatype_json_value(json_value): _type_mappings = { type(None): NullType, bool: BooleanType, - int: IntegerType, + int: LongType, long: LongType, float: DoubleType, str: StringType, @@ -933,11 +933,11 @@ def _infer_schema_type(obj, dataType): >>> schema = _parse_schema_abstract("a b c d") >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10)) >>> _infer_schema_type(row, schema) - StructType...IntegerType...DoubleType...StringType...DateType... + StructType...LongType...DoubleType...StringType...DateType... >>> row = [[1], {"key": (1, 2.0)}] >>> schema = _parse_schema_abstract("a[] b{c d}") >>> _infer_schema_type(row, schema) - StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType... + StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ if dataType is None: return _infer_type(obj) @@ -992,7 +992,7 @@ def _verify_type(obj, dataType): >>> _verify_type(None, StructType([])) >>> _verify_type("", StringType()) - >>> _verify_type(0, IntegerType()) + >>> _verify_type(0, LongType()) >>> _verify_type(range(3), ArrayType(ShortType())) >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index db32fa80dd3e7..a6cf3cd9ddd4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1130,6 +1130,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true case ShortType => true + case LongType => true case FloatType => true case DateType => true case TimestampType => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 69de4d168a372..33632b8e82ff9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -186,6 +186,7 @@ object EvaluatePython { case (c: Int, ShortType) => c.toShort case (c: Long, ShortType) => c.toShort case (c: Long, IntegerType) => c.toInt + case (c: Int, LongType) => c.toLong case (c: Double, FloatType) => c.toFloat case (c, StringType) if !c.isInstanceOf[String] => c.toString From d12d2ad76ee673b819c92dd8093ba0a560847761 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 18 Feb 2015 16:29:32 -0800 Subject: [PATCH 194/817] [SPARK-5879][MLLIB] update PIC user guide and add a Java example Updated PIC user guide to reflect API changes and added a simple Java example. The API is still not very Java-friendly. I created SPARK-5990 for this issue. Author: Xiangrui Meng Closes #4680 from mengxr/SPARK-5897 and squashes the following commits: 847d216 [Xiangrui Meng] apache header 87719a2 [Xiangrui Meng] remove PIC image 2dd921f [Xiangrui Meng] update PIC user guide and add a Java example --- ...IClusteringFiveCirclesInputsAndOutputs.png | Bin 249245 -> 0 bytes docs/mllib-clustering.md | 95 +++++++++++++++--- .../JavaPowerIterationClusteringExample.java | 58 +++++++++++ .../clustering/PowerIterationClustering.scala | 9 ++ 4 files changed, 149 insertions(+), 13 deletions(-) delete mode 100644 docs/img/PIClusteringFiveCirclesInputsAndOutputs.png create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java diff --git a/docs/img/PIClusteringFiveCirclesInputsAndOutputs.png b/docs/img/PIClusteringFiveCirclesInputsAndOutputs.png deleted file mode 100644 index ed9adad11d03add0889a9a6e7ca06ee0e91850c0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 249245 zcmeFYbySq!)&PtM5(+3O4I&^Q-Hp;E4bqZB4TI7xq990jNO$+pp>zz*4BZSd^e}XM z_-etKb3rVtXJh@~CdPujc5#`5(m>G#iZqp5VQ zN(Zu1BGcH4*jZnro@(z}X#~_|z6d2TCaEgJjH)x&BYoh#keOXw%|Yr=*rNErx2=9N zO>lE4(R6>_ds7k#@c}$SbJa^wPl~idQ>S}H%J>zFdt#h7DOVB=T|tWE>4WHQ(#H?i z)+{pxISl4-pCwz%_{8x)D2_qxTuo zT~#^1r|Q$}v%tm0r7zKLk266Bfe%f6l`n?OAiv)ZsMxAi1CHUa!<()Axc5%Z=!?#?ft^();;fQ6n9lYc<_ zjE>Ltl7i=_avXiXGQM^-uky>tZ?`tD+l6=(`}MPbV_5mser3Z~rGisAT88m#629@< z)eW4dA65?2y_9?n!HJ+geWplgnF672hmd|_ zeC5NnfPQq?z=b8CT>ggalhns|nqOa#DzAl3mH;HT#5_(feKv! z|6}4WMjfC7`<=i$Vs^97m@=QS8ahmD$s-iv>#>)4pT&J?r~4S-@sdMfaR8HX^wz4Y z*kd-(!P7oO>;-|?H>1ypHNOS|nui$=Dn-`P8(>aUTD}c>=C8QFK7aFsVK!{*gY?uz zWhmO6Byi~kjv7wdSNX4$7%VIJ--F7GgshX_{_5+~>>%}RT}jE7y2v^l{YI z*~}X1F&e>QNx_o4Ytzfqi1Q0cF8~Y4_w@rxV&?*~K>-=3ET^FFBZDl~kA8FVB$Bgy z5DhaE5fy2CKzG%~CP52&etaPCyWtw5>w=3`@yTZ0x29?cP3V#s?XnppPy=)FE<;=P zz%qQnsxj>}6NvUoihBQv_y>%k&-_%eyiQmWbZB3{yn2ciBPEiA4iLv8mYSOAc+Eu{ z7l2!dUitv>%eoS87v1~Un^BC2fQeE_a0kO+tVl;faJ>!Z%4+?F+A z$WVWhmpxlZZx<3FD?TR0N1^(bIY+D(n_ZSHYjBjJ7Jng7AiH3UZ3|C3v`q?&>vNI$ zlOc?nUa_hVuki;vs=1#DV6pWUSBW^#LeZB{{3@X+Gun?Y-t{Ioh#O@*^WYio`Ee-h z!PrK#^i6OTaAfZxxESfrIEwuUL-dOveN2?x^0WMB1K3kvslPIOb#{_RJzabjBJ*f0 zbnC&HwFq~jQ0#lfd4+j3G1=Xh5={6}3NOWDsQVL%<;|l*6|HhzbH#IGb2&5#wjb)p zJE*uZ1xNGtp&z4v;`bE1z-at&9VS+oon@Z2T1Hs#TO&EE+Q_O|>BoRhJWc$YP^RwC zRb|EObxpXI-K(oto3GATK-xMwsyeOO%!S7~n?=w<8uiw~*|N*%V%fU9M_SApjFVQA zX_H!$qLYGr6p$EFsliN=iG;BoTYwFLE$!r%9ll+(9aUYUU7cOHov|I~l+GmkCijY_p)m6)4Ff7fM#6o*tqYvK-rXWlH`@; z<>|$SN;#p~TDt^a5}g!mgT^HbmIMvFYe#HOw~oiKHS!U%nUs|@om80Ao}J@;OMPv% zN}Z9zjOE3=5{z}cc(VA{P8&aDcB!eaY51}UZ>(p6X~a55HJIHFVV!DWXzx9RU*}xE zHV3RlP)R>uYmm3wQkwYP7)>pOY{qV>)Y%UF;Y>D3Qk zi4ZS52VWw-59y^c(bhSq54CSqN#`pyZ&V-xR+uI~RTff-*wsWMt$^!x~6zpZ7sSySYdzEh`CL&>XU3_$A!ETOOrTUSMgU3;4KG37&Jo|i! z=_zxUT4CBD-!B$3-QVA2yjCZ>7D5hUzQiOjFDQfGCA_T9FDy_|DpK>zpolSuXH}(4 z6%gH1YVs42Y!wcSi;k&cu=rrU)x}Tb&D5yMn(E?QJ+;0a%c(k+5hJ;Au8k@xEehtr zNiIrm=9R2jeeYTtP}*0Dr+>5i;nd*t@jEmCe?cN*uxwRg@Dx%$;5UriC|gCr&o{L< zPmTgF5Em7n7(NMZwvGH4oJ$sr%SdT&|7;Yc@1&i{*hln}osadW$uvtXQ36h$?4qoF z=%wsfHnq8onEt#`DPE)qkkGzgEoPsvJ!UqBU_vvWI;C9Ln5z{Sv=^4gn;st&f6GIZ z(VspeGH40C)?cW32mqx84)v!@h`=27vm3>czc=?%)RHp<@6IqI9v73g7$-GVT1Bs9 zd;XxgBfiy=PBus`+=ZHFbf+dRE1xURCN{k`u0OFCF1Y^m%|h0-dsE46VbYp)|u#B$!X=y{_xc^2uE2Mb5xVNOI!p;a{WZZVSmr`WUq0_7jGEvrN(`PFS zDY7*Iv^jP+jUb$%ONDsF3+=x?bdKX42!0Z91L98Cf@)Uap^yylVg39fMO$6faCNyC zSd&7^ED1q$qumuIMdt8ng>s z1hj(?em%FL50^i$cVx6%-+CcGocw}Hrw5->gN0gFux}JR8ODWjs@ho!6lZ@f&U^Ch z*x3J28;o9}@|*JP&7P5s?#5pI|2ll2akm7=zZmWfEi0`Q_}tQ-7t1_v-V zBAe0dA+dX%{5XK@oyke7$8OA7UQVJor=ao2_VH;r)NS3pekroHxn9r&)i681GBq|X zT#l4U^oMQR)5C^MBU}77qzvY$YFYbW{gOXdm z)2K1z+BdbLjC8zrS1OlA$ZojuAZ)AREEqQ^=!l=09114Yf3hiGUIGHkWqyS!Ib5wQRPIm|3U=2jdY zK&SiBXlP;{BKNOAD;G0551@mivxtZIi@!#Q+`s>6=6pf-*AN$b@fYt^)#;=`pRDNk zIruoZUP$25(b0*0va}Y_l#%<#^!s1pFKk>~oJ2S|-QC?e+<7@bp8%ZP!otFwTs)jS zJnZ))*quEcUCcb#9i18e2gpC+$XGdBe6n?Nu?0EO{ef#{4svx7fAQiEqJRDU&pNF< zZ2uRNqw_z=x+jqHPY)+I2N&nRVBb#_`_n3-ZtG#?@Lt9iXyxd9k3)i+hnG+6uL=Lr z^}i_pZL0piO%>w%e@*?ju76Aw_bXLlniZ@EToxD;MWb^Qp+&KEf1HZWJdAZv{Kw-v+ac`Z zSvApr_WT*`gN`w7gN;r184dmaK3>vE(Xl^xiT~zrfJ%Lqq5%F~J6Zq+26isv=PvsH z3+^uzPZe_T{tg#i)MvDST8twqp=JrHe{HBMkmqm{N4%0)Imn-DfHE zd#VYgR-gU{v;MFIJ9q5S-*Qsno@xVh1*$Lq$*X((0erK6+m4Qd{Zv62EkycnVbVp( zQU0COr051YY2sa3$nE)JPTzW+Ozk0O{J?WSu{ zXu<}Y#glSgomjCw@j@~a0SS?c6E=`~G_!LtDWi3pw+gwT-%;*PszEl*8Vko1VE>kq zooztNYm(ubG7#ARUUjhXopPdOuW*X-gMf8=i&ncOcPTgrXR`Nso*o&CWMZ>YW*hTr zU5RgdM2qWi@iYIl*ZRfRKRq(wA$@tSkO!UwwS{|A&epA_j~!dIWF2eM0w(PIPX+BR zX4dQ+k#a0Q<+~ZbH5Hv(u(Gzu&5(bc%BHKrx#x$1K|uR#4PeBd8MyuGIXd>sZ&JNE zlsb*Az;mmkxWfD<&B5d9Ab$=(7qJsmKEQ3ad?Kd*{D##Ur^hXLp;xO>T68cmY!9=S z8x=5Na%0)6J5?`pWlqk{0Ox(*zUcEP2a9M=_gH&7B#jMcK+fXHoVB}hr~uSmhElIo zZ5e5>ey3Cb-~=$LiUgpQ#y!Y!qW`}2fQ6O1LRTR zJ9|ne9DwLJZO5t0ak9OYA=;?n4~87N@4{iuieIawx7nTbj4BK?<5vR!#oJ8%@3o+T@$@I>r z%|4oOEYa369XBtdE$1cRCqKN?-tZyN*(G6PCiNBT@4P0gzm6w!7kfKaq+jeCl+jo} zg*167-!LuI96fUlKf*KTS&V#psR3oOrHqWx5}?1gSe=9lmYI{4gzF#vP+5u@s>~MW zUd8{N!AH16Fd~&6P&VAoP(P_te0aHFkgi-`A(174EME3n%`2&8~bCF1@^j= zM}>6ap3hy;hZyD&JPcUDv6AT~Z&N|FKsGj`y+Fxs1cCjp-6urq)T1+t^{&^WAkb92h-|J=)xY zJysKtX>>?6z;|Qu*MAr1+h1ubg++5&4dxT}_pema)z!&~p_D8XuCcah^7MxfrHag+ zmbyU&guKh@_zPD{Z0(BKVDC|Z8DVER=XH|ga~{`JHDrI(Rp$OP@&ctEUAk!kYT_#=%hYmMcSGNFbZaxqYf8rh$3iQLw%2aa z37qpU5mT#bd>w+x%zHNzt_94SU5Zp z+33314KlXWY)gC6F2ra|nHtA0E|E;JSZTa*LVqK6QdPGXmEwPD==;m3ZS89( zb+1~U;KEgV!Hyi@DMYJD;|}p74>yB?L!3ccInlfJ1l)YGPGF}qNBlg_L1|vMqAb#= zq)F#uP4l|nNs^*CGqyaOAl~eh;a&r9u(@h^|2j_u^uKhH128ma3-cr;M3=-*g|pUx zfSwx6K|&*bLWrDaPPcX$38PvsKY;1#PoV>oF^5IW_utj6LY9eoF z;n8;Zvp0NK^gO0@KL!q9aKbC7+@4=mZ=0D?X~DZ1E9an_aNw0t1(P8qC7?~wr7`P( zc`a>bcxexBScjRUuwj~L;O(4aojbEU=bU=hPzt$UgLCW}N5Sd<@bb2&k1l1crPhJH z{cPzZjkP!Y9S{#s#AEhju04op?dFx=3X>^iWQOVKi+l1yaD)yVL()1m?!U_&UvXa& zrzrGmDC$(Dfa~$$DE74Qi!1*M5qzrAgLJ`PV0ZM+29ELTZWy8`az2w$Si_41m7xnm z9Hap743vvG1|cEGF25h3Tw3$buCbCkIdmOsYlrBj_J?7<(mg)55MEnZ7E9B5J*?T; zlYx+Wh=L+-!)stZe8Z%O5oT2IbLf2_~ zK31-48U0qkczQ-b#2Yev#fg~CY(gDt_GV1^jxkawBgIb01P4i>RO_zETe;5T18>D3@ZqAO2DRMDq@6tlP zV9{@cRVT$7(x`$IgB5Xsa#3?-3x+Aau%o+?oH7F4m(wM$RcUW_t$9616P7R9#V7Mz zzlGp=?I5y+1f|9#d`#!W1jqYdHE%PLAP)ZyxzI%+RjP^w*} zM6vE%%cmnk$v%?kO{-|#yQ3s_P#m2 zklc#b0`M=`b>x$nwr$xNTlhU%Z#<6Mq1n3$*+ZJ>mR}ug>)6{b^X^XVKkTio<>QPi z`_7-ZGmw7o4)I>UYt8IT11U{dr;JSUSwWCnDTw1C0M`J4@>L^Fsry~KsnF(EUIN{7 z15dLrdqT1oY!F(!2(XQEC9`R&-z8-Qt}<3&RQ`f%>uLVo**9%1fbb0W!NZ4ez^r|s z>LxeNwW-I0jTCYy-4Qa&1 z&oAA!wz{2e{aOZj75*RdcC|`TG~RDX5Yos#?WqIkfY>INAj);ilodJ$S7(_h7e8Ya z%jF(LQO3^&4FiE*2Ff)Kje7?XR1$r zT#tj#ih2YHRDP94HWDfzL~oUKv?(ep^BUa(eM}+Q;_)B(taxFQb^0&T$6c3N4Dq9{clIm$Alin}n2OvQZC+&Xo_y}b z7GFND`hfWA^f|(|QLu2}#Ae2Em6Im6Sy=_`7wzhgk8hovGtzGpai{PiYC|`+MCZqP z@|1YV3>z&tz$SZwohpJeLCKFeEeu)aj4HfA1Q^(tCxck5wF|=IgH#KpcqF${(i`~B zX^g`iPVQH+#S7~8h%t-i&!9RfojOzKK=%A0HB)-*Ie_1U(zzM9V2@x6803>`p1z-%=yEwR?K99Dlx7N&9YkbNy!JP8;YfgNetqHwI5zg6`1( zZceD@%BnlWiyI8W5u8(zLjdvTci4mPk5VXfDhx;k;y#`LorTVb*pyq-u22Uhg55&} zoO75Y#dkd-;X!TfNKgK01vW{rOtLUQ+l=FB13@kH#$bUdiuBfbC=mi|v{IZ5g z&RAjBPpW2(w*VH`%*M1eH!3yhmMwE){Xz>E-bAf`bhp?A|Ly%i%>+Q0SYw&9G_>-b_T}^r3b-^ z4}hN@jeWsU=^C&Zv_5w5_cEJXo9p~yqnqXMYjOe$HFca7Xam(R3$MGkCN&lEF(^Nw zAL43b=@LRIZe)GIHxy-c@vMC}R_vp(H^jmyke-eHzy9){GO-!hvShjv~+&p9S?tn*&Z@vr^p&1>4hPV308B zG+1y)s^v$6jvhSJIR7TT`yNs9tlyHCzu|M+E76-6x4C>FEycQoAa^rxyad!vX?VOc zYnK`2x2zW=)r?G|+E9dnW%e>m5sEzlj4M|f%LcVaXw(&dKKV)5qYOZm_lm?1FnToU z#OO#-223``fnIZ<-gGfYmFsf^QlcnSHK4ufO6yhGZ1HiVsl{7P_;!Y||9FMJkp-f0 zvUulMerR4jFg6chl$WaVvaD{(-x1+Vy~EonP+oa$txUVaqp&gXG9B1}wZT;`biDq-@d590&t{$zF|ys90rG&T0HeUi!0 z5LmuY9X>@=ldC8l@gS>*oxqCcB@f8&{rhP7pCG(|f z#KuM20JV~}i`HTw_8w@a&BU2>cf9GvRV!Vlpg_jd3OKwi!n&*+$2X#EK3-dUC%?ow zK6aB8g!o5|c*tuGU(}@hh zWe`j&oy%PyW{UVtsfnY#4#dr{aSm*cC>NWG-<;EWKIOG~7E7~tDOZ>n>pB*?W3ZrK zsX(TdxG3(a-Ol&sSe;W$MLSFcL_czWjR9SSC=>1Y^TNSZEml%i)3r6 zW!2d~A3WR{aIbh`@1GunTV~)F3s*r*{9td^roCJdT*bHb1+_TX^TJ7^Gd9nK(>)$c zxLWyzkPj^Kcqo;$Fki)OR^91FdDyiv1gi1SEZ+3Fa{zskrl}(C?b{Qz&r-98EkYCz zFqi^9Uj(`jS#><2_TD)2Ms@BqsY`WH8DAHrNuQ&CRPO>(2}f z`Ww)Ct!L;^Uw;}aq}^-PBA8FNA6A8->f=%rW|tG!pn)PY?MCSzqO$emA8H89#iz_h zY||!M2g6OaC9^is+uaQ>UK>3ZYpU6hk5F7J>yIq+G(=DAgP;nXws{t=N11?jRM+V1 zKy~CVHM!*Afy=VlJIsX=Lg0dJ+f(mrn3*{xg{rH5WbSB^5&&^OUOR^s>KR_4V-31z%Wox3x>oLbuYw6RVP?{5j-X!>mx(wBY?KoRrNApG_e6k>bj3x!!yLreW4eZG`a*8$hh7F6n|bhfK+)y#=my>kk1eF@CbYi`#g>x(+5v+K1mXndKgXX)d} zD;@+5!W5-q4qu^Eo4I#zpPj1vrZnQ+t*ta(uI{8{endF?hn+u^_eg+F?vJw?O7X&K zWV{{Q6$w(Vq?T!*x_1`T7nW_m<&J}x;}5UC9x(fg?^LFAE?2`vBW-p33h{Y(6YUlf z3Wi3CDjxa2tDBN6a3nhUL7I)1OSFeN$pQp!)Sddn*Yq$6ln2>IDUcjo$pQ5rD_Y>U zdgVthLThlt|)+y$2zc$)QA9g26T{dGU*cv=0fW+VBmL<(c~P$R$;m(j}#C09WYjDU@Mt=Ft^glipfAImNF!lXq{5c>%SI6&Tj(}s>F=)?V z;UjFp!2XNPaK#o6F)Kb@H0Ssl5*E7&@KZVFZLCm_M_x=!NURISSrhdRZo zg6Rxi^}Dbe>y`0?2l@8JDY(O?{991rrBZV=-criO_}Sl%EufEnn$lb@3D&sJC33T* z;^Zf<>OZJv-%$SJj+wnV*s5eFu*ztWARej(>cPzy>_kW~5@HK(3E!jI@pS|?$0FPv z>qO>eIQO_+f@u7X-bw^B*i`%N~6SW zn&yd0@D03$>TdRmELHEt4`m z@I)-z$)``xr91b1bmhkdE=3IM#3zLnyLCkGhf3*+4?=B&JqrgLI5Fdg>1IDX8vm47 z69h%IaOo*O}+X{V->5Lbu| z-cn2q@{N4d-UoS`^x8H(f~$74WHC=AB`dfgt&v|z;=6B6sGsL+zdG=FNlV`&u|%s* z(#~@%6gJ@_i===CMX$!FAYV0`BJAYnE%?*KPjM7kn{pXpYERozEbocxUhC zf|dS$OYD{6Ql5>0%}HL1YU;@Xb7YdA+lq2%f{tWaLAuw#*@3BK&CBc0LI;qiHR9m% z2)Hr=_%Y~{)q+#1ZC=*KXhxGX=r`2pqBNos*m8MY1D+=?F_(3SWpwwO{kqxfUUcrD zIGI@(|F~D$+MpTbw&g|{-M^zoB1=%1%E0^hAi=_}(r=R%`voCoAf7F6jZf=~hl;S< zkw?_q5#wK+9bprJduK%jx2nqH4*@UF?cgQXYID97=UVkLCviq6&%6iu&6X3$Zq$^h zR|y%4N{BdXXEpgHa}L55Zp8)%^J9U4mp(@SC&2=Xlwx=PD0zNZ&uTvnGZt%7?m=8gcw8(d4b*Y-f(hPmPABUATcB_}Jv z3=HV&JI;MIHx!#lJ;Goy9<}s8~Fj{9rVv@ zgv}nndQlM)++LlP0p+LaKjk6;-^R&@4F+~E3@qPaN9>;n z*!g673}X-y?6XIgJ7E-pErO9qV^7NTsRnL@Txl zpT_S-UZ~B7S)jJ=?jIy)-k%leHeT>@dpaHMOp00;t}b88>yZ&BOF{@9L557X)aF5O z9VK354raTK*U2xj$nvdbs2J(!o$D(hHG`!RFODRVNJk^8C&;ZEcj?x-Qe~KK0+w@B zpHgj%;U=nC0HFM$@w?y9`*>CB=>>J=`(TbA7}0ATvqN{SUeHjxwBQC9(f+upXu9NPleZz?3`L=>fP}0y~Go$(p!}8qSM_ zTFvCOYcQ@#P*ey%Ap5^nuC{n5z5|>`JX7b z>4z$G`wW949N&|C#(^;X<&LqH3h~)1BE#i@-9_zN0{Gpp!?Xj8C91dxEkhI{(0IT! zVQqTHhc~8)BI@Ew&WYuffl$}T7IB`bo$G}gTrjEQDI4$aU<^;F*Vrh` zeSdE*(xmQel`|X;D=DwXW3$!mMZoJ1Z)3q|Gx9FqE*vGUb|~K_-tX}=o`XvhAOXtn zIPMiEo6C2qhsV2fMEZi_HOcOvAl_;ll#V^+WHjU zXk{etOXg}#&gQFhA$y&6zjn?fDuTX0*X|1H=!~MCnmMT<`{EoV;)bylR$9Kh2kKEl zdW<(Hr;r-eyxM=(IWu#)8nQy=C<=z1aMjE20$y44U`vRe{+`pM-DU;)3Qy&BIv|iVfJh!@J&K6bntPxW>agcJ_t<|mA9-rDv zsxeKYqB1u4&d;1`i(E^uU~40B$)4emc^{L_-OP~MsgR+4g$nU6*jtrrebQh^7f(id ztIP%^;$)liWb0g6e#);%m-7ofC6~e8#D3l((_My%1YO^rnAvmAa2Hds?b#NNrRNthwP;sq~F= zAJ&Wr4=@a(XtoZM|pqD@OeBm_Yyi8n9n!(cb)geRYhvf^GG4E0?Cd9ZA zGZ;3r(tB@_oC#}C+6IFmWZ7_kmSiyoJ}t!FKrZ1JH}Uee%*eIk@|`CZ!rAZThyx*EH6F-wbiV+pm&P@;(A-#B7 zJL-ots%9uVHI3S|vz{g^*hM&E(;NVOWbh<$PJE8Hq<^5-=oHv!{F49D%fEY_Kttds z&o+h96Kc;NHf|B}C`DxkO12RS#_kB87YaDemtqN~`V;J_ZQ0s^ABQ1!&J*`aFM;9? zxUBA7*xBTdcs;MDF)-&r?rWO1$*}D^wkp#ijqnkdS-~#-gDI`03fEoJkI1*ZS012U z(|k3zj0oYBgD%>+9OemFR`AfK88R@n?jurqhwlwhZ%m9MjZ}2SxGObru^A8oi9!e^?PS~Vd4~N}@>-*WbN-%PS+MT| z<~h<{J5Vsrx)4|;UXYo%zD8^5V>5H zcJj=CsX;MT+g)WvuQS~YZ^05lX^-!o@QK4Aw+J-AK4Ga;1v&cV*YP?iG$w;i@)m`dlbItVXl5GuhH+M!d9;l*wx_3^mO77vmN=>8 zg_cg-_a=fmjEdTf881^`%+cCW8IwnDpCmsQe2zqJCY8Lgoo%Se_xNLZ8f_U#0g^4lht z_u*3iY(1wiJ)rR?qXpU6tyib=bGjrQ4Aqum-dgw`+N0>jY9vd@$g6z4Gd48*!j>(Vo1J6b~ zArq!%T`T9R*pDHf@g+17R6E+08t(9!AX)iz%ek2B*QO&HcT5K>dN~A3y9cA?O&?@t z-P-CzGS}e9QM;f7aGk);VbSzb7ytsn6&386`#$`VH_kWwZem|Fr%v~Z;e}gPtrvb0 z7H!gXEx5L@uB;nl;kU$jmNc`=UX14{jU2@5S}=|Kc%qEfzeJ*^t-q~Nb?NpfASrp* zpq5M^q1A^Ab;THHfQ8l4l*$GcD;Ob zWW3Thqz&?3Gh4|!dPpsx&U7Ju+9#q_{-i9FqBk$JWMC+^RXTNCV%g_I^jY{8|N6p> zKURT1b{eHS`>G$Y-^sVdo=$O($iIYs2-|0rRr=#-o_yLj#5qPW#|dGtvq$v>l=lst zxfdM;P|UeS?%$rjn7it*13Ma6xGLjdu9xzwbz`MTFISV=C%Z0;nzdoD-k&&+Ttyg5 z6wY*~c^hSyylTade`x#Hr_lOzaO=BRZR7n)Cu)1Fa(K6RHK;F--_z=#TamS@PWmS} z(s$td@Oa-}h9q^@^fg4-c9;{KrG0I`V)b|l*L%D2?HTjSF;tiLRq#m@h4Ih>sUKea18_faAfX%n9!n=bmrinB+x_LSB z;SBM$%6u?zmRqmd6bXL1U;Mjyhi)Fe81MkD6JTzU-N{$d^S!U4P|0@o2Ig zN06?*tHmo}wLC|f{_`<)Tnli5jt6X+7VA2(@=HPF?tzODD6@Xzq@1_);`crvB9Q5V z5@g{j(^&f2`%FAn$@?=@%c_KpY}Wco;^9?4qxE9X@_4nZWXP%W0M#gBp+eVC+^B{w zpuRK9$A+2=sALkXlY zuu#ySv5JO8q?l3Jzu7j2_Z!$tXZ|QNU3g0ewB9=tHn$fK!{`xQY+dvu;K@K#l-6RQ zhfnWfMws}Z5dN7A1!-nu91|rzWvvilXQCV9uA5iKm5-ak)nuBusmPr>L``7H&BkrD+NF#ePo>$l3kQaDTcpq;O4 zcAu9&%wLo&$y0{2psqzl%xJ_8Py8M0A)Hp18xAeq&(I`6AjqQEeX>)O zlJ&?GwS9YLwu2TE(Mel-wG^U^^Q0|LmU6&A`#wV`a^2W2{pom#w0G*{DAn|VWbq}! zpYrShLtu#Ns4{qe1}qnT_6>ayFIBfBD^uHkD5`F;T-weKENjN}D$&shs#m_s18hMB zwlP<>ozD9g)AOw)9T4iAEo~Q;Xw*ZbdV(7=Ebk)KlF(GN4z<)le(&nybaxip^@}|d zb9d4N-xc087St^W+^2Qa>{?t%V~w|5Ww>3dZfezUV143Wx9?ef;T(VYp{)gveW$c9 za;r?-@+~((pRWGcH}tH(_>^_pTg5&(@*#VFzD2UV-HNj= zYv26J%K9S?dPkPVg&e*}_ch1FXgwO_S-*LelsG!@ZMH`mz4TAo*#6_E3eHP$#LuKY zgkcW3M`~~;Uu?`aHp<)Ae}zlh+w*iXNQ2T~uVxA>%0h2s87$1&4mgLiDsG_HHqj5* zM_g$IwrAVaQ|Y>1?eR*1^X`%2z)Nh3DuOl*0N$C zt~0N6Xey+AOF5tz(VoBISw+!^eC>Uc=k2Jas$XX3kf8{2&H;<6Mb|fIGXhd>Ol3Mx zHx^Bb`0r-S5mnn2XX@DEyI;Ss>LeH}4M`7vu?O*h95!l#)Kl4i4C-Dyw6*qCywIJx zwu-f*kQPv#vp2_|s7C)W=*-aE8jb?+YqQ=Nw1A^^8k7h?@w#)~wqX9?yVed2p5J#u zPyOP%7729Ru5Q13zx%2D(xo&Rt7cfY`*c0Dq%9{N3RnU+j0yPm9t5VUJT0p&m4O8MzpMn$9)v)z zWcKBuAD<0k!g?4mrtkF%yMgr(R-P|s{>NNC$IcyY=u6mnl9v?m#_@If+`*IuDpf~J z#@tG$ZuUm1C`bNn7t8izEAbZI2I8c46Z!4d1W(t1H7`bY(k|-#g!E<=SG8Sjoe<=p z@1aN_TVA{&s%zL)Bb_gab%NR2>F5crJfw*f_Rn^-k^#lI zjNtkLEG+Hue{TA#!m3?e;XhpBEs_e5&ym9?UNcQdoHD}{qN>4Kc$Z{J#3h09K-Gxj zGq<0fqS|wP#!X6U{8J^xuV=A(b9l6RBlq~;bY*!maPCi=b2jrwnpw6#|*Zbs9)C0qi z$*j8JEHyV~^f^tv$%K|<<&zU?Vx>F$(!Ox4ANIHB)gSF!(~EzQ$RKGG6KY=H9#BGC zS5_7-3%pA^ZkBoc<+@_ciJEEg()}OQAED9z75#sYb3K(V?tSZBO2};dC&pxleNuwx zqrX`h;Y76F@+Ai+By%R|r__IFBjp(4mlkG{XYp!o6*(@zcRf>`2(#UE8Ge$Vn&{j} z`nu(9@^)1l`#a0UtEma!fgzUo?Cqineg+2VZFmuAEa8Z4?z2}5waCdan|1M?ExcVD zGQdsXhAl|7rL$QT>m|vrLuTv}!Z9)&!;1(>WH<1L;JZp{Y=d*fMw{_4GK@W*$h zo=qyCow~F9ZQK$5CJsq8eQBSdOK)$NVpuIaZaYQ7B8Z1)JO|4~W$uVZB0FMjA;sPWg8Uc7&%maBUyzC>z{T9qm&1V$Z{_B^LQ8^aG0bCo`Q%=Oyzf-gQ^d~d8dxdncKsd~444g(YO{+V6CG zDr0KlSV9XoDev0clrB54To#AuMUbDIL4D+lt@O~F?*bA-JIhXg)ZpuVsfHYOukF2= zxi>oZr6-!WCudvBccR5-sw=~3%?hvu&B9mI4!*LqTZp`phiuiKP^JB4f$n31n=mq#~jG_v`tqWaop@V=npGxN-@eKUa&tKs2yL#rWn~M0ECC zwG}bXHJ1iMACcJ??zYLd6;UAiLjsli0SP@B$~;bVf7Jc2X&uC*newy5cIcj@1GSi0XU?GuPZe7w(GTxmniPDTUCKs?pge5-`pHIBRy7aw>~RYyR|JPMEj-H zw_1<{^S@%ZRI$JlkZ@!6#vTm|4T6c`=Cq4ysx5i-qRHLCmH|ShxPz=U!&A0r}*}tlK3ay zKgj!E-$yc%?Gvm2h5O$|H89=#4)~P_|D>+{laPPDDOleZj`SNf$Kn06<=-l4I{5E9 z%Qa%2{)57QsK8%{rFiboXv*-}Jcob7^sfQs_ri4!kNNt4(ft7)qko?X&jaJIlKz|Z zIH`V?O0kc7`Ssta`v-=op?fLEG`g`+{>^$m+uwH@<-DZ-4-%!&0xU`I3&w`(lxO~K zJw`Rzgg1ZH`;0y$};1Jv)xCak1xVu|$cMtCFypt1h&Rzd| z?zi{jZ>?FZVK#d=U0q%E)Kk?p^?;DVI((OO6LV~eb4HjUI6OBX>5nit^t@j%HQyw z#ak+Ra{0Ok;1^I{V0Ym!oNLY|=K1@BL}H(o7|Z$P*_~Ebp%T@&NO7?{$BS(Qb3kQ@ zO0ioDGvN&D%nnaM-XX%7j??0Rc^r`A!}1q*52~^Ct7|iP;cQKhUab9^ky4q9K}nZI zLF)uWqT}dcNewlt}_6>rxm{YCHp20!WK*mS=Qp)NWvp9Qu+}5Id0k85U=g~j)24$~)kms8!#O2o_q~fV_P8?~RWdd{g6vD;b=5~@O>^W4?yn1s zkA>2BHyR7+(|Gha+&3{XFkDenMpx2PtvxmYSB^wu6Y`HzhZM_WqF6dPyI9*h4Jq;F z8c#nX-X^0Y;I{`%5I|pa6nSh7IxN|^MKcdQ7~Wq$X|`Hi@6r7*=*b<>a&NY(f6P8J znumm*oO}AB%aNaFLa(H+8!zWvv!A3_^ZoDy)@x}EKpI;*b%hjS^@RPXbYTGj<5k!o~YOWaJ1hwJ%C`A12M)|KK=H6v=rn&-sTh_R9! zV~d|NdyW?+Qq?!(<`eQ>8!Gw*6eK^Fj}QqyUpc#$0EP3R7Df(k?W=F@@MPz#YzEiZ zQ<>o@mV#u;!Cq8@9CT!M-mlDF@%I|j^ZK+5d(2lgkaMOEza5>=$_qGDYRBb!C@oJP zoXe`#V+F%3ov|&H39$qY71Dek9R}`i!n(zpd0h)xGPKGM>6>a;r=%3aZiiM^+s4Fi z`K4k1ylKlD%h_GJj+&l^=dXGj_07f7yf`60N5=k99=_}lXK4@bu(8Uxk?qmbhds|} z|1_D~AFif`^~d_k!->^WI-bqW%=t>m^i{jBG>z2c;tmp*zujcsbUgVKrdr{t1?Z4u z*VQU-A0U1_| z(#0>~7w8-+W0n2;+j`F5aS~Qgn_#{tHl_#-l~(ou8wNFf8BP_O+2xf-IfZ8}f#?Fn|0sg70QY>qEqFakb-Y_iUMD;|mn z4JAGMNW=w$E?TS|S47^ZwvJ_CCFB?{WH&WEqV}BEBi+hP(m^(!>2P(;eIzXCO2q*| ze$8~zdQoJ+aJ4}3+E>YDZ6}?K6W@&95 zUrd7pyHAUn?g!;>3h8gAthKV62HQ0@)qu*^?cyq)8LiG=2cw26w`K*m^V$GU-Vth$ z7;N)Z=1CRq&<`;K*KD(2lo4DH+H^G4VlCzNr;g_L6p%P47J$na$pd)!EjRvChx58M z`epf=DMv*d!eT_~m_Hjb@x!QxS`ZIs(LY2x-=>e*YLq+QYnkrs-5;!eO z&pE>o0%Lo7u3jzkHT%K7wH7M+dC!k9PsbWh-dBlU%UjUZHNz|Sz+ZgY;16se^We0v z%=>n9%z<}2%R^DYV2Jb{uI;bcA*AQFTpu?MUD`8y_X`^?cQ}5q%_hlGtS|wm9s2Pa z;7O%9qL6g|crgRWCszZ8k*Sy8SKlLlzP=p($V78Q2M0L@RcnG3U&uWwSqzk=_|*L6 z3x#H8#5nHbLW3ak(0mNZNQL9PA`a)=1)^)#kYBEh*}y_!!)j2I(n{1A=r6cg=xFrt zh!#wscoFGMma8`uOYt~AIslvwpGy_2E5A_lT?y`fL>`h7(7DaUyChJKU_!_$kU@H0 zXaIfMV0ToZGOuj_)Cg6ha4Y88?7%YAQz*Fq0U^X{;T2*RI#AuoZs^r1@3mctB;aP( zDsY@~Je1LE$n7$)QYquAmezjcWAQ;BF6%ae&A!-UkEqaBl*i%5$cFD}Ks^`lx$r>r zg{g4IQM4yb!|tI zx7>^$f1HyIBr?IU+LRxiR?h`a)$ONK8Xr%D-V0mOu2qDqG{BmAL;P7*)?W}HoN)VB zjTmTBg73k1HMj<B<$$D*;WcH@qeU9;~U(>Oh4dNmiI0;dq-CP{|UF;F1SGGddq zQy75Ddp;&x^{z#wdOC+snLg8Cva*d1E~fFrw~>gCUYf0ltas3t8hH1P{8bDFKs|;? z&LOZ%Xi}N(0X=lX?^(_GC^JNl0gI58Srn*XXr77U^$Ei;D?45pPQmWV1scat+t~JD z@Zp}*n{&y`I+;mxCOZ%l&U%ep;zVsEk~7s~IFENR}rY-yX0 zo}??*-WE5vWRhNjjlae9KyyEWEt>3M-4~{gK*y9}u}Ukx^V)g(Zlo6uy{AzumDpr} zmad{92jPk`3}r7Y;l}R>kMpu{Hxyg*KxG1SHkfGAN+G8C=zU#)2Xuo={tF1nHy%ba*FM1KiAIZ z+LBDvNeCFfpWHrHE9kXtA076aQDQah0t_qqxD@)}beqSs+=Wh6(j%9Z>v^rp5+KHQ zws>^RfQq1S&`8et^jT?H9w~&c@}lg4c92{9wTRW1=<(4 zzGk2QVn$YIsa=|dzdh*GEFK^rKChn31IySC{^A4uULoPt5f>)C|1Ij(mbM@$Sl4N@ z%d->W6j{dIis_83 zBZ9E!0YnBC(rbN{Zl8{OOxQe+>%a)ybEc=^Dg=L6UyTHBjmT1htJiNxgPL3HC&mOK zg%fWW#vQ>_bWr^PwVyzW+)2Li`o*=Cx#Gw=hiA&)d!9gO9TUaah_Lo}fePltTn?Ml ziiaT5?oMm>AE6VU)!gsI^h&epXO>YHr&U+JLGVTE$1RgKR#P#8#b2i-&LF&FJ!744 ze58J}yr$~Sz)M)j%>9*k8D^n8}a8DRvu#!vUMH9RIRZBK%a-c2~v1eqSaK?u| z-@^;kily{zd2sKzlKA}ipu(!c#dtf{=nT-5@Xl16Fr?^7(SI1xwB7!0sFS86;c3{T zu==tJ9#lF+R0N+bJL8Ek{l^};zs8>D;vd%jRonvNZ8uTPJhb(e9<(16-Av`ok0>~H zmm}vxjx0x~kB7$+XhIz4F2I^^w>!k9qR*dAGwlQ?FH_r;u4=so`MfkRO;6~rY1|L#4(4)la8{@@1~*H$T77l zd?L0B%b&BVE}xixGO)VvyNHDMv~Zc}WmWK;J>W=XSl|upsmONEs8kv@mo#29MN}WE zcvuQ|;CMZp=2xAd!Nqag`^0bd1UZPx0!1yJHK^09Gw_Z*njO~@aSZSVtDa@#b6YCx z7?0blzGMorE!L{ow;0Y3t0e3mG*UR;%;}j;IBol~Xx0*a7)b*iTaDJ=_t=pi)D76p zMBey5fnXAc@uE4{93V~F5F7G;v5@3orSqDdUr-c7)D%8JdFTBEO!XL4Pv25Jn_Co4 znVZ6s91OBsYU+^7|3o~b)%$L*?`Hg7QR}Euz?8|xTbvC^jgseN`a)Vuw^WTXQDi<% z!GF83*c~eS-V@K-K8-z5`fgqr+=sQ%cV2Un_k&GEol>|SO@hD!ZeMHtG64oPyQNdv zRH2h_SPdF)uCB3jty%9;nx+B1LHwGvc!Jf5-9IQA)Vap~{cJtuZwsO~C#W+}%~B{MTT;v8;ZP~@ob@`GWx;W^nv@o5@W!_9d6cO}Vcr=wX~7E`ay zb*iZvo##yB`gVzk=fp5F*VO_sOWIJ^_~XR!Ld`?ZZ<7M=n4;|FMU(N=kr=KdCRlfk zH{Z`EF(mp9NHEwuQR*2YEdztFojPoY(;)@peCFo4!!w zGK|gOT3}>P4a`_w@#ZGCyWfp8zH@LbW)5RsT7^{q<2GPXZi@Kat{$%aw39;r_&7+4 zQLhGRaUR{7L7J`I2R!SYtW^Jjp*P_x_&bMV7SZ#kQvpkxJNl}AG45w3loY0D;h&eX&n&yFEVS9W4 zAPma1wX~`&m9`c*FW279TfYK!kZYEea*emDKPUIflTRDhc%F}Xu64~?a@_|C6ZV4r zBkh-Wj91{Yva%H|MU`Q#zY@eN?%E}Z>=b$`7#PV(&IKE;OB4-{$5=yC@V)6>%lMo& zg7(|PvIid{aFu1OCxt3CGdR;DBHFTsi$#W2pPcmi8jcF9P_eXWkeRJ zTWq>Cy}oe|OK^DDPvvz{5)~CaMKU-}d;>c6p}ZU{%}d52Pc!rLmU$EeN4HFQE|&K# z&CSk6l4?$CEMl3z9B2qt-JWkpuFA;SY7vumKDWW_by{kSOGAOi=~oDi)i3___&C~K za$nGZM&CsT7*)aD`Fo_|9r8hG(5eDeL(;zEHaCL#7LUe_LBNDKs+j$|cDxsN^B+m& z)i0NsTo?!(BXTLAB^*IP^c}O3;A=5z&}xBBPC37YIk~MvHcw)X z_~1R#3p%@T{2qDr-1hAxv$4$Yws6WPiHj)-Ke91hw9Fl^3p(dNZ?m}`&sG)^xtr%x z4>%&&R#B-}8sYrC1^%SY$)!LiK+wnpre}s|Cu{C-F|=j?7OFaymW)BFO7$x8#WbiR zhxq$a#2)`8QweeqpGl@{84yRtVx3fTFb6yuj&bdk35+*?YBW0HHO@tQ50|}8_h;k( zy^)C-b3iNpHaPDNY=PO-$aKM$yrTcR;N(A6g^>+p0cFEoNfQ2B z;IF{ZK=llkUue94{L5ouWDr2Qpq!oFNA}YLTGe|xU7Fh~a`zf;%+ z|Fhr!e#hAUtBRx!ypH$Zo;wLTBF@gX7p(o_$Ns+Umt17ZKoa@CsS^L+s=lbj|0(eA z?f3uxUVV5zNQ2W>QF*!|!a!G`R4{vufX+75hSG!8Xa<(cEW3`L0~^07V2dk0DbpE3 zPAefLL<0e=5|qgW#}~Ip#bfeiGV>mrd`E_XhK~9{A>yp08tL9fkkh-iv)%Oe^x(e1 z9Si!(^Xm9L_&dR0FEMh#w?lnsZCjGwuOdrg^x;{ra* z3mf(7-(KDTKKT#fSioJ}zUJJ}dq2*NMSM z(LiHt!hSseJ+1%#nJ47^J{d5G<|g?v^Z)lxJ{p0>2;&9&l>Waa$_k?(!4mFb!0^9h z6($23Bbvf{DCU1mG!Y8y4Iqz6{($V?1O>XuI~o==#@XvKm;W`<31ZNKv2F!7zxy{q zfuzUC3>rf${S~|X|C%T+Xu*MA=IGkPFui$ReXthm_jn!FBoXzu`ux~kd5{fE(9H9`3%IzL1*vPjrpO@HrsY^Fuo!3qShmD^a(DgPA~Sf#Avpm=W6xplEQ+|QBMRaZPO9HTOhX|HKE21j<@b72*i6yXs1-T83c<;r8 zA1V#~gUCZ6pDsS}B6C2@iH_M=!i#fllY5zklsk5EP)=A}k+^;wjM-SDJZ5Ff?VBw_ z)H~)1qyEvJ$ZU$sd$4aUgisHR$Nbd2=cu>lX9GCo=s;2?k8J25P%YIc`Nws(Ay5d; z4C6&4BD}cUHlXOLrVhSn4&x?yH!&?YM!q?21q~Kc*OtEs@#Sh#0CZT3if`q(Jj*mna9QxLN{%N0-f2I}k+9-`2k@8bg_KFnq3aY3r;fvT1m# zNBcsQhQ|#1aR^9=!s=PB+!~A(4Z)RM} zxSjL`w7I9hFnli5q@*NSZ7)M-mFFt) z!C(%%!i%j&b`0GPzY<>O>h%iT`kydLhKA9a@nm8#0ZFGT!E(Fxz&vtdb|z4hZa6xn zIv}Pwbu=cIaNR&pw#Z~$RFu!xVq-z2$B}_>0t}0PaF0(O2mn|l!(IQq;My2qQ{Cl7 z>+6Y0N#Pyidkx9S_O^cO>y+}iH#;lQaWTGW7XDdTdV_cRimHfqH84!Me3+r_K|@8N z9kkKG%Ae_y=Ni_7M~TNK2yTQdk|Dj2D4NsNvPDMDWR3+pr*Huk@d2r%)WU)msap|3 z9>B~Hv;O{mPPZ%ChldBNhD23IT}|kS*Ad6!HOOanaMF&3oJ};s#n7ZD`lg5cS`1Cd zyAcnuL^fGmwJK|a$vPK!)H!*x7Q`YFT)DGuu5c)bpvKyIQN=uckpG~b#djdIWrb?S zF8pFuYoUBBY;Dm|+75Kqey;hT5c)#W^X=2JXw|vE&h!R0`_oRWNLh-l;Aq8qQV>uN?I)|NLq*s2#l6k#v!0`TjP!nsFEOuRwDU>M z(akADy+Jd@r($lZcrzuDnC`qvgyA$E=ZWqr!)H2Vpm?W!8*&&vpC&BKwWJD3Ma4QF{Op)}r{EKX4jF1dwOIZS`1Q8^1r!mT zYouQ0MiA3>X=huEmRaJRQ{XcVdl5o<^Kd>g*6QVzlar(EasqW+UUuMx z8kb$g2yre(j|>bjOx3df3=Id@80Qlt(JyMN2xgFej?w$Tv_&|zT$qG7zY@vy-Y@hV zsdKu+4t<+^qh4_B3YQ)3$MYTQ2Nb$4?$<=z?m_$}9wkyT6J`#mdMpG0&)WLxIP3v~ z&4dq`HF@9~7M;-yi;j9yn5S#WbiI4)oow4Rt~^cKF#W6@SZG_cy~%!}C%7bZ54a7N zQZs8iJ7%fej)imapZ_CXD4_tM&Y2kUeUjf>*+(m_E!@ublTQ))7`a?(P+=jpOfst^ z%9Li^<{hCEexR_HUm`jd$yfN#`N z1`NYO&l2+33T~HNWzXakS0de~@&~Bd_I?%^9zs{R4QX}*k2JDm^sz6NU&bn>(dWQp+ zRG0F|*Tr`SQW8SsXNW{bnKfu^^wKag(8r$cW?Bu_I{Z&gPcsvz;VqBI|JbX#77&)! z(MvB5fAN1-0C0|h?lI&KDk3qb9fREh^|4v=Lp3t?1RQ$hebiJ`?WY@L-wS2EBP)>; zVhzGi6>%teJYS0=61(wHhTyU(K4ep7c~fg692@9tfl3%As(^X=99rR#zeHc zUv-pP7?h_vp~@s1Vur+bf$aHoYbt|~6f!Wdb?`WeiA%>ODgl$`ExPM}3C)Vp(3Pz^ z4dIWs{pMh-zDd>K+BKTK1GEJY>VWf_7Ey1DM=KhjiP=aYaBzYXkB>8}Y!?nOrBLCh zvOG@Op$A&Nd;t#uT(Y#W{fS3>@FWBo47w3M`K!D{Ic;=O-#B>)7T%0x6>&d>dhaLp zuKoOup&nA9d;*xSJ?mDzmY&RiC`30Qm4_u zpHgJCyPd=u3rIw2x8TzgHqv-^rL&l?p678jX9@!Yqfpv%Dy<)FY`r1~Wigru%&zRy zb!F?ub-pO?2qob$HI%OEsV*>>UsSfYJy>KSGntR4(_%EA{0@~bg)+3TfM>2uRc;D=G!=e2jXJk>0WUbnb=L8*E)!20L=@CzUJPP_%PvD&EG zaSSc`8`jSx3FZ^qn*+L$)5W=ImZd>0m^-UEknjO`5tKHPzPIWRns(; zV28J5x8kA0G@=chM-3g^MHR$g>_=Kr%F%C-JWzxxiR}G8I+-6vCXQN`8n}?tjphW^ z)Xf+;94``V(`R%_=j$nTRnhA-d}A}vl$AvwmroBXH|$-1emZ`hjHoU8*%OZE=oJZ_ zjMN2vMf6QMUvB8?^dfgcp^J5h$>DgJ;Lae~JUB1JjG^^`dHM53xXC`VgP?M1FxI0O zsoVp6AsO?bcoVm0S%pSwJ-%Fa=cF)9#K?Pt?+H$gyjb=JyrSvD4eQO=fi**cMWd;O zp*8DVSOV_~di{zHh7FJPms0e(rK{QqeG+XK;NA5e`E zxe){x55m6(At0ByPe~*h2DjDCt!Hv!mLzM-AS3OaF==9Yz0Jv;h6QLw1EAI`rI7c? ztkb)E@6k_J(_7(++hgdua+qaOHV8 zdDv_Ei_l@Q2Z%lKS2y<-cT96N@p*!A?P)kKj{LDM7P(YVWYfOYgaAPKr#H}(;Chz# zZ+xK;l#0x4HR$|wrLQmh3I@Ezv&-YR>9Gvdu}SP04;*6RbX7uZx~ z3O*%XjY;!UAd{E}EotsvQ8xj5~pGE!@NI+-Ig_P9rRp&~Jy2Wt23~ zT+Fks*-B1L5(lt79-WgMQPVa=Yjg&RX+BZZ^Wuuy_=jt=enI3slsfonIy%-tLFD@h z@c`>fcQ|L~X06~Pax`RC3>IK{5GFqUqLnBs7d?QD9N23#=jutbe0xYhrkhxi9gH+J zq%4rK?F(vFWxpM(A*a?{zlSTzI#-7C>5yP;$2~VF55L;cud>ntjU<`ZoUX(^U>^hY z&Lk7Ru@VwRHvJYlRA4nc-I~(&>;uQhR)tF;$vg^*^U+RuHYGiMyh>HWhcp4wVGMG^ z-fL1G7PNZ*24b6EI20dFY_;<}2+K{wF^Dvf3jBmlHQRM7)wUs2Sy%I>W!pA-5A|)v zGJfDI(cd<&hXTZ*-PpGg{>E$$3gF2xy|Lf+;U0L``0ZH5BJsg_&Z+t+r@w7Av43x@ zuCKScFuy5PHub|dF@fV?K>EykKu6f`^JYnd1Y)y}@@@LN4&X3q9xDgVd52$&M%j10 z`3Kyt#4`VRX;r%`h!IW}@lLuuB>h*~k339PZrYU|d=nrUX))-25F$j4g2R1|~} zFkwc!3$t>Ewz5&9*wE&!rLNkkM=;J^9L?pUQF)Yxk(!fc=W8!r(AQK6(*_1_!gj81 zV!d7O=-yzO2{5t#3JdoKlfR(W>y&A8+apz4591P48FeW1tMsck?2p?CDbxITzp$N; zR+6BKKfLHs_!Lj%H!>a}R^x;t93VJ0y2bZbVpXC{rO zCQhr1?9WwFwA`%2N!a$I4@`vsoYMAlr)M?F+mt@;xCxIfRRfqR_{zSQl&GS@e)qsJ zvYLgQi28M7jP2>xU{bc~(q1kI>z>r=hCO_xCR4VU);xXG!tqP^dX4ve1DiUQ!Sj6O zUf9hpZhy&_sJjamZ2j7wl4sg5uu2I?!%TA!ghES zU!z=f7<{_Bf*y15I+c&jXzd<{99fN%g|>Ebo+bPx+C{t9#Y=f4rR)&O;#_dO9BrN_ zpG9cV=o0~M3Loe)zE4Ei=RARF{EofwdX*bdbpTG_+;i}P3K#mc%ZdJlO_X)JI#=tV z3tIov@xK6MNDIfka6VMjt8G8lhbJrDWHwH%@RDn({>eMJ5I8a;PR+=RZjiDGdS1IzYgC4=PR*y8kX{4+XlRaLvN znuY7l-F|83yymeaa7Aw+F~_V?C)hjnKqNr&~KK%uylgDv*US2z>CezYxS4C$-9FY*S3Q#MH{$XBAbctW|41bdGN)3kl9{8C&Km2;t=8+UZ5ws%W4^i!k|r7qt?R&i zbxAH|2o;e95P@U2)8P*nk{F+!9>HYTLwSEA^jE|>S`IUq#&aD4rlQKeUu4r=zi4X+ zGfHyUW5ebOOrVwzcDKp8$OvwwB1vA{2@yh)>l$3MdxXa4ZD;uQuKlmn5=V&OP`9r{ zDBOE(DS9cIABs_gzGEo2Cx~aLK6A&t_-nNo+e86C|DTkYw!`Ug=%@82BR!0d*BEzJ z>5c?+vrV|S21{;!1aE5cg!v_Co>|^7YSr0;oPyyJQ71OdAp|?!#2y;*qOoU!)c}#( zDiiIUS+vyxx|$K@h(8<_wT?A4Si9tqqy*Eh_8}MNm0g!bs~dt2w^Y%772SA=zb;S9)%tGCjmE|ClfW{t@=G1z zQe3&y-NmjwU!eiPFAX1EgN09T!pD173vLsX|EaHzs9sKza-19HvA2i6Ip@-0?fscC zq1%lKv6-($D3wKBGc$wJp)et+KL?X+7!?_1GyiCe{?LEHtl+Z&mS~8Xeyu+hri7i_ z!mVXUO1`#rxbb{28bz7Yp=KYa#bJFFNI=R(WAz+}4fe=iYv0vwuT?i#E6K#K8%KYy)Aq62T0+)G@wpeM)8leJ9&9FGn=f#vIRXVP^X5rcB9Rm)b8{oHnjIOi zy*UXKyJ`zPjutY~^TCgv*=ro{B%Ljd<|7li)eBeoXDjS+QpEz>a*u3m+%dNz$2D7V z?v|lT2RhnAqG53O1l6qD@G>^^tYvyyxKR#e8#&hW*L>EN0t}Z6TyAbVMl3627V2zC z8=MZ`czI>czpye@7T~fj?w~LSp2+h?zwyXf{8;<1sK@C-$j<%pwdd1yU?RCTkI$NV z`ZD*-FsSn-@9k!c6q$G=VQ!RaWQ@gM;-?0m`k6~vgj^m|e6VU(YHdZ%P%2`ph*QYw z9aCg4i~>}twvhk&B_S-BJd`@iac@#`u0`7ohv_HNjoMUe>|sz4jIRNRr!e}G3;~TO zC_tEGXUp>#6d9?u$ZLH4q*wbKn77EwFs0k^WdiP`)1Qb~hp)DD9qAcStYD`X6Ra*q8r?PDQ&9%WvCc^aP^W?>?^;iCUS z0LT88%g*}S2jGP@Nn5ul@8~0#d-@p{eh*ilc}Y2pVolX+mGvUaUy%6nETAuhtt0E; zvwvTkg%z}^_=1=v&eL)N7FOna_3ZTeQP>!wXHs-&)-{b26L}zriE!E)#{nkh-Qp}Y zI4aXBBhbLw>lwI1 zuEWFA#ahD~B;EQLHj*G$gf`YCJWa#0fKLScsrnh}FMuMb{F?UD;o8o@A_%OG)Tl<( z>-q-ekz+HwhX6n8@3!isMfpeg_@ns2NJqU(9e5}VLePH0`BH`a}#fllAJM6 zWD&{vNeETORH6)&#Hr;49$6clXG~TfR=S@-wk#nKOLeg(%O{Aumv$q&KP*j2hDYu5 za1P`5cmN$}TGCB310u|CPOSwqz;0Mil@IP0TJ~9wO*%vu6y3c_(Z}A&uxN>M@-(u) z&Faq%XuYn97?y(4NDCrgZJwa!kXI3Dbl638y*>g;FB9+Uzx_if znel;mG@z+*vZ?MJ!t>k(KjG6kYg4Txx7$_kAejZzOl>P5BCz>40c9pAs7YMFG}faT zt^x{AKu{4aH}PuykO3EcA}5I4p9KI?#45WKPTMR`wnNuuE)Zu##oXVPXP*-L^n($y zy%cg^BqxF-uiqu+-1$hmnUXxo$i&!Sac09I-s~=hvbzdJOZ;17k*fQI!1VHr7V3oO z)I*`V#iQ%4+C|x(kC*!+(#5^KKuqE_h~U$aVdX-Kc8U(+F;%u7$7Pb$6N#}(%kZKv zKEp2wRMA!DTjzhFk#=TZW0L|&M@st^Z=YrXqgzMls&6u_*OPjWg-7zxk{yju@$r5w zyn{M;B<~rFfuZ4BqNZFtz1yF5vyKI!_0wjPAoX_Uh)4oO>Ci}oA4<+nPusPuo2R}9 zT(wg9lWKrmG@Lwx-=Mn7+1I`Z>VcxtmN!&6{aM!0#)Sr7nUPQ5in8uT-0zpRQa9$P zlPu{7zkeTVX|HB~IY|?rN10XNtI*8~3TuY8026ai*0Ie#1M^aeDM_Tal}3w>3GDrT zRD+GVxny5_eNhMrOW;+=qfd6W#y)+8QIWBrHc4&?BBybCe!O1$IpvyQA`75r|Fq0t zuR4`L=zDte8L1nixq%ZagDjW0wCs)3HGe>ovd1@jqp4UD!%lAZ=55A8T4=+8RKQ6Y z2bxvseROhsXy~0|NQOpgGt3NAWap3a=!{PuKeYd(PO{NHhOuVLw}Mo!@On@sHRa1S z>XSd{TS+enx59(v!vgI$%8U@GnIFTgTW+De9yW+Dn1H1kdcXKp4JO$L)dsGX$AyKE zMDki=^Ggt@-=`hJAH+Y&SbFw3hcR`T-IS7*4JSc=IO#-cXwkYZ6EjdUhyk0<)yk#{ zl`IS*?^e(-T8w3AxdHdO=_GR4INqUQHkk+U66J2!Rwzus%CAP*s_s`U*&HnrkKm-@ z5l!GeAfcmIsxTeC{SjC;>ZD(3kc|HggqJ!Bux!-r)q|7RwT!^xF^k62l;fi$-`;#) z>K$;k5C|FHZtE5(S=(JcrThWoZwVlG#}_k%ZW8U#r|9YqK6o=RlSQImVWSJ|N`_^J zk%V{<73dPmX(!^5Ryw)Qrun#mGhH^^0PHJfZraO@Sx@E-{0qUZs*<;119~z%8k}&w z5UnJT=%3Djsv^R#_+9RwOEg6q-=-Pnf_z6Uug}H`842Y(7$G6o=Zz~tzn_?*yzY>k zZ5)*F_g&L#J2)o}FoGrpK|Q53Q+=`_Lh26Ga>3q<=2SyrJ~5`mS(PFe$LE3N#9c)6 z>~M-X>7zmk<7vkk&ky;i%DdmUD7Vs+Lo1|>hayy~W$+By#JI#w{FIishIo<;Ee|wf zFhNcS-vh<6FY4{=`C~e{C@btY(qOZ3YB~yv?ReC&P5g9z58UqVZabA=I114!#cZ*) zz$PNanorArdI5kO*QQGz<1s_?1!`7ywz2c6>=Z{3b>tVBBwELtmqSVes(FWxwqWrC6AiAUXS@V#Yt4ZUSEnL7Db#7;MyJVg-;C;FEZ#5WI6 ziP1Fw)>^~$`s7+i1Gn_7tz5x?(0|7J{L8*u^i*IU&YN@^loa3 ziqRcwebJ;18fjei@4wCd_jr)h{-+4F%dW-LH zjHFoiF?vF?h|Zt%1mGghu>XYlRfeFv+x+RKw)9M4(J*Dg^)Uk`b8j04h&JK0Ukhno z{3`RLup!o)S9Jfpvmoh|=CD;CZKEx6CZEt#-V4_wHD)f1R=ow^f$tfrd zLDB5acy1^as7dv8b$O79zP=C0BcmU;+|v&Z@eE%-nRitQ=|EGtH#!`B8JQ3WjER|0 zIVow^xFsa{pP6jvW)TcbkJSyt4?$Wo=T;c+jon_!Ietm{^xzceDfpDcy}!|L6l?O) zU{vZ9(TLA|?{$3x>A-tO>8X`o#~0_sNDkI4VA*p3wa~Ks`Yuu{h+tjdFZopofV(1@ zXhxFdhOPkwNX#Er8v+?xO^8}P9qO!+5!Ms}uY&gdxco}g+e}qy-MgIh@9erh4Ym#B zu$~gv!nX{&3Qam}I}c*0$iy&BM5q#|tdmKqmFA%$r%tAJrTny}CO#^!gARRoI5i%-ecUWI-BPSFG3B~Wy>(sO+2!W(Awo$@(<)dOXi~OYP&>D|$MgXvo86Kh456myj)Z8Uod5F)hN<;z8JMD? z;%tp2US*iBP$19mOnsQBHzuHmWW?!LSih=D_U-zkD!IYtWGq3)-tJMy>Y57*^bj)o zVP}GCz{$Erjr#H!oT}b(9zuU0cp z#jdIzrins&wYBPs83o&Va6~6W_3Kk^{8SI*Ts@prKc{}f^{!xU=VATMfVKPM)>bkR zRxX@z@Ng2`6OiYjDPQ9TWz0NJH4T&Ez1|WXKVGHVNq(~?KDxhaDSFn`f*1RwONVL_ zER(u^*_Z^RNryo|bv$lE60cUId*r*ITPjqv(>%0Pmj0C}q_|uXGv;6KhQoNiNy1*- zxoiRDH|5idA4=jgC39$4{@!QMzxG)tZ5h(9$iapNh4JF!RV6A{J}hoHVM{Wn@a*?Y z^&q34TtgYkZe0aRH?0%BezR_ADcnc~I5*YNy_8cEfA zYekYyaV|D?w(ZgjRY~`;_I`LfjK;ZQcH~ zJM)7x)?4n6Cf0E+O5Jw&)jpR<4j$U(J2=La=vmB^V8jQh9+rn5OgNICJl!96c6LH2 z*+eH%Sy#{6$OZ#!Tf5f|SC+-XZ??C#0zh9IP3D>Y$mU*3N9#rm5XUzW1ENSm4Jc|R z`nZsaDq;EkPQ-_*$I(EbM1Q5xH9{xe;=-{XBQiWH5B$XpsMzqkZSJ zKL6#UqS~gA86Q5HEQJJ?nrK!ZMRs>2*ylcj=U|~{H&r21F_jX_rSdPfpnX6Ax`FI) z_Z7y)-XdIWNa7VJ*iE*L-cUHZZBDCWCV=oKnea7z77JCjxJTZ`927XO)(+1ynzy8r2d9P@^}znZPhhXq$@HOO&Ud_DN~9Kf5eSF-fU~NtDV~5E@cb5 z(Bi>kNxZ_! zx?vj_DkS0g1gQkM9n%f~DUr%OZEO8I5K~ZgGWlK`{rRZn-YUp6(F6@y0)>v4_*DwK z<@@91=5ps_-M)A4{-Q}oQDE?SoMf&mM4pZ|9~Xx*RNeqMGV1%QCEgBg=7kZEgG~^R zcKB*o5Ve%x7l*|;DRFXu<&OYPy0VdIj#BSPl3iuBx~t-7)JRxaKiDSRfyh`#`e<{H zLrW@AR;pwI&>^px>U#Ta5_0mbczgc|u+=xP4zIw&MOguv6y3H&L_{(8I?^P`+~22z z0HDV;9o=qc1Uf*P{0f47BHn{)s83D5mZbG$KwSM*VPYiB$IjNAx=7>m z(>)3n1;K;HOG(=59SUfpKB`xc|AiKOoZbs+ju$eG4SvkC-~LqS21J>-5UnEQb6o`~ ztzWfcngBpK4%bD`5M^j;3PVFfL#<1g#g1bRR4p%v2g`~%mhQ5Jb1+{G=e+DZzWpl>k9$W>Zg8^N4z z{;wt4j9_;{%}h*8J|8zZ z=0TB%DH(Zklfs!~nDmGBY@FgC=s%o3HK_in@r_xN-(L*pHxIO($pSR~n^g$1^rVB` zfJ(e58}jy#`6;Lqx}LYjIs|9y9~VCIu1~jA3LmYs68Zc452zbIKc7Z=G4|E8Cv6L% zf(`)O-rk1XZybAG^SFZ`<;}j{vpFt-wI~CyxVS_^=6+k?xxiv~2**0>#}6Hwl9N;` zqF=l1*K5@k`K8N7E2F5o({XN0Kxtj6`UEWL}$@h+42XuJSvY^$TQ<+|a z{;`rTuVN6iA*=S?VL=2ii-M~7Z}#02M!U#3nfvRqNptyCOf@-(P2!hN=S>1_85ixZ zg3u($goMlbpj52v8KierRvZ((*8bMqOff@)^HIs*S(#5KW3>&nXcgiG%Sh%g=OXC# z?&CBePdEQdxoq1d$+r+d1wSB(e2LeX$tY=Swi~5ntAx{W4ZXo}FJSX$KQ=a_LVd(b z%9JC*8<$14T1HE|MIBH~^`k0z9=A#NihF<|VWVcaR)4MAA3L3~NmROO{lMW94R2`4 z9wR+?i6LQNLN@!d_WwuMTgG(}ec!@>bayw>-5t^$(%mJU(%p@8mvnb`cQ;5%gM>)K z9rX8)=YH<R8}e)a#CWhK7sKYkSm ziD~1R53xpT4OjLRf)z`B%2E8Q<YaD9XgST1EW#+kul>MN-Qzrrq@YU6D^S ztxo10f7+L~4K8LPc2td&TLgsz0+7GIBEbm%sa9RHl3~4)pi~H=8$qRTEJjAzg=?QN zQK6Bbt)=Jz?fPW6LfM2QG3{3@0>UE0zE%KXWxzRr1;>#iyw!Dx!rV_J4+=xIyfmTk zuf`w(HBk#HO%gE>Tr#%+mC37LKUDyVf$RfH?XQlOt*xlscm|D@#=KOuE{z{!0QAG~ zUw}sONl2?`6dVjdNal<=gX6{eL&-PViM2s%A*A1HuZeXJ8)2KmSE}Sdu+%{I9 zZgBtnWeIdLpPt$tOOVI>9NtYCPxwB8{1;dF@Y+91OJFIlK>!CiQyGJT|DLyxFrBs^ zeYyf)!Ke5qD+CV@@!YZ@OiZuc zx$9lZ_s#E&4AvldN{lvY`2L3?m<#8#qU}9vd|D^bPc0-=6)ZE65sNTi*9zoWliHq~ z>}afUzy<CI_6iJ7+pOt%$=xAO4yE{zwDc#QWiAN=^`>Eqjx2@JACn=`~PjL!1UEAk2Od^?tkRANWh|! zrF-ld)_o4{07?Cd=1aLr5pjejF-Gq3Mf@i(UdLg0hh=mUq}yo^lQ z%YV0EWSGjXswAo zdy>7Pcd!4}A2s+vUXcY#@tb95GSTY7FrB2vNp6#OUhzx#pyvjK;S5rIxdq zltseX46T|ZY1xsumR?q&eN7=dGAj+ucEUy(Bd{&=(Z^G?^CeMiCQ_N*01aJ?W7-mZ z9-Zd-Z%Ljf14!iWgD8HW3(^t{jor4xSeCr2j_Vk+(0#mwTij}_xHWXOfGb=+u@z-e zOZ&FH9H!X-jZ({E;Dj3>f=MdU792q;)=*Q!)NwP#o62dIT~}S;CD|)GxlKL1zrPc0PQ<5mG<+d^(qhRVCqOF!!ef+K}Q-zhJaew%~XY0Q@&MIwmN-?P@ca>Yxmu6%Lmga=peVuP-VwjHCkR&xf+1KQlTjG zxRDkG;?ctL@`xQB9Sb{-|27g(TmhU+omp61ymz~-AL=~Wo&4G9ig<1}06FDbZr&nyJSbiO%G^*E&gMPpPiym&p z`p$x7nV4QaX=rRk7Y9RQ9X5j;Lkv=3f$2=p`tPnCw-j9a8HH1bHT=-^*t@!jr4`9d zb){z=wR*>qJ#OIef|fYd`qpWGL7Wis%#TY6;)~bbROTikpUqln&;D+Wjm!0|Xztx2 z{sBRFW2DNEr!;yu`2oIuz+OAgDLL=ST_xOT|L2oRMQj_)w&lR?P2P@)X`17|*&jaf}M7p;5b*u6t z9yXiBj0?{0_UL%+m2VO>{BKcF-RK|n%SJC==i}&yev;pV4}10TQjoWI6A)~^)1#zNt+avXU;q^Eo`peDPs~w8-&amVi@MV=}!#&Eb2LV!S0Om zW@1^MNLGZU28yp-QIpJC&wX&^2WJUTvR9^}HMYQK_cW-da zRDtg*h?MR2Xttno5u0lZ6I7|R#G@A76yp$-mX~YowA;IG=)MF#agt70ED2H#ajIsy z{@N9v9gf`;luJ*Ee9mpHcNGv78_rkTNN5Tm88K6++-u;F-U48J)B$q0kDP)0GrcLy zvObvnq0V$XE;Tg_P;5rJP5)#3;mmpWWKCz%ze5KhJm?1de!Elul{MH74HZa-ZX`rv|(uHq$wj3wK=-Gk=St4a)@ZOQ@ACoe$mxS#fY z4cy;12|VA53S@yyr(uT;1w- zt}mN9|A5a1*ZXJ;zV?#rI~`b#_S?bpvEqt=H(dU_FaTB;1F*LZO5?c(wST6-?Cfku z!{66{?o$9X*WGr4v1aPc6$3)HM{RkMd>9)6z=Si#FKgYihU1Z8N&iw7r zwx3qDUmh3d=s8ui6g~ENzgKu6uu7oBR=Q-mVg7|48el|Tm;O_z=Wi}gz7f2|fHhk1 z2q5o3)xa4RgH=DzOU=k2qoFC|cMkUcch#dF@QO+xA#yw-0#;>NjrI)zVL2|fh%sLZ zYZ)+AqjGR?gp}nGI-XHDJpe;njJE1=&MGf#)$6&SIHxyFwLgW;AM;fapyV-xHqVVKC zef+GO89Dv!bDX*cM7t+Z|3s<4ZIy14n55R>JgWkK9Z{qx zcSk<_({c-ENKUEkC2}U`BSjMvlb+#WC~|W0RL?IACg1#dCx2OEN-GFW zP!Q;iBN17UWBA9LqeOFmE-;CoU}pdYA9#u_+~DFd_~AC|P1Htf;WCtoc(15(QuWd0 zM(-vl<*E2pzWzSZgPFEe3rBOnDOST_OK(zvhl$Dr=|4A7h|N!iOZzp`51ObXvF|sx zJq}J-nD?Qf%NN!F@iv2B3XLFo)8^0xv(|o3c`{($6Wx!TyEizgn&?veuwe za=N*ri(^EO?T@tFTD#`46D;|4zxdhDQKgZ~wGx?HaQsRsjOhAB-FQ$1_NR+mD=}yw z_`BFRI9X3`w3fi1Jz$e%r+o5~wfZaWw-oBdWqZE?FQ4RJLBsy$T@P}vSVeuWCJmlo z>SRSySPGWG6yq(P!z5o2K0G!*gljorYM_f0HjM zEVBKnG75UO2Kw7+rv6V2Fsy3mp-QqqWz{@PayHM@KabciI#FQgnk|XCUafGYSpEfS z{PO_cS>*FJTL%H(=hJUm%d|JP!gBB9d|G0%tDgTDMp;=bHh%kS53DjNgf_8_SPjEL z=Bq;$Fx$i?mrHM(J$LQo@L-hL>IJ#U<05i>Z2}LAz2H2!A@g@>2IxSojta<%{(JB| zk+!ze2JK-zrr!-d-TVlR${P{}8zNS06~wt9kqVoZ3d)eo%|p+y^~jGE z+|Z#rX)mI&TwF_P(0N($dvCw-H?}cBwI`|ig3G1)Ndy);k;Nq@m(%?W^}6@IXgx$m zLn~PS%Bo(MA{YwsP=95->z29*W41X#uYJ_Vtdhyj=c)^l{t^a*j##y`&fKLG;y<3k zXuQ^7DXE~aoHd03kfjiCE!v{WN}Wi%@@`dz>_aA`yWi^iFl-gE+W$Vip?8$+cDBm$ z_oZor0m3E&;7r3awG*PCIkF>sqH9_j z4l>-iuH-5v5HC=8@6Z(!YQc3L3{~l=v^kNfMMOvE6)ftGeU7krCu#z(_r~Dh;EJ+V zLZNscXX5+-b@WwE*WHX6}l?0)dId5AIM3CmQ&I#PnkxeM>(u{?ldC4N*wMgoa~uxV`LRQqTX>o>$G# z(gpGOXI71jepUrVuJXPp(lXim%K6?pC^U8;98I-^CmmweMQI9dc`~x5fsks0OWhz= zey?ia;zLRo-UI#u#mzM&eOPi4;xs5Tf+NsmGgeISyTuMTHM?hOygjo(QD(B-2 zoqy3lLjc_F-Q+QZhOcX$<`Ttkr zrLg>8M^t)!wt^cO=X^Ij8<+g zTAf_ZB_r7$AiW2cP$rm)3XC?At^-UaBDh0R)F+3xd8UU>c6Gdx69YQ>XjV3#u+8Le z{VVl2Bmb|t!j$?83(H=$3cWiF`7q-b@q5FueIOlKha$ zGy}3a^+)6JyhVG#dWhcaPb}KVvs+o~==z7;1naB8$)3m;ag!Ji?)^bu))Kh&+Q8gB-db_oWKbHZOTV z&WxJ=^ra+}B^#g{$_lF_l!y3;gdHGGh9@cCz6D(E4R1Zy){CJsp@4bbT?`VvUP7s; zFKdsERZYXCho%msLK>ppe_a%%NZR}ew7>q1cr{ngt%V@q8QAHkP#;q?PMa-6U4Hy4 zVE~7wgDX!HMz)x`OB5F9Ut5_{IaR|)s*&-syJ9kyPFm7~STv;lwS4Ny!H~%(CH>l! zEn_A^qF;Hu#TP0GdXnR84~l#MVg_C+(4J4c+OT!vU8>X%b=VtX$IkHzlt-xqMcuPi zIWS80jDwE(6`MvYs=kk8|DH9$hRYk&xOBEE4?qvGfv;9d@l2ZGlxl#q3D~dr=-!S~ zZL<7x<7e-;-lqx=qpml=x!k6ZlkVYUDqw>k#K^28TR4D!zJH6ln1F6_eo6SL%3qlcfaO1On1~*Raj?90FN znAH!igrg@|Xec&*Kxw-&1kN8Jw^&3EBje{3hFR1R1R7%vmq)%9x41Cik)Wc7&1r~c z@HA3pySHIY!7`r*yGvj&%eA>T0X=@VOXBY(mp~IkvowbY@CzVoG=5S2fC7jOw*6(P zs?eE)ZiAP2Z@M;DuOj~s>HJ4}!zvbex{>wfWpvF^B_r1C2jgt~Sn)iRQ-y8Z_X@rd z0B#ZQ3McQO9V#2y(1g18a2_c{j_PHdDC_k2qLr*1qDk1dAPfyFfOs1qtdr?BmZ`@` zHDdK1|G{hu(uc?>qWOa^>!(~iGM>)&m0bOfJ*8xnGG2+dxY8;a8DKKc8z4<>BrWaj zdqC$)^zHS<<*8+k5GJg}FpmTJTW@91CAqPeyM|Sqz%c%+!+Z7ztYKWxZWK1x1A~y4 zPjHO^O>d_B2V_o9G8&@~LBz_@iQLK1Q5n%9?C~S;$ZEPvS5#8)!Cr5g>7Dq9Wc?>q ziw^=#64)-@W&dr$t|!eu{@T>h0*$eon-s`;R(VRq_mdlNZF6ZV`D?LS(LFwm^T@EN ziYlMR8?EJY%5QJI*=CupO$)xk+u7MY1NLe_4kG;F^JLRlcG|X(h_&YyjJ!N?zI{d| zH;IcWVyr%G4G1c+IsBedltqn0EWi}D;~CKK6%rN>_*V0&vRrQbM0g-*(BLxjE0d4? z=CmQ4QNT_L)gsl(BaXo9HE(7Bh8JZB>54fM?yG1+X((SBxHGufbau4qc=B-*;C`|d zlO*C&Y%y%k$L@Mlth5`n^PWZA-4%_Jcz6;Jd+y>C>Bl1fJ8kQ>8HuWL*f%_<++i}y z*vF#&e16FitpDqA^MD7&*#HoXSzq^A#M7@1{rKMrMIN|lkG+FL3zrPvbA*QYiLaO_ z+HBlh0pJz3V`4?d%LvDip|`2sfZGBl=G0+ZM~E+IOPKvlu07I`GIM6nv;`?KA5Y5Q^JP;Kt9fjxqAq= z1ARBFl`u?%B!=$g5R_}MOGOq5^5Zf786ast;)EuEIbv5tnH|7AfS*POdHM@NMO;q7c)X4oGckl0Na6< zfkFImD$4=L3mB!T$`9N6xf5&LJC?-ds7Li(8l}7Y1O7H1dWg5t)BoZ*MoGv2Ny{0e znP*_rd|p~ko=6q!+hi2|Zg&)}cgTvo%M0!+M_s`?1_7hP4`@i~4L>HPro^p)(a5^I zr`YR_#lB#IF!WuGFV=LQ$Lj`u9la9@#(vnVY%fqM{lxDS#A+1sX$YcEgL#Tx>}9@d z1sXJmDvMi;i%XwH+aFgCk^oRbkOZ)#7@m53EC8(}3*9Q6XDnXlD8^-lPn^9_spr%i zXeffv?Q~MXaX)Zt;9cldi%_wv@AVMo&D@?xh&(yP|BV(2agdkwh6Sc=N1-;!Ws(g5 z<+rA;j^%d8<^I(wVlJsgZ)aWIuun1c@k))Tm*J{leLBzcMIch(GG1`E52Nm&LruD7 zsi;9^D!wYdE|Z6>i?4W4I?Md~hPro&5MO?{SNx#)zF8c>SV97|oL%EiKRbeF%;D*F1`JX%iCva>`slir9IsZ2r*)xG{o3HVY7vaJ$~#9hD0hQ9YqqjV}Mgg3uuK6IBYN zKbnDxgZ=mTDLlE&5s|_>Rrilk0Qz&oImR(@;6VPv`z-08dhGVqv z>dL@xz@Kg#r_z%+yu9;N1;jE%cGvK%$50XN_?3dg)%&*=Vz-V|P|0XjbwQ-zSbNa4 zp9(zh9FQ7Wm5Qm(eX$L_%QPVVUnw4_-!n}MvT2ha)zm!Nc?V96WBEg>bQ@8yfL_-P zm(#)L`aPD9MaS(r z+F+|4rnmP(%H(g-RGOoI5=RH0DfY67x3oK4Juhbe0gDo-|_|8KxX|HNIg>1NDr=14oC^Ukmd+o zRHc2jZrQ?jLSSSj9bXPwfp##!nL%JJpGkl^cQgZDMl&{W-%;yv zh%yLKzb}D9{Sl^X+D^!&92|l^8#hZ^hoI_YZ-kewuatg0rY>FUKSo7ibx*w(#6_|f zKn4fg2hRmQp)eJ|VolU)Gc)lAUnaoYrWs(tk{&XW{2m^ly2U5*#lpfiN&gg)aG8*> z$m^(KKX`H-*P=4alqpt(*;a)bcRoz~dDs2MuLhekQBGSJ=B{BzCfC2- zOXgymR>vS>B&-U@sO0k^#S>W6J3j@mDYH<(u0$4!LW-e%%|Bj-gf?X3;a7XOtee8`9Fqcw!Xy zk;3?5MHAP^f&W0dF(@;7%Lp?*Hpb+BZIYOH)E(7Y$A0n8=L2YdS`yj1F+g$($davc zyn2RzX+dVvz1g3@VKSq%W;gM*K(ed}4B<&w*pPe8|F|?Cdek~@+3Q^f(emb4%%Obl zHJ=Q~#n>Kd=;-CXpN5nszC7KLadN63+e%78yPmHV%GBD>7`H4wS^l^o42mS4-|-H9 za@0(+Uyk>m4B{=W8Q}I@h3V>(qo=>~3q#dVmxikmavO4oW1eXHMcfj(2~f2%35RFD zXZIr*?f4R65sf@k7bKxC;*cyXsLWeiBHX{4*6D$PC~tK!Rh5KKzj;5mYB8VpLNzka z8yOinUu15!Yr|sARFPJtXM=6U7)p)Ta-nqSB$uPi@86b%nb~YYjE;^QSW?T8uj>Jf zp3=wGd~bN*imAN5>$oUY7q?6%492{XB6(8N^ovurG`}I7blDe0SBv%7yAFYRW|to{Lu z{J+L}&w|D3B+GW$-6{6VzX72*>Sc+f=}~)BB7FDv_lmOZsBCxd2$B@Yy4fT}n>75^ zQz@OxWEyY(soeKI9)BterHJ$5#a;plD=;8{#B4E`I(3(|*#8!%VcP7-%oGyH!pTn* zFX&ZP;5TJ4S_6iv6EpTPXw|m&Vg<#UZNDs#h={sX@T1fKt(rt=76J?m+HV%3JLf5s z5f|>pvnLwyvo)L@fnv_be^Zb!aOmKWZ!+QSf5kH2L31iTJuyo_+c>-$u4rk*>~5Iw zVNubA1Tk^S9toMYlOnS`y6h*b<=1iH!2tm)WhuSo!_-+2TCs##SA842Go7A?v;&QQ z29C8J9odc20mn|``DMz5U(S{%>4kx>i+a1hUL;`0;X)yVCGx&6vXAa=Yq2Pq5O4Vg zL4kD499d&;NEZ~(^xr}R$AUpA23n>^!*5hEt?UOfqb05t?bP0-MJUGMNXEe!`V~q< zDHNOM`>XfC^$M3lQYv@1yL+ z^kgYuo=qj<9QZck?8aOTQEW(2)1Vz>jM&T7?7}f0YJ|h5FWA@Nq#=K4F2($%2thn<}( z0m#%w{i3Nq0oCN$maI7^)l7$`-KO?w)7`Av8o#0Txb+WQfZ9x}p1S;VP^HLfNQQcB zj5e=2M0RFYSui}mV4NQ+lAQZfo~lE(39%0k$u|oO42+3!$x!z`OO!MUVo@T3V86D* zSR7*n!<&7u1Jz8J;YOtq(`kNcl+)U8R~Egr*?!q=;i$%F2Oh~FzK4^NV^hZa_6$b) zT7zBZ{;gEFh5xL0jO0X#oOs`7B$(2;rn~Zf-8f z&EYgwR2$&3g+##pt_k^#J;+ZBJOcUf5HTeeT#O#7ntuiB1optQUm6DWC36RVAe$w) z4<;DR-V~|e`Wpr_RMS(YV&++|1!V31$6$lhgl92(2*k|m!J85AsbwF8tp;7cS1aB3NxRgC@B-O;kGK>ka-!nyka}IOEHta#P()sF!H<;v8#>7( zT4(1~3Kz4Zr#JTMG(jLFGc4dh;?j3F!If@bLATFLGz5_#D3%0~UsKMa!x|XeS4S+! z!N($D6*VT<1mI;23@MZhU6K~iU|nu6w8|MieNt-OZdF`8<;YB>NY50-E9RBwD*P>P zSwG5F_33)UX}UJ$y1919X;N2r9#J^A9wp8yS1Q}W5mm0Py3}M1aLtCQmaD-ON8_+U z0uezI1(nqcQR0kvWb@L}wtK;gtu1b~uy>owQ0-r%dj`S{b?5Np9)zv8y0XMx>+B8H zRN^f;ZB>ka-n6!OG~%kMwcY=!<Wo-g8y71;6#lg0^mo#rQf@cKrsXMvbDB-H>AsqKYQW#I#HmjTC9p}hKb zcCflbBs4hI!cj!=2#lXQjt)5E#0FrraHfvxo1#+e5wO>3O%yV=RRy8uW;{Q{@}*ih zQe^jd9Nq=&3j#OBE`djOLbq+uQfe(!upl8;WMa}v9D_z>RnoaNn3anKIuy&z<1cd> zYjB{g6u3x-``H$#;S{&4_$<*07Phvw+l)HK!!a1bq0R#f;h&uvK7#XLfW9K%tr*9` z8#PSFlkg>lLQs-wd_ZVX{vzcd*vi-4mjJ%2N+= zF}oxq3J!Nu!9j(BVv$Xuh=Gmb)@gzKCst0Tl)Tp2E#9usO-oXDcC0WZ=PIN#s(0#3joA z#P|U`1Jc+zXZ8#G{QgH=N_0d59&pmxwPOnE=}%#yP^k65-mocU5OTEUdhY1lImhZ1 zy~SvyFT}Mj6NK9F*%_g(ULRZ@yJ11b*oCq~vI7?uxzSimxPjiJ2 zjW$?8NIV?tww27uKJ7+k2y;m-=#i9vhWVqnm$I!C84%a#SRtftR5oQzcmNJ?+p|caY)==LGpdk`PnQ zHfT=Wgh;a4Ub}T(x%K}>XiT?sM6FcF5jbo23vT2jfmaD6CQ*h474}0e2?f5~+wEKg z;NQgH>9HX5>F9mj4{01c9N3HIDuK~#;zgrUZ1l-XF=((HXXLTcp2UjLqFb$(tBr72 z(&Tj&;7T_WJ1rw<<{Bx7fod#DFYa`_<<6PwN-d+8-Y47loX0mXFZ+ZPFA z8uM0{U%v*0OT0wY8Vz1tQFp@gzcgUyV-aS@id>zPil~}s2?_b-hEwGAHRpJoP+QiI z({rc&nd1Eg?RvPcvVCr&o<^yRaMLqm`l#8ybMjbcB+EDICt8%0{93iLrK;YdmR?<5 zZSdMmw{bPOEf`$8z5?HeBxnM?k1Dv5H1slrX9t{|8yBrkPpopR%$W&xv{)`C=`7(x z{oDYCradyFX`bJ1qS?(+osx?b$<5*9>v%!VNY?)4?6U8pr$w4?WQ;=l5*q=4zPU4@ z&LxY2V&a~97nw)zW(T~%uNp(z>|z9I6u`Sh^V;19hHsSSaS2Glfd?*hgF;LycT5e z7b_c@v9KWCaTyf*eXv~-5fuGYSo7pg$J?o2LAaVqk$HuMNW5ROhbO`y+9iYI9knU; zu*<#q1ab26W?Q`2uvZ$PYc*NyI7>Y^HJc#PD3Gi?&y>h2bWhg`XCGefwotIKHSFN) z%_jTqwu*OZhf|i;r#ZPwe_sX#GBII$EZ0^w?KN-te3)*bVFV^PKeDj+3<1NgKik_U zTDW0|JAa3U=!r>3Ja(I_nylTPEBsDUvKipjAkvZK zD$)_U7&YM9@9%3mNN~Pb)2C7N_VG7y&CWF2E#LO+e@y=%!_`-RE*)%TK>qP#4fuz> z@48><+u_28hO<*rD5GLM7aWC|nXllmad0--&AJL`qcS{yUeX0-%C>vsxzQ6>ubSZA z(IiW<30z!W3SGkIOj4JH`HC0p5SS->&Yprq?2#ekW`%q>E2kczhUboIQB=4{A~({cds{kci@rYV!+1aM+7CE^VlFS z21Ns^t0&2f7E5&K>!lIv2;}tJE1$HlIdy$W!v`=3=(LkC*=BHSmwNNor;~^n1~~m% zZ~WxCqsrF<5xJ?H>$N4E!-!~IAl&ZzJcu298ke5cG$|}3Ca1>}yM}i4iIR?$$Za%92hY{?W&KDf z2mV%>D-1lAAfc~9eqjw4_86A%01V1yhHeOwg=0Yw)~Ap6J1*bC%J8u1BjY9fq>qO~ zUBFCs_89U#m@!nL2+Jot8{u55PgY%`JKkScxK@^@q)fWh<2)uP%4-$W20LwxWAy|G zmTq{E#*Qmpc+4- z^Jdo_e^aGz)%U(dtzsSb$~-f$62wV3m*^EWXJ)rhYx%D%fNwV^^_CmT(bCi6xrHWc z{T@hgJ(hP%L%jomH8nLyt3q=}z>%D*O#*ys&25=%*uX(D0}h@pQ*=(d8ruEk6;*c2 zej1Xm@sv7Mt~wfNAqdok5>I+=`*c)v#vV7E#Q*XJZqU7{k#PXcXGaRTI^f!`HD%3B zUsQQHx7lPm zUGKursZh@XTz0<9V;B zho7$p29uepn%{VsG)ok9=+&uV_Q_MH^>avFWa$X{ zBdwUGJNa*f(0ZmR0xnfz9HoGSCk>Fq%2K*oKWEvdh58UOG*Ow6dkk3Lm;kzihao*x zB3wGRdh9Nom>BGXW#!R^mOwGO_>hu<+gRTf5@&7rsZ7mS0E(FUxn?b%omuO& zIm2{%+zJCi+Mi!r)6PFULO74H^u0(d@~>*07-h1V277vPPjxylHP-N1Z17~{?Jsm7 zZoC!)-pY%jWLJLBp&~>hzcrV+ZMcEWF-k6At=z#8CDMy)++v9@Be&Xdxx&aVe>5Kb zNNYc=Qy*Wy)4l1>{uLH$B?X&u_Vr=fclTymFf=6vD;^Nl`JV~^kUxzfxVM?{3)Y~O zt8II>noz74D}rb-8-X-B8vrV82w*H2skW>XxX?gp2ToFd5XyPCtIJEjL_gvlHW2TI zN9d<+`X@{~%uebG1*TNLLJFB+mq!Z;@CN!C6B!F#sKx1BuPFLVj)69jzsS_Ii9M+sdFjP`69WDp}JZh#*f5qi(3IGrum;&vv}FVR#bbwp@;EASm`wk zq(W&6RV|^u{h7fE^VLBM_k{`@@a^s;Te-uyseU=9VVaW$`FUN9V{Hj*Ty!`x>iyM! zgNQcHw1oWb19oc;8djLgR^3PFysE~>FFpF4X>gV2$iKVkzgbAc;46VljqK1;lIkoW-*(5hVVgH?K#c4Wn)s!Rl8Wq^=uc zIrKjr+9Bey(9YW64KW0~YV$S{5+AJ4z+quwnS5Wp^60a}qq^x7Nx-4~i^yuMmKpqI z_snHyviz21^+l|u=!~R|g2d@u=;HA+;EhF$JyxuFDr}VN`S2odB@Bf#M2Q<|_|XX}U$nZ0OMH3D z_q{lB)W~d#?UTyB59mi9-{uIqs;NGF0`X__XFCWAHn0b9`aZL?hX-zBR@WqPX1n51Ji)bzT2crAJSK(zI!QPSVv z4uluN($dCn|0J}`l+B1^^Z%`nK{W@+IF8^gl+W9}fx(b1lpSd}Qhg2hlgU|9qFUqj zr%L81@k<2^oCr->$Z-jl;+;{7UJ}1mYUZ0o&LnEFneBpR>VMZru{NJOlccX$yC@Md zV5QV@QKM;SVsl>6)oLl}F+_})#V0wzz#UF?M2aThS|PquL0;E0WcjUnb0i&I-NGU^ z*U@utzyyu{=eVY&9uUS^g=Z9M6}tLZaX55?bbTl)(ZI3-{aDh>tSh3emIpw74Msw- zAT3y=BB6J>e5`93G)afuB@)BAb0^2I;o$r8qUO_ z?ODzNqqez}@vFL>ip~(=LB-_rtnPSDy1|OKa1^r0LbiYa6aC%@*1^G(#5T7EE%_w` zC~VN6F3{MxxUjg=9Wj9Vwf?4wFW{!}DgVltZxn=b_I`ri_vrf^H?#$wc1@adhFOBe zh892*bvhiy=);FjZ!adD6F4*tuJ?Vs>U+V#fx|E`@B=2D261M=C=oF6tH+Nz2?Wqm zQ&S^+v+$x5&9}V9AYWG@>FHhv@HGQ|n|Fr08Vl;=ObyWwH*e^s&Q1y7;;9}owf`_7Mt6C7{=0ti9!7rI;C?8d6n zh6A>-cUGL^Q^r{aL}cRtH`f&@x|c+S=FVSY5%_)taDnZ|FAXvpNTezdS`zrP+|Fj9 zY=dn8vPX;4VGNUO6*DRZ21IdaCazb__q6pYPl?Nl%%!EJ7N4wRgxc%grzLEj28JLd zcasyoc>q-bO)sKn2-SJrd+zY2FN-vTita;AKzg465Xm3m-5?C@>;iuO_3<0I3^D;h zCH7c}Qi1)^3<;)V!8|c@g=KciXCqMROyz4)W%x4J!qRL&;Beg=LO41;-q{~b2+&J* z^laF4L6MY|1z+x|sKV@Ie7ZfQd?uJkV=X>KjG&_gZ$o}x(gRi`-yF{yZXCorUWi6f zq?i?t!X^n*p3dN`{)8Ic|B1C(_pT^TkE&%`fKfZ3Dz_!IwTw*>P-2uRW?QmXdIO=_ z@xzB8juR3R!bPpNALlQs!u@DaZ7w!iC1qs#KOphV5Qh&*lmqv?(XUE6tLBjkvy)j? zC9TZ&uA0~dj^o+NAH{nHGzg8W#* zIjolG{6+EjJ)9%g3KH57!Q|!TfwV&?%6Ugeni|*5L9+x)9Z&H&(LFLGI8OwCv(<

    NS@7J`zCWl!*f|D3xhXXDvCnF;x34)4+HQdL8u_}?iXlAaa*XB&! z;zPnXIvn9}s@6Zy&BG#>Cc5|Wad$A1*U4hxD;`;{Y`(*FlzG)p6I_hHiiQU&82Z|3 zLX|7ruykG)J$O}drdn3$_m?-ZJnN36qfpBm^ZtNtUUe*D_qXBu#|8JqxCZlws;uO*uxUKty&OjVJ zM#r?9#VN4&g|n$}-3v!bmWY8xq@|`p_npiaE7-}^XZbKtO12a`tEys%N6ieb+aZW| z`ylwLQ-`{HczV|VDv`Xjm)CRS;#{8j2Hhr4M2s`@1sr&dzR57~@Ig5_IVXqdY65r% zMT0v9L3R$MvP24Wzx_@{nV-x%KP4U#53quhb#C>Y#p}lQRB>Xi0WwfHNU$phT8(Ro zJfY<|2o1Xw^lvKgvNzF@d@+rWPrtg3M08AGko-|aCs{-HNEI>-wE^j;Z_11CGpwi_ z0!p!?$v{vKD;42#*#Q5Xx%7x~jf>;3WvY>oCCdhZ#7=Mv{&CF^!!)y!xfI%Yw&z8! zd5(98y1uvSZGX{FOaQzU9Jtm=8b!E=6CQ;u_GDiHVpA~p`5L5e?e3bQ4qmipks_CR zd!JnNS5z*=)2p^mYDNlww2t2{d$i`A@j6ZrANFG@4(H>iOn{MMN+@uK9uT z0JAjE@uCnxSlRm22kKm4?O^}6oHhO7WR%c7}Fu#;Xi7czWTqVea6<|{ijZh zsp6`!qk13qg$5()QzZ#1)2J56{oP+{FaT3tgVFesjC@25otVQPY-}&&p^4BU5pgd_ zh_SZ4h=SUJ(I4a@B;Gq+>vd=&11Wk(h*6Dm zCwSUFCZya(aKZ)iHN$Dgl>^_I1=%MVfaw1IB6W_PUMM1TXA!fh%+giJ1efy&A*WHc zi=LEBZOUin3^b4IE$4O7YqY*T6D}oM<6Z8i;|ZquWAGP2_qO{sKGbf=vWy#DRSH$=g;W3*deP}s$aInk)hm_GDinNTLFcKp# zn#*mn)?}mZ@W&d$qrRjJN#JrAuqrNxCo3-lrIc90&wS!%--+j+GefBl@cW@{gFMUE z)>Yvxhe*_dzIYWXDHT0yP~S% z**zI5kfLX50D7sw+R%y@od!%DH=m;Ut>VQXEZ=qhXzFu@(XQpIh@0vZkR8Ysbv+;6mrtuA3!Y+ay7TG zu$B|eEu0Q%mL4?c>tEZ)>fn{S-CL7;-PiDoD~%a(mg!OB^rRrpZ%a&0E)x*)1USDR zvIW`uD#?gTafEY=Npp=y%=WVaq{+1~a8BAvg`rMWKmSV+W*=aR5gc042M z`I5!D=jx(eY696V9QQ%SzmJ;jfffYmyvJ`{kw9R36Z#jq6}>!|V{mjQhw^^yspG)i zp5z=wtK6$jH2br%YcA5;Ie}wbX^Iv<$4aVQ^xvw?WM9}|?di#ayXZAPAgF-GLasF1 z<(LJCsJ&D7=6AWr)&a*7IY{V`glUc)jQt9aX(@{Tw*{V2{^^%{7N;paD+`V|&(i47 z!ZRGH##^uKS)5lZvmRLmyde(d%WHXxZ?0$1Q#B%hpo`1CxVR`>0vc~wru4ltJ=Bke z#pX8Bw+8;}?#AQw&v6tK6cTKKo(ZK-f3Z!-Q)lo<{)!; zg#Qs)`aaT9&KAk01$1=q?cl_ZG#u!1mdODpyPO2D-Xdc$cHs32;rO5Eh`LRDGRUV( zFmqw9T3%vhGI=yeRd9e^`cZvW49P6}il;#`5w$ zcL09UVT_@Og+^F?v#-34#{(GpEtgDoS%%rfW7v*W6rCl zL^=+~Sndy<#TRu{a8}lMVwBx_5Qh54|HzFg9K}pgBT= z(Zx{lo1PxNxwS8~wpgVp>fi>+z@v0eMLmK+d(dx<#|zdWqYMj>9mINM8sRdu-0ts5 zVq(}UQ7~>J3zqhwzYheq*^@6z4GLHCQyPSrvh~IuauO)y8sWgbl(|8Wld_Mldc$k~ zg`bhOsk8B_VsS(T@c0GawSO9^{2Z8#psjRp>99O|7qL+LL4ey5+12&2HyPdTLSrnXgT9j+XT3<)yRWEB6n{be)LT zB(7kqP1$pW)9nO=N$%NOIjtBOiSfvtFBx4U5OTKg+ta1ZPX3gl?xL7VG6j`2O{F;G zl7t(<=^iUnjDuLTY*sIJxrui1Z_m+Qf1i_P0CyaqIZ$AQ?@!<+g*~b>-VP=-81+WU zY{_bG3l1`H4*ScvxZv~oGv)@7tbHB*ax6Gp89zZau%qCrmx|{N(uiSB z=GRYx;g-4U7#6>)*MQO{jl0r+l9gL?FkGB<#3tSI8s570dqL{m{?qD{U$vivhez9D z#YAOCYQblt^~=bchBNba0E5*Uw8Q>V(!OaKsXk}cu7muOEdYLj2&N1W)6*B$NER27 zWxj0?w3NrFY7{P=q{lpXXMpUSU~n*VTuYHK+@nu;n5v%2a% ztc^`P_TzMu=97-ICK=ARQ+IM#{&AMg0_Q^<7J%WA;q0f}0jyTxbDFm?@*1O|=WVQs zCA={wbUJBqF?wkcr$N*qNk#$0XuWpslT?-8&KQTr&3y^^Cozu=!47XaA;IubefC2Wlj-hf} zhd{?>&R<%Ty(gL9=$#4GkA@~=fMd->eBR=a=4tXu=TDmvAgVHNT@s6DD-@F>Dru6f zS@5-!=Oh_<Np*_z_hnG&vO082AbC79? zK>B8llq{#-HxD2T-YNpk*4KJ27Wn3s47FkhY=Y1_)L+sr#v}e)PQVZ4gU#sZ65_Vf z=;`jy8hC2ats|WDH&Zh*J>=B5*SL8agRbjh4)gylc)FYIf6@cUA|rzY#12<_-Z@i@ zH%@;mXvcv4*NaBB)}(`kbJf;lriX{`^sf+LUr27c!I;RxtS-oNzYm5DfH+AMXv5Yu z&d<% z%l}?$K!>zRwTGgpNV3FTukfSq#f2jTB$91n z)ZAWoLaP|&W30HI1k1I-wYc3t{T%b^e&lfp-L>GNktL8#fV2^^VW;8}Fw;l~3ry)y z$XH;!CM%F^l8ih}d72m0iGb2qR#tw^=ZJKBdt026>-q2OKtjTZmEwVgo+6yEjg5_~ z@L?Oj4!nPpnlp?R*J{2>?&-(-5lR#209XQjSEap0^)12tI#s?tOuBx72(r4N`$MY$ z)>F%;F-mI&m`_JuGQT>e3N`IWKEGk?j<(;>PqeACg@#A0&aVqS-8%?8TukalK|Da3 zT43xs#+yg>0rRFJaBl)`EffhT8}uMM4?r>g(P>Ai#7j6pHWfZk|GV5S zGB1!#vfZ+IN9(QvT{~C=WKR^p!3*Ma5$ZSdLTUaea&tvJo2`8bo{oMd3-K7QHF_Xr z08vpzx}g8~ITI7pm;2M+jsSK4$Z=>B`fiI@#pAkMB+< zY7Z8bHNhT3atYUl4Ia)_Z?gBJL$c`LcKvoe}Ep@ z3xDVJzYoI)?~97Zyt*}<>L)z{6!aQFU1QwHqMwJ4gV$Cu8RK|z2fUh!+g70v{%@Qf zxS-Xy`d3XGP-WuW@kF;u0!v!rhxl*mMsX2PngS9@3Bt;XFUB`9RyBXPR34Nz-z%NM zaqF&3S+gI`n`f#5&d2%2vz@m2`@a`4h7TKWoy8Wg!%H{+>^FRhU)yo(6B>zLN1Pu1 zM*^dLpECSq^LnlG4P>OeFe2RQlc?h~R&q<&+l3hElmI!ogz0(AF@jq3WYvEB!46CN z|3<8LH3UVUvt`ts$l6MMG!&p2;LNP1C}?x;g~iFlje9BMnaFx-LAmHz+mHI}C=`Id zGc%rZ?%vIJpM*YYq`YQky$2t*y{%1zFFoPk;>Ccq@p+agSW#WwS#P(#ki|7E&v|^E zo5#Ih%@LIr9&k8JXs-5uR+P(ni~vtopPapx>A-SV6sxyVqVl~O2D%@oIQer9-1uqAe(VNXAS z>q+|aT|-&lcZ2E!@R5X+Dt2Y&YNcBUWbLysqRh?}HCt3EOjjmSvy|gqDF>z!fzBy^ z@U>$-0K>=R2%;1@X7PzUj}XFti8u&M0({RuCI+K@Q0%~8=wF9M22Utv03vP<-{qWo zrC#3|hET`uo)~?ARp^8mfI}eIAZ@W&BIh)y#3{gf>~5tPbsG@sKd(hDj>-Zr_n-fH zQ|RF;-ws#wCdE?yhfX^%1#}=0v3bX~%}*+}1?9#{hczrC{7 z0!NIY!-ujr89==inc0htdlI8gpAroHCSC1e6>?6iCcSB6EmBnxZA zP-RguhD8LV(Hr{usPo_w^mvk>O(Jwh9fjDN9p5 zeVnFP!!t9u(D;{TWGz~#e#Qr52k;otm)FZ4iukb^8MwGC+^=LfRK-;!3tE7IGOT!j zY5|OussvMny6v>a1)vCor{*p7)ye`3{u9f(w_)Rh<1i7DrUdv%gwa50M(7>dCj$Pk z1(I)2UronipoWJHlMu^;4)65z+^dotHRMJMC_RXvkHa}db$Dz750yWjyeMhL!BJVi z{~I(pf6n(Z)^dPK>*-O7*Kr4fnr1cY{d=;QM9k;H=;#IM8<8>JB{mZ1(Ad)RVX1n9 zI-`ltufjzTn@BgQd6>|Ay_Qc-uEedie%velc6UP))3^(rKMUGNn^C@FTFBWWrQaOAv5{d5fT{Vj0{*_%)T1A z#NySKt7rh?gTe=sbG*Lt;r3F>`oK9Q#PXV9)|r?t&azy`TXIZd(Zi~IEOZ@S`pelR zEo-Ks!f2q)bN*ThD1aOyv;RIZ6;Te!Q4t=f(5Wz3zBp2IQPVXmBgH+^AT{v7mmj;e z=o!HPwV(Qk{%?>DBvf2SHu-p-`9!Cr(dbCF^xLH}F)JIeF%y%RbnU$*p3i>D;ABKe zSx5++MI*8*`NhPm`sGB()YCqLk!B_n;>0-(EV4D5hgCnmkDBgp=N~cjnT5gWl_ull zd-=S1F8Ti;m_9RyyS9@z$i8S&K>nj;XqaulHmX?R;X>9EVYjo2spL{7=@Vz^ur-94 zB3mx={AqMhYqs`={ZbA}*)3k^(6xz>LMwj=6c?j0P)P>|At$Quz`(m9FbGlq%V}Uo z1(;ZbMDR4&Hn7gHA8YXe;c<);T$lN86Q7N@nOgO8TFOKVd!hT)Nf`Si=hlRS%S~M5 zxp9IUd7CL##J3Yxu8DT2F0GtMr$8%II+E5%*N2IH-K*<=A<7%Vg8zjmnP*sJdZ4N{ zwsIMsx%a`uO^Y!;ut_)(K#`o{hcyj&d6-BE5sGo4hYRIE&r!ka*S{XFv^6SA z^*n8E>8}{dzw;O9B-Zgi_f3egyt;ZYFq+V2y?MNI!hCqoIW$!0Jo4al`)kPO^nVoS3+~y?CM4 zif3?63r4Jfk7AMuM*H`{cnseHNHBi2J&6L{%jmGf)KK~dNaXlL_3$Jqz+dT&M==e7^GHqPi59s9oGNpda@2eJI`N+}XE)Ax!&1jYX2bqp86@8;(X1j9E!lpXo7NQC9S1a$(r}lRQ%lx$^E-`qWpxO! z;4B6~8?gxqh#5m6&@TWKM1kPQwF#uE6$d{M2LYarCYA{r0U10mLhSS!cH+{85vKj_ zSiiG~w5toAf5qdQ8+NllhxyF9R{MAqM}qqv1yVdE3lE-T1V?q!&H(2`jve0<-D}$y zhJbv6Q4NIRo{}shdDLVNV#I4S|5KN<+i|5y8Oe~Hhamkg?C9r02w+Er^ntF?a>My5 zQ;C8izmzLXSepnDW0QqQqUnR;d*9v;TV#=<vZ4Sc z*6d%V{nXX#A!%pvnMGaTn#Efb&jz5#A&~uFKP3mH)$9oqW@hGYO!ah$%((GstySCG zY7t-HO7A z$Rrly;~pp1(!VvaC~qI?i6n=AQ__wDSy8}HnLu+K_Q(HfZLOeZl9fb}AoTimfuAYW zgkvTg)L(!)XEZc4YCJNhM7Jrk!XS5h`$naw%^{=FA@0hkxR6fm&(aJLo3e;`iP;kq zE3fV>^Mff3cyZbp6C4t&(Xpz}lvHB;>toads9@2I5j(f~5h54|1AvrDY6*v&+DK51 zorKA?8+(ky$7eTcxsQ07sRh_|=eanPivJ}Z`F<7v2{CM#B z?23~U=O=b9LA9}@d&)L~_j(0uJb-~_KLkFKj?XfpqFf$7cESP4v<@A?G)D8_#fiG0 zP+nTX;FTFg#DfM!+X@sjmv^~iVW20gC*~L*h)I#ZCMJn2Wx}ANqiX{>F@Xildd(p~ z(N!y1J%vax<{`u(*;#iZ*=jOi!q%-B{N(XV} z;H+vh`2Gd2?AjK7z#*=3X#tas{dKc5Nv~S8?CONsKZ(;p5%1oAxqc3FUbv>tA$O%# zbduw5|5|!Rq-hUp=G2qju9sFqHvaV`qJagAP5ktffx^7g>lP-Y@-;Bk zxosj}?v*EZO_k*&4K(LABg5&6OXcV9>%=_6v{*swYZHnEC&v9BuS1t_Z55WIIwnhE z{*=LOvzosijqau52(n_Q?0xN7!BLG*Vtl?<#`VHb@43=zzGzcXCiA>BA)St9+aL7L zf)68BEr~)qcBsXir~NiW$lnCG{z&4e{oN}~?8py)^BIYD%HG}{$Jpbf{dr7`Ek1}5 z4D*~odGL~vv3O+X<0FK6WbRsMJynKVwM-1HFzUgSJw$X8M$Ss_C$FUBN>UAa;xL~x zO8sDGg>-2B#engBM111$N$iCg_F|EQFeFu&2)EXIUO;E-|M21^O?==0?F#kKtuza| zIY~;44PdE89hWXDuwQ%^%>N}$rUorohmx<^VH*vk+Hbm35nikC+phX7oaj-EbN)HA zjK_3WQ^wF$4q9C*S#5xQ0bw*W)40|!VY7T$jIzCE&o;PPXbfyX>wI^Sy_7Wt_$-^a zIi-XZHI;shmm<1?EwLJdVxtW-DI3tnxB06O>f5kc9@QAjC5}koR3L19YUm|mx%W^? z*}-yl90-QlYWQoEf(`rz^g6UDU9G|kf{h)Oi#aKpHG~($HtkTviMv3 zMl3Zr@==1{?VOZzVRvc$CNt>y>R*B3{#7IgIoL`V61m4-FmOG|M-ICGGzkY=+wamj zOd1s;WR-VUs_DC_Uc!Xpst!>sdd>A2Y;KmJH|*PLzd3BE4LEF=X|zIC<^&GDe1^PM z*p6@CIIga6?~iXhPThm0I)1{}&*Y0{e~8sMXnCz_5(zu z$=hUWpt;{*&a#T*Tf09JuLWE&5)u;rNqsnpVFWcarafutGYf~sxXMiKH6PZ44spsw{EGfdtP33(wObJ9$t0EqpLxrNYWVkT@@FAl zm_!POvrw|2G=D|k7di5SrKcuAx+q)IE-&Pqt>n&h&rROy3Qzd50NpXnElr|c8iauW zTj#_I2yf|0hb_qn$p(7x>blIbhpY ziW9O^Zvh&VW2d*S!Qg(@p%H@39=GIe%Y;koi0N#mRm$*vJA~a~5??LSMbPg<*h)j7 zD(0aoma2*uh19X|39>J3t*w@rF#mKIY{2Tbt-yMpL7Ea&^S4<)g-NXhcqP(O2YN*aUh?66>D*6 zxM!xl*=H`{W&5%6;B5n_@6kYgB~j{EB8N2~hV<60$k7n6(;m7&`?57L$*> zNN;cu#V^)x(~!2k^c5ItbBsP2P9EHmjMtc9Asg&e73K%V9$w81vF~``;)9?i96Pwe z<9tO+0z}lsvPv&#IA%%e(;{o)aZEPoCI+(wXyu*iUo5634C;y01e39H5${UZmAtOH zDee^xKk;qCW>2TAG)BJW1&|gJPM&q=#y8P9rvns^A?$EIwom;e{Y|WoJYM=!-y*(u z<8N+6RbX9F4CJ+WCXN04TVp%*o5{(E%Lz!X_9qL`osX9zb(@@h%YpQO6B9;UTs$B! z&=(Aesw(({_*mR9nN_s+Z--nyYaj{h+4+`$N6y0&{Wo{luqTBnBt*fCB@af#kM`HS z?(6b%SGoE6`N6}>g<<+1FB5$(P@TGxcGAqkG5wul5>vw|ocoM&7{n<5RiEd?Z-Xj2 z%>xj>czU*TT%LAZ{t1C#XJw&N!-L+OpPxUku+VrQ{>?is)6~tkNQQBWq4PPjdA0$; zx7`2^FpaqcctFrZY5W3md5*ONeAQ45qeGd#0X(IDxJml>$0jl*ulGreQvd4-necx_ zh{7L?;p_!uD;)j65o9Hp0PVb<$%$h3_Or94dDaftUO5vWbG?Hi9U)Mp!}Wb1l~~pr zB%ND*+!9>)Cc|6xFz3z3^G}zGx1AVBr&Q z0*F7=-f}s;IzO8WS9be7sQ4RG%Zd$F(o#ccURof%0XZzdY_J9OeMX?tH9jx=Yf;rN zbRL-UJq_^s+%|imM%b~1u9&fxq9yd0woXn6c&%F|3!#hOlKQbwfw>`-%PK((JJO}f zhKO=;-sM+8e9q)?+rk1Ct!cAQLtuk}-sz#Qt;u*HGC?an4Wd#ELMEhbExVodHK>@` zp%rtt?Z1jy-mvtNgR!iv@^nz3n=NVHAgw|bK_RE*?^bFD?x^1YDsTY2xF8qz@z@sz z>;Nd{6B#xAuxS-mK`2S$^anlD<#)dv);ejdnB(8+yBdAZ1o|^u!fOHa%l|+e@`nL< zcQ*Rg>?jNmnt48=Z_w__yEY7ILI%D*jMJrxES5=S-R@D_jx6`~qt#u-vCVtoKpb%? zlb=Fl4Y#*kg46Lhy!53L1ql{~(Bw`kV zmGiZ{oFvTtd&TZ+{+G{6N2EefJ$B9o!{$F+`7NM$-=97w@xHeMEKzV@jeAzDSuPUh z=p1D`(=*6~c>AZiktU_)U#MFGYH|ZGYK)82EQvqZ#A;FFiFiKwm?q*fZf=MoVt?9R zPy%oL9eh~=sFa6R#-8QtlyrkxWr2B9Y8Kst`ekVA6b0RM4=(x_LYrc@a5YUI^5^GJ~uW+Az~eRHB;`!#tBF?ig6ZMDZJ!?~eN znAzVg$&}orfs%hK!qQkHhz-i#|2l@U_gw;xXY59mOQv{n)F&PFcOVZfA${+_zpGRL z=hX%o^Tw<8#YR4xXg>+xvr3m6tF8%^Bk)qp^qPcv;zZ?kV&=+GhJGuf8g{#Xo(4^b!G5tB#s9Y8B4Hd}rD= z`!yEm@B?9P8ZB@X(fSxva%bMM1}g$=wmP+>oAN09>Za4wID=-zM=0jyf`M@sz*hm6 ziS+koJzAAZ;-ADJ9IE5MYxR-mdb+wiu1qj}vygD%p{$tg4m_MPvq)22pW8Gzu0+1u zgoOSmY5x*wd!_ZpK~@=$nKI-%7hj{_`tvqDbKADuU)ylN@d6TuRzXt$4H*xCSmE>+ zsP=_vbv8ex#Wr-J8h=z=J6|n3W1b?4cI75IW~;kONi;EDlyaFVhIe4n+bG}m>`ogu%TgRcY>hK!Xz`&qPAvhQ?UgtS1KpFQp%265*opNRnDdnq>T}b^M&RUfN27fl2t>mR6RGzTvlr11zcZEMr3SnXHAfWxc(B4 z%D660xLM-^5szA`u>5Hb^VIP>N_)avxL#vP5wb@^)&1o>-Bc&F=&I4Mox0J&wK-~D z-M9$qxYB@riU0B!HS}e4ATVLxA$E?argXJT^>lvi_Tt1J$-(D=g0T<24y(~ne3zz`)UwHt~v(axLu{(u@c1oAsrmYw7fq@)(%@y(cY@_+w>v?bZ=6jM{Xf7Fe( z*NEzvt5=fG#p1o^h~M;#mC*bTa11XpvPZ5Qe*2EuV^}(k8cgBbZg6BBNrcB^vC%C5 zoqnVji0EwB?1ctC{$cxEdO>X*GOzvhy=yWtzYE&m?p@n&eV$%b|B>#q^ItDDpwh`L zKTBN@b`mUk6Y}E?cGPLzZ0T3S3U^qCoVCF5}i1evQn>11zl%#qJMteGCd;Od;%VhHJrb3snHrlWiC4_AeWbOSF z&-%V$ZT|F<@y`cDl?624w0e=D)2K2R;-|)bKS7&d0|Q~JTElI`piWKvZM3uxNCOG0 z8$w&epRK}Lrvj?=hg<2P?zS?XJDvbbtn>YMCR&TS>K{Wf#&F-JkevV)WuRRSe3URGG{F*tt~LA@ICu)8J4*?+B6fMuVl=6cQm_-kYB|9`Y&{~7r>KSi1s!(lv1&&eR zhsw{HU-N&UV%hJi-^8nisef0sdR@=trx71k{h3P0%@WxCq?#7Gv?1$H@wkdiS;8pJ!#9$iRJc+M!=eHC)rlm};&s75^sj@)b`YUmDHr zO|8fa&0CgDdcxW=8w*4RE`3=*V0}Cy64&5#ts>z!qqfKi1gLONDI`Z$M8;&n?@WwJ zEfF=v@_r5kh~?3De4rQyeKDRPKNS_caqn3xCA}blZ_UXjq;(XvH7#RUAW0(mg)Z@f z6q#YR*Q+Jv##7|-vp*$}Kg-W*Lv8GKY!S^EG5c$BawtHqFx)a@Y&&3?h<4-9co2Mx z2UcP*wxTO~rHnQ&$ZyW)K0(a1%+T7g=Urm6@tu!4Iz}=fy?e-fA*ZFN{67gAd=S?E zUtb?eN_%^6W}8pD(N{{)dxA`pZ2f!eWz3HGZA!D>U+ypbWpZrT=L3SBNP~`@^i5RF zkXP=OKJB_>RkCetZ^z+z8IxVctb?(mEmnB~9K7E~Bv+4Ue~#1!yMGpkKWmL{s|%^4 z9A6mY{8&7Vgm0udWam2H(`Jc->c&>tlT%@g^iOJb#zWw=#$GjY`I0rjMhnfr-Ky$C z#$BAqfP8XWv=~w-D14}~Y*aGeC(XOi%v|BME~Ty~Ux8B#HRGAiQ&<>U>DEn0II=kr zj-DtvqH!RcbT>>8BZpO{=*O;lqDy8*k$Mi=akbT*K=!WBmPr!pg1B^aQ8W} z#9`OA@xsont@J}^?>pki9=Yg$sxb!KYF-qlo_zh-N83CQeiqX^F}gQD)RQo#BteU} z{tR1o@s+!n3`VzX4TyvE^I)?R9BA>lem6W;^$ID%@ntUz4d4D~$G~tT#9;6Om$>=L zUQie(c1u@5rC`OEyL_Ds(YFMJnaLF~6I^4lza);Ea)4z!;Ak z%Fb9JW4ka*5QOj4)>Im$a165)Lp_ExEWuBFjIITzp1|o5%r^>`}%Lb3M-$OILDU@Ym&_!4>XBIzk@d{jjgQ)8iBXwEzw{1$>l~Mcl%l&vy0T`#KxK zu{7-#T2XD%XvN_CX9t#Hsj<5VS8a~sQU}fNizL>?vG*~m=YyV%0rTE*wLBJre}R1A#zy zxsNTxgVhvSpPS&MnG21BK77R_L-*23koor8SV%}{by;A^-_OeD^60nrIol)#>RXt8 z!AaM-(e;))ACH1dL#lDUt>Vk?V zyCpgf-FH_}{b6>>JK_|pO2gdd-ep*RSpoXuc3F;py0=I}!}TIYU)hykRYRU1;A##u zkt!Iw-#==f5?#FS!7c_ijtcy(maa#s-bknRucE=HKP8KJQ@jb41L(%FBp_@Z?*`%M z7H810htpZ9*I>oWwUO&LyW9+t1~v0=X;;0YtKH``?^2mNQZR)OoJoh;kDqiS+#)2_ zLml%vs_%~s-9t8#X~-uUP2JwZpoGu+SNS3{4&~{0G;)W7w5uVK_?JUv1_hj=n!_?< zgWU%~V{E?P0*BHw)Ae*e_zVW-(;03zJ`KJSo(giXhXUUHa`Wlzf#!s*zwWQA$e|14 zEkA#5PL3Gaq}StLbA0@%%#(zl^813=zH*7ohG5)h$LjU*N_0VB`8A|;!Dn~LJVh+$bN~R0i9Jv~9O2U$ z?J+G(U6WAp%|_iDInGrdyW9WH0%*f^uU!r-ls|xH`;=WJ7NPC@vCL(SE7`?xK2trW zzl0Ujwiof{9cbt3=D&t!h1=M%1FO#IL_DkFk=BU$D7C$cb)XPGiZ8{FX`|!l+(53R z4|kFVpA4BD^>qgPLU8z4NWv0@-*20~igpg;@ONnI=TFIwyX0Py zUTJ+V7^W*&aHeD^6bxqw64}|=!6qk+^wsR<&rW@{{A!*eQE+OV@%h(Z8i34G=wx}| zyE&h@BfMaKLblL=4KDaCs!}44IR>RF*pD;!%n2LiGIgD9(Ht(h+elv zkQ2!IIwgDArMl+m+>3XwHCm*m1JkXDwA+e(gRMzMmY+uCx2##^ftQjW=tc-p&*K5! z)^q^1jZtZ}lt>Z<_-v7hWT&S%F!f;pEeOWy2GZzA|KNw*e5EUiIHDzuq{6g+xD6VK zfJ`;5{WH^jW5GY~<8-z~^AsYq?NY8}Xy$bNEoNiGbc5Cf1Mr+4tC%02TA%KOyX`_H zCdbE(G9U!)(@3v(g9wn%XDXR5M|RgpWo->)2DPWsd^roj6!; z(UR_$Xt|Mw=Tx(9{Wy+#_Htn{Se1HHZkng?7egf2dXF7R2}U2#_<1Pn)a1rd*CXUV zI^pwBzTroglxzs`l$|pC61B@ZAX|ex#2SW6gEWZoCN)!NhaKi;ZMY3~lfuXN+C?u{!0Qgk* z9i;a04R5?DIGIsDUN>mzexaJ?^TYcr`N~DnSW^DGzOg+aWg|5(} zHHt}UBy_Lq1V0H>*I~bhM?!KcSB##BaXbOlRO~G`5E?I01<6(6!-c_}4+Q2uo8*{q5HGalW>BtjRwh$$|xAO$?F#WP@0EP}zyri9!)OfqTI9WtOC z9w1dTy(dj%(Z}sD0J1;SjC_tcPJCm>-tWZO#b7(-Q(SA@uKJ@_IPQL^JsfMVlNQO) z5CI2=N26y`kZDEJghdhvmr6@ZI{~U>l)~1Bt3@U1Ao|a#GmFoCtrL>1<535zO6W#- zV+e}+Qj!v}&<6W6!g6)CsTXpu3|r~omWLMEAbbNY#H~2hG-})VtJH=4NZ3n+-6%T2 zr!zleL3q{5`DbFd=E6nNcF5%Zw*jN<4-Opy<+@Elz{_(z7g4&j?*Q_6DC!R;L3m`P z_DT=uwfCE=es$i^0?B1%3)!SX<9V*M(#tE4vT|hZ!|=oL?5eZ7u$j~-pGGBN);EYw z_Jm?cs)~PZ)j{?IV)YLCsL&V*6vNRk6ZU{w z4y&`sf55ys=|q2(w79#l<}^7wE-eR<$treHHO!!)p$W59@Iq6BeMLg@W6yCx8VLzW z$-;bl1xW%b*4x2A1+5@PQj;kyz(ZVVYD!BP5x564bc3#qSJbU1-}RDJ826$9-8rh5 z`4#pZ1z8&wSa+(H5s)~6afJ+lH)$iG*gvi%^3O41YP8xy%6xInP+BQT1oAO#B23i8}VfOkB*uFhF8B0gT7vA z20N)jC=D!bT3d(o0%}@q>@+h5b)WQK8b@6^Pd4s@fBvq+=ckg~Hb*|^ljjA;CL8zP4ncZ*B0ED|TF@7~@= zNGiAdspoaoisz-k_y*G%yhvmqcm~d+hPuxyLKNd0WX`L6(N(MCxXpR$dF$Wy%d@Ja zsCrbAvtMt$iM6;+Sq^uaoyn$9mnkXRQvvG*puB!il-HiT?}eC19Hzya-*q*P-Ou$8JIcL6=KElA_nR@-0V-uMTG$c|Ot9sH5 z>OWMb`0oF{+zOnk=*a{gYM>-U{Q2oOz3UAKj#u#=Tn|k;zXx z?qDCRFSr+jVIc&>(Ojg}%$|sHkB~qN3rtuh^2rgk^jEhNM`e-fFZ9~6`7?dy9mzj` z3AfIQ(+U=`>aPJZU4SEa(3)14e`aGTScr%!A#i z58}7jH)Mqu$SjGNXx-`Jf7GOg)oF#C{drsq{?L3sunDwl`?KZhK;5iR0NEWyYJ5w7 zmzz!;4*at}u*%@Q{pHbEux3&# z1q7n(OlxYezvy0>->VapcXXiRPIUSr>U4vxHG%QvOG3r-zxfk1OigUCIxfM%F4`Iz zK6k7N#yN+Y5QlfV?9l>Az+^oe*C6>Iz;3fw#*N1EaVS4VykOwFrQmRx@gR^bqo=dG7^2i zV+gx?xKgsx6a3kC)D8SUiV~;ib6RenP{f)AE0TT^cO9>0i(g{=HGDC6!mrXPHSaKNNuTW8_=nSZdFjow%lL8SIsBH`j5v=Y z8*$2GKj0deL1596gOKnID!O9cUgyGT+w-N1Bhg`4pd>q29Z)T|{QY4&;@G)+WB%k# zKT`N@L^_Ka00)0*P&&|}H1AAqIEyq!XH>Hzw$7CsUbzRU4-ytZiEd|Z-#Pr!=1-JiFq=>kQmc0B+ zA^Jz8R1}^y?mHeG7hdP%`G=Zg0zS^AH zrgyzJ+UnN*cDE1=ohUl@Fk>wxmY-xmqoAU|)6;f{B_xRIo1&-s3uCE`9X;!8%>1WJVhCd|#Dh?@~rhcQQ`|6Y)rM3&E#nD@M>X(fi{ zW>HNMW@y?ULEBu0)DKY~w?>nHM2lDr2?YQCt)7x(TVAoYjv>J7{%JPzdqz|R8~fqA z!&e8*HOeS-N?t9m)mq9& zKO*+He*DD>6E0%5_VzVoFmQk1Fa8_F(FJ76olk#XT?NRoI+jdfu^%pe9;`gCMxD6d z8f}F;I`TeJbC~XJ=cbz6dSp|tn!`x3xV~t4USu*M*kJ`f*nfMox%$?{13pP+|4Ob8 z+W?*q9Y(YOnOv00v<&O{kJmA0)pw&kiOBjLZgtg;%t8_|Fv*Z69a~L@bdL0JA7=K$ zKi!1Qbq6?CapESiLbt3T_f@6c4POP*`7MrGz?b_b`HJDM@?EKvsr^-#zjx1~wlTlc z{gT_LXcl>>nWp;c_;_GgerL*jA-9SAB@^*Q z-FTRRjQ|pMs5wI`tcrT^3xVOt2+ngQw=UQb-Gi5zQ(7C0M7h5TCEQKEOEgE?Hj67~ zaQuD_)o=!*GgRK{d&|U~ow(u~W>@mM+P)EvZfa*o)zq936Y_p&=KMoDl-c%^|JOGe zPB#=r+CiCR7nx)gONZ1w(EUChX zt%_w`Yc7zn>vg+}g?o$hrw!&+=+&~v@miqU=UD%nX~t%k%FF$TH9)mC$vjSF{NRgz zH`P(}6SMADUD~i8zj3>}LFP-@8ca&5RS@LR0|xXqE?JjPgg}P-lL_+c5A}0p4(Wv6 zG-)&z+yeVRamZ~7EOS(k)3qs$ywF7Y*L$3sXx4Vu6*@XK8cOR%|APB(4C)o`mmhVS zoaG%NIdt)0bWIjA1UEJQT=W0XDe4}}zS__uN0{M`yK!4`N>t{kD*tthf)eOQQ5~ia z(-zWq8ZC5-w1{rqn`K$N%8}9$Z}?V|=;a6ptS%7@%|zqX?a2T7WE(_aQ3>Dx0 z(exJ9?{$iR8@q{v0vKs>u=o=lMC=~%aVRHbS5$=fk>DHe{ZW$WDfmiSV?bt!{Oj?s zkyD{thQI8T{(b2OHS%wqALPWbsuaF+zLg~ZW}m_kG>__`K>b0?tj(B%6)DUo(t=7)!`GfqrILnVZjknq0knu+kKRps2k)cXCd7tw2YXO_)*0$ZO z-uq+;Z0~4_(7xHHhhy)x{ry3E`&m6c5@Je_jGiQ8g$vOM!8Cxc!L*Rc5j^d}{z~H$ z9v<$@HsV4FPft%zb%=0CzT$DrYGV4}N1ItniU~nFMvr<#hsXz8;B7aPP>kUuC%7Hb zOR3`;gMj(b;$b_x`X_4Ai-mf$Sv*D~c6szx2Aj8=%Pk(AE2HQ^K|veeg%RH6S?ZsgOMLZhI&z?AMvCuEt26&nKe^V2D?=3iECp&ejSSR!K?8J$J41}+Df_6j zZ^wTw)~pB6eexT~&(;&$DCv+H;?T0WX z1Ve$VI7c%3%fKA1J=BYhxVV_~2ZpYKeCI#gmE457v@0n&va+gt%oN0gYaz(+yGA_X#cCGjCns-If7}{M#*#u*R4AgHjKbOB`XumI zTihVyCl4r0SEU2A)FE3}lzE1K%R?kYWeLJiCNM9)vgn@sEw)_Qp6A#}4~r>@afhLt zSL{i;G#0jP+=b#)d%=TgUy^hQWcfaOX1(j(n(uMCC($W~22WOA#=BhlS8HN^a6ar{ z-t~t4HNZ!0nbS{|(xH@}+hd}oejmgsQ~-)GO8T%F_4F<&30qs%eocKIf}P;ZUUn_r z^|=@c?HUVA5N~PMOU%}_Zp0ETkdwsB&Kep=5{ z{Ph}B=IyI7`tx;6SC|v26 zC{?bfGSxasrFOGDC+34a?~DyIy>7O@p)}jo>YVA*e?mGG%pP~Cb_C!GH2eQ-*Ri_l zOREhA2GxT8;~5^xUaoxrEjJx#U{JCzM~gr_wTcQ=f43M%4NC(jM&1rcmRu1;9L+TW zO9+?TASNN9;H#^)N+jw&Hq;%PB!$R3n~|5;LCyz7J#ZD zKq9^1?F;FjnI7U;@}(tXIv9@=ylk+;lB?0ek3OlXL}%ycT&iFx`GcS92a^{q5&HDdX}Q+Bg~vLwT>K3a3o;TqADY>_J6z;bVT*Vugc!efAHX zy6m@wkevQ&Q+D|@;k$!s9_mS8wPpF6)Xel)If~PAK6KEb3()kcPJUoha!HA8{g~pMtR>9kK?#~ig zYuI-^SYHCcNrMl=1=|r@Z5vN7<~F!M`$FMbLL|llU{}*g4k(6-NS#=E9WT=5bDvJZdfTD~;Xl+%1R|uRJH8G)zh*W)a z++hD=0`EvZ79&wzS?N3XDxKBP@5je4(cmE$kib1}VXqDGBJ0)K+B$cu2oF1zn)Q!tszB_3hOPL;-*-3 zU^3H6WlQjLQSj>(dbD{A)!R(6{_Fso7y6%mKnsf>C-&>^(IPJ; zrxqTG-_`omqi{en1c~eRT7Pv6!CTAT!W0%mSl2X3hX@>UEp?qCNWWT*(Qp-~?MHOq zhXm{}-GJyp)1u7a+6# zYP_k&)$Mk-4ciIJMSoJ`#wA2uM_;{-dgCRy zKPNx!>u%iVLOA(p+f_;}O7?lV4_O=bdQB~4sN_3CXM-B0QUYFdgM&J`zUFKIhP#m( zLdQxkDLGpaT}e}Z{~OCPSoM6762jjlBJM-5--6hS? zAkqTTH8j%Q-3>!YNrNCE-7O)lgoFr4cgNXt-_LWMbG`2ezn}tp?=@?$|1U?d+D;qI z+p}Bn`6{MB?cwl#WSdx3|r|NY45FobpMczmyOU1Hq#n9M*(upjF$LxFirVpw|AZj6eOPP-3(_C-F9mzJP5*=;&#^|}5lEoyENvU076+Qf;k zpUAcF*Cyw|Zr>?qksk7xw2y&+b7_(+j@1rk4izdS2;Z3Lk_CYZKgIpc&W ztHH1lRXub>>XjWMxf7-eBh>qX2>P7vN*5`@^jpiZ~B0?!DYk zw1*<`&>l`U49^bPuk{Q@AqEJlW?q_jZ!|~Eb&*Uh7rR$nP%x*D<0EcNQ^%EyznN#T z!#*3qI92|i0JH#sH~$8#||74|7ysS2QrRuD9TsnyV}E|{Lkn-CW&b(UVB zldC#+fP7*!uqh~FW_+|kYI%`_+3TYsp*}-BY=XpdQR~PF$^&)9HCI+@D4RF=k{ z{a)vWXROl$8SBZXp{+O|tu(0^J60!QO4#4O!|Y{Cs_F#hisnCVZ#0I#M#%%S-J`d# zELD9XMS6G$c5+f!)h6Hc%AgoC^IWQm79BGwOZS3aqy*ANM!Zx`Yx=0*7QNJ~BnvJc zbYpWaJf-BajkV72&{Z8Wh&+V7y=jYvkDDy1TuYwj3$OiNkSyP#p5Qz7nm7qvF&gRn zs2?YC#`3h&XuEspsJJnS60IEWt2}$Qe!I&prDCI}8g7Ws}>D+3w`Y{ z7>r^ft8@(6RRvK`sYK;)MTe(A#m}M7FT2S2coT1&Z^D%w6vkxoHj#`vxn14j=hQL1 z7g+EH5yzT`uZ}ka3ZrG!djh9Tv(;;oDfT$i1kPLGf%!%7>V*D8UV=7yXWp?Vy>-}% zI4y1%iThGC{7qTT+fr*bn`}Ya+V&A4?P4KwAU8auXYA8*tlG5(Nm!IwV3wv5C!HEAH*IG zVZG$%S}Zao(A83PLc>hH5y`o$`_apYbqb~WmQ12-{K@Ud$H;s!()ukmMJs{J zF?x&V_WU6fOK-=2Y6z;<>mPEV)&S`(_dnKaxwB$47p-TQRtYHwuOQ}_O@DPF;!dcP z@ivOy8UId_r$r;6B%C97&3t-tg#XL6591X^eN|I<=Ix6Q_yCrfJbPeUCFA}-O% z-wzrWqb22GhEOYLJKMVuwiA;^bEPX5qXy9O$aafH^3JBK)@*BP;}Px+l)EA0?f-=6 zh1ST6si~AqBn_93rYYxsrKK!*;n>q3Y^bo2k@#${>BWFh9Wn+TZmhXFlk#FXn36CK5<|p?({zm9m_)mlk@}K4^1q71b1b zJ1zThuc&sWtn2xMz5FAV`41Iyf{5}0^JpS8!p0rlWb%YhN|8TLXlUeZ^MTyyvMO`^ zmu&L87L}t)f0r?)d{pzZB0q0PD->%Hyu8Y2l9uu0r(R{z0{N@E*J=BN*AnHsFgT*;tMYG8Jol9(cW2PsAoA(JgpPa8BV<&Q`R^3T6dzzx$*#iy!ftrI22V| zKxe0*kc6!%octNy7)r?&vjUUD-2&q@^A{ihTm&dc0J=+K=d~vI2@z73XK&-V-X6m_7WV}TiG7OJgZ6^-T7VEx1ih0|1R~b*rcr{B4g4r7NpS~9-i_XQAz5sfr zXj6H0!yIy?Za!I#vTiQ$i8;Nnll7t*PviFzFW$~+{R=3SOnXC7Ja@}WGnH*Y4lWs9 z!{sUi?H@3cQA!%mToJ23fb0ClQPYGseIg#!W1^GX_1IFn_zHf}mqv>q#&h;C4mhJ@ zMh@0Z>y43uZLvUrGP=`ONbK?2Y(%yExMwU1xB_7c;N>Aq;c}>EghdrmlO)E2^+kz= zL|vst)?{SapXl%RXF_rlHdsP_@1Us|z;r&(U$o_5FevYwx*jatIZkvLqIqS3!zLzX z*h)oFt6VZ?vhc2k#zTM@FccmpIo2`LB1*20xXNmN@}N?mEX(2C!}BJw)qG#H^4Qx; z6FqGmTOK2gO`NUx#v-KrFWt2sIgVx`6sY@!7#dP6FE0bvj;chyU_W$2bvpi+lYImd z+4agYmL#5$xof{A>8rVEfk+a^4e-YKg-PJv$7PesLS|oug~jmrW^elDY+FrTefz2Q z5OW(ijRN}x8DbXA48M-i;Pog@PDMxf61w+#ePkoi@xJm%as9%en{tWUg!9}{BMmM> z^2{9Bz8-#XcefK+DbA8Mo8;Oe-uoIOJwN(}S=rtR<;J99B}=A}*ieKUw+r;w@oBHQ z-=^WB{!D(G2&JH)X!gDNf`^el(jeO-D=CF+dA%UBi1&%zh?alMjWb+uH}R-CckK=4 zRPvWsDFUzEm`A|Vspm@+wWVlu?<$>S-W^Kq3#o59iR0f+lTSaZJCPBD!+W|$2vrcH(pv+g^u(S1sSbS$?UM#lzSTpv2NLN;VPU& zR=CI>Y4FCHf6YbWBo0EpdggP=rmh!Op@<{Tgh8J`ol@*2F(CkS0lH=y&+R!JK zb`k43IjC*HulFX1#vG?wulm9rn7Qd^cBy}S!L_*2)#K7A!+$#d=>R*n$&He6Yh3i7 zVIv}D7*#WHBhi0Y##$FeMcLia4Tfa%TI0|sf0m-3(Q7h%p9LSl2;sQ!*~IG)otMHC z^dS47(_nLNcgG{KYwL%h>lv1S8FVQ`;_r#pat-_ld~|{{)X14Xe&0MxRsao7Z;_19 zGVn-v>{7fOK9?C$2^F>WvEnI}Vk4&}Z^|HuNw|c3sBc=d{ zGd4CKnxy8}GR{|(hOVWCc$vOkd^XYk{jz$BtYiv#Or0s45Neh~#UEi^FCV0*A-K|6 z#@q;#nWEq?RWSMj;CLZ11kTq#`1E_IPZT)iAtSi3MlQwpCY&4-QIx`ToTjPAT(r3? z&I;=jrGr2s!(6SY%i0@1e2R68m{f>yHE3J|U7nAuW(f<8-!mi!PfD%t#WNaR3=I7m z#3l4tOF3Xx;dOLz^wjl~^fHL*J_{Od$pTxKL`?(jfLb~>w}~HBlVo{#_}sGCe?A-$ zy;g=muOZ_Qn^Q2hq!IxRIVRz48Y`aEBrA?L#bj^clx(EFI>R}XnJ<~^DFxRuOVXms)Q|2rtz3+if#S#qEPA4=s;}wO^ zWK)Orfr2ra$$FI_$KHKa_UV?(2uR6+N>C3c!@;+lBY8x0doB*%_L`xrZ6}cle2QuPG94XT=;eE2hpPefXo z*CLR<^Qe=vT{RH@q?HI2L^Y+IKso-`E)4`j6PacHo2+5Si-Fsj-dc?6%2*^ zfn(tc!+-jLTT?9+_~fmJM5<$o2qelD90-PPYl6#)GaGUuaobi1(ife>{>Xp zmfZ>b4)ml3LvSs_;l`wi+}9C6Qg>sGt?00pf?gm&aS$Yh?h%slU4s%6%>-7d5Z*NN z0V>BCK6yIu7UOq$_h}==bsIs{<>$FePM*EEfN+f7=s*oNni)iFhS_^UHoefE5Trj9 zj^&_i%K8wF`4q3{gxjMbGU-E?=iJYD>~U?ZUZRl2p)@Y`_v&-kIjMzZ>14k{Oo<9> zG2?5&Dan|u)Uzp=tN0rKjZi5Hl8qu+cu~OJA)omG@UuhaS)G0lo}IwVC~N%m_>c*( z0{t=d@^A+&P=ArmreESRt6cauwDh;*JQ0c-q7H3AXsG;Nl>D!@7w*41*5$MPX}qc@ zyT5#>P$4tmR?2f)m0=p5kxokz_nAuBztxInr-F{WmY7_FU$!Ifa8Ic2CbP1 z=hQ4Xlr8vw8Sfyt42!f`f&nK@gaZXX&9koAJb|FP69CUtdP+%VOh#Y;-i%Ou(2L@W z@5dM&S{NTb<;4VY>SVzY&7OY{O{zwon!MM49Qf+P#ijt{>mUkhTN1TWF~0e}nQNTa z9;e#l=oc0VKY({GI4C-vS;z>WijE@`l@ubDDk*&D<{o(b7|>648bv=Q6h@x~K$8cX zo0>qpt-5XuyoGcT#X*rrB@qfF;`9IoF2-hN;2T|BIX|TvE*Gxl*O&;Kdw&|4q@8vn za*t1sWonFYD*9Hi1o}X~6i{g?W5V|a%so-IM_oThH!MuhU-x~`Mx0A)#4uozN(T{t zifVZMK8UEihA`P#$8?!|`{ZQxum3Wdll!=&e@!&xE&&BLgSEp`p*TWm$adKEYOq0@!^3h>l{QO_Ta(!D?@3Iq!4G^Y#gy9`_Y#6$AK8|qXuX5v34;gi`l zl3AC(oFaqCT*5E35x7rYz7H2N3Op+Nf1_3AOL%9@0o+q2>ilxa@i)-UHSG|6#s-lf zj4D}~9U>G@ALjjVFhFQzr^%xNoNxck*BHd;0L~_JAvb9_1S=Cz-}{-O*6;3;LRrsa zWx#}{dap($gP0iC+Q5>Dggla}Z(~DKKe?A-Q6$DIE*+Q9QMSZ71zIQ~ZjOX%pV8n~ z$R}GkBFYz`JVQSP>F_$+q81j`bC8;hmx(+%3G_%Ejfv&;`phSId15Mvg7N)}-E|6z zTWD-FXThU4aSm{3W+Y>fRRI@BF57vb!h1k37t5J82Fw8%yq0-Doc9!-wR~Y5 z2_I1BeV(+{`DiixQP~_*QT1at(6#2&+Dz71XM>{>SioN=sB#4?;GhakfS(hXqbDdl ztTSU|pR52+6}Pe@t5laXss1M9UU4P=P@;yTf|m;k)hmp`hh@yeymnC~sN{LE>n?yg z9Cc)$sTc6ir7@xd68uJang9YM%1lTb{#zeo14&;tgH}!8<5Y6hd-j}qOekBicg$tr z^fd{F1RS?+Y)s?*1B2M2j&B)!)3hGB57ywIXqs2kN%6*geF+@A$bBpK&FHb{&A`82R3@21`VTJ3G0=H z2Dd~*g1sB2sbzr(Hfu9MzO9JFQF*uA9A1A#-nrsf>YA&A7%vF>_SP)SWi$)AXpyZ{ z0?=CTCgI$i5Ozfz)iyBzD>ot|^L(g7p*PG4!~Ia8#oq7+$f?m=qFvZ4K#s(CSAgON zRMaT)_&Q!t3OWV8f4d_emt+lyVAFjINBU!_CUJ0ZoQB9fcA!y?d@m7nbaWIdRTHhZ zYf`#4l65|`ZZ%9)V2m^%p%FlR3!|b7*{FF=9fRKx-O&GEc6DbF4i#PkK?+JiNd2sA z-nvxmBJq@8?Ate`px7n3DfB8da$f?m^iondcK1JVajRUj;E7E^aN>(rJy)W8XElmZ z-yB4D17SqeNCa59?dIksT-$b4GucVjz=mDc<+i7dPA=R~S8@%Os5X}utVVp>UL$!r?9 zp0130@=}{OfAq3&LBqs^w{1?nDcZ6G9-YK07LllT!zPX~P_%K&@0RDtg4L|Wtpf4L zNZHOQhy3cK{&-p!hXs`{X#}=Giov}zqoIcF(vebtlmWO~2%Nh`OsoXt0VqN9iPncD zhgnRB6R-NK0(G-oCK^ORUlmJ6z|Jmv32Por#`$h5wm;l_pdBeuDf_sRNb!fCi%7Jt#p+3y2nn7C!Coa&H;OYM<{;9w9& zl>66e@(r=2aumGj9*y0(vs>#2l@!`cF8xP1k8rT=TTaQH!^5P_1h8w=vT1-nfk<2-9I(KL3 zES%fDZT+#YCskt5rXBNjm8rMIg%1aIdDs>*^_lW@!eK0(JsYekrO<1%{9_$ga_7Sf z0s?|lIx%iI&jeqaxzX^S-gxwJpum^TdnS8x5W^8*LM3yB-TnTya{-d|$%frUo6yv- z{A)d~+l$?Zamj22citJoNYA`ji3n-{C5n(7C+7l@!{t6I(83PQ)8z9+J^{X=ri-Pw z*Lu{7oy>oKNbY($0zlZj-`m)n9A>8d`ytARw}>TIACA}5b69>BZ6mu+oa;-Md@^dQ zAl@fHW#{z0b{ZG5`tJ=4stN2KA8yc#PRFaiA?v)B!e{+k{ngLJnt13`{w2GiG_bUl zQ$`wG7AAVS`YGYkBw4&0ZQ6?7ig%qy-52XM=Pv=so4KB{(iLSvUHA*5mp~D5|xFIIz0GN ztjXK87griXm6fV$q<(0np^Tlv>QByRRBZ(Yy#Cq#;VjxWDnO0lesjv#POKR^o;|5# zc5!~*pY7<6m@X&m4^a0ro#pp0ItMxhHT~qHcOQXnmu{Znls|XzMihr?s#u&rTT|s+ zO3Nv)ew2iQ0*d8i9^0H45O!MW_!VbpqUtgs;QsfE(yhKjlJLo{8#16HW(HjaT8zSN z?>Y)SciAm%nSfo&yx&Kv(Io_P9)kyX>0o6Yxx;~nMAM{lvlP}!{S z%05o#0fNp}cN}U(qVGF>b)#EbJdO;VchT(loBCa3OR~PKF1+v2(Axfh(&%{~l=Q>z zmIhtG?^y^#DRk5+9FDC0gVxv}=ZN|c0HF=ELcghQ#Cf^{5yBanE7HUNZf{2bX`*o1 z$+XZnaBjUfTm^Fi)RE8a`F7hsul-$5?&RV!^WN&l@VQLFNRF@@>gDC-v@f92#5VQ_C863bZ=n zF5O03Q_~o5NC^fQHZHM{@X!b70eP?Nk4I02_6-MsuqYE^iwBEu`+*0$?3q*Yycm^< zB+4~42mp*$?;e{e0ojby=eWI0<-KOQjXd~st;S;fCb!utZ~hVV0^K=$DCZ~9wt2aA z2PY4LezJdeIWw-Z=ln~TK|$ytQ_PnW>-QjM)C%K^@4T#80@!YRS_7#!ZCLG%P zA72BE{{8Bmw&ZvkQ4l>GUQ>WxzW{&lfXk<%GOpp0HK6o)=NIl?P!S<*){m8yI&z6N#Qw9L-x&;`pFph=&4qM!1M$vlsMMXREj9JEneg44}N^4$IQxN~N zV`5-2{-{U!;3RVzWdjM9qbX%3L@OUl-vpoQ2D+ir{J8&rn9;zz4{cIIU<}lKwl^}K zo~&<VM1Ys!fni$>zr%6*FpCTDwdOJ4S#{%*E6?|KB9QPXZ4up%MR0D2PAfw3mPHzm* zf(GRqEWV{v)6s?9C!z1?h5VV~jgnAT$1(I=M;d>~Mrl$mw9el^2d>K%7g5=PGAgWx z8fbh`aUmutmvyj1w2-8alU2A;b_gWaMt3z>mpV%cD23S&0oUUGezkRa$() z-tbZgesUe6=_VB;j}f>UL~eKuze)}c4v375BqU;Uwh_LLQL0OJcA#Tfe3iBssd7cQ z2$F^4IYPLmqB1za_EJ(H3jb@9+>2}ae;&^Y3c}I{u}CsGFn~GB@6elnl9gmkq!Smw znG4v@C^2;?reD}m{*z82sN+j?HzPl_Ph3qUI~c;5Bv=no@LIsaWH+N~FNYnHqVK0? zZ6p9uuz(QN(oz;%Siq(DFQ#EcNvkcO%{v_iC)0%_3IRGGIO5k$VojbPbTy3JzycyB zCdLXv^1eEZ14UD_26vfuw>SaXsl(1Y2XjkH?x=ZT!PnK=fSd-osh2iGq2Y}_ z&S54TC?!S!pw?3pOH`Le;BD1bKPSq(wo!&pBVjq6TA>WkIP=5kCR35qZjG33yz6*F7YQ}W~c3Y>zWube(NHutU^1JQ)-G!GE&{%Z+;E{&ARhs9G z3YaFF51N+FiIW`!u7K$V%MMIKV_f_xY3hJ};JGDy`_D%~Otp|<48ym}EdlfT3a6p| zg9i3?4UtY2ssoxz#`TRVni);;uo|MR@07uh>ah*DVY3QR8?DdgP6z0LAed}1zt#q> z>v4fqOUjM4-1$%`A2(FS;(p-Lisjh2484S^2#B27xs5m#e_5oyFgr?PP z9t#E@3J6G{(pd8U{-rE&| z$EqoV#FH$rqv_{!vfjAP-(Po(X5+69iIS!8S9_$O)w0%!mkv-Mrt61erCd0 z88)_Db*E&gbLM(3d^0shUGZSK`0k%Sbq^!|g)88zyb_y}N-nIix@H{z0u+>-j5C%e zs-Lw77^=9S->Et>1yqy&dvy8d1d}fYvXBX5EWJjME!pH4Uu=I=(M%lFWvXC|tjkRa zSww88iPX$+$XXQaUTpvSa1ZRV2*q*q>gyLI_%LlBl~Xov9&P{?>wQ`Xcs!%DTkRz8 zxFmA=Uh?&uihHN3CU)!&;xJeY7fs4#Q9MVZ%<8~bgFC`=TTm279Xn)~W} zYWK$&v8pO16H4T0p`9;DMqo}b^E**qjF>GINFRo_#Unf7(QZh-g~lFA8l4wYYtuN;G3(V=P(Kte{ec)>)st;fJ#%Y^Wn5t3A2vF!NTe8}t@d-d1C? z382bBNTI5{y0|I;u!Zt+`i&CZLBkdljvXNHfTw{IYh@`!4`Uh}f~6xuP`R3e=@q8; zlug^Ca5}GI&g8}=wj3#;w#Dvi;8cEivBA8?dt)72r-kyVonOFoP=_d}CiK~z{GMWE zkJ;a30I*nuT2N;N0_Xeep$KxSuPD4E5PKJhZx1JJqNmGJmQ2F_%JPc-SYiNUrk< zwSTN3sBIgEtQc*m&zEb|l73nV>hnG&bvRx>jLsxVu(Z6iMFB>#kLpSCt*FqcP`xeB zofr_!1Gn}qWzc?R3(?_vTx4_*=YSe(sa5d!W+Nl^fU6jX>w|+9RnL=J?3LjgwyZ*+ zU?%Ud#?w^G`(ffH7JCN)c{mHIk_0`)1NNAVNjSC^Y2FFQ`&&>tgwuI?pmyB!G4-S$?B(MN^N5Zei?TJ1cjY&ysVDNWCxJ<@y z*tt#gMbIg((6hJUp#Nq81TJ7|rWnNos<4Vz2ZK02afB(XH(5O#KQQ<9nSJn{RCF%c zi@Fw({4^Cah5npK7kwa^R+xNQ2ftiVssyCa(>Z(SjkiEg!_AF9Kx`W**y4~7^qHJN zJ?Q#VQU0PCJua@B>OV4}_K*}+DlQSSs87TVQVQ=H9_7j*6dgMxM485^0z#7EVI_H& z5@;H|`H}ffrm43!t7{_v90XLT^T(u{J;;`WQ`B;a;+<{{#MAMf6|?mS6uaf`c@>;_ zRBlj7?OYAp+On&I`>*TMzmuAdAmN?&7CQQI@5+6VtN8suA$;I^UCG(Txt=bhlCK zH|Qrgr|fP~vd^;*-_T<*%D+RMWJGeO#r7&kIy^bj_`8f&PcKwvqZi*(*;6yD^0veE z*Td%0@bGBwuj+I9#dpjK1Fi{F*D`V%EP;wSj#X4fX7G2q1Zj|s0_hEmI04G9LKcpZ zf|?_&Q*!22aOd$2kEa7vJKQI;0^=I1+JW^(VebSXSLw+tZU7=to~5@2QFNnd!d5?t zVChLo)HjfnaiO@_SlTBj9RHeV^4=QfhCI5lCbtJmqQ2GJNCs}-l_QRmhHiyb@K5)8 z^}OOZPelv4qzqX6`Iv92wI-66{URpR0nj&#Bc40ymppWDru)!<=9+0!zxCn~)u5;$ zA<~jd?QGiks;c(%OrBR}Hv_7q;;U>(lwSeo0dTzAA)E0F30iR*D1PWZbFIO&qdR0w z_>=StZ%x}(EyqS3h`9{K8ZO5S%IY;B5@ta~@QLQ?E1;<5+*LxSgN-d(KeS$arM`FJ zP%LRopMOInz5i+2Rh&;=8QP)wUri6|XOqmo21oQC|CRLef(F4!@+3%Enn?)(pRvSg z96Wbw(9G56&d;Q_kJ4Mve#PR`X5p0EhG6Hml73K-lD7Y{S*oa;8Q+NgVYsrm111UP z^5{V$D!;6A;R#3zgQ5ILK7alu>Ei^-3o0om}lZc3@!RG!w z{1yLfUjDZu0yrNH;g9tu*-L_&pCQw`dYTy&(&th-8*_$Ix5Q~K)x~!97~Il8yL^+a zfPVESu1~EFC;|b~9;nVk=ueH2QRqFH>v_{`)2$1|K(#Lk5WGE9XEwyUa4setZy!R2 zUyRcHvH&*n0nPvVh+5CuzQ&LD%mG>{I1QN zEmg64&;c670X&PqM;eJvYxnA2O#4oP0W><{M1` zap0R1&JSb>f3W3D%-r!mA|TsbD&i|kI7 zD!)Umc0VX@oAo}Jr=`mt5ypko25I8xc&*{ZXP!%J2kQ97LWSQZhaZlcs z+;VeU^{zQHyK>NGN(Ub*!G_2u7hWwsa@(a4H=^!*1E3WY059$=s;<#ldNfpVsvg`8 z*LumFEyfc{S|Dgw!iyKk-Yl;PsiC%+6N(qu; zm~a@Z1&Q^9*FTE|A1n3+rD)-x!&OL5iIdOk24-V7xj6_Tk{1%yHEEjcUV|q25Y($9 zDBp&0696w2JuRTCZ-~Mtc7&I&1iz(?I`z3rf^Uo#H8nD-YDqnL~G?P^rMj(F(_!A3KRc7s+z|YNv+!gh}<7XT_3J!m;Ks z2^k8IT;eO)|5ZZaGS}KvZ={QI%y#K-qErJeuJfr_F1HViG-etjlvbP%`)rb9avKXy z)Sa(yi2bkEvHVMUtZuuQ{xG`U2{LyJBUwNEuZOfRA@#E;&0C3GK#YV%e}fr_j|=Et-+VnSumA6Ji%(C@0EBzkoE{u*7!|BIcBg$ zUCNJtHYg-t^QNLhts1_T*h}c>#JhjGe5S7?%aD7S_{hg=)hDt5|xIbV*0)Y8@}P>C`{`Di}UQL z{#ni9l0j3LX|d*3)8k;G0`JfD!@o17z^?*^@~-(KUf)o6=lRN~Y8O-_8}b9vpuTY> zt-0dr4+3>|l%`#lB;JJ6{$LO zR<%;cfaE>IRjUd+#To|>5N)^$A%^O1Fb7bMU7Q33$kTp_Y2cN&RZJx5A_#hXJbqQ| zb3t>Pj9{V0|G%nx6~Xh*lHTT2zEbWAv~8$?I8|DSgU;+7`8j1$raK~nBS+_h@xmVo zTm8SXZ~xb-)?;U83(m#czoR_#@!?OO%n2SFQ=Xlk^SuqY?pn(KHZ%nbR!08aa&cB# z)}Rqt>;s6YTXBNMdR+uF)6o{v&bZX8`$61Ml#(9bKG$A?db-P>V?$A3?qU3G>4q(g zaTB*qOSMJksLK5RK%K5@k!^}PvQC-U&$}4=&8Rq1!!&e;dp8L0JLO-W)P#I!)7|?>D4~KM zmyAH+kDM5!XBeudoumX9-2wyaqKG%9X%b0=L9d_V*D{I+3Jjt;zCg&_b!KSc$!ELQ zH77aqBp`_6ms-a8*Mp|;e4W1h{Eo`=E-%>j#AeyR56fi6D#^G0#k`e&D(@6x6o0=H z@}v9GDwa9{2&N3Q5FBy3?tIT~_~qhA`X9j%qU|YI@-PSsE*LtA;a?+era?UnNa%@r zg?}yTk9700U|8OV;d^Cayg4++t6<#?$tJJwL{P_*wJ5jk?oP4IgFEVSTl}g|RooyQ zSyi$G@LKi!cxScecHj)xU@qb#m`s>c!$#Y%x%>s(Q*=!cA5>|aZCj=J0Byc}k{>=~ z7#yoi-IvUC>I_1PE1*nFb`>#UDCBDEnVf{9hsd`dSVqCyWhe}D-<77H;!Zk?F}9j0 zo_saj0KiDju6UJ%wOA!7_^6gej>uHKux|||WxeFL*ZSu2`O&<@otQe$BMVL4*~(8& zrYFkzj!mV7_O+`4;cFI6+AOE}sc$6$HBTQfZa&^WHzE?AQGyv2J<@P!U8XJaE}V9i z{{r{6#B|gNHWG-bqns4bPIGF~{JJMQ`u_e8I{3R9GVg-bB8v{Y@zaIPc}bA{p2ds1 z%nNVD!l*Pg>)JPSdd2sdA(6sIK?z_#TCd^av@V6#Vsd+&@reoKSEAUi?mINZ=AtWuVGtjH26?4x4FXL2|)RL8efz>!6 z^9f*?SH7$nwC`{X5o8zr+8|?|b&UN5NF{1nwsJSglmR6M^a z_a)$|8Z@1{rA#{8lKz3uEYFLCZ5yS|yR}f#(mk<)dlFO9GbB@;rI240{uXv72 zgQl((kfUZ@CwtRt@TjYk);n6I&N_3`{{5njk37O1(d?Sl2py=sR0tZs`&BJh2z#7E z50{R6+fohU>zZ4uemAW`dj1^$b5Xyh+sM!qe!)~pV2s&{?EDXP#n`aJKaMC_4^ z#!&fz;9||shE4T!bF~u&_Qz9ly@%{sCx2$$3hF2VY=6QRn7pCo3LJJ(qN@O zbN(hFeEp9m?YrSUW0{o1_s-Z=Z971f2KrjPPy+l%ZYo4S^xMQw;n=_0jtygmdDnH= z7#Wcr+fQEH0>N^}-@3PB1AkG$?UUM-0`-5QfUuwcASo3r@`3PTlZspigYps(orU6v zWJg+cBW)mbvb3rU>yH*6XXA&!#+G}%PX>kM?Km=FR%-U<(d|#<-D9%yye*cJZb6We zG$W3P++1>3>kPR?rID697#66HH*}? zOu|v@Ayt!7g|Do$k_xOgAuQL6NZ@=LWRpL8YI(aFs`>|(Gv6=ZYeY*!EqLY=1n>J< zZ?h34W67*_Aqvhov{})mx~YU9?IY0S#+B|yDzIU3sz$4BxCv?YzebNP{M_w6ivyvo z=oPU_dyOz~UYg%-8n!u8$~o8cf{A*v7tE+IZKD*AM$5J@tYQls6kMp*&9?J=4iQF@ zl}i+m&*Xf-YjRn|G+Cwc9HbZ0#*{81m8>$id}%LsM9f_56n{{!bafghjf!U(X1Ob3 z@$MBhb}1~z3CI+VSQHjbDwqROv-L1ShYA>dFdp)g&3IDvrdaE5HC^rCY3Ex5fyUtK zM(=uATN?X1egJ837BDh+R0N{dMufV7;FvL`^d~_}7SfNW8)YUNe06HNES3tb_{|CvS#-z8xv`6vRzQ|D~XyP-7wlmoLs_ zZ!NG-0Qjl`1DHX9XS^^YOqQp*4w?s-Suo^u?2<;&Z`nhw(O$kRbq7yDC{@LNQYsmB z7?|^AS*ysL0ea}wfG6^onD;bDNr}v-FM&<4ff}@ROkLkEj8i?q`T-F}M{>c&V7fCU zJ5FAi%5dyQ0y9}WuOv+XMn+nS>ig(jb;B`JZhW0YQd#lk&+IatH+!FeOo|-wfX-ih z2e<~4Xa-&S;al_P*;|2+!t%UR2|)!2${)ER*0Jz4nxQ?Qzn`=qC#erIdW)~X57yCc zE|5=JETARE&&Z-_Xgku6^k=^_1rPQ5=&YxDVuFzC;cV4x9R?v4Via6JQiD z{7P2*ZmNJZ9Dvd*{)o?i_=6qhg4zY883yJH8;^jw8nfNQxUZK23VB!K^1NyjOFmS2 zA1k@Xn_-i@fTZc7-_l9dJ=es}nyup{uWVt|&Fb#rax4VbPx-F^(&vi+j2GvsaD`P? zDd33RefYx>V1lFatVh8S8ydA6`zyE~7r@hTgZB6BMgX33yJ6RYF3F-dr@juDD^*}# z{Ipa34r;Ehu4XDHeB{W>%D5_R6;v~jX^X9L!1@YUH);klxtK5fKKwe+09xpCXNYZ^Ks9(h*g&So>TBJp;2GGsImi=osq>)O_XPR*2i9z4(i@@+kX! zs|7Mzqi*MM$JQ>n+^F>*r-F<&5w+1mM^XER&`?B?t7Vtwqs}{8dp487mGtuVb~25u z?~s|pp>BDZk`c>1v`FRGrz@6Kw1sa^39apwEep<_D?KqO&qri0=2&_MNNOfg`ud9F zn!KL>EJ+UFv#O^l-WR4>ZdYqXEqeVF8-O&Hu_b%-eurnCNGCwsW|((gIVEr-!5tzo zEVKx5qqCkm|61EUZ}_l=O=vL5ki`$1w{?uBkUg{;3VSQ?089|weSObqY6Fj>=(An; z=j(xcYco2gF>Dw^z?RHA1!?g)>LBFe;-dLhj*o1&)TDBH#8UBVY1?pkD<_VRZkRVI zi~G#(-%JV5yMwQvgn=Xa4hyI%spK=u#JIS#iZCB`opcYwa$(m%yASsH9dW!>*?XM>tc30fjF^1r0(vzKs$1TC52wA_8Voq857k|k78!t62@d*JhC#ArfRd4 z^Znsr3*yVaqaW6qUigkw&Z{izPQPS(b2uKs`irlR|Z*dnHg(e77X^+F2RvMt{dJ= z17!3mA%*&6hUkP9&$-$t@>B!%dQhJIN^4q(EH4J#>J%;@5t~T~+p=~yKt#vD`1CS) zKf*!!=7q2NNfaRblP!t51!Pq@{58MI7N$fk$n;$B^k`T8uE^n}<$Ey9>F24&@Ha3e zfv`FtpXmvCN%s-jo1;bM-ctUaICem?oG&8?BX zek`BEEUiqFfs+|;fp)?JM5(yV}kpr8_HP7V90!by%F+Qc! z+8nZjHc^SKJ08#)A=a~*X&%NfQG)0ru*?&WbANyDHJ(F-;m|maa{)`7aTYG1vo$3MMn6hm2GXTRFp^!pvGWLV(8u%`b zL;^tD3<=G5qA2Cj)!5 zRR|@ZhFsjrDa^XgCFQq!hs$>SXN$@S_%tiof`=Pe8g;Nd#jv{G-^f_a0foN6>8Omc z(A&YFEiD9n2a@Etiv}XIlQ7c0CDxJLGeu&Z0(X=%CK!N*aQQxylZOq`9n8WXDiwMZx zDZ<(Bbdns}ss(_Eqt`g-l+~*W0^%%+3H`5ESEBeF-{@H1&u;b&K?UP$MtU<;xBAS> z80NRrZD}Z&&uC0Gc4MlCD%RTXbjjma2{n+fTT>~ILe}$$e`w@{MD43KViHa8Nl1{R zueDPq_m&n+eo`W-ht&7ra-PJC#O5RCwHQtbo)mz(hCX)#AxGREtm(XssJQYW&f16| zcXN{|D7c^P!@wym61O6*W8`2CCC~G|KGE8XaRkpi-l_go?<2|Zoo(y*tdu!C(K8%G z4L;NUNXu+5Z5fRZ4J&kv4*giD0`8CfF0qSe3JOgSE z>Hi9ZrJj`}@F(VUA^am5Zt4UyQpZ94Bb%mdn&HyJ9q}r z;YFZAr>pq3n+GY<)HkCpwLxz{hL!sB@-v{2`dA4o8Y)ET0_GLHabBr ztF3pLkQ7PlKU4Gb+C{Ju&H1KxJOEkeJ;QOZm6;|Q55cWnj6ahsrkYaZ#@;vzAub9J zX=gF{=>k?W(50q!bkMZ&FFBP$%4vJ|Y3Za*Q{zD`Ioe93Xbr#P;02C~F{|M)paQ`G z36J5ds=P*drwTbWjJ_t^#*jl}iGrNzeJtSjK>UXc?#3{wm+hHQON7?^$RhQ59z}m! zL7+&u@Wqj83jLO(8w#a#zE-7IVEe9&OS{O*ZA9>h8*q0ZU@ffThpa+o@{68#rslzL zMr6FfY7c}pOq-R~h(96jk3gDNkWr*hHPtVe4P2p0A}W6d1x(;Bg|jNq^zJ zw)9h3psJLb)e6fjk^MR|wn*AIGaQu{#TfBlfiv@~^-o=$v`Nr=e`;{AoS$4^6tCaY zi{tw?JUEW%aOC&;mbk28BK!wN56i&kwZ~V75f^ycOV%+1fI@R?-K5~}BcNw2$ULdo zYG<%a{bR*6|B-2Iu5(U&DYrVW1PJre3qmjl!o2uVUkLd#eAKAW3`upGrlJX%TIk66 z6Eh9mmr3y_5qi};R_l}5wh@LEP|$rU(e-358YLxVRkQ+9%TT2f4VKKk9%Ax*Q~(uH zeVK;_em@QEcS^CxpvXy9%{g4lsJYT4WyClt>&>Sr{S>ULY5DTlgXV!UcF{OohWPdV zzv}(B(3g*#L+9T$Q}e`vaEIaxnnloD-VeWg2mX8fjN^P_^_?m%Fbg=_pup(m8-ThH z8YU(b8j>?Wr|}24bxoPd8ggg6oH}t+--oeY@OYc0T)hzadk;x@XxIz~B&2|a!oCx* z3Swba0xP2x3ya?Wi)zDEksL5nC{5!nXhj^kVZv{!<@lG-Fr)*)w_ob_LYjZD)i%Lq zg>ZLzI-0_j+dd!paq_<%aZHH!#8Z7bZD>%KJR%gmj_K8~#fC$w$n5N6$PMD4UMoFc zU*CFQX)G8Hi7P$@@hnWpziJck>5Af_j5EQhj>{OzpM7f=4L(fNI%rh9Utn^a%4Yd)5zYI35#*qm}{mdV+w842A~ zYS`a*G9)Ev5BHq+CP=d4w?vGBG!_ZG z%2v#>nVJf6#<(`c1pfAq`4fdvQRzgf;$9j|?@Fyg`iYU&@3rFcF}| z5OX3;1P?u2R_2s!perx(EB&vnE>ecuy(e%a=6OMX6q>@10?y9$ZNqP;vUGE*ckACW z`0PU1J*{mmS<~peQW?87;1xLtWv6*xSKPQe!k^qy`J;y}!U}50p^i|sBcIoO5r2o0 ztHxQh>nIh!qXO~%?3!Ib?tVJu7ZEt?-c^bl`^N#(>!p1cz1reH{ctmX>?9v!bfBVY z$C`$R?Lu6(l}fTwv^$NBI7`BLtaHp$!K@4EO_#Xh0kAfL0S>ySe>|v2 zgy=tQBm>rxC~U*i!JoP%254A2uhNCQ}fIEi@-f@EyJsL(A7cQ^xNP)mk#DDzqHPYQ8WwZ4R0 z4Humil`-KfeTHYvU4&?lhzIGjlhvtPAOIegfXCD=Y%tSic=$+z?KKN^$piPAnW+$r z_THE{Y2NW%>j#ook)#t3qXino`3mNtjd{a|CK{3+pjq&lS*s#P6#j$lIDztnfwCr5 zto9EOuGBQ#AWX?+oGcOZ61>f$Cxc8Cz|w|67k#SXZYo<%`QrU~r^TRcYFHY0JqETt zZs1KVJS~k73mbc6d?Y^(&)J}j@NG*RJ>Hf-`)Fuv7Ie4o<|V|1#YT5$`28j2n&{PJ zNbaLSSjB}}P>2f6+FxOiJj{3DcH}MT^8EcbU&V+1%@LSmCa-)vdnGr`*e|8Gp^~G5I`Gq{O{n^G088e$p*7dAZv3`du=GZS5$u>@X zqMDo{^6a%-&VSpt5mogeLh)7pfY(4DOzMUcDOP~QU7UvJ!hu|!*b&h1;Ts8qVts1J z*`2-Ys?Hq?hLYI5(Qk%K8^>Bk&f}8jl+7(&8KvIp*!z6S<6QcUpVLkbr=8`V4fgv& zcC`8UZShVwwKa7${^g-;QXsuRu+c1pf@7#-pZW6>2+(~6tTHGUw!US<&0&hz{k?Jpp56T>fZ#3uhT)1*sAdGmm@&jq1}Vh~@XVocyfZ+<4H6ICm}9IH@ zO1M_*i%9HhE5;RLnjX1qyqpD>G}^J7t5=Ex=RDG&U0-c@vcj}oz(hWg@h6Tv1>8%H zp;`Rwm>PHfy=cy$9oeA=?v)_P=>`Cn@i3yo`_E1(ojaIrA$JFG>lK0be(8&u_FFfo zs>S}G4k7LSW2ggy{!f1z~=|;0IWHCz6 z)%@G$+qVG!z{ClYXxX13@9&)(jn6z_k+fE!Lw208cl;yNkKkrxYwr~ou+&YKP>-AB zNjuBQsb-q2_ZpEPOT9JE$Fz*2D5B5F@*I&dysUivgJ+ueWGU#~NwwvSQ!I3_!sFqC zLBl$-h}Xo#NwHFbRC>OA5Q1UK!i15D)MVn&Zo&8UE)%L*$G`P)ml<-p)#7YSaMf$k z7Hf9#Re5J2KP*CT&;qG*f)_?h)ZdPMmC#tZ6hP%!(h?XVUQ1&dIN@RNbDc?0J$69fM zngY_0P7b>S1Mw4wSuCYEwYod> zpmezYC3xBs5r7d%%vbeknETQnbW=KTt<7isYS~`rJ}D3IU(fo(iZ;+zQ)TFsGQ$j% zpTjkCJf*CyUfI)Jjpg7za{tTiBpx~6OcmdnwUUE6kAJLjZb>+3sU>thSqkhzPAeWV z(^(d$?{*yfY{w_^mJtcNF9-lseRMwiW$eFJtD$Mj5txn%$N z;~$MkGE|idXi?UwAiqSUFi-}NS~w-6dbYpX-i-A6GN-CcNHaZ3HVEedUBcZs4AN51(Jdfz#jn_rsI-CqswSWDQkY zBDHKcwP!=DW1tp>Pb}zq^uv|2(l6}RnzPy1Vy#ZV2lFLwLNoHj;r5%uSZC47Sb3n*lnb+aqx)8G95Il%p@h+#h&U zLlwX|*58{j&P!cMZ+~eL*S-uuB`G(F#w0C90|FrMa4hA<$m+$;7;Tycm7HSo)91;@x4>-hNaWX;It#+BJFcoP63I-T@#sah^757LWb%1cLIw za@vlJm_JaE)P7~?>Wj(Ue4+Y0fs_c!zkhi6Tax1g=yJVznm=ijP~^(zr)6Y-4;L!6 zIz1Xcp@3wWQj!9DoRO8z04!)kuY@C4Et{s4#6FWgd0Mf2zVPeQrLj(Lo197+vHwax z_M&~5W!90LI6pdyfzc5gv$ZC{-}w6Xi23Zm7!M|^O-j^GUBU(p(Z@4ROTkTgz(T9d zGr@io%K(+{X(>Cme;nbCz8lfex#wG4m_a(N(xzzO`T*-23{vGbS%P~8ZpyQB#vJ9k zJ3|9E=Jx}2Q(O#T=Y14=5-j320jg5P@LNVa&}DEGQ+LMHHv9z-EA?n@UAK!8Io9jG zk-hv~W!aa;wty)L2+@R_W1Xi-(;zepW|pM-?1Z!*Z_d_v6_12G*Hrmbqvcc}AtLM| z!~GK5cdHv6{U_JI!MJp!HqK?;0Er-us8jCr^~JK;+J=WJHZ$}6i`>uN-C%1Ch9XPM zDcb4O3Zp0(Ua5GM|Nd>IZZq9w(CJw}w(^`_!GX*KpY4iRx6kvi^9>)zI$41eCCA>V^j&=J!ALOuiMJ1+DChGa@W+cWXI@M&5X1f9n2r#>-BA7 z30jGwJH2=;1}g<2V&siW&X(M!i*4`L^{Jo{Yx#38x>GU;AlhV|HJlm=xDjDEZ7$B8 zpRgy!D@Dc;b^o4BYPS%JZ==3bz5t@~xnDexo$w9XeV#uO@!9G{3IPeyL%HIC_D^>X zej@5RT~ipvF+WoCV8Yc>?!gF-KI&;flMl{#0d-9%KG^ybcY(abo|G%sLq<(dY0HbeJC%&+W?uN?j8Pu$6u8*J)?*FsRHG{ zGpF4+r6YwF!VSv5B%iK68$Xo7Mt%`78Gx+OdKxq#Im_)rEWeI-Ix0vmX`Fg7^eX=zuM|CVh)^T9jMl(}}VxcbFMx&F# zYjXxTN($uhqciA$xI|!*fd+95(ANum#*{53gHusa=^y(lHNst#q=A>rZ?OcI6(imA z11xCLthXQm=ctP9z!Me5s;bK`V7}xN-c(F|{tcOPq%#7{@#@(sqo^c7*D6(SzUvo* zkCX27fhJv|#h7%A89&YSc$w#|Q)5rn?W<78XT4a1gSk>IfKnxe&dV}W#c5+a&^MG{KeaKdc-r*=8FN-I0y7iV z41Qy;+Vn4Pob3Q8Vq=bEEGo3w!fe}CFc5b1 zZ7mEoq$;!$aPH6E$wtu)mLqogmB>jbk zzcQ{qvkl&ed^l$7_-NY@0s>#yULWiboSb)%1=QEbp)vP}c(fb??T7e$A!C3XSG&tQ z26z?E0K_Vz)JW0WFeoCOZD*4%thc$KytRWVEE9iR&dv z|HNZcIZ=iK2I6w-QwXBU+mm!T2ufP_PoovSMx{j6(C7t&dF7-6j*Sc;6wA7!sbiDMH zXjoCY{c*0 z7AGkpmI^Z#NsSBU=q31Bzs}X%*tl@;svMf6S|HIcjcT+c>ye2yZ(FV1w7+?Rk;c3&l+5lq8E;5&Yw6>a-jl&<|b_1 z+;xcioEM@oN*M|C!os)LU;qW(YoOityYfqQ>*k?mj%hk8WY9-)Nw1VazFgBihmVX#6!?j+39wmbX z2!gKdSCONd?&$1v0K&IF%#Z*0C1ce_L-Z&a9u~&nWC$C5a}U7GEWO4|yyU+q%;IEC z|E)QgRhXrnUTjG6UXblh8sJFu1QrrBa1?s`(LeOw)>o_B=*<6on46Bl4k~`m^u*Wa zia*m7XJs}FI@R=sZdWv*^!=-bbF^xp$dWie#|;CZIe;`WU^_Rnx*ur(a?`aW^R=~g zAscM8n3^REzED8mKTo>3LPz!UH^RYdPZIK9SrLSn&zMwIN^8>t^R1F=%OzIeqSHQRlc(CAygKhcHirGuHgPAh=^ zA@|h-)oWlFzSHhB&$iThbC1sz0Gnsl+4YR?2^CAOR_8yw<4J}bZB{J((Z<}wo z>++-jE&isMP$WBo;oZX95{6J7|)v#*J@)OYwL>Q*W^{AjekFt zt(yMZF5gzq|6C5}?Q$Sh%)0-3IV8}$JKrh^@X*B0On#Bpti_bedV_2h-`~2|-^XG^ z&y@niDLRJsoeOB`+}*&%Bu9f( z)aYzYW(22tcynf=X=QUD*^OV@yQ?;o4)o=GfFvfl>6bEwG}8kWe%502G|Icfjr=VY zQnBHW|91h_Bh4k>)_oL&s@rNr_g~pIKUyJ1$LT78MHSo(-mKc7nAV0&FJd;JaJ);a z@%A7nR@#RpuC(nBwF`M|3LLI!0JWD4^?2%$bE_Lz6z$A^_*9kbCuXU~Ud+-R@Vz@b zN=dH&WW^YCv=W`8lp|d%MIV{NfmwyTAxGHyPOHT%)3Rw=4djbTtGMtEYh1SJ;%KBI3(-W|0OnL7Hasnzs{fqSn#JRslaTc_YGPiJ3(RgDvoH3r0{Nyynx5?NO->!KRiDFz2rDw4talPfNN(REa{EUX~lz z=fHrjfBflvE(>Dem3nGAKLdm+=w|)Y)j^&l8KJhQ&xD$sEK`rPxR8ou;E{K7Rn(`?kQjUly-*>;^E?%n4M z`UewhlgKD>;*Ebo9itD7C{;pmSOwpWqF#ODWKX48+FHQl!<+VhQT&P$od&1et2{;i zb)}Ski0v+-j*XCc4MvhXh@M;{W<62N9HQvDWw$MAE;%V|DQOKQ32bx9AI@I*0C&|< zG*`wuItbj=xc=qzFq=Ea@N8_AuSq59raU8Y1V{S_rvmiiP5fH>tN2YKKV>wV**z}Z zoNvwsN3ZB=`VCSghmH{|K5>$>(b_e-Uiefn3lq)ibztB)lQcUfqQx!@GI(|t1MMTA zRHB2nUexZOpAZwh{Svsd3|2T?`%3K4+O(+5sGlewh{O)rJvpg@Kn?qpY6;o^wsYEe zb2+1UkZSCYZIcrzX7UXr<@KWeszplbZ;TxtjYD@64rSr**AP(+CjU8P82xEw+db-3 zihks(u@9#wk-n#HB+>so%#9{yIBcdPt|7wdx`wcy{blkOoGxthH}gHp0V~d0eE2-bR$|f6V_h^%PZwc`6hXc_u?feD(c8L*r?rs=>kZs%p$X#zAb@XbTbFgW3VCvE!kTqosxAyAEYS+ z90f>LKY{urn8t?swdA% zqFmQ`;=d_J$FNPBP70qC-X>5(c&U325tDj*HI(|C;7(=^H^bT=izSZa2w?HP{OtD) zLbu4;p|X}`Bca;LERe-KyF}@q4;=G@VuKr+E@WUr^H~$OH)uc?Xn)?=Blma8yY!Rn zy0h!G>2oJH2Qd{m{X_B@LKQ#AH$Pyc;qgd-o@Jzn4pXG2MrmHq5i7S3|s-SiO~xaIc#2c6>|cvtuP0bHWj4ONm**K{w=kZjYBvrZ@ z5bgM4dh`U$Bs_8O zW=v{fpfevlZ^$cr-`0;;lDj{ngp}n5_{a13eiVoH2#$Z%bOIJ6i3(=7vdRj5>!wCO zQIP&A`IYy*Tu~8hrK22G~_>@TS`4s8;m=g<3j554BEljNoGnu5@y*oaG_Nu_$3eyhrl zlz&iuNsbijt@^%!y8(&)oZS(Z9IEq+oav{tWgN=4@6R6^)IPqyhoT2erOw2PEiN=% zTX@I+(Db5)N{3}Y^hllJS!#~;O;kW(qqds%Mtv~Z3DhEY?y+n6N#Uw)TtyhE(K zX1>-pyJ^z7GvT?7>r5B2^{GntrO*PyYyxF4X(&l^@bEa~2gdzs+{r{JIyvj7;+?O* zejfYj4!29(1b*u2P~J1TBqrvtNI}0Dd^N|=;^lpM+yEk=mrXc3-``K*P!D>5M{DpB z%P`Z4R*Os+lwMAey8cLxdMc-H?qU1Rh~PVu9+i4jkqAb!ko(F$GRurGVNu4tgUSfK z5Vn0Zx;o-eDT=~B=LvXzX3P=@>1v8jW^#}-51F=1lKJ?;kAsq8QvFl?{%op&wC%IL z#dj`4dlGp8#G_elCOZmdmwb0TW7YooSmzbtG7qv_asjBHV@*jGdS!4h+U5e7>;quz z67ckXwv?WcJ*DeHF~Ve;)w`1LRB?RaNxRYS<&M#ykrCBR(K$El?ML^eVzp)tJ4>)VPqX^O z+|JJb-srrr*BN@qoa*=BS9z|V9kdTEXWRZ4sqNqRENbrg%gylqoojLY&OmF~`WI?T z^s79041i-cHf>v;46rwlllr+Mb`OS0A4fo`tT-c^Td#i-wkV;@8`0;})O2<+sf3@% z$YalFad1>kJTKvXR8 zpi9K%W%{@qpVce$2pWkqjlc@6I} zAV0#TrQ)|W2iGLazg|0?pT{r++63V9-{hNH7UU@1<^~DxN+~&6Zf|gn@?FnUB0#vM z{WIYKSpBR$1vqI==kTfxyf5Sn$E}|b>ka4YsK1!vgynu0wWjo`_);LK^^{G1JR zr?kKI{wZgjV*Kue`F_6v&NdXBzYO9n5y{#!)1cfGN-2tGR4wkw8Rj1Fcz$3h!Y$h} z82~X(p3NYOV8Vx`ttw_QM?8IC!8DeBE7O~kaE&>m3B{~_G|g`Qy)JP=sD2vaf!gQl zo7CMlGdN%`x;=d$*+b%8e81Vbc^r$;L?p5$-P3g{@i$5{uR5Xir_-)$h}@1^s(3kZ zg}`LOOX8_U5__0KMv2iVrE-5&R`k%^&U8tT%DN}2{vpDU;@*q%A)fDI#hA)@tF%d! z_>Ktm!G+dn*KgXlvr5V*-J3>dxVkE;T_M7XijQH5nK0XhnSI40to`Mk{=-&fn$1)D zG?k7DhBhUQj2a{Knv0IkwI`7Rc|FAm>$zrqbvpp+*>1X6k(fl-L#=tLtyNJR2CwOc zhNUM^YbKOi&oQ-@23t{Oik(?Aw)pxdm$dug!l<#ORD$ZHNtog&NL-0X$u=WYixcqu zD-@RMP*G7$wYgXU#3;u0?K(V)sTK#YmX21>`1pHbKHGjEdY{Lq)KdJ?7@NQ!I0j6X zjMQb|qqR;oHYRhA%;A6gcl%i%shUFFYcu!xB*Okeg2<7x`r9Ap?-f_BeN{od5zn2s zYuM8?Z2E15x-yVIM+p(H+Q@at!x6%1+>Pu555kAFhJV}Bg-DFWC~aQaUQ$fE&kzTv zc``f#O5GRJEV$$UT%SVdNXF`CGwy`XushDIzq@of)9{@jd=Kxo4_0ZJSqu8~%k2}% zGi-MPIS&^V750zx+g%}fW?k&uEjh0Ny1FsR&Cb7Oa_)iLyUbtKH>Kz5NhlmS)0!MY zAC~f5CUsjKX}t*!B7uPOFz2NE1QPOSLVf1pY89werCXza_Vlm=v)=bjWky9=u&dcV zM9&Y`*e11UW-r%XMpnF+32Ao^^-w}UQSj{gnnl79j}#wvA~w_#2Rg!ZTXG?uTfG&x zwjHz7Z^6t=77{P4;1Cu>leT{YS~y8U8ym;B8gwGQbk(Hq19-K z`_bhfNc4rmH}k)m3n#Uq-fkji9w?WU!f@@K4+s2N&Zjv35%C6yqG06Jd44u|Z?z@c zRd!Xhj*KxSWS5IbHm1+3=qJnE|7HQCtF^fQ63=kYuxBlT!fwNa?nf(S!m?jC1U!EdVSLv*AyJ-} zxEILb+s*n~8ELs|C7NqhIbu|vU9w0h6WAfV#={tADKl|6M#}LPFv&XSU^}~*XGF9& zDq=`8pOv>DilaEu8heH>wZ5S{s4FXswk#I2-@u;82~Xv^4o9;MCk0&MbiEWa zJe-K+Y%Y;*N>uw<>i5&NuG?9c5Z6Rp1vj{O+V76-{6`~8oFXy%s2RnvO#iF?sHB++ zGdN#yf6lx4{gCG*l&Z8S=__msXz-_72d${!KADMGG#)zk{HJvDl$Rn3Bv5{u;E_MX z#COi_JS#*Yql+eWMlc`DG#`{I`^Fj!GbABNl*~~0(_}-eQK2RNkBJKCuPA^l%+O~* z8PVRgvWvo=7|m)iKTgL`<$dwu<*3)vg$UcfpRzKK&EfsIJsqsjG<#>G#(Vq~=xu~3 zFp01RYwzgTh*6+G@Y$%6`0YY%!6VW-xy#jSloc!Gb{AD$H&JP_C{j3>ok-eJ?pa4JH7VIFxsB7TZS0i8Z0N9?Hv89qQoezuED-;4?B z;MA+8YQ9k>98~o7{e~442z%}gIaPK^R%rj-8p`|ht@8-nL$iGKZx;T>`{63iScijo z2Myxq1MG{%e+I#^2~18IyFxLRRJJa2kiy#-`huP9b{z}4XXA(1MIaXIG7?dBMH|SfBLga*#w-?5)*%S~+dzb+2>i)xGi50SLcO#L z6OE=^D>_kIwvCP`Yvz)fX}R67o{>^hu}&}uTQHKL&4dvk5kWMnzY3oPQL)E1aOwzH zjn3;c%H^Mb!RB{xo)|Ld33s{jW4EXpi7()lMG;E#L&5*G**lL@K_<3rLR0#CgyxuF z3cTubS1Yk3Zx!crC;Xn8C#!*YPmF{Hk90MXwTW~%1`NHRJkqy#8?7qwZE=o9M>yi^ z@T9Dm_?in3&{!uOQzt=Nf&v;&$6R>Ww808stm7ojuRj5B84LGEb&Js#V)7%DnBY7u zJP!nT4j+Q;io5;-zlGjH^-1s6M5ms7BhPTGOV*|;gK7%}dY|a%yZ5W)lq@3Y)?ARm zmqRTa&dR%%AJJp$IssxH?@4r<2UlPr!Kf;a^WbYvAlyumDT&Z|e94!`z4~S3s-D#P z$9u_B6J*Gssg=;?W{*U?!fcNWQ?We}2m0*ZM4-tiPhl=DUHhfOcda<`rA(Eo-^ei; zNsuWJ7%M7xz5akR&6Uk5#*+{h7-q1TCK7g9yaIMf;Zqz^;ydV>*0{I2Edx@jE;nc@ zFDXoa27Z0wHppeUT5HN1jWk;t8(xK+K$-}nKpJfH%Ajy;8BVd@;I+8vXP}VeEYz5#->t;^QPwZtMu0ScQHsZFqL3-HHq2}t~_~aP%gktdEK3|P- zAy;}l$}f4r;?FSiPDT1-*@Xspe!qu0Nxx9 z;4!<4z|9t0&-S>F<{~5fU*5h%aoEIlhi>C|s~>b9A`*))eSyc_&tA0IlePZ54r^d) zR21lb+i(CVU3qpHl%Tr_Uy6+o#IBoL%+oJ;9%n4q**1nNypz5thjTQ`!}vt1u@p?> z8NO1I#SUsFU2MN`TT@y?f9@6gbA+w5AlL6p@{DoqP8tSW!;QC!NJ|!xol8xx1HzC9{8Pl%%Xgb&)7f-Z!M#ML8b#bumM#v14>%5WEb$K2Ufk1hY{`M!#tXN5g{9gGzSm&_i3ol1i5y#O+slv^;_|mev#+f z&%eH%c_`7&-p3VQm#_FhtUVk?`xaYmTYp5!az|{U8X6l1t(3R6V1txKPw|kt697es ze7?)!f}-AJbab?goE+Be6kv#|kiZV(e-*LU7DTY%5b5py>~Ur3L3j6UHcdfc-j&P*c?9-3{cpLvY0JSf++my@@_{`>g~eS#{MFAO%dzD z0h6F}8+@~B^w3stH&;@^--l}OIGt*ANEgO8t#fO_|J{&x`Ml^$?|lQm8;^)DI4A@Z zQDY5555_je+3w@I456646R8E#>p9~r09Uejxke?s%fZ|MbUA_*Wj$t;3A(D^29Ik0xsr=EdpYTbUHp6OnbqpllT zJkXG#g3U(7!iArFqhv1k;a~@o)~+AIppeeqL;xvHR43yE))AfHQb{Dozu}?5!N0&; zrFyu>MoMcbq$NFBRs*@o-tqI1#n{ZTULhTQ+z5mHgM!dk4<8v0EJem5A!DKhb0GPq z%|2-I-W_Yb{kvTeLu(?^oI70NqoM5W=f2-u`t=Z5iCe4V6a%CC_!zikpdkxm3-Ee@ z)JC2cdV0llnx{+qDP+3o#loFiKrsBc0iON*6842`nsz3ZB%1YxY#-YL+Q*Xq>0P@Q z1M;~6e;Qsm*H-1`Dl?qGO95P5EKxIQ6Dyd5UI;U%B|ERf+en(O?(1Y~@7_+zKz;9G zhv0a+52PK^r%sd3v3iQUFRpH{8>*mcQS9(2&}mUHGOUO`xa!+!fi7@DU)$6)IFdj= zP^f-1b-(#;24J^J5upwDq;gy2b1Kfu>ijLxW6hCc|NANcaItp=j9vi`&>047TN~(Ew~Z@(D+LwcvuE^eD;0TcMapTM zl~G`S<$BA^=6CoDN5pMTk&PKY{w22Jp$I7bkZsLa8m2(MowT6!;fC*pMBkIhQB3of zMg?MLGQcFBj0M?p(Aqey8TdR-P(TLnj}DfMq!Mt=EoNqNCjml7)a5y-mNm|N#SEJm zk+aWQQwd9Pn}(bTElinRA>Uy%1od$9a#AM`JoJFzUHmWJoTMhVDt(vJ7Whh9O2d6J z%R3L=yvJ(T1z!_j_&zF?kxG0nWzHa3_T7!)rqvfRIJUmj0WGC*5i z1BDjoM)6JY3NyXfDscp)6}++;tO_$h5HM6L@*=JxoH$9Bj~z4aCS)(i5_$l%kfmV! zM{6FxYy6H{)@MQ1Sl>rSF@Gr@Zy6f-V@-Cr4c(U zf8JUo2m-a!8BG{{KHwuu6ah389j95}*BJ33gYNHRz|(sS>-6qeLeBf2YYfU+CtC`9 zo)L=T&GljiauL`)%S5Uzy^0~IS>yoBL@EMLB2bYYvQps)H@>+6I#s?hn0ZXtbQ+Y~ zw$X%v3{LEN%3;Cdu!Hg)KFU*rgW}UlJca93yx-&j!X8%`d-N$^@QeQ5TF&cY3-`4PESeHc=HuFdR2O_wQZ-x zDQO}}kvG}huOWcMmCu2ta)1S+LS4xkEyt(y(-(}w-8IeKIdMCtay`qJytx!(T`T=;8Rl|UGO{`9p|7|5}oYtJgzBi|tS!KEm@TtQ9 zA_-)BIo-TtYhKd;clYbVC}%@=i!4r&5=)M3`Zvy^-yjVOKaK?R$1l|{Ti=ww?mArf z^Y-VOrqh+{l(QTnTa3ihWdKnPRMgaf&%TXjxyGmugNVoy16*|Esr^Y>dKUEXPEmML z=rVH3d4rSX6NdI~KGoROybvO;;KR zv~4+0UOgvaTG$r-yX|(9I6@UPeq$QiaNIQijQ<)N-PpW9mGk7fkL~?-o#|OoS?q$b zH0dHV@@tb_aX6RRnVB#bwFw{^Gw~6KMYYJAbYfy=1_nqS=%ksxqGCjsf%_2BV~%{+ zyNxElmZWE#(`8jdp+C_8*k`w9isYrn1zMqAFludAKL(l*e%wE*Cx$2LmN-b{tvaL5 zG1g?$>uGhuk0s2zbNRcaLC{456zthHfRuk3xd^Nm-utk6<=m;{%}3DNzYm+3)w)8NPu}U1jskt zB#C6k7`XopDN)8>M0j}I*6+m3T4aexLSk4x z79`?kuPWP}E; zuD)KXVMN&TCJbu=Q9J)dHH62(g=cmVk;n?Eo9ozw=6yv=+h|=uXr}Z ze7St_94*$%T(%N2sr$#=ze^^+=IS3YUH{fWQJw66g_BzIm)A6FsF|r&^6AbS zMDLD`#h5+<{4&Uh1Gl>5mSE2`w~>hnTsoe2+kfa7+1@E18pWTU1=4O?gvE(o*$I<4H%8KvGi)!izoeTK60c zjg4rY^8b|&E-n+$*(k(aO7q%&RW3T@)lN!eJ&%DE8N3s3h?|@U=&&NfLj80t8(|{~Pn9b3ue&en ziiU=c&JwxF842{+PcYkcR$0OvOLH1IW3HACzB|x(brhuJg{B~w79YX)Y{O}z+*Qqa z8TXw>+5CbdulL)5VfZnn;8cN4%m>R#<|bZ>H+{P6c3*NJoBr@*-+701ByX)>DOo+& znrL`*PNDN2qp)D5W`FxDlvl&1m5D0rDKzxL5!mLNtAg*#6a16c6R*X6TE4gEGr5>f z13PU_Hy*zi4e9GsLgd_>+B6Iomn!pPAv~so)wZXGz+g}#56}#Z2IT$WW@gr|%ocWd z8xD;Te(~X!det0lY2W*%#|v6~9zEjB`rGag+2_p6WtXA4n2sB z|MkjFFzqsK^fwoJ2_Ai-Rg)4Q5YfDvQ6amen=7rj?>i1x!4oE9K9{Gjl>6nA=R3Pu zy`f^(;kX|~Uyz8B0{0R?I!iTGGG(5Wvoja7prjMj)!LhkM)QxKLxih_3C#Ce@i~tN zQIp)8%dI5gY*(QlO`jVhk`Lb$W2`W(5G37t9P-K~$a5Th>{>2QW|lQiDQos}Fx!Gp z)CPGp)oQVDrmm$o!?5CDTNQ#DkiuosF(hH}6pnWJ2(=KKdO?oyYEj>;sjt&QHbZ zEDzMLP^X&Q({3ec&%c2~Vc4jZ+}O_#7nObz1PNN)S;3^b-W&}%9Dn=xwzkkkhi(g( zTy{UfI(t>1N;1+gaip(tboC@xif&w?afrsrmt7ru`{o@0aB<2{ApPpL~~k? z%WpwXvskOk_|fe*4qvO;(~`xFb@DA^qhz=2pQ}*aS|8=x2F8vNG5o{hEfm`HnLlT7 z*&YjSw1eZ&L$DupI0tVfib+3aD#b;plcovuVJ6UdN6{%hF2rZAicYiulj+mq+nbHtjWc$xBh@ucFcByI$EzSqH%D61ON$!ZfthFd*Zab|jO z+q`T=Vc_MBeowr(Qqbp*yfHN)-x)}=MM&?NG{2#z&%XG+&o>j9Mz|)8ip@xk5X_i? zi(8m2ppSm}F>}M2+8TmhCp#O4NgXuE%r{W+PiKZE!?;F$*)-d@T~1_LjfT*S)8Y!@ zfKR_@Bq!&Yo|BP?Vc4$3qOS{SsS!OJRuQsKgwAJoQM4sMI-4_EQAbDb zD#9iDxiu@PX{(cek8@py3Jyop^(JzorC93(xL41((-kREzKyK)jGptP%wXvg33sh+ z(Ee)#Y{p0VQZKkstKIuQc7!;%7n~QvJF7K$u8S?_#IpITTQ!Nl$1fd!nRLiv=APoL zh*#5lQ2up>)8f(t>Ipq~Ms4xg+0oaBzav%!zn>(2QFVbX3$b}_JD>l2AE+kPIcUS- zxQl#r@d_2}!63R-!1u{$H)XTMd8_#B+~Kl#A)e%EttCxcihZT$c1*2s0sAZY){rD| z&1MN{>IY_`n;x^R*Tk4}b+vBhCLtu|uN?#^c+42uH)cmeWJSbi!7OXjS7Cz1s)`bwVSFXuATAc2k`5%Yp%$VWEaEJG`Hg z=`h`(Fl9usS!Ehr{*NXmIZ50nG&UTr!U)U818K(fv5)NxN1_;3URZ;QG6mMi7K>(? zifRE!Je1TLm3RW zrT%ay68ms{h@#pE#H~RVOzFG*Gj&50e>K2j0H>zGU17t8Z7#)Xl#AIo7n?+i=}5&a z!&S=+%YZ~&L{X~g75rn#U$&xm_tG1B3~^sVz0TKz{vhf0(xlTz(@Dj;%TiR4=^?Kv+Cq9V!B_W-5&Clgu&D`0!O)Tv z^GA2)8i8W?YMR!qO~Cxwoji!O@k^fYeCo(?G4{G_nN746>`u|yW?oPX28 z1PKC?$Zb@t{0M&(9;*e}5R~3~SW%d?qM5vOA_mfP#_a}^hJKaiqW$i{ZHneg+&$9( z9#iHz|4b-CXKq#K&`+Yc&0eI?TEPw2^Bw{FkCb~GG!lQ^-hpm;?cOi4haTG?n~0{o zb=Bvy$GP?M{dDOf3$vmq2_x|$i(;D<=Pat_RSd!fFAiex;QJnO-;O|OvU>#|7M7qd zRkCp5l=d86B08RWpR7$OGUr(x^EGe%GpYr#el<$A-+a}=&p;qIOR`nf$P{JPD>nSo zs^Gl*h*ebIo0ci&%=rbEw6z7rBe)}6yb_&Sj|s}Z)P!uNdrT^@nK+Lu&3kQ0X;(bO z{7)J5lZ3|K=UMWW#s`q;AZO%KPm2j~1x=cdaAY9S-0qLV5nz6+T^>CndRCW`Wm%pZM( zC~kFHoa^dDD7e-=CwuzT;SQnP-y2_3pYKT&mUo;CqUvWfTPsN5U|?b4jdIzbejXhC zNp-(X{&7?fNnn2tm1JM5LHHzzprHh7%m_RL(oJ=AbjVC6x)glaPkRRk;iSvQN=w*`mPj5P3RcqjW;3+6piT#0r^t`S4nqzE%9xdb8$T=B{| zAw0MKNVbZK`r0NmZE!>xcZby3L6-3AP$rW45kecP)k$3WvZ0dUkTM&`4x$5HPo%%~2zF{)BEm`nkD< zvEm4(L{=ss#$3X!QsU+yK+9v9ZW2H23i{lC1>2;<=8SZu9$@(QThp7C@ z+Pkjph$7u&OUo?^Smd+?o(?9WhS~H*bCr>#aH<|-qKNA1YW7RLvo-aa(x3O2AE+MY zHABCxehiN-!E^N~kF`*Cebxvu(e#Ux2MBOTvFMc>MRt8wj7zVrA>ercGPt;g!(al(z z`vY(yL%KNp~Y6At4|o-Mlxy=Q-DV&R@ob2seB7?7hBgt&iCHv9ZhA z_umXVmd_acJEVm{kH2bOK+8)pD=)Od`KMsfk*&B61yK<7B{&;OKrzl>& zQ{v1dKi$uXF73St6o0De}vU*pQ_1Fyd_DjZG-A3Ltn~zRRd{nFEj$G{K5j`lYLj9nU?NrN|{hh+dL#{5{}K;|`JDd=;>uzLJnk_6W!d z!pUxaNqBg`Ff+%2HeX=;$r-={nWuW`g6M#Cz0C1PBRuf=1F}@i^LlF^Kd@I?g``(& z|4!G@oJQn7{cP}PVROUwD(cM*v0`Kv-xyo=s7c8-ldh8)FFz-;H##|8p42{U;B^-q zMpYF8oHR|gL=~aA;yE9BlqwOp^}rW@_oEdx%@H+M06(`&xKe{U?D0@5mRbcjCuk#|GpOB;Zcu2N!PIhcGL-cG*p^9#c`z#Wi#olDYaSozii%(TKvqku5v)Rw*-n~C8Thc&i5U>U% zP!=oiOM{9RS3>|Rk-V$EC57CF3&p>(-rCs^A`w5nuCA_n^HHpBJwR2G7YFDfcGuZ} z8eh^SO>yncOia@9GvdKD_sYYN^fM-G4bYjiMR86_18^T9RFMaMWQgeK5gpzX-fSaa zxneEEKj)T!SEu^RH>`lRsGr|YW^#Z|wuMUUfM8ri#iEz}g>)jso8dE_-S z09`$XG>r&A;=1#M(Md$YcsKl-Wba(p)!@)O)MP;P;7meEn3~pwj`BHi=u2P9Y$9E*on3dF*P-Dh&wwwn*bS_`qOn2l=!tN<)2!Mi!^TGPYU9 z`uGsvi{hVKq$tyfRnkCZjc1C*Y4UK%u9!Dd!R~70$SIZg>;FlBD5VdpbA_lETSNr=C&VU9ld% zJQFA>2(IX6_of)&Bpnhe(>)X>j$|clZzSavjT7x8qm*($!vaXL#bDOKb3K89Kwv8r zHq-!zT3=p1hVL2(=|}AXR*_mcU(p(%juH)kK{C&2N^wnoU(G>m?ilHAktgPD*pIn_ zQE2emVt@30D5ssCR}p1tz9A+i20z5qQB$q8p=1mx5A~r`&kW0TR57G`+g+9g<*d&J zQsPYkpsi7tEJ;SY*nDfP<}6oxAOHvuzNuZifjVMyrxVI5N%b?jP*E~7u)gAu8a z7yJ47IoP6Uhy@|0zHv@8vQqF=k5ISDo`!`*DNfwBYK$9G=Q~-_-AA1bE{Q3@DtkWj zz8fUiF@&EFr5!C_vR_Dz68qTR(*O;#kHn&mS$ZYZ`i`xP6aLcZ*@=fy^3f#LbE?Zn z8gxZ#H6Cy{SPsDJ9r{&RSoj~%u&H9LX6-SSZYwSZOq$#<>e7tbfjTT!Hmc6RM7-ZpsXYA3Z}L3IKj+}@oN zmR;zp^MO)v028Oiiew1ZA>v^@>aq@`3q& zYAr7Ipfc(*5J4kK<^xX$?_3n*@BA=HV7|&6*l8-49}xqCO`@1Uk7%Iu?Y!fgRGXS) z*B-=FYkSiH3>bzi?y3JG9@{;JI8s1Q?UNPIoXyaVm1cjp2SH19^_*#SjfckHSIN9ZKlsGl}XD4 zU2RcPLqV!v%3?IFIF)p`fQc>7q8X~LSFz{izFH_X^ebV<=Mo_Q0+fS>&)WVF$|~5s zmu(h6)@bp}kpq*4_PjiQSeO)H{$X?m3=^%AAl26qBE9c9qlc!GfVFPz!|#1=PR#H6 zeUnQfRkuzS?vNgKtl~vr^~v}2I*$ks8RcLrzJ<$B!m6{1eTliIoZ0T$VTJk3%AnId z4_4Fr0M1Xs?cFm-2B^p0M)9oBZyCjq9Q9?Ia zYlY?(VP?bzr$Q2e=f=~Qvu665jypx8tSt#=GYNC>RQ%`}Qv60FtN?eY-y#_cG=pJA z&uF$0{^0ClsKN($k68qe7*}vw*QxdPcZ>AZ0zPU}&{z`E+Q&%5)e;G<{;W!9XB+5H ziL3^bk2t6-T3(UrfpdpWU2qrrt$*&ceugS_{ zZofrjKn73#$HKoMno2bSp3=~s1k)DT+QR1N}tz=n2C_0BlW*?Hvz86dc)*&@w!)VN8dMP^S;iiZD zD8Z$0H$)YZ>6}fU{vG1b5WHXBh&m_lI5`$JIM|N`Ape$gbR{w&P54W1%2_Q(9-q-& z1EwQdUnfh~M@`d|)YNeHT&bX;_uw7%Oo%U_nvZY~=!-)X+-=cefJE&i@{jM@ zckHTeEY*VCe^aa(+i2`!ZG@IdAm%DGuog06$}(dKTG0&eC)o-Td5tf*98i#|u{3lx zriSu5ixGIX{(L+&^(R%O^)#d;hL94GPR!hT@50Gm!=}Wf*?! zZpljnndGMW`d%QN9me%rzV1+jZ3?SoXma2U25=6Zn4QUpKiB1SVTmYTie1O0COmcV z8Va>;;a~B5WOMH(689Z4aR)>LUHh+F$5O2OEwQqElykbK8tS9=aUY8+wmp!4#Py+M zoL3Ictlj)R!j6KyUuFWPR!p)3HL(mMrK!CDD{sgk>w@nxS6cZ(X3JKe_U>e7nAl82{^H^h%H@AN>9u(M?kdJ3dsZ zMz{!*1~x7WHdjC+e85AYcjHxw2MjrnyLDg{(%DYSsgB>=bxs{Ck(?akE!i?nj;M% z^D-ZXFWkb-`>0i_1W`@)lhADx3K{#WC zwr`D7Kfw%-FxO1t*U~rcvCKvqmy!pu*A_@K=OEC}HGe2yj^k3bqXwqU!aiP*Sal)pS9-ce6EmT=hnLgKboH(+zC& zwClR^RI_R4V~x!&&WxA7x*4eYHu8WiNfxX;v|XZ9?n0)!Qp!zEE`}|od{jzCJIj~y z2Pm8XHFCk7vUq)oDu<>hv;KVtYrj;R<>p#yNQ<<>TbKrwBIn>xv%X~9%O&Rc1dJb{ z^qt&4zkXGF09r0$fDaTC`qs1h^XD?!8*?@q*z};FM@7#>^3_l7wf?{xfJ&asw^)mH zVe7w6_WBKqO2sYV(~S=M$K;E%CocO;EP{5p^a(=z$V5PgLA18bnf7P$r8Q%dix7ZT z4@CPd#5i*1boDEb+Cn@?#VFw?jgjJ6FSQ20S}f6Idqt;>O}{S%nwAcYlU6vq7ZBsf z1C^PXLa!;+YAOSspKgpyY^z%o>Xv}ors9V9+Ohnk$t_7q$#F`Q z&B9dl^q*H-X47~sfNkgR9mGoL-dcy*s0pUC1Z%c5ItFZ(dMW%N!|aq+bmA;Kcrb@b zbzk{w{dZs@FkwWh-0Y>+@&BgPXG3r5wFsljacYxn$`f(FwK*5UUt?FW#Lxo9q?+(+ zpm#?WONppZp#5a0bk6H!wgjQLq=Y6~TVV>u&dv_(xF&18#0b*hcdbY}lnkO;;L#GL z;=AWS@JgoEzuIE6^I^xFnS=LY7oheniVuR%_t~WiL+c8&YjZW}=bhX`uV?hbTU2IV zw!?%;01tihU&%~NgUW0Its}#)P1mo+9Vk2jVM2|j!6PsBWG*qg^(q!>ar3D!8?C7I zH@>qQo8=R zg7c+M4em+T%amyDaadyZW?V@{tk*z=S-1ze$r58OYnC=_Qp|Tc9Y@lbv;2cpZ}o%v zBiS9dL4N?+shX*o*}seF{vU90HOQf&X__^FYmS5G!u&-WurjI^d)wL-yFh_+v*sF^ z*GKtZQk(I->zHJWk*Eaf-yhpfDAfghrpQDC9$m!eZ!Hnf zNrj?-KCGpN_3;yVI7S4RW6!ZM2DJ?JznJy5*O&q+GFz<}a#*!XXbE4%g+M9P`p1MM zq!-~e}b1Vt7qSkY=K0p1uUvF!JtX;kZrFDnZGa zd+~gyVg4%9Qp`9Z!J5s9Wz4Sp+eNy>7eF+D7LtJ=dASPh5t$<2qk-~wD8n!;yRvXWtF8&C8%+~afDyThdxl%Zi*yW z{(>oKaMl51u??BrmHr>2yT{Y8YFv-*WnKpX-n<6r;03)LGqKg))d}3JEwyp+?8RNo zMoV>P3Wj}bi+248wXy@x0u38~lqRHzsIzH}lSFOd0#GO#5nn^Pq3mwq#a{o#goun@_^GHE%Rb z)J9y|Jikak$O-*Pi~6IMV?C9>jt*~oM+XHXBXT1DIX64{w7XiiYslq{@cY2f3B;Jq8^T8L-)}OQl+fH)3q53q*s}Hi zx6NV!F^dm%I6Fxy3N0(-`nuKGegj1T!<&-YgFDy8@f2(elK9hMc3{oR*Xq%itBPkI ziw5Cs4Y6BAcu1&&4e6R)UB&G#hcjOLnws^ii9wP0-By2}mNOFcZQIgyz0>i#jW^J5 zFMN1TUY{rUvmrmtOd>JwUlJ=B3&p)h!PkMqApJ5SYsAuRj2Pvu#v|_b6!q=RzR-V9 zCtsw3NQ&;7?|A6f;hFQ<%bKFX!HxNwZuBVLZRcudY^q;8-_ncBL$K~`7RsqwnRBUm z=H-}_WG&xuYT7Mc|11=LOUzeKHk?2*Ac-Li2V_R~h0t-9iFqmc1@KRAv6zJGxa^x2 z#6^#^mZPjo1@>hKffTuw^3<=*!y{y!3Lcye@ls4tega0iQJ(?JB#o zid>FWFHE)Pod2z1F%pU1?^O(Z0?<@R zjrdg8`f^-tKPrBj6yx#SZQC47W{CPcsk^L!=&#G?T_dDf+kHVNd1}ezKJ1ZTaGyGamj;jP?%vZveL|0R1)>DMZ0e=8(A#Qk&j2?WTW;g-2ozYS&@(qk~Ku;NRI zzUa3gi-?HaD3*tZha-j5Gn*44kZig$quuLLI6rHn9)NZ2$I1S;LFj)Aw|C4Mpi?nT}7-l!$Kh7lL z$Mx7o0(r+@;dpb2@0IxXZS6qu!mAkrnQK6zHp)hSaa7A6(VWJ(NrN3*T?#EjyyC0PIO0lOVt;gV!K)Wu)G zhJSNOo>8Zo80L6C_bb%;YdTl<(7DzW2`DIN`BRCvEpYheKYBRriA>kp@*eOm6+%Qb zr||oyaHfkd55rt;W_09->Z*N*iSleuDSdZORRD{zhNLJH`cIo3H%95ocum&g$i|P3 zsN)}FOEp1kh4E9>C+mw-j>iU1UNwd7ksJL&b|MQ=`#PLKx8@BFKhD1QiU=w>JZQwx z*)Q4ngzUy*i?GoDM|X^=0IOAc+kq|zKEORWYyTPs_6l6J84O9&#VV>h4mi`Plv+Ew z#ixtI0!94Qz6e>DaLD+GDnVNSVE-uQ3k;;Zl9jBX?1$RK{W#kiV8DcAoMkUXN=$!uxcrT#)Up11{<79eZ>@)`}OaK?*KQ*7wsFT~><}LF4O-zEF))D%T4g zM1f>6wUQ6Tc0C}kqQs^>Doa-bJ|2gSM7ztApXJpELy5->`3e?Jxrz*cDL3tXaIq7u z#U#zV9NZxxY{!c+zfFBTZtVH#aSJFjJO@9LeiC~jR3{Q8{}9w7lddcpJc{~m!0KqF ze|vIHjV*=^frsKB8q@IH@*Yu!XT{(-PbeT|?EJBqfE%OEO6FjeV-z1&ZBZ2)V;j^_qt}_hn?`>?juxaP;ZN! zOI^KCoWao4n-vTSx*IxV6Auf#uE-TCFHcK&ob#x5gH584+l@LcSy6luLS?X_O|$Ni zb{3q`&9fH`Ye_$s->f^LLAx`Io}OM~zoXB+m+GXK=8vGm|B6f1#1s`#iAhMRnPZ)r z4dIj>u2ZR38^coRm11X-tczu)yV*&1&=ei^?joUzOdZs+Tu`j+gQ8|4=Raxs3NGti z-nt@YSjf+X8`nn`d?}vk`9wjS;miWO;?tz=^w>ZYFjHdQWg@F#C11BLyL?Q3@czsg zZUVhGkvj zLT@cq3d&1~!ZKf&D&5|>n(SHywxGu9+vWf)7#wfjxp9n6F8VCQJHs6Q=-l@y1b?my z7NOQ5komSbnaDm{E<5pZ?pDjy;~K2B!a|UQ?@VD1#aHF}8eSJ2scHUfkhL4)4zYnU zw8KQV^ChL&E7tapJWVztnF#`04||YobAKtjSLbzK^VKj?4i+^FLAdD|0Rg$V3Xz?2t7?r+7-Xv{oOL|50;I>X|=}59PmT`snkoZ&tSUIkadUk?< z^}71pRsvh6OteKW)<}TYi38AsG?D-)yeb-O)x1mtnIoUJCZm#0Sm?R>IsC3f46}FT z>={j2lQ@|~cWY#z) z5^ug=XOdh3S$gpLP|cm@B;Wq}*c!>ZFR~3`)U|Mvlc+W4#ky|Kf=Og#;Za#QM`sBA zLX^oA?L1nHL#eG{7J+Pp8ih@?WO+%J!(F{kSuG4Ca%u_|>q^Z@21Gd$DC!w|Vqy?l z99q|pU)}re=6>R9d%^c?FwiM6*`=~(Ujx;J;+YVXd@j80QE*}z3uRK|sVHrKBzTrs zFqx-;JbmYdkD8={1(h;NV*m8sW0V}9@KY}d05n~LCFSm)yd|RVC>|=k>7~%!d}*Xz zJ^G#f7t=>R=`i?~?kU4+cb9^S8{LE=4H7#dWdXJ3^wfE~pM6+VT8ghRl+fp4vX||K zS#tcYkC#0gv6Bh0Na_9fk1@)MuV~qXxV5%Z7!>QI&NpWvnL0mSf}UK>%3s%2U55YM zm!fo!5dLx*vQW|PX`y>ncF>mZqys^#mtJq`1pj-i+U?jplfzlikjiKO%EfR+>{E4d z>y2;zA39Ance#$2rg>e21<1G-+I0{k6+EQGWOAHpToF%^{d$)> z4J$fL5xPZiJ(RLz#+M{%28C6;@Yw1XiXiV+VC zY@+aWR)nIU4hE-EmghGyariJ5Fa=c}g2}%>zN+^`c7!~AXWahptHx#DGkg0`cdh-} zD#BDSOg+1PzHpbMh)PjYSA>I7+Jw{JX*oV0oX0zx^R+ z?n)dQ`#^8sr+-B$<11;o`1i~9smp?gg=9I%M%8D9$sVlFQ90il-?4y=*$Q|6{Jy?bmmD+WEBL9bk8X2^N9~^THvC z6d@L)!CB<%%^NV>6wmlv^j}q^PTNZ|)PDqaG>Iq%!ky;}?eSbj`Hhx0Q^L z1plDtbdeTf!L?*cMn+LKxy$*MHjb8f(^ObO;e+NJeDTJL@1|I&-}W;Wzsv8i2)NHX z9)gNv;LTK`<`|wUMOX_}@@m`>~>mK4JPi z;TW-veH3X$f>R_0(X>Rg$d)z!ExnsMWX%9UzF;MnB&)BkvU3jt5QU_(?K}CB$?N0m zz%v7!#9{+br8XpIa)>E$$M^!ui4~8`MY=49Tj^Aum@u6^cJccIZv%C3T7r@L;7p@d z!Z9|fYjUdBEjSG=yC~tE*Iu?2yU)3u*R7NAZp6ghJN;&Dyi3VY@iMx@LNYzQo{h5C zaGiYX>T64Ft7ZJk4mrH=IT?V~K`6tYRJ-K>gEe~$6i{q}WEDwy2A94&WbvQB*yLl2 zzr*1?zD%Eox{H{=tqQ^pFHaBWJ{jT+E|Vpu1yi_xwSzKU5w@fu-s7XHGC?UL6+7{6 zXVJq|0(Ctp9z0Vt0@J4B15z(`p7=f`^DwuRmX`XUxz@`3oV4j)#@TyYPjAU!z^o~& zPY`FbjnFR~23~-bnJGtlQjXycv*3@b;u^M*RtR8is!OMv`>|N9#mvJfG>8}XhDsG1 zKnX2|VQyH#A;M6(arN2}%r(WT>gOizD2dC`DTVOz7F4|}Kp9z{pbHDkaYj8_D`+MP z+TU@&w^ie?YiLx`TJ4;I)?bh zMm!JLNXV(>#8QOW2I(p=Qn2xE*{l&!Cz&5L=#2SOnp>Ka}Uuhi;EDKsCiWl1>vNv<`wgOXrw?b7G)J^j>zpe!bgw z8^~s534E$jtWNOaY`{#mB{}QsQ~*P*Nl3XOyrXCYR`x5$yFL*OisH<5P+WE%l!W0> za`7Uh15BrbU=s;Y_JYlBgyKu&zwvkHTbBjrC3fu>*S zHwJ7bSK>j@&G&^v-m4_A2^uA;gguyVYoc>Gz>(t%-iVYq%BiV~W8s&%xw$5E?YSY_ zh@{r?%6e~S1?dT`TopKeNvUL6+Ry@g_J~JY3O97LZ zIJhaTi)9U?kI8e}r-A)kvA4mE5Et8a7fg4_bc4w%Uj>0{*$Yp)D=~I3MMQ}-xg91{ zkJ~7S!gg0kvhH8o2Lkx7o!F^1>EtUba-V4CGu9c!-erd(>&d7Ill^K>0H%qgrP`_i z&d%)p{UgyH*@*pnKtFyz5Zz5LB}`lmhKGx`6sR)O%U54}Rto=nY8ZN`Gv}H!i$qf4 z5mvZ-lD3utVhG<56|euK{Uy|yKQW+@M*;-=V=Pye;`o6Lip7VI!o>VpYtaS1Q zXM;6BD=J|wp%4vAlqjmDD{amKKRH&m3{+K>FmeKsHLg>8R;oW9N1u%=Sk(X|a8H=7 zo}LNNajo0#Y|;WDNVT?oyiEmRig@u(Yhg1~c50i+oq@eju1evI6j7u4fBfa1PIpzO zhh@7vyQ2MbHSR%OLhrx;a)te1$E-L$lwcu=c>}t8CnAc-G_v#2EALp=jgBG&@=9*{ z2TY{R`ooX_&a&nL*_EM25{aUvCGOo3ci-nb1eaNAIe(aP&{N zQDeM7UGCcu^87@uN6)-yzgqmyW?15N>BCcH?b5BZ&~(F{AZ~x$>pUij$GHRNnOR6R zvLbMeLC+KXTYZz7rAXDR==*z+JP-~Ztq;f-(hC$YQ!T-S)2siIsK|d3BPx@9IBtBm z8rxGo*YTctV%$@no3}+dE^ZK&gKSVAYIit)GSjK2T+E;caN3frSIrzoo4+V%dc{V8 zDV{Aq{_C>26Y-mG&9bwTdnE;ki;KH9SjmY@A5ylqwnB#$zF>E_M~=|UNoYvzZry9g zSl(&MWU*1BaSQtId(PoS@Xk;Lno0-`%+IHS=VdRh81FnEFE|YipUK9m1EK+{2RcBs zEv`sM%Ra=b1=`GS&rB*ud4N8kcXac>izX>8jT(t%YyV|v$Zzdi742tk^8H**>2jn3 z>lpqDueF(`xAL9Z)VuC-U@I#tzz@d5p^FbAUD%3EASnges+=KDWTJ>#h16y6^=l1pvJ5m8Ar0}EqAv+1!AeEj)IXMp9 z{axnMPY1TJzz>S(;!-*#fIC}}X*hWrKlq+p^{J{+gvOOry0DhQST?qL9tmf6k~ma_ zHB`YfMj1x{&#tmymg>I0&lVZPYq#~h>XZ90g3{v@kou_hW*l7zZJ>tgHf)L`*3gn* z!^8@zIz(F}d0HX!nR{)4;k^i8RPl`?71o!Q`fIZCcD$2CravCeo^b_Eyd3FnCsl=9 z@uMwP5e)hB^4<^Vm@ViA&4I(;2I8$DZ`k`om(GjXK<3+iDKa^%wb{b*I0n@;WZ*!b z(o(s2_FSb`%z;!YmRSSY?Gh?wb`V2>QtW(wJj(uLMESTkgL`6!lqD!%8*OR*Idbr& zaj;>5s8sG>#eyRC9Tk;S)Z2`52{DR!={Wpgx8_d9VsF~)PXll750_W12Bp08(h(XU zfe{k{18w#p--2OICbAHoRJ0qk^)=8mQL1GCO5k%_*!9xJ^L3j~%Q-NVOuK^v>G(-J zwr2KkN^>|G@<_{wa^ZV0aQ(wkA;3z$#%tu{1INWTYKst`NH39{gIw*;9vT{zB##~JOD2_qDRDvUy`0_D?h#6 z{fTJ~#oQX{DS}YU1JiF>|06`;kncsPTd7pRR4tQaBjHG@Ee^R@$WR^-pxPrNVzSNx z^7+M2YHDhjFbMxGOIF0sEg8}_7W^&Ga7%_)y^ZotZ;X!j#U%tPjaV*2RIZ5xwy7&3+Ww5Hl4N-)wUD!O>h)QBp@DFTm zi>4}+p9_|^ygQ#~BzB^&2n!!Y5 zPhP^LcO`HjhH;yS5)RTJtvlN%t3I%^?w3>a#F?02rzX{f5!5kS$^3P#ml_woqxO~j zDnrcLQ5X-g5&B>q2c`qcW6Hbh-Q-QT-YY^gR$q^2mQc1q<) zu*au?FM_VFuBC5}c{=bJ)O>k5<-Q|z6qUamM)}!fgl^78%BoU0jyvxOPoMwNOt!R@ z_O0kmcRUQwmkk+IB^>8IbIuEW23xAr_Qb+$gk!mQ#630~Y}F37di{=8_Bxkqk^J{H z#m3ai1DFHr0*!yDQ7a~uqOj@7HXB&kt#!BW`kgAD7_T_k`&+MO+0Thtm;gt8s?F7K z7xy0=@eln20~UDC=b1gSsr^8RcS9qSF~w#u$+0Ex;8dUuZZJljnZfZu)EH5c?%sdZ zCA7JILhXD2ZLO=y^1_WRKh*)V`6t;>$1VRU__D2T-hL!&)2}RciJ<^uT!$NN$rDU` zal_po&jR1>UspP1zPdY}IosXIPFJhXNHcFp$vN40xWVbbsl@Q(mW_6$(YEQ*uDmo_ zDt~4Oo6v)@Hz=xA&}m(0mxqi1MtYMJl$6hAozD>9t!N8nLK;^|!F0m7iatMRhV8%s zc?~9z^C_K&C}oFtVcXPwdYUG*nPMP)Ppuy$YIE^7I-ALXXDL~+)BOMom(7rQoJ;4i zlfl7yarpiV!yA>{HBiiR>Rw?YrwolJwcwnNx+)^}rN1CZq=6LQyo8z0a%U_AOcituljt~=bdU<{_9mo#{ea3`4iC~u=dO)v5XV`?&Y%4f&OG|+DVgE1caS}Nz2ZycBH|3(&8^~&+KlG&%*oAB){uVM$f`LuB^G+gcD6OENK*$N0$c&6lNP%kYkBNP> zsAp|qgPMgJ8{HVvycqoaXyuzI9k7%HAI1WWG60IV(&mReODOql*!4I7w|T{2Y=vTo z%&^8x|Bdbi$`g+0!NJLCpeNozjS zXJ^yYks@w_83ZELXNWF0&Q40A8wInsmB&5=GR>IHZDI2I zPk*(&e0_DIY#*irpRr@RjlzP>HslpppIAcWn&p$XzTX4BvT`K#ad%jF{29Ej`0vY_ zH__z*)Wow9hi0eaNDfIP)a>VKE&h+ESw>#-w88mFL6S`cxE|A~N~K~3?pJk>dKEa- z7pYh7ls*DE8F2@&n5{y2^o`n9#$S4V4b*OQpL(c6=#`sjZDD+Q^9SPCm>6&{TJ!u} zR|h7&m_e^b7EU&Lo8I%juK`8p?UamgIR{DUg zgqOLR^a`#YVVgLuP|7L)fMc{A6x6B$HkqolEQ-1$5aAF88?jw3Ea1+iafQU)Uz*hm#@s3MThj55HY zr>(PwAAaDDUh3<^|FHCVk{FG}ezrPOvP^$6Vs8l(24qV)3;E&1gIW*3xN=v`EV5#Zx2%nHB>g>qMKZeHbEZfzrXwgb=KnoRlp2H~W|J7gbboayD-L6zihs zKbR_e?SKjoDWNf5hf+1x4lk=vU5LaOoG?vH3MkZ?{8BbQK~Dw#8>iv*CU+B?_sjhT zhLArvzXh68hjN=5hkU*DdN-z}*Rd`re_7`P4~L!19IKX-x>{v?a&j<>!>p9Q0+E?~ zf-juk%EGk9iuqJ@b!(RKzcea_(AgWH*=2}Y-@@2czf%&p6npUQtZijP1tljZgLxYp z)Ku8lNBt3c@LmiZG~l!M%uc8Z)JB?t4NBp^v%D@o(v`RNO)c)2yi*Lk%W|EqtNler zH4wiXn`}s%&`N$?OkUSORZz0BQM#XBjVLVeC7yZGObB&%(XYaxP|ctsE&%PO-ExHb zKN+CR{HHwowr^rbwOi2NQGqj4Q&TObFFOfj3#_$1HPQaZowlek>?RCFz^JQgm!Geu znup-b7Ypy8oD|CM;WlROi*cR93;D;ZMG|Mw>f47wAMaA|HVV^b=(|e-bH)k+4 ztKK~-q89iE$k5#jNdHHM2L91$)}XA22Py&&=s`PM)m`>gYfjb6&YE4TIU-QDYe4Ke z>ql{RR&`Q*a<9RpQ>l7cueNp8dfje#fbJuYK(+T#+@-N-nOfEKpRpX)){6qkcKqKg z=2Z=s3H{l^F#vIO9w~U4$0d0!N8`M)vrY-o@C?gd49H< zzHCc9?-*}HbPU395)I7*TDUOKkA zYVJzvW>g;gl|n;ui;bc9Nq9`lTrIi(zOU;?h3Rde)Tgd0%XS_l?N z-PDeO;(FUIsM>#M($bj1AD&emX`a_nwwt>vg{L$Yc@t}z(LX2=UYg#Q#S=ijCMhe) z;tUrCQ=aT1Qb{KQFUV4MQ^$idM)yQ$j!15Y(9S1lR^LP=tTe5iL>z|UDf3kYcKw1t zLb{l_77b>rB>K@ofwQ*$41HbK3x-ZPD*21P`zW~KuIv_q-Eq>>1F#QQU>Zgz)J5R{ zT6igLBl-NCQyJC1t!m!rYsroFHnvi;snk9)Dk1oD0NQVZI58p!u;zLAKChK6#ftfl zf(IuqE)UM(N86{i&mOs^+YH(D#l}z56|rd}DC>ig3^DLihlWi%-Z;X1v|Ch~p`HCs zt!90bib#kkNTb=iVp$G zSoa>)Yl$Y+`1>A9vj2Fv0NtxQ;ny=-W7Y8s{Ei(%UlKxWC$+_{B3${yws{;~M}g`F z4-aSUS+27`g`Z)JF()$%b}+zB1>kd3Z&9Mw2?z*GXGH~5qdQhFyDhHUKQ+rgd{q0u zc}!C6S(xP=v*9jfNP5Hh+<3y)p4_)xY}_v+9TW|IMrVg*K@GkKh~X0#9SPGmKIciJ zlDgeC+yVmb3#I?$o-lz`0NBG900#Aftz~&)?T6(;-y2uAtoxP-w~w73XiZkr86q%J zGZEjvL*Nh)xWq?|;&CH`$mfCdutjIsbaTJM@~C4i&V3$Qy`MrD*iDo0qt5H|Q}-Wf z&f#5znttrmBft4!cYAx+as#nO+Q^k4(oZ6dp${w=-_SGM-JnagM%HJ!{X8ZKC_tJx zM9vG3Wl=G5{e%8RVj*PT_{#>rqxC%r0n`<_CuRtL5p z+Q~w6PBK<)em{V1=BVQp;ArP@&h*#+83NEenTtOgGG^bPdrBCLa(-P1aLpU=PEEgzI&Cz8yk zdiMI>C6(5G$C{6^e*uJ168Su2Vg75DH#95lrK?Wwi4E1;&&TdKAl9~x94uA;&cyHx z$Y3?}bl`Fetl>MrK_Ip5?R>}7QHW!a8UbRxrJYYwi&~NjzQLM!oJ!LMBytPAGjHB) zqy0M=C(|JiZ<5%qZpWlvUF&dbu$&~E#O_~9eG%byZaz%y`X7jzs7<8wS`g~>W^mn+b`5`tcUHChRg@qLD?v;pAe^QfKop6g~KcUcA zQ0q&CkpB##l1UiZ@THxnCT2O8q?m`Wm#sp*SAP->6xx3UTgK6>qZ%YSE}gJ62G+Ky zt)9jJXODZIdwOP;v|lX^&Em!;&*QrKvl2kSciIR2 zi=RIRBA~Sp$|RERj#S(PQCK+`z`;e+$~FPLOG;7zW-lel)5U$Z^dNuXf1V-{=j$NW zB1}omX;^J<f+t1B{b21r;5J9svS~(-ZyE5P~Cy zUhmcqhQb+G=RgNxPLy+Jc3YUeS1;z$eCdF8JpQWbz4{#Bd&B{OE=2`0iv3|VaO)(! zf)M=VURw*x37|-J*0l3g!A)grSmvuhhDnDB2>kL7>ON}YI;M`x%E)j*YNIW%YLnlo z1210e>{cNser3FZ47(Vq@>q8FNmbfeLK}njm8E##ZKl@uvHi+{I2d@$L>IiKDqEz{ z-i$B|=d%+LQaey0T!F6tH$q(`ZOI5eD6!n%pF7CcFRYHXG!~<+UXYp9 z;o@rYWz6AfBZfnD5O#5up`Tlx5+@^4Sh;eMS_xn~k~NBk31Mg=jrjhSDM;67#j`c+ zZ*I%ks}>>2G;erd@M%~+&RE$KCHB0Gze1u|3g1(_xe}fyTX!i|@d0>$O3JJ#Q`Y%} za;yB7LX~I|T-1Pj{)PGR?vKOVHShNnRbPA}PwPTlXO47=!l$~=F^WG8>IpV*{h6A2 zY@ZlOIs8i-i}~V#Sp_1Da7?rMm@GT&B1KYaYA8^OIW99*AespD@@U@hZP>xVeTO&$ z+5M5Jsfh4z@EM*T(CyyZF^_+U2(1cj9AX%#*xW#QjyKwSqI|-yNA{pXh3F+q5ZQi@ z0)b&3Qy=pQC^>(_ta%oIr0%*RFfxqF-@Gj|M|Kmv4(KGrV`CkX#OR*x1PU~X4LGT0 zOkd%hVcJW0Mt(3Rhbt>sL`1Q`??EA(ock702^y0W#U98&>iBF{TVEG>;32+V*hNra zrudWPnmvKAMppWg6df@>-xA ztO(wjUnNq6%)`g6g+3$`ZIih~3y2}mL`!5Lf=uLTpH{o|NeUZPUbqVL0&SO9sVi{M zH>=DB1o#SU{yC<%a3n8rMkr+}axL=?pVIV<4^86pzO5Ghz(Q5Ndo*goS&I~*n5p>C z^YS<9V`tqc^x?|)jZt78wGvU?&e8U$33eQUP9Gwe7dVTqepV|f^~iAkx}jOzEv7yK zTyz+?XGi?xdrCw^lyC9BoqTJQ>s0pt$i@lu7r3Pm<2+Y|mDfcV=-6{G2>q?9rhXXLpq2Ssp$Xx7#|we@FBXP6;9u=ZS|L50eDz!dBH@a?%@q3Z=y9Vmn@4`jVw3L6NNT|^@STJpfL9sF* z7WEN;$V`akaI{jy=W&E5R~+rzZ&NoTi!E3i9{z$Y_usAU9SAcR9;d+icbf?yt69rq zDTy&EdIBdx@)ne^Y^%PWmoUcU=BOwLHq@Hz^bsOi9R24nA&ce?$kfS+jr;gDq6-|! zIit%X!bSf^i#f=F)gB>K^MsQ^Epp&wIspNw92f?|6-B-h-qD6>GS%Vu>%Di^={NM1 zymo!yvponIVbO!*qdW%zkD!GGrEWG*jT0BQ>2#ckCdgwA3})VZVS-vJwr4CwiVX<< zwf4t%jwJ}jO$=f$xqH}nNdS}`+tt--9X22bgrwplTq&oa{9VE6?4*ylTpy+slwcJ2 z#Ql6X9cU|$E7A$*VL?er&Pq^MO4`U5-dis8KXq?_15P0t<>u*W;>3hv``53(odO>| zG+oUsE#U&7$==Bv9No>YtG>euY~5k-PHeqY!Y9|B8q3y2^21@Hxwq5(z@#5``HN9z zpkOg^1N%XV(bG93$tI`_e4$zF&1ebS&Q|-wQOc&d)p!)T=cgEhY>LgpA%Ivgw?OZa3|DDsVNoWxoeI1jldMgP{QfSn=v9`L(MTPvQ9hM( ze`IQ>1&$%caEz4oP8%X|B!ak%>GJECeP{qgh8$Pvb*Gmm=jGMGY7%{U?{8JL8NH@`d!XyaA%MPTh$e>|Ff?=hVeeYXZHT?k}TsAku_ zw$YB(yL0548P)98h=3;sK+$6g0`8-MX=iwzf)yNoFyZT#_`Ns^Rvzf#e(UJhX+ciT zQ+j>!3+12k`wyDa?>|bJMnf^{s=~io=1UN1Qq21Na39{|$}B8!sjE85U5|PliuJ4H zk=w2N-+W|`UW)2nR&56EaDX4?F5eid_TAkd{Kh z!1(iP(UOoH_=}%EiGWT$Y2fIdQz>P@0vt-%VS@l|gOZR`&;wOP_zGuW0I``6E-a(_ zmUTH(QA5C1qI}YYhQyepn-GvK(#%D8K)@!X58$=WfS%dk05{&Q7}uE_c#l#lCLkg* z0|IgCTuQgnW+uS*{gtX0qW_Cyn>bD^mHcck$4P?Y&(Y7eFS%igP;jN(;hy}7^lO^V zY>Ht50`$I86q9B{k&RehU*Wxo{~uXz6_rQVMQa9kcXx;2PS6*33+^7=J-EBOyAv!x za0{-10Kq-DyL6H7KYexTu_sF{fp@Nl(nJtlB ztS4$0yVm>6?iH9VeD*u$p`n;Ahn36bNywhdG6E81c)YG@C9}V%HCcb`P%TqnL(c^g zITI5TH}>~6oTYgP|FE8g?@{~fjYL5%bNWm%p3&~ zfG0-psnPP{qLX_j3Rvv&lF#Dz%uL=d7FBrkcJC)!;P(pigYxt9KkV)8oz(BI?6$-T zyUqysb@@iYw&NY< zgZ2KX;|4&B<$r+rT_lNYPuYYi}Ik#aNM^)RLKMB2F;bU|^^!tA6yl-lh6lj>Y>;Oc9U8^}+|0n8vr670IJ)}Xk?-CQ_D$_=oSLV^&ZonV|Y=YRU=++cKDrD@ajBE2zPs- z?EIrfBWIRjb8_T8Al28iLb=}YQrK`&3lp2Vf9elb>RfWi;nqXa`(*%r?Is#fpMpU^ zZ(w5y<&Fy&0GkdWQuO)uVKeTfcH6oN8W3PLC*wC}c@OOv@$boD0Y4!QxU9K#LxF<$ zw>lrl@<)YR4R{D4*22n6GAh)NGG)-e|KIMpQt4}QN;|=N_NX`vk)eG7hJ@U0Fw-0O z5vg-r8qgG^WFr#Hnnh!G{eXoX$_iF`zI;ux!{kEb{9O8XY0I$#0JCEv0N%dL_+zH1kj?@&^9Czou^-^Rn_{{gusVIT@9?`5 zJ5MPN0CiT6(3-7C;OV4N`nOd#j@S_+MP1h_D69Y6Q?b>8eh&itqVV*YB|d|wB~ z9S>YqZ7%SRXzQiz{u*lp*kn_qyW6w)s`)f{kPH%K4E+`9XNjRL9;o8Og?_v_^A1Rh zKWwF;qiYW{^v672SmvwM=hwk_>3mIC3i!?SR%>(U|3T}2HCdll8%oPLQy@pScFz=7 z5UuD#sL=5+Bu%3)F)BsB*zi2}DfCm7XG=Y8EAjZe+-l5H(NM-wkjKz8q!>CgOPi-3 zU<9KPLyW?+iqOcGHC&PF%0Og`8l(aJ)F$6PKuIr5Nh}!C%bi3!2$K|w5kWoB2UQow z*UK3z)hZr4JXO}T=1UbpZKng!h#fr@fL@n+7LxLR`zbs)5aoaic;eLRQ&ES5APv3t zbKFreG1Y8+YZ5_XtNUaXM`cFw_SNadJ?o4}0mZgNt&UI)NqhbAO%+TqGZOJhIL~3z z#0dtJFDQK(TO|%kr7ZJ>!`xM(qzq;>T_dgfU0O1SiO3YxJta1=xf$qTF>pc~;B7%3 zj}J@fhbmm%`QVKz+x7H^I+$n|o^Qh3a-Nf_Q7dYj_op?B4g0XQeeiK{K44RE^zO>a z>fSs#z*=QugK?eMXm_-=+=dH}kig#EhJIi0GdNRp?2__V?&Q_#t(ahGb{99?vAS*o zPZSLw_{KgsDR&94k{EH0=5~IGKoL3tA0lmZ?sgJnbrknqGMS)wD&t50hB&^wX!MxO zxGd9ewaB<|VIMm-G+HQ%mHI6a-Mtjw^ix&GkG9D!7KD7jLhX17+Vg?bLJ!a6u9(@`e%w_*S=^1Vsf z%LLjm|IR+ANMIs`5yLs~5!B-X7#JAVlryI_3wvXF`<~&4JmldZX9RhWqYor`VWInOmb z&eDfNZzjEx5`u;k7Q8PmHiP%u6y>)+-6EeIzX>QxQ6aI#;1)JMqREf0(%{6`HTdg% z@MREPVHGr_T99N4koB{Fu-tDz3G>AXqaHd%?ysaRn>8?M_8k3`98ASRP6Xg*nhRZdNEqJx0HL?P)X0i8U zm>9+AE-N@6EG8Cw4AyK}w+B2O3<^0Xh2$0JhEg6(;kYdAZ$sK0D2yT^CUI79dcPIRjEzc=L{48`vp6ZtW{w^%Sy)U~?u8Y>rSB`GY3gE$bVdfo{*zx%bwJ zw3N%SMEs`O!oq;h4?5iIf3M50?`LJ>qAj2-J6QeU{sM1v(RRp-!U}-ZwzFS+T(PtJ zF1dXxzT0_pUO`Obej9F(HhR0*pv{dMH8Q(945&eP{+W8N^tM5Byg`}{6_n0W7MO?(2^^~;jn=$q>M#fBL4c>NPqLxXSGkW?HVb6IXa ziDNH4M=i1`PG35U@Fn8$3@H+?~2dtFEdR+Xnqc4JumOz7x<)y3I1r=)grWt*zo`JVoBXq%vg;wg*f*yw;++kqW z7EdNt@3<{-Lg=H90(m+6xp!?1Ic$(bBV%!KdARZU&+6u>c((RoomrBq%p|YEwt_W0 zdH?wM6FS@rTI?5((|3ZAR!WP`4=GeyNa8G&+C z)h#Snnu7F!g9YpJ0S%BounZ+ zYEB1|m^AO<_>3A{M$#0n9G;N!oaI!RzxtII((TcEmLO*qxOv&jX@cJJ0 zfE89jOm~wncyCQ&E%RYyow)het=|tTg>U&?o_wm<6*r(qs z3aBh>Wf6J5*@`W23YvIGMzR9qyA}xS?F7}4z_Fw=vsHhfu}ovuIxnawN5EbktLJ_v zg~$r}D=sih@RU=297mlf7^@ACN9Vxs(PXnPWOwRHOC0iFXTMRQ*vR4@9TG-zgHg{G z;ZE#)bhTCa;%`HhlYEJc{SoCJ;iAY>Z%^z}E!HVuJISMjT}lsFN?5Sffb+R_vz+lN znZtXq!{hOK4v+GL$G17tP5|BAo8T;zrO(w=c~p)S!=nPR=NYFiQwS8q-$MHvRfqlb z_*M*b;>H67$ZK$cB=&D3on{;=hH z5_$l!U(CF6oCFta&A75}$Lx%>6#{lUYLiW@p3bo=C0)Au1C3X1fwFM&+JlfX&kbNO zO`eR!;mzd<<1S3hN3Mn!^^|^At0!5S{ESWMg791;-P}Zx$k#$Oq?$1g$YHQmgym6) z5&f%H@{`&jgAYYy%}A&IXZBgsdZ=DocEB}&MWUh7%XSv0Jz zVEMnSnAyVFC}LXySoO{~A)Q{x2zi_%nmlSRcU-q8p!)$Y!}yqdP{?@jpGA`RdI|-7 zIP?=FljfOIYz*0s{84eA2>3ub43#<2a{vb+Ys@k__x(%NG3Hl>&i=mr2HZyVK%fW! zi|N;-H~eWr%&C5M49@6pWBu6*JYtfzyjeMxUJ^l&U2G@k>zv^$DfG-qtM#Btk#xLe zMa7qzW+3_(2>gU}2C7@jAO2lkIUFP`L{=qpoJ%!CpU@imVg#fK#rK4QA6OI8_SJAV z%Bkmmjan%j{GJQvx@3z09YN`&RfOlidHliYw^>1JJAzzUi4}!3L|RRY5kHisp+-If z<98P#oXifHS#T$nYI=E+HpvA=uLBwdvOtIW_eyq-T&b(ldfBUuOCsNwR5*dNNdH3Y zHXWrH|NF6vbe9?o&i?M6X-X1D8P}6Ex~*29G~=~PbvPbx36TQ861`E|M5Sws?CMwo z8vo41zd#IeQkh&SU3fCl{~%c75d4Q4=H$enXw(fc)OAGyIMYJQ*fZh( zN`7mA_uC^2)sj4q*ibX)O^SOut7kt+#fRg&{`v?PeFyHde{b|6cC>L5PWPI1PHbt7 z0*2z2N*PT$gNSs4y0K2A)1p5Zf*@BqzgLxv_8qeBZ9}-XtMtqB+FGlGf<}(lw#d_0 zl`+PX?_>T&%Yi#vb%E7Qfgx+m^^GKfT5vdMyH-`TmE&)Y_!TvHcopu8-piHTdjk-u zdh=HMle@;%b#Z`zEFT8qk%r48Lr%4SI4@dQl|rd2(DE>|O;95L6RDvTKd1}?b;~Ta zmnN;Z#G4geY+r`G`vA>%(~xR>q>|j~^8DrJKuZnt};@y zi7T-z%v;l|Gk$X(lx}C~7cX&cgFI5CA};>-kbvx5EPs z$hB*$I=#&_w6u%S-*uk;VOU>(QbNY*YLBnISg674SL*x+lFQ9)mS%b1EC8yBbEb); z_CNCJxle;Q$#PlhlK0=LogfDAjf=snFpsp6N+JBoDUDukIM)2Auag$Nei;^T9VnQo zlhn8MO=uiOC@sOVprFMgdyg#c(*hb;-HX1(Kkw7PZ{JvReQ&koiELkr?-%T)!VyJw zl48@@8@OK=yOC=#ZRcz=5E!dTMEyxwI-db02}gDWV``=y+1bH4`z7|ejqgtZD zI{P|5A02k5%UK^8aGe@Ni3`W6iOK5N9JLxTfrAySasUpAOR2yC1L%qoGZpI9l%P>* z!CU}M9P*G(!;Ij>(4|5)q{iwXhLP-X#Wt#m5o7Zos1VNZjq*>j=n~G2#{;&5tko>B z{&U%yt2zTOD~g`XR=~c?{vd>x$$vQoykH^_?K>_cYQ_bGo{rgl04WC>ctO|KC$|dC z#5DDa(%DrAC07|B!!A**q`io2?pmtK3h0Z@XL`b;p8y?GF;;OtP5DeYU{jFdS_4K@ ze#z9#-|bcdb8-d0o-XFNrB}WFp`}jJ8yr_{lHr;7JTBlW-L$b`*l{(MvmSj0Y&nlt5U#9&p_%D` zs#wDPi@W7nu7BMup(&(KC9;gex z)E^hGl?iEU+}|LbK*5v;CU2?^wFnv1QcZxJw#a?IH;{WkR;#5=19b2?-&%or;j_u~ zmJU$2MUjyI0_@p7mH&I3mlhX?N6@Uv>1h{UXorW%#6pJ;vdQndMQtv|+1oK{FP*H( z=Kxp~0s^89;sF3klW}@1Vq*Po!3zaVwf84;67uqx|5zy^#22}b8@e9o9NaitUbi?y z{hz)eLO@|5`Fo6IW)JvThiw>5?tvN=EGH1&)eSKy0OIN1{{W@lr&UJ)9EpMd^3JFB z{>P^pMS26O8a%xB25Suh(09u8>!@szw0doEM0jswrou*(2XyI20w0}~cUhz>KOI!{K{q1)i2cMcdMc8?hu(`b$x@y_SeM|xeGwW=RaF&eOt{=nqd+-*Ezmt8{^80F z8n}e9e6^r#P7@Fia5&!(c}VAqphgcK1E-#GLF>3X@_XJU$vnv3=8FxgrOtFi5|axx z{5Eg=cR?IZn3RzWbB1~bw7lGGodHtq*#v~hih$uk6^%nS(9)NN?Eofa2fw5{8*x=p zgR=kds&m!-2i-yWocF>wxVqLfFm@?>LjtN=KnlVo#m2-`18WbhSI(ZXx4&O@5kdVW zS6~SPAV8Y!A%&LmVSlB`5lHRjM^wlWMJbdC&+m}oqR7*cx#t>i!TED*VopQI5 zpe>-)bCoY&6xU{yrKRBvd@N!S!(DT9r#&+xb|KkO~Gd@1vl=RSnhdRBMQ#Ufs{tW~*Rak|p{B|u~czC!B5g?q-S&nN!$F@vyoIF7g z{|!nenImgj)x?JZE9evil%={(lv^**jZhrCo_nP?E&3kD^sUWy*WXF&WGOx5%gQ2drbfbT6_W={N;C9_9&3rCM~^y6@Pl z{-R=9&fjkG`Lrs5K}m@XyLZ9!r^j+_2z^i*_7$qP_tOuAy<}*~4gFybv|{0KvX?*i zYZ3xH!TYgDg2K9nKg8eRWd7@wiOCjDIJ; zR3c>Do~-%!ybk6yc#4R0sZ4a!{TX$gM-HeQZmN@s+;E~G=uS!P38rq&>+U98?JSc@ zQUEk-tQnvAJs#*J^5@A7msjx`MbkDi^bg&c5XYV=GjT;y62JlXxxL*4rJqh*y^CqT zm1HPY*wmZAPh+S?emXEZ4(h$s)vo*?-h-47hIuE`e%$F&+>v~-9rk-Qfqp;V<;>nU zu%eZtWTdJmitD)dB|xHG)rA_l^c)P?kf%++Ht_?SnYFvvB#|8o?5_~Hrp>xxvF~XIRde`YM$ven@uvfog#ku$d{qSlc)RH`Df#qJ z60e1zygA7<_dvmLiU*@%SE(j2*AgpzNiC4x$9TVe`C%84b7IQHb6G~DdlUVaaD~c+ z7F6!AyEdY?`oNczog+U~hcCzQJM3zsp;=+$Zfqg$Z`z#~z+9$;9cb#cX~#+DW%n`D zBT(c=3EK`gsg_07`#hLsxURGoIJgUqUZLudW^J^8G}V5baH;NU(Qf&2I9#Dw?<9iJ z6klZsxr_t6o15z{_SRY`AOTR1sN0ip>`nKhvg64?WCgbBA2d;>qN@1>D^ZLIp$!4M zRr-*kB8nC}`GI~EKYt7w>~qQ%Gem=u8lWr5-R|<2{Om=)5_xU&^fWf_D^Ik@*Q?V} z3LvZEN5OB!v4DF$#s6+%H%PUIG-XKkbe#(C%fZ@T4Ie#?{5z#R?=Y$5c_aC$29;Fp zcH`(Q)YA|Es6Uev!SC&9y((t9JbCaZ;Ai4$y8Qa`0PArg5E&(F^?Jd=W&er~5IKTV zVwr!fRRB^@_kuTgFEmlIng2fO!l3>*#cbl2c5Nz}<)3#EUdzuk@86SX<*{-Y1x+}* zTK+mE->1jQaTb2o{_L+J9%8=wDl(ay?G~#$=hn@}cC!A~C8yl{5pd`9dwX4Mw#FHs zn3$PZ)?<%gVth83<4U%!LIc}ebO&&>Rv;pKZwU$kTgsw$K@8e@)utm>QwYI|29NRT=Xs#bwpR zbvKd)S`+~|B9iz$vlbQ>rpwJi#n*Ad0z%#z{iK*_sg^FY| z;iN*8aqByC=RviPwX-@R_Wk2({mpg_&($(s?r-ifDaR69BH%Ey{0Qs(w*2W`wRg7H z6%{T-p(Of7$j7r0rPw8WcF|cPO_O9bag~B*x(uX4&@(8?GR)qCp8l-Ah9)N`!}mfB zP~N^l`$;JCli1`AIn|_)1AP3_($ZxYkiNldx)yr3=8Xb0L{s)QLGO3_5iF)*Kf7DK zw>iXU1)Hx2)9a?dupX%arPntz^~!j;sxA8KtDX zKSxnfO_z5@l6Gqb27GMRYh&s80WgB_lUMRSSxUfyoE7kA09Wc+BL%SFdtJLf4Ldt8 zpDOB*UXwE!m~ZiYf3BZQ#y^=t0(da@nui#<S2VruAK zb?D~?pB<4fPgoZ2EZsil}U$(&!S&srJM>->6#2hi`dvOw!57ym)|&T z0>`ImK2(S7F2oth*-d^c*uG@yr|O;z7RB(0`|AS}sYXJ`IKkAzAOGa1@Cx|?46&u= z2l8PItK%FMDXB+95hK652Fhkg#`Ym2ZetyBtxv#6G*x20f^7l>GoLbVFic; zHnJQZ3q7p=GI0kKPDXu@69S<72h1~8G~uukGjazYzeX;DtweNtE&Gl-wN}^b_d4g) z5kt=aPPA|kY5sS=x39?CbEHZwtD->1nxk@Ue<}~CuRkr@S^P1a0$^nQrtkJuIk`aO zYtFBm@BQ=h%}BQ`H?GoYAZI71ME*V$!C~{P8Y2#MdaU5vrNJm1GiNbTQBt>aYwhfo z&@M-U_dCG|ZFp=HBzTgBo1xXWpslki_e<5kdR!MX)#I z9YNgyOnTVlU6Fv6B%Al`@t2XNx0f?N2WRL0v+lRpcSmQt^Wo^{nbLRu3Zw*9kU{-i z_^n9}pr^UM{GtUm{uV;CWh+)z*7NIYn+&AA&zwBqI3IO|fkTX0o3WmSt?lZDt3$v% zMURf`{C2-PUcl!VfHKSIypYpcI{r4*#?Sp9AI|v(LdOpDRm*hKOzDywc`dG zT;Gof=x6Ro+Z^F?*-=qUGs0>v-Uf=NaAx(lq1UN3+*<$H?zH4AM^{zA50f~et0Mtk|bqdkbNXat6cQHI~5le7T(qv<>bHz@2ew1 z_*bz^g!qR)+nAYARjsS51V{6mQfSAhiN6QZlN)aribqw_Mc6lghU|Je<>y5~Th9XG zl?{PWN=-AvN#QEFQ>F1F-HK#z2$0_KF}4}z&)|VGjSl%c$A>N%@@D{cg#z4Inb{5J z?8lj))hL}DpO!W@cprC6{7jLgwGUxaHMWzrHWavL8EP{oET>c~5 zEqi?R%`C=Mta%UUB?ke`x8T9RPcfXbV-h{Ry6%d+;EXkpVCv9)FMXT1=Yj%6V?MjT5`n`4BHb`Z z_gcc|X$&e|>r=ENvqC@1Pp8C^l>EMlYWRQ-Ou0tFK+SfcF+c1?ir?fplSOmRtiJwG zY;Mr`aT=5xoCvRbBjM5Vl1f{=j6~`RrMo%H5&MtcU<|QjR$b0o@=%&WUF<#>v{l_g zsCW^O9-G2$grRI1pXV(d_$a2yQSTr2^T>vqsz9YnfB*iLDvgkU`2t0%#5a@hEJXa* zGDs4cMx-ted!$xiT0i#2gE?=xlUOjCz#hP%CKbc{pt`R*4Drjit$! z{dGowdLZa(J>dDC7Bo<72;*pMM4T{;YC!VtL1-0Dm04^e68N%tX<6X-jc$SSg?jd=lLtA&Oz>>T1>c?4lkVT1Co%jT9n| zhlif%4jUx!OmC|sQn0wk1r`aftoausdr=#$o`{b?S*+rQmrk?cT9r^%Th_}&cne{Y z_I11bIF%QQK)JZ}cxROJQ2|%m&_4=F$rzN3=b-Ev9xE~Q;GYC@+++@`XFD8C`GP+H zIHXG}SE!+a$ps5Z{8Mjf0U>qTwmzjVUS+s*>R&Q_JYZBASKHfGlPt@pydPr9NL{+h zqW#cEr!4BezU<&^_hCKWxUkU4;<#U(N7TX0b{0BXKKiHL>{?OO(2YnB-s+Vp^`^EY zQBZsEMQNdRzExg9Y5NBqX;9osLAMPZ1oZa*H0i4cORoDs$KMSax{(o{{rDsCPxb(C z7uh0!?Yei_-b8u>q%jVp%&Ra@1}yAcwbO;@CC_EEQ7sq3uX|o}^8lj=d?CrcXG7GJ z>_iO8`>Yb_tDv43hIn8k$b*Z_|65pF?y1Lb*sL8#`Yp(z)`f1glOg5zu*F@VFmhkm z4~kV9u%AAUMuqmCTAb+k^YLVR5Y&cvdxn#wvOYU|68u|FC2!RbbC^P!hhDGu+bCAx zj`p?|mCs`!hm2lbqIwjAW0}FEAP{p0OOO6XoL_n>z=WArMK*G`Pw_=qJ`X2e zanzooFw)mEtcNJJ~n z3@=yTHsoND-9N1bTQcF2onMHv^1^}pXLtYHnQcKDP_sP~qLIHU(b8&fR(UBxPCKA&&06bCmU19~a==$nTXxl)amVe=l0 zM!gwvf~DZhuzVERT(F1f83L7De|^@AHRruM!~fN5B966iMaRflmrKf9S`t?VBPyE_ z_8UF;<8=It`-w|pBn&=4T|8QJ4**j=fHOQ9d3fX*itLf7a7tQ8wE&hJp#Y$_b~pL| zR(2u!6bX&{;Scl}%}$vZk2lqEUn<@;k2%;8E$(OffXl*GNt%A&o5h*VMrGyfcM?oY zB`KwPu$LgCP%0%B=%fg6GnFLG!c;>FUa$weG!8Vj%zV;%$Z^&J1WGA-Q&E#y@$_PJ zr+hH#BJ>bU8Q`0Fz(?FGA*y%d8t?SI_112xx0kGijyiRJxXB-elJJ5P zf^mxaCs;|w5{kGB&PCR?Y^;sV({?V%WOCu(^Kc=nqOT7%Hy_Qo9k6ra4{k&{ZBLou z!Lt_`6$ufD6jNuXe@D4W@oZX90Ws4A(%i$Qx8i zD;=4z+O^)XArYz4-oV$0m^!7!yaa(yrw8*x$kBKI}BdfkqC;V*V~2Qei6CE9VG2xnA09#L$3JZ^Sr zmG8_e>{4#X*lNp?f~AYMPOj&c62}Z}E^WC`xct|z;f9lYyOap*8%C6w7*>_v%7xB4 zu8?Ww)B-9}@9@uiMOD*3K7Tk-%OJM<5!Hl3C!Z}Sd)}#feZ?Dcog&1F?#hOEt;SGv zbhxTup3U#5d*M9mZ-19?7<^;eG|D-H8$nH_P%15TP9Z7$2`wJzK8`e{??I1SS5wPh zRZSQ5P*^d)LEH&^S77-|c(oQgBU`$9;M`UKNyf!j?~KqFy#dOuw(ZCFEU-9SPHcnqcg;Z>EZf+7Y&J#u1d23U> zR&QvN&MH%k1A7qDJ(r98oNE%*~_Cz$zu>bZcra%p7 zd}At57Vop$v*F?4o1MYs608UmbC_W~e?5I69GO7zsou$?BN|6}g|z6#?PwYRH_Q05 z+2}B_H zl%$>*l1#(a=B^?$^5ot$duEt0H$2nEUR-qp-=$}Iq($mnm5%TSn`ErnSk8oth$h+2}wCdQ16ilD&uv{3a(mPaVR507>AfId;#b z!k)MaVYmT@^ZTjEv~wocWAG+2<>_e*xNw>-PnoIQ9`LM9JGVjqS*=FXI*y^(mFL0z z292HLPDzJ;@XdENv(i^JD8aJobIrR65>z&`e#h`g6T#z9^9H)JCeGi`rN)zug9HDJ zAwJj@vO^|BSKXV5J$-`ED2Th z>NH1hW_bE7c#YzgOC+Z*cS2}7RzfN#2HX>5z%=dp-%tZpZiHZoUo%jJMY`eSEZeBzBYsA1Hm?L6xOiSh_m4=iVU&(>MnKWBH3c2XiCvVcVk+3_&0hQ{DvH8$ zxqHTZWJ*X(U`Hm2p%9ADXHi6B&+|B9*a6j?2p4)<$;W@$ASqt$8|FxT++t^9W=1*o zZn6ITF`9&VutJ^Vry>tHVE|wNb^Ukl{O;~-`tUtdshEjbY<;0rOz;Qla8r@ibozV> z>4Ay|rP)f4vsU9F?pjeNqQ{@a<-6_#tzV4fVqtg)*uA6SxkoiLj^Vc)D{iZGq-K0c zmWtIJ1Kr(Udd|9|Oy3R2V!h8g-6;!vu6~kNfo7~SBU>pvd~?{Z@kiqu9vrpLYJ;p@ zxu|bw11|CL4B5*W+=Vk6-kd=-4Z(mrpWautbds&sj`_&80qW&0G6qalcN)jaCsa0DuEi(AuxsEkzBw@o6= z#-BI7wm8cm@UaQ;oCh{J(~GGO@*c=>21UHk4@{Hj-vl;(Loi0k=|LCL>16~GL_z|! zHZT+g0y+VU0XF%}Py${RNb%Bj=aP^@Z`SrA0?~eF4Pvf|GfV0~yJ1v>yWa_DZpe0B zAEnMy5Oz`hb-KS|SxmMhS$xCB28M8Y zPj=ydVPsU6Si@paP`R>0=nOQ?Z-+!`b2NjOVw)2I0pew)R4G91>ktAlB0}fM{+G@?ij6A{?&` zMei8fDP}Ou+J$B~SOUI%ivZNGl_SGVUFn`|@*EMMxrK!xU|w8IeecLap`;Oce=dC4 zj2j+*?lG%hd&H6%`Ro!LHvQ6rTD+&>(n_O;NgZP1ojQ9Z_qB+F0WP3Q1K}%SQBs4l zSTT#DucR>f%)3gmATb~SOz`De_1%A)Ub?C4j-^lbcDt?Gf&u!}3gwj&D&R{$^@FqC zpFTa<)BfBm#(Mt--ZD@jE6Bc69pM}6T!@LmgaBT|kWXF%ZB3rf&Ac3uPVPWAO z@C)mgpLhhFgE?+%@Ob3I4)XRu#gz0N`=P zGK(9;>{`0<(Mh3}zEehFJR^ff7LbqzPH|xSDrA7XPuuqEeV^RaClF^lp?9b|W0FnG zrgH?4oJlkzf9BL+Qr9heyauNwAmhTNNB;&fdVg0*Mnl=)|Vy0B*Gv9`ol|6^oIG_UX+0h8%LmzS7+4R!L zJ}JSBnLs!+P(fqYglJZp&~(JJjjwFMJH)TRz(y=<$)Bi&5)JJ$Ghsi!rkI5qmg-R6 zX4~{tCLmUZ_aOP^^0~^~cQaTOegd@8q8w7UKPKj;$)^Xiug**-+rBu(iRSSC4)hl` z-&+dQv1@~hu2;Uz0HQ@%(POAe)n5TDy7T9)#8^y2wdJ^r*-l2XCTchj&lrToO_PLq ziR@fItGB232joXrQEf-^l^PIv4dqw5YTvT0pQ#?2vs3yO=qD3&_FJjnzHe;JM=g4M zrBGy#nz7<=Zk8o3dOFMeXZc4fB7r?T(`VAMyYQ7O_8n-UOE4wUv}DVBN#X&|c?YCDe|s4N zK2_UIBJ9|ZXpjluM9}obTI6#ovnBhP*CSc)8^zVj{qHh&jn#11SUZHR#p;@y7&X(yWFje2oj5Bk23yTt;VRjBb zW3|!6502?(oD;exD5M<*`0@|?UFUd?H#HzUl#pU>E&Wm=k8=(v;_lA*6nDE&OgdeB#YkUcK#Xd&M;8OWh;4Vn(g*MI*`YH?&>e881n zIyw-6!O#7(4hA=1Ysz~pjx4%3oowU}atUm|WeXe2(pGQ+9K*hT= zQA@Uw0$oZAZHLN^@diZE*aMz%dL7v&uLb zlaBMVDUZr8zNXl>RD9W@I3bk@@WoB7DZremvY?>thm*8!WHnGjM0=KFMIs@7X=XP> zl^enRIBt)vco-&ZXh>>olCh0w_;*(N{4T3fWq2~v%EsAxdng86d4PU0wss>c`44LZpH0oEa=so;(L{og!@ zk6@C*wrH(B4-I`Z_!w-)PQiwgOF2F5CNAFda;nrb*Ectg0CuNP=nAtZKWx(F7)!`Y zZdsaBwrY=!R4tTgHA#$t`0Um|Jya#xofPNj3Q&U5Ofo+`)^alBn z+CA~6fV(Zoc^_7kVNzxW1W@0e_~s>9(xVYBW?^Ar_1U9DO;*$t#2nDg1VU@OcV3rf zEM$OwY{&9Leuo{e^ndR<08Nn%P^Q-;LRut6J~pgeI*Um7F%?|nkFVUBf4sO5fVmkR z@U(R9?3j)xX8n~qx!Rr(-|!k~!W!GHB%#dx_6=L*&d0yv`^WaSzRLvwKso~0<&nC~ zX=G@YpWl(QGmFkBRKsfRMe^$!Uk25<2!bE}D?0@b8jbaz`XT*m z2null(^D`(fWw|W&>9_SLc4##|0~vx#wOoq&iFD;O(otx>Q$0F7#HOKAu9D{l%Wpx z)5x;vqN5{BZHwN?xBcAVrb$Zt##uvx37h_mf7j6$&C+1AvMySe?Y&5VMLWN^=&jWo zFGD(CO*uQi3`u*EFJ<*ZN#v!mrUBALU|}XFJ@IuRYr}v<9;Qcoys{DzN^Qc^dLJK$ zkiE_dB=8rqx!4x{UIdMg@C|i(O}K*ggB24zdDy~7#z^#iK?k3Pj70-8*DbW|9+G&ZEJ5A1;U4s zs6aISpR7H%U>zWT5r8QaSKK{2{N(}v63`QqXf?jnnSAVe@@>Ys$Ymr>%wO1VxC!M> zzemF$;&h(dCLNi8U}otKHGJZS8V2N}Xz7bn4Hko0v!VBuGv`l}pN&VqKwea=rmp4X z<>cT}1VC|znwK3jN?}wiyc;*AsD}aHzLZI?A$$W>GIk_jMwuBE6;&_NTbRlArr*!~ z*9h2rl#Zk7uYG=T{>rChAl82^p4_)V{#4`%OX1f;Gjc_+hEBS3-+S zera44S;BgrwL@~E=MOgR6E_<>dq{WC6UtVC83R_p7k!etxbHs|(Z4wUKV*FcSCnD5 zHr+LJx3uKY-5mnb($bC6-Q7|m-64&1x75%e4FVF<2uOU-_`YX-XRY%CX04fL?q}b7 zU;7Gf$Gd=3xsE;H?+myrK#f*BKpds?n}OMwKA)^9czj-C${K#GRvddl4Sclyk>}EO z(f}Mh=q;2@+Zzq${X)!6C@-;&(+ZqU(%9&9R85aEi8GI%{7k>t#jzOR{AuO@1=a74 z9rLGOWf@!d2b&|&cx0~K`=h2Y<9{`!5ww$SBMsiy^%%XDxd*rZB=o!f-7$0NPw5yb zrd5?ZPw<~{sm7k@`|_Y(Yk7|l0lN&$d;DhvS&^=6Tm%AmwLdD(wKLN^(J~ew0{ztv zPmT!TEGsBfv{5YBvIZORFkI(xBU^7xt|rG$1Ft7qt;i+;ljYey1rDqDg6x)4=;C&a zTgdS#C4qfZg0NI;wMdEY>6r}0Npt%A)lTx`Trw5-_RsS8$hr4EYymeNgfC%kfM=J% zh$!oJ09za-`QqsKIQ;ewS>BWhbcmj%WB4N`N_Bp;L8UQU+R**gBS5#_jI>K}hP z%WV4Y#xFeSYg0=OInt*T#J{m?@g{G4JdL@G2xRm6c2 zBWV6~UW3i$zM^tNxyO|9m67{&wVAK;PXOlgg%SZ1uJG@!KkT!)iX-@RL~Gs$WPT8_ zm~Ohth*f|F)_H>6%KMJmVuVkHpS%Cy<(a?qWfIR~<43T){ZbV{Rw@6w9I^4%zE|2p zQj4}eyVTOXW-epS?;I>f^~;0wI`V>F;pLS~lGi-!AEq&5_oKwxwaw<%Mc?C2rumH^ z+C@WRBE_?;LU=D^%WK%=y}bols%&plpG;}E-Xpa{*;dKFS{L?iAXi=aYzkSa)S62f zZ+de*@8QlMYDm@eCii+ts-@w328XT$YqE_d65X7~jWkbuvdz@g)N)UtIAvgt{k4L? zORZLazoOt?#6R)&{ zBTV(PEn-pR*E5nLoUl9>1Od3v5E#Z;Bt)wXRC&`q>KQ6-N6!JLy|K%%zMkg$*8&d$ z3FOprqYCYGb9{(jWR+sdxf)#xC>k2v3SRl3oJo}O^5v4!FrAwjalpl?q&^PJH@zFU zC4`UjoApZScg}KoaOgCR@jYllNO`RPy4oR*p&^s!*)tXTH5psyE%1zx2#*J^Ga#UpCOXV`y#f-X7A@u zGhn4*V}tW_SidcP)=UUF8?-oCu75XULx-zskuIC@STJGps>k?Kb9_)bducZDM;f1_ zN*0aHd9-&HJj6Fg)IG<*9}Hm|ZmP2E3Mhd!%$A+&XUz2qQ8-C+s!ui8Xn2oq?0d?~ zW>g3BBJ=@;t^HC#os7?+7noDR%pPo1d&Qe&^K3-p|E}+UKkvGXbomP~mvCvfZ0+%c zl@zmNES8+R?-{ac#HZDOD+PG3?H8*7k8{5n$x`9pbrN#JG;6LS6cm>$h=J^$!Iq=k zg+)ame0=)r9tmq$W3U&dhGYkcau8XsVJytZ2v~Y~c*yBDvuU~=WlPLrBUf}J- zsz|tYsTAS80Ud-;lFiAGh^Qz9ApG+9Mo&s6%m|U2<==W3p3Rb(TUa3Q3+TFk?hwV5 zJd!J{PqsfILCw~-S_eMWop1Zz%vDPPy!gC-;p4xklBxjKQMX?cP}sxipd5<^z2mWt zae}>V0`HK;pj5E{qao1QSq#2nglN-@onaEdF)9@O0n7|2+N~KzUjRS2j#m&p$!DYA zFiT0sz}H{=P=i`)9ej}>>-GXs4S65eB zC}8_VHowPlEm{w*0+>ORZn{_&4*R%<@@geZ#<0N>HGxWeN)ddMN!k3V zkSF>UFcV8n9}Ew*=xOfW1>PfO}27%x?Vh|$;FO@-Uuuon56=v=&Q^5csG5cMV6achNeGz7i`#v+1^q6K9 zoN6=Rp3U#Hh9WF1EP3breiMy;_0Dw{WJw+t=8xKXvu#KKzk6(ni2oQy$H|7XMQS@> zp52;64tpbO3drjel$Etm%8QCZ1LYp^jA4yqM1F<6#@)A?{At$pN+RNX89<(Xe$qXk zRptmj^}8Ab9ij#UoiLr(vS*~A!$Acd+_%ku4yoA6%1ShB@RyMcAuUs0ohl2-lB(a} zIUrarcyv@jwj!)f*L7O-57SrqxoDZvcePfe12Ip-B`M-0rLtpl8gw%;Yj+yO+QM7$ z8R-&eT5po`64psJw;WQj2t~F--`-lChP3F=xF28@Q}n>BGpE*XT{P?Q_V}|OD>YtB z{Z`j-D+M+F4nzQUO~GTF-ame+6apeRlp=HpdX~+i*Prtds-HSJ_^z>0ah0KXJ@rI=Pf^+;C&J%1+RO^WE z6Y5LqE04KoXe7<11OfNt$77QhDH|J`9>u2i6zytbig?XZCpvWihohH`q@H2K*;ZUB zPjzCVcwdr*5$pKola}PyXe3G`I)NE`j zoLA((1TZ5n7WSM%i=63k-2$YmjJ6skBr=8B_>A`+eJPJgaU^zKCNz)IVAKl80fo+@qZ1N)iz$#YGD0*0?;;xB`Qg+TF z?sJcRGCALg?}acG%SO-U%kr>8K6Iilpv=<*WWjTb4B$-M-TBlvRkKk6(Y8x- z*kgc)>ID=nRAakjwJfi(%(l&KogP-GmMEz{5P}^dg3t@c`^z-QcI{xDR%+rCrsof_ z2peyd*jDp;E>TMLW$H~cGQZ>F^GjjFk>LmoaAw+P%J-wRieE+eW-d#gZI}ra8 zw^Z$mkRNhjD!s03DTQNKNL&1Zz`6&)iTbgpz$YozQU zZ<6!lpJDWpdm~z{uFNu3Cj7E({tHUO&z&}L2wB-diNVjsV0En5R5fGad%*ME7bI){ zS<;j&4Z@{REpQh4Ox|fYu(piT#~gl1Isdm;AsukwCPD{R7S57wA6P)9Vi0Qosd3$i>g~F(ZJ=IzP}Al3 zqvf5+K<8qeWInKNOInzn)ue_jCznA&CG2Gxtk~5h&Xeufc!lT1T=WIIOTT53uzng9 zJhgRUm^#@a&{4s&(z)d<;#qgLhEaNpSd@$YVIb)fUh0V@YUWP1)`Bc^EWolFpoEQ9 zv6#Aii%1=1-yMc+D7$+=as0%^?}NPhyDn^)4ja=K1IY&lH>;F9Xp5u5?KlBpvzf5-)D)j_ySQza&o2NIEE+Q}ztzBIHBe8lGk@4c z!~|cEoguNh3u+DjR#eIUVClU_OU>&BFGBl^*^ELZEJt`aeWX$NdIm+>*c|4hl+QG< zj!~rZgwr_yZ|(IUU)Hd$zCKB3SJ%)buy*7OLg5U|WUIGX1e6G{SJL0{@!jHT?A|4$KD7NBXPKiX?UJ(OJRpf zg@xm@;3aKC1cUE5cE9BetmSdG9l{nhhm(Fu$F{lJ;%>TeqkZBX1s9079orZPRTjO0 zg%?0fDI((_VylTC(EJ`7b%9C1a>V+!wyJ7C{N-LfEG*0gQ{k1--LG>g34$zO5g2j4 zj}!)>QidwVwx4v^RtY;|_OO#Ba#W6mHse%&)LS5D)=nu~4wFdJlPUvk7>C%yAssNS zsJ@Bb9#I4%l85?9Il(S^qxch2`jBn^dWG^{nBhFccZ(pb#&kLj13&G%>3e2elR~#_ zUMp21epma;vAohyj!%tp+6I)5Yo~omO0{WyhJI$w!{bM03+S-tQInDcwvE9`T)3yT zflsWK+$5Q(xwRbjk!7F5CEf-wh;ej;xoRReVcXXqx-Nf?a<5_$vVUtPQ3#!)^#?W)R+D$uzba`$t?X;%X-RyAqX`kReN3Y?2W9BON-{KnN6l<0T?|2@C!@$x$bVR5K1?Z)SJ>fT2$g^F+CxUoB}{I4iv z->Y&yelODN8P2l7^FDeN*QFgN8+jco7E83bf^ffT&<+5OQ4-P@G#b_5tpg4a#WUu! zfT08t$@w*!5GXshEh7i%`B5ZTdWM+{ah3`SMjq9G% z9}4*BAbNvUfS!f2OzjJSt#PkKK}V*wzMX%1mxvmlEN9J9#Cfr zN-hHTX7juwox6kFe$F1%=`62SYmzj$wm)wMjn&n-!YGG)1rn&=kI58h9LY;Z_Zc4a5>hAkIopO_lD>IdR#ZeI>yA zMHgcyoX8kv&OyUNZIZVF|NMghz_e^^?av!RiWT8^siow8Gq+hu?oQ?iDs9F4)HGau2^d-YsOvZo3yCCb4}pc=SC- zMJqcpSiB@2ObaQzJgToy2D+ESYOdeK!b}65%tCr<5dEc=ucvQ+@cKsd#vGgpBw)1hOD*gCjxE840@6l zIiafkTbsiApzy1MW6=E!@E$C1 zK|DV#`?S4CDN&j@?bY01d#^{EqBZqX^@3=ng9bP0ZeJB^b({Qh>j|rc^!a`nWiy6pZLpa^BrRD1IXfuBF&99M4 zl0YFn4BznAwev~c=R5T8>X*uTx_yXq6@tFhi^&?fOIX(PW1V`zec#*9iIYJ`r>hyx zP}%y$+-=I$AAFx=&cW}m2!w&R`h04mI*6v@yhd?YX+W&)y3Lltq1;x5mCaKFdhI`( zf#SR=_A{t@Jg8al}t)zgcZeeu{1|Kz=5&9jVIU0J+I zQ~+;tS(t`Loe<9J=oIEWi`}0HcD6XRZQ(&#Jh+oVnK`bz!_+4Pc*{Rx`&=`#4!*wY z7*zAQvz#Wm?esn@hy*gW*QA+qU(?L^{ASCvj-=KV))N-!hnVqUH0#iHR5oWERy(@f z#`Y;IvuUKaQHWJ#psm_aH_Q*IkoCN{g&0f*eJK!|uuf&JyHO!N5)W2ae#POPg76O$ zcR_Gshr5)Yn>-EsCgy?y*7c925PngGocib(r;nBPy6{Nq85? z#N1s1y4)jUsz8rVJImuMaGE<*XoemaOG_%P!{p=tJY$lN%9geSO+Gf}RC7CJ4D_RL;5BjvYokLMzNtdgTB9u6g-i1nFt6`lTQhQb;LzTp0uDW} z7z_ddkPAG?C#ji=0Vl2}W+E{4fWLHoL2jhHiil>Gs6e-M zuQ8b#AO3ZXe>DibmuwG(iH4dzZIQ{XI9xO0#Wl^Ild-O8?G0$CeuVQjoTX5@n-tAj z^URul6`&#`$rW;`7QrK9CAA^x4rr?kbBeyfVw(YYmzXtxjmkJZY3#;GH)ASesx)P; z0GxDC5a4J6=v;waIh%fiB?>*s#=C6LcJ9rJbx1`^P0i^!B}PU-F96YumfcfSw->RmN!ic9a@!?$EiKz2^!Z~35Lp-d zaovCRjy3_4upvR-F;-FGbxSrWN*<==oT+X3{? zQ#%c4TA0x)T8k;1DNot4hIgskMFfS@NP%XUC-elz!Fk^ggqO*HE&wo|<076&Q>G`GSl@ zs8sDYOv}*D*Ecd-^Nb+9j#KotMls4C>61qpNe&krvN*!-`xt;l(_-yI=lQ5?KKLWA zm2WXMTMbM%_GIsw;8|}rp|~29d}o;$b1jj{ktoZ#PE-VKJ)K6YL{Zk`7V2&b+kLSt za?RY!IB#{(G~)|K3Z?BPHzIe^9dhnF-{CoyE~>E2>qmJWZxTpY1w^!3t)9PhGYflGSO6zE-51; z67YjV30obEE}ZT$6QyPsk(U5+r1ntf6wI>~8?Egv?61Kw%YrTP(VsupG8cM#un+p3 zBO)PrOzkg;(rQFr<8ZJg(x)irLMH(#+j3N6wPQ4p0z{ugl7XJ^%Z)o%U05|O*>q+{ zQc=<^e%&!s%fkfGgiMv&?)K54GDP_27!MYW&SVq{E-!?Mq-8{rFff8uNL z5KsY7%PdIa0frUka&`)1LQp%c3*cjbSU;^0P|EdvZVL*Z^$q!s`lLpI6PIpx&i<&L zW{?BjA(2s0wF?LOtJ$~TFv z_S?gBqS&)WH{ZzK=h*-}zKR+^F#wd)Dl#f6Y=G2PK>~0t|E=rE9)4|!0pAp>6@fh1 zEHS7hI2p&W{@AHVpy@bqhiUOFUdEzijiJ5ojL`K!;}eur8^6%l(6B>K3NM8@_s(-Y z{N%)en2fCG>u(k~xWJ*>p2~!oalg{_fWKa5U;~3}6inVd`fSpv*F;c34^R~e;hrwC zJZ|jBXlVoR6q*d3vZ8?W?95b~P~JCBCuJP;r1oz8nW$0OxFxgJ2G+Ro_anRX2I7oi zL{+0JtEUq>DBWT|woC2{6)tD&^4^`oqe3;nLI7 zIeB>rEUUW>;>UHk3N6&7{>YSmWPoI9sS#!dMb@98D&CBpv1aLL&nsBvEQD-fi#sTQ zKxNlrDEcIY{(edNt24$e%}6U1fN@bm3B2^q8$D?(qclxh9bC9E@N(plEaUs4fm7on z7Ty8%f8Rht9AqQwps!9Zpfm~Fk{U%m4-esg&&>y$D8gc6UWk4SJ83*a*sH#?TD;pt4pdNH! zpAe@a5piD*-B8vYRZAZ*@FJn8sWe@ zpj|L<1})ms2{7oqFK2--WZ%|3$Vh*X*8h5BcwBxgQfdRJ3iNq4)wHo2X-%&cL8o%> zEgnk^sAtHs((?jQIh-m6-(K2Uct>L4?r6Cjd~v?M#GQ|s*ct()B?Vo|MYGvUq;zl@ zi9EW7$OQ<^5_t^&9{tq#K3oyMVsQbz9#bnn)&ytoC)}>=k2iGr#qD1WV8+6?0G?FD zYdGGvA6tiH(P258)KBIG3`E$USNm9*5<466KE8zKI~>`!bZpGwBsD zK*GI?uLxFGR|ir`(N`yC-HYolw9n$Na6=QI>@0QhrzRY|AXG7r3B6(pg#jq5-k-b^ zcw+LVcQimz-R8G8GEe9&mj)^rm{3nbiKEt^q`mfV$8F_30MJIl!mTIU5)kMn<@mw& z(R>1Eo22(zbA>p6&IUeY+ec>P{|2M6G|+An8nVS1U02ElbD`mFe}WC8U?1trBqpz( z3?<{P3s~Ypqtyu^uEiG!r;z*JM?5%ay=y7QnY5e8F>YCTj0Nx476lymAW}Mwj6`el zP#M@8Sk>(~K6qO4Wa%DF`Km@>ff_gd-}MLS%((?&2$Rk2|K8U~D5)gmH|v-fBY2e- zl+<84&YHAfrxjUManUZl&hF^wphaV(o6CLpVVjl~wVwloM_;Qb5~>L6BeN+?<{bnK z(utYyuk!co#3$K~&vvz)s;>*eTml8qd;A_t)zj|jU06-dDU0eo~k#6MQ$w%H%NH+@o7 zi)Je5tDm9MZihcl{QygZ?`vgRRVIyWB5kl+@$UGq8B3)vilZubE={+W-H#n=E)Ovs zA9E_qlI50_*r_k)8Tv>X`rl%;|H;*WuY?(#LN9)|-&){h4YNO5g;M&Z!Oc(AXc@zz zM3n~ERJ1UBjrTyFAe5hBkmjy#8dBm7uUI}*ClIk7Na90c=UPficFGj}ft8Sem1$cY znlv%XF7*zO4JvQAOqv1{#OSD>h2k%O)-)XWd_~o#o1Qk3IDj4eoMhXqYhT~&RK+>k zVYL?Sx#4YnhncB70llJ_0Z@xf^UnJKI7t$KS3Y)@NAoF5zukaFU=1bSh{|B>l5yC& zny?FQO;351`hDQjv56G^QhGX}BohS%MW2lIj~2(7C8Fkm_Ea7HBI>}uJSZ$!y@_MW z`OJ(H!}h;E=&CZGu^kbNlXBx*$qmQ_@`Z2*4R-ox*3J?twXuIHBWleig;Zs0B0r}6 z`{zDL{#%wT-`M%Rkxe)!Hb7KqG)Hbl*8{$BM^{r9{Ipe&G1?_ZvR7hLR< z8Eum`)QK*f3-Vk8&9^ial-$zecGX1d8p?~&{5VENK!6(60J(s^zWbb_#|bhP*$SHh ze_vZo z3$l%Z8V%c}Y;MvdF4baP;9KjhqC81hs!~<-e-+|ySL2mFRy>e7cf#(kEw8L(rP_ue z|I-t&Adn`tPf0UrcpW;1MOH^3|GggvHXf!T<6rGi|ExI-jV_uib8DO6OfOrj28e*K zt^4UN!G*JzGurr)VdU#GEix)^$abD`%p}OrvXKb&_6Huu`~H*-w&umj0AyBaTU+Lw zoSX>We=H!$NOKE|HOb;Gqz>~C64N@ex+{U!=*BL*2C%l-ivM=Z49uaOxV`f zyxa6^JQ*NB!X#Vfj-FvErdD9e)CUm)atY81`S}lLRU4tyNb`aKx@^nI8A!*M2?5WY zR&&vHs5YND?Z(pPjl1|IgX)UMd#?GzP{~wc}h&ReahlDPN(B! z0nLQ0%$g!-ASkY{ZfeB-I%goVp2#t(y`VK{n%tJcf^ee+YR2s7OObxl-pTznSonSy z7TH7q@kH{YA&9T-hqTjARDFFR9bsgT88JG0b(;94g)l&A7sqwo!kIG$z!uqnM^Zy$ zW1BjgYrFiOwWp=Q#V7MSoxg?UZ3@(o7Uxa&%qwnI#C`sZ8fl`-|?KkVfCwh_u1*|IRd zI4$~U^|pUd9w&^wz~DxE{8<+gaFvSK-{+gi;*s|%;NjLRS>O}dj|OD#z~AnNpp=?! z?h(gon|_Z{tS9S3J&`d&D^LS-d|*d2w3)3kwVtHwi_=b0h1fV#CmP8$T}LvjPRKCT zOZ#2vFtfd1h#G_}wU%%4BHA+XK@u+HqA8GWgHaWnp3eRy3^Bn6pX8p+vVl*}z#wFU z!UfnpY&`a1Xl;HnF=C$REdcfcM@)?8IkJX0`5f!y86UcQJ9_TL-`n zVaON(kWAGD%=#cOB74$I&~rOFEgE&(XkIzTsxXAMwFv@bs>o51-2eJU=>;x)IQ4 z?>~EBvbduT*s2|gdHv7uuDbnp@81{vOG;k{ z+{awJK>KIId4=rGC%QQnR#wE3XJL<%Ai!!{Q#>?qTtz#Jfiv~}_((hdIvPLIpbe4)v?n#7LHcptUCMgzyO8ahEE|nk;m|5~cBbNux~UiBkcOGDH!A$vdAoo{hx7$*wX6Aihb$ z(-d{^7$7Z{(fVD^VF?A?j<`=nu(7#P^4ts`bN=1kM%Zv29;dj4?hWlNE$wLTQ@>Y_ z;#H2J+_E+IkVP*~#|x2!UI{t?YPSd|f~HG207k6&9XqGc$*Bn7iyj6fx)tA@L+8C_ zNdP$>e*8jDPrp@BmV2Z#T=}TtXY7HxTimfnzF`j8t>XvnX1v`iv+x||7x+;eil}~P z`l%~R8`q{rYJZACK{x=mU5mk@Z`Z!wuKVz~q`4I5FZQRxF0)EL2yP-r{aYD1d*`UU zctQjNL&FOoOqD6(Z^mDSLScY!MxAUsVV#XXnJSxR!2tsx3#DKF03f{5+xz5yKXr{0 zE)cL+P0Gzp0Q|ySPnsQ>ft@4q}RT}L3< z0E>JmmFNlJ(GCOALikuK$nD}Alu$NNJM>H)<_3!S06bA?D81Fnih(W7!O_u|SQ%an z)g!caFH+x_B_U^khqi`|2Zv|f9G2f1;qd*y01fVouu($PkW6W_4z*9RU`Y8_YY9%$ zM!IbfS@b~kkb<#A__)Q``FUac_?Rt`onUfl+-SM*()|2jSzbU)Bvk-0V5d2>Skoh& z`dBD>9u$zM2Wz*{)!B~k35T%uHV&Y4gd zU>_|rz+FWT&(2q**1JmM-5U!_TYXup?|>PH3Ak>NWU68r+(`R?|B1KnoFmcviPV~@ z(|_?3ZGgN8+B95hnSTGr1+S~m{Nq>hCaqV0E!4i23SV0Ib~u|Uw1MCy;u+( zHFa+&<~fH{Ey}qme3#T9ItDEq(Ao)SaEY0@lP}w77XU_vl>*pYM>cQ1?>60%Vn4xhv?voMEnxSDpAK}<+g>h%tx}MCIc$t zA;2lR@y8c4e?JO?l=Z%fc(oZI4W-*pU$QhgmJq@DIaP@}UJ@VznF5sdkO<`O)@tqg zGl;&vz8+&xde{sJEP!-86`7ITA*JiJyewAD3Z%c?Gxv2sQz8v!o^z=8Io&j6UD9@P z?j~rvC{U>I=q2NLoh(1@&?B5O1w3=;8O#otL;qGuKexR62_8p5&&Jl4mfZs>ogDJ(g6j(-;gWg$Cs-C6cSw#SI}2lrq{16>0-2mkQC{Idz_OOhaTP2A&?xK}NQffo zIP0K~8b0>mrfJx}_#L)gCl>U)TB2~iSOam+NQP{HJi)pv$%el41miUW^TJa|%A?WV zen#QfxA-PH>DVnsYd6=@ml<@9k2k;GrQyXsqLJ9KvL3GC{aN7zl-Qzr(;wktI0~Z` zzU3uoeV-fIwzQ>5CckH@N{$}7W0ViV$pnH!^0W3252Ngtf<~7D=}cbVu+%4m=k#FU zO8uE%1@=_bHKcS$YC4H0#95wSNLtxS`<5({T9!GWga?! zBpgw+uo4xmDEBj&iMkKI;uR*7@TFL?&7y$OM#x>J7d|h+z-P-f|2u2RS*UH-ZE=BY z#d?dGRAD`A=k2O}jtrhvN+1+E<%eqI)*Zrtf)Bj7L3E}T7IZSO7No*mC5!IXuEA)C zZL58&eV_!aMziu}(v`P0Hn#YRh@!K%$(pf`dyhj5nXeP_5$(9E_tIhiGmuv4s@uW8 zZXK+YZ#TPj5Ec*;GA*sglHyh_DFp;8F6)-7bH-ZuhoYBJG$A1=OsmT`+43LdxsyCf z{)=Au2vI1~htvzZ{REb#3K5GD-1H6}(tEji(R%y9+WZ5Gwxw+U>-1j>FuMfiX=ccC z)n=q4$h)A_oWv%WhcS$|1e<)q;7iJ?zpWNHDQoVnW|4ea@w5^)WwZR5G_;Aodjgq4+Mh_IEe2U=G z2ZMGOsU3)$S!DCW2!?@1Y47{>03%RLTi>|RSY*Utv7)?A1WM?9QVD0vl|;fv&KS@3 z{|v0xq=cRb&DU@cO<&yTjm5#L8ai$%8@;>b4$s+2X7UaEYQOzyD0PKmx$v%&m&pI zQ4t`7!eG2;h$GlF)DQCF!$wCJi(}lM3B7@1;Z2y{l~LoWF3OFlu|%}9ge9vq2@F3G z3D=Iidyh4Gjsf4cQcQMv^2P-~2ZjS`jZv%m-_Ewb@6ip0os)x=k`|VEOBR;&Y1e-B z?6a9j!8-*gmSMH&*lXF1y)!1G-vNlki(NbCo%nFe_evQZf2BI(w1f^%|GP_q?4W#Z zU?Y$LT&u+UD2Mtrl5Y(87AN@ejBn%FGN{1`*I3Xk?{Ik&K!IlC~ANiVpMgPRk zjCLejwvf)`jhX9_r?`jHR^>kIWfs9k970^io7kB8Y9eBsO6OvM^1omrcz^dFsdn+Z z_2*+w(h!ERgZ#M1d|8yzXTJS!Xz^+E_sQ5S{Hp?}0NJ+ji>h#zI+v9WKdzR^Xj4Q) zl^1gp+Q*NlJ-VKoU6_n&4ukjEbaWuXX5!RJOey`?R`kp_dNuS_4A*0R`hsIe)5*WH z*{}ERsi;`!Z`+*7!hpq(7m(=F7ovPV2#jLDkiZJXd1`p3MO*7AwKSmR#W@5mLnvX<J{V%;! zX!>IupL;miSQif%1mguq3h@aDh5*i8G~_RYENj?U>6=4^jBN#m#jjuFw=se$Do(u| zq&^HJO~ZJ9-}{TJd~#S|Aem13`!aVNZaEW{^-0vB#=*>9?&Dz(o;rDOfyDnpl9;k< znEJe4Q@ncPvn249c~UAr9_wCGImmF{Oop8c@3BnZx5QKzHtY5{`ZLQ6Kr)=^Em6*> z1K5atRC6};bK~Q{jZo98E*{;ZtEl(}ihIuFc3n;J3uKjEza~G_0*>!b(!z?;&ovdA z-}barpM_snRY|@5g0V`dj3c=^I<|77K*wl{%4JgY6`L$9PR6jyr@3xezuAs}aV||` zzYJPh523DIMfTnIHiB?PSi$N0dC8#T;N@$ndGLGzJr8L@!H~;kU1Tss1R-Tit@^Wg z!}yVgtk_JhzM32sENoqYXIYALv=o@NtSy3RY+S~w^pT?jVJ0goEl009QK66;{$n}w z_hC8YePu@C*WyW9_Omh;^s70p?sa;e^+Cdnb8_oTwcmq$Y3G)eE6y4T1fHoN)oipo zf~yysAEh~px>7OT4Hm^UmOJat$avC+y49pp&O;0oSj~Z)uDipc#I)Ee_k`YFnwg|T z)4rD0|6*2sws!vi!K}75m7PgPxGlp|$DSQ@XJycPw|DHw*P1fiksz~jJ@!x8Fk4;zs$WBPf)R`~u$9k#r?Lu7uu0ZQdJ44ZV+3glF?<*OB)YMpCjQ=`0N-#^r z{d01>X1)R1%Ve8bBs0KfP#}ws(|z1<8#wh5RB%eZkO6fsb}pc9_ks>FVZXoqk$l^O z6hmzlso`x_YSyCeG~38f4 zJnyJ6qeW@lSk*Hi>;xUpUO-m&C(p6gi4cMvc|}=HSzZ zrb=+1JnvSASe9r0B~EYhwbZpsv8`z9*S@>tN!dnijlc~xom7axVf$?{uMsWt7a)Ql z(mHN-`|APJ24GtMxT>`8^PiS-4=<|KmFEA9No$>-em5wT{IJl{kn-D4vu)Aqx^=GO z1;K!h(yd<61Wbd#M@RiT9IV?ve__z^8YwP(D(>faU^Di!#$$VRp`W(+M78sMw+vEj z5xo#>Z&pR$Sy}&qW2}cr2UCwLN9HzC>VToOGn5;zj)~K>qz>l&Uh@r@f_rD4l?Y&Z%>GNkC_`wZf=t zi1Dth5k0Qsqy=p>!&`GpQf23!r`xE<8Y`<@2L}MyF3S)8z`2WU*6F$Cimb_eSzB{T zyKQhE`egj%{ac33#> z(@?-JTo;}B`DA4~Le|}CD_!D}{_rOt?C-efm;SbW!jHT)6v6~SDomadFbk~{=tue= zq-vBOr8dnk3IkGr{o|8NNm0W|Zl3jrm7N8}Bc4l;Bu<-!Y+LA0P zBi2qP`#+N0Y{7qYj@<-HXM8Q=z;jU2!a{ya?nc5#Of&QOd(1l6w*<{Lw4PLeg&x|~ zq?qE$Y7i-Rs;QG=0=Z}i?-vE|7p}rA5YN-f?FiK2>J$kPv4S2jn7IYc$nH#mbbrDb zfPFpCV{w2*a9X`CXrcHCxp;sP&H44O-TFTRUs|0v=3+!V`u#Au@}!@9m>p}RG!Jzi zRp{Z{Tnpu}X!ydxT%w|87R}`Lr4;?^l->fG?+~o}e41b{rXJ3DRi(8WgvYh~Ie*Z> zzEh0ADlDMiAXyaFE$kkZr>+F3#ufCjUxo244FJ}4oc^F}*}L+|+X5n$d|;-1$?I|t zj;%EKaaRNf2Zx42YGad+%w0zvN;Bq_4;+t{1|CEvHKYy~tXz;;1vk$q{sH)!4UkL+Je~cFFi+9ucFenP~j`zmYdh5j?+3 zJBW1LlXsI6HSBK%kh{pLVLRw&#a383v6aMdBxo1+zPoGRMWvkxL%{1ki6E)3XA>&s zz%o{_%6htf7{^depSthzaS&Ut}a@ayJ;f5P_|We26yJi91^DeeJ9gD zIlzj}=xgyn9%CG;S~%DWh0oi-bic&3cvu7*o*`i!|LYV&>xT4p1uZEN{aqjPFj>l3 z*;crLa?|ZZnZ?+S{P~YF?%U_|ks%B;v{^;2n&M&vAYptUi^m$@eQ$R+bZ}566YwIu zg(%qriCULCFe*_yd9vOhN}OSpuTSiiGNgF@enrckX&Od#uU>E%&|!^NHY2E=5e9ki zcldvl|Ihm)|bz)Ly3<4BiaGk#**z|Q)L11V%H>V5gPV%$b?K_K!LcIlJ> ztfDw!&y!_XR@V3j_2PG)=BKilk0xA}BOXWP@FBg%ie zdkjhLbuGwFF4MOPOrg2I<11uy&8C)9YM?beB}llrCs3)=^k;%Z;8-F$kgE~9bYi$0 zwKdDpo(w z{>6m|tlRnbnV-lJ@T>nnw%$6bs<`VGmXhucN$KvE4&l(aX zyi)n9&=Ktmns!H48#;GoVWgEo^GB$L*gfBZ=^I#-u`*+|5<=ctVprl8!FLT?&WDHD zmcylbAUI@BH%cN$+ph<2zrS}#%^x3A(*eE@g@`L$k}Hs;4FifYy9^fC&GV_be%QbR zJ8WZ14SUp^UKnMgb~i#T+j5RDf!$GOHviQ``N5nYzH^)TU}Rpg!~vM)(0?cPj}E zoS^YJd)c0*rRvV+OVAnKN{Qgi%5i)S)3iFC@m$U;_bVrq!;(Ek%_+l?3U&N~u0U#e z#F5rlA}wv#EOZmo!&~e0e-;V`r)_=}TiC@Jhx?!i>DY!?+g7?*;%y~>Gs&W`5BgRN z>T1pw-WF@f)ximm(cvBz{h|S|vzX~x%7nmR=p>x<^fq&&05 zVZ$L0w-OtW&cD*qudz`DzVB!v?#b^-xX4diCFFp4=Cz;Izgb=aPA%UlX^2FoDsw3y z*u1tbhbtL_E@Te2dGmzc0U){iuhgHN3oThCB0G{)V6;8C-nLgb@H%| z?p)khp`O-FXD1HJ)A19|&e>H`T}VgAADyQ*01eh#$>WrN$C8zqkkYwOO5W_s!zId+ zuaNfEUxF*~@A}+#T&_h_R8&Oep38q6|0WV=BF8?}$bImIN=8=Fc1Fi~HA)1a@GGmT zFj_@YeY8*cI~o?Dbq4D-w2bd%YS76#(NSEIKQk%Kq?}T2PWA>Qw2l@L{hUdLqQl+nZbLE*N+!gAC=w6^mN;U+-I+ z8;5IZc?5U}<$G08l9k|iP7^S(Q^H%3Q%BKQiK~2+kjI=b?Pm*Bf=1Q>zYsS?(02L4 z7EH{J!7d$rlMY$#u%?-5T5&iY9v+R5kP5#66^|Em;CSYkHdEuxhJU&zit=-I_;8Oz zfF9`ufyK&J`<6Q#p1yuScd>Zgt48y*VlYiVaIhRU%nT&Y=}p$zYSo=6Qv-`zQj{1W zK0Y6{M$-2!@W6<$QURtxy9gO)CI8@xAh0wO6E!ILGev0J0BBehoUTer3jzk+3q_=A zXZOAH<(|FsWo(`Ry8^JzPcvo;lsa-zuLZA6_VGiFX_N_4GGz$8DV_L4%=kbRMuB2+ z?LpicYmP5;Y4rEpDP?-y8E{)Hivsv-n?>UJ(`l^)(M2guDnZ=sEaAiGk~AY*saX7| zHw?*E`jXi4wo%p($x7tW=G!Po^nLdn^gkAM12?@TB1@(j2-0s%7=9M;<+CN0^`igO zAs51z$NQr&w#aa4Jfc5}@0>$a8r@Hb z>;xCLs7m6D=f01-NV)yUo67^=OmMB3ij0Sj-+3=Acz!shDh`{fPsT1RFP|UU7+!I) zn=jSrK5lIB=#KQTGd7;Qb8WU=aG*EMKfm(_G;lx@B(t3Ji&0J`3fg@H+aA%y_HQXc zup)eYo)T&Jy-XtvUp6d51JsRvd|N_DVLUkVNDn&RP?t3eHBLwa<&h?bZCZ?MmBW?G z?7bDlL$8`=h+u9P&Wfu0-9hznW@H4%!r~1$Y0cckXFP*n8@OivploK1m$8I@Zl7`A z(K7GSiR1oGBY!0H*gr7u^Gi*Z?3_;g^3RtvvKWfhyup=55^V8=O1=~PWB*$XC(M{I z=c{j~5hsGfkHXhR*h&Hbk=*F?>0p}LLM1nB5u8<7r}09nF}_D1 zdoU%_vYk+8uz8T=?Vsw|_uBeBtS-0{2+d&wGb|CgY#l?&bHerCE z8QzFT4aCUEh+WO=!fSl0>LHSMp-wWYKMlmfQeQ%9xhzpfrgiZbvn;I@P+VlpSp@69 zR7sJs_K=KVdzPi(e@ztNifW1_eKaMEor|;@Nm(*JGgiwOZHcX{bcL*CWWol&gQc2+ zX$>yNYrj4&2?-dQoa@lI3EE#ayLnzPKS(hHrGbhw&|}Mk<7gurOf?bV%3~ANz9iaJ z5>^|R3*oW?Y`7>kVto9(q%yN70ircCf|gbq4TXZ1#n=9+Wmw?Il}eVPR7*n?zNu** zumRouhSiR7Sn)2a8xUQff>xk*>M%{qGK9D2`xt=8@j2al$VaIBYu_J+&xv^#wRH_8 zQGxTL^SxlJCoXeL1-pEHQ=$f!t$9WL!p*={FHmtZ-P~@ThlREMbcknZtv{HPR=LbF zXlT2ifoVz-@(WF-L~+aqM@J+2w9IEG4UgwQwjQNmy;I#}s%tt;ZMR6?Q2)tiD3{4^ z?KRdEmy1P_ng`XSI(v)=c?MPLXHE@E z1rxJLL2(9LW7{)IBDm5%SHHM}SsB}GM{75GY&dPJYfR0w8iK<6TP%@?lljNn;iS0y zl6cpTL!DVaZkE_!V>tZ{$_Bi>y)UGq5`8A7ZRpsJrt_k7F=IR)hMxPwBM+e@dfXfJ zMj^pl>K}*IX4JLz{FJ?3(Bt_)$zr1lOHK@#qZDOYz7or|5_4dFFF=J9broM|1cm2U z6vT~@+4c9GMP2#F=vJ%0hYb(l!6K(nAhwKo009s47#pU#Kp|9{T^6lFMkmt{)6Sqv zu=8jZ8}X&q1U}Q$L)i&D)=aI1cTp|_Y#LQQ(7jQ$w{JvhI|5r~D~5MX*;;{1_DbmfM_o$?Y zy3*~0IJoi(p zs@9JFM*bGr*WmJ;C>oUz`JqFaDnC|03^u$IhDKC;8RoUz7$zh&Qb%-R2Rb4sEE>3! zNXk;ZSYCf)v$71RL!RH-enZBnxap9&C6c*-R6&0;g9M3B6fkGreq~_NGwd#n*WQw~ zgnzQ^7ipv$nvJADBjy2_@LwM}a>%|Nl~Gr)27%Pv_jd|JE=M`FTinail9%fvY5m!_ zlk_mwEQWL{4DE#^7=IT5mnIPF5A^r;8V|-(eA2Lu6ktDT`J4p4magg?9F`+9vQr3% z8j<1i^$whGov5>*EX*xa%%)&J>AZ#kyP4pjji4d_K{Q)R@VYTK{PjWv1@kvwFoUI& zPK#Brl1B+M{ZR+2VRLCh>2$fL5KYXB2DEC_KxxscFkGzmM$WqS(yUVQ3W>7HJQf`C zs4C4O6hzAg9R#GabrnXutK1~KFQy(`K8L#@6GSfhLX{sn8c>2;5s$Cqj5gm(vpMUQ zZHul2$~Y<&d!Cy@c-)QC zjMmntYO;EV)v|h2g>vRdrJ%ZrEkI*CQ6bvQB_~qdF)omd%baPBSySm7QzLAJ14wT7&Du|`Zo9^FiR3x zSA9k4F<;{I(=X+6ZQekmMpy}vVxiMGSX(dEDh|4ATJMA&d&@j-A&d;}JLpEhig3dI zfNt;V5}L$O-HiH-|CK#B!a4~~&t#7)B$GeGeCFtLo67m1rCyPg>fItCrDYzika!Kd z5F5&$ir-Rps_&pZvcy(AAHB75Q7N5RVb%M%JZ=@4Vwd66IjLBXk&%Do=QC^putCoZ zjCnn?Eoq>xJEec%Qp7zCq(8D(C)GQ zU&bzVc?=U*h5B;WblBx>CArotS-ek{YGlGs`nUbdo;DnBGTo&}6hF(G^D3Rca&_fA zzPIU^kFtW;&jDd(uLfkklapMzIj){%l2lOjz1%u9!ibPJaXFcm|Zr6 zy&7J9yKIx&oJFzbq(+5v(?Xb(KZw|a`m4n29x8I24o=3?BbjdYz7aSxtzomQd5U6b z-9^6O{(;O0i%CA5wA-Q4cvMwVn7;u z1Za~mK!O4SL~lHQoLb-5_RFE> zRyS{I=rPK8!06}&=rV9Uetvs9fG+a%>U6DCx3&vQp=silGT+N9l|Ie$T#A)R=OV&X zPfzny14PU5DC$nwEDIQ z;y2-VTIx8vRU(;!DtVrKeWa@9a9tTO?|}t9JQE&17S?h0is7}hj_MB%v$JF7Cr1YZ z`tk4}R>9bmShtosKValykO%bj7eY&0bVZ=JUI8P=1zER5oVc@?z{AIofa37^0wpq_ zeK!su_3S91$P)C5%*2P|g{MWiIy08Hy+U7Rc$z6+uL_c~B}386%eF8m^?|w9T1;?L zgEgNccl_>i;$}W*l5AOJSK?oBdiUt@Tq=y~fM@Y*cc(kZM6k2_nEIHpuvFNv9m9-p z?&o`A$qg1*;JZ0h4O=}`_`V_!R*hcs0R|;UW$}q&Yb5MtrKM@qI&`>-4Gp}KqYNib z`|6|>jcfYs%-ctdgYU>+9K3vnP@M2R>{)}vbS*%3P) z)As_Xahx4W+I>1CYB%@ALo#JTvsQls3!kXds8F^lr=qw>9iKjf=fKdTyHU2=+-z`l_1ASF?%QnUnlLSQPniQQdXD%tMhR;ASu?_U}__a{6E^MT? zJ`08ujgo$bqqiw$aQYxmaIsxrJT(u^ZJ#9}wdxD?7LE|%z!5es^}Mv`qvZXf*f&J` z*0U4jX@RI8A&K%ykiU<3=o#-OJKbyGYOta!Li1T(#7SNipXX5St4DZg!3h(ra<(i7++K3wu&+XgF7;?Ti zt;}B$b;xU0>mb05%6*a+Leda)^dG3UAb@X|Ha3T#!ZjBY1hYL zf0)|sdtnz`h3DTfIPJuv-I$cdXcy9(!SO-zu4{QHT_*>QO85m@6)(2PZFR>3s~zDR z=WESSZ?=Zp-mllAX_SI*3UqMsryA!F;>`ODcx6Gh7(6HH6xj?pK#xo+Ufz_|Qtv{N zBE9s(-dbe*Vm8ZDqfuYo;cA2Wo|}^)C!b}b!L`8}zKd5^%etpD`ETC2;9l;(gcKJO zZ3cEHGyBweuMATXeUpD~YXqz!!D*4tm&*bP=qerN)PsT}Ibr4Fl7v1+nn{^m{i68! zC1a+}le5l+F9C-%g0qXWs(cjI1H2pc`7#Twp_5O>PhWq*wCT#9$39#qSf5E!1J(Ap zq}4?&X*6@0Kue3t#vNbEP3;7W&<6nD6V<~crzk5MoNYF|Gg#-mP%yi(1=E z;4J1BF3VVIo8JtbV$%?<@W{qHuH+0lpYW&lTO8Nd)MrZVBUG+*V`=SC!_0g@wz)86 z>tB+aiuIH6+t%uGaKddNH#+I(AW3AD%oo&N8AU{Al?*ZV?;DEmT+J-6)3UPald)&rl_dY#*u{Y;Z-(Hpd1U{N|9-?A=g;PuH=kjAHZw%%u~H;vnlzrK-Sew zF?s>g4A7@34&--x@IJ?ZaWlU=+&;_T49;vJT_fe(#1T<54p&JQ!k z8#ybTsv-EmCtEm?Y=LY_w60zKA>xUIy8^g(dp3K*LYUlb&~h2xP_O~T9-Djd>r5Bv zeH$&SP)RmSx_j1p;J`6`kvz=va?69jE2 zj@P$2;z8S8xaF@c81?e!YQV{omJdA~^DrdkFvXc?2oC75_xWec3%w-Q5!$@=hjxUi z-Ko3%3y~o3%^2@`K-8rAU&volR1l$dWPIFW0<{Puupd$OEfy$)Pla!zqT^XSj#QXP z1Z)HRpcMu599@cjKw^j#=b;6RE5)GY5)kKrLh!lh8J+@P;%uVKUmN->i&ZhW|0UUlapJo~7{hg- zCU9>1Gyuf_5yV~z%ZRH7wJ)uu^o@2BFY6fFH zC4l5QWB^o)_PRd1Jy8F32T%&ICeMJD<3k@!5CGSD4nlyQ8*D}P>198(LU_m)3gn>$ zyx6F;bIxOfKO$nT@Dm@Zhq=GOuprInv4BwLHOsz$)cb7?H*CVq0Sc4`W{jT0{?3k2 zB4t`bLj!q6LU=x>qvPrugpr`JGCn&Z%_3WgEtRmc!8Z$Q!Bqz$smI$k- zMhEXzSmZi~H_fZu7HM=j=a;WhiD0uRGLSI;M&Grl!1=ixDrf+NNB{$<0^3*;{{8z( zp85VDgp-nk!=VK$OM zTTWz2iHQYH_5$J6m({Fwoqkv-+bebs*U@GKVBnn9U<1Ou_ zHvE`odfy4Uee8ZS9XJVq;DB(XAC^weoLK3VLDt_T6#**=@F4lPLp$s%Il$@7tOeMn zzX8GI1#Ba@AGrav?U2=4SOenxZ5%s(BDnoc?zc*rWoAV4k`zHo)m-QH<&gw9#R$8TA44=yu_xF0}Rr!)sCy8S+{;CU~t(#HOh2(oEfGwGn8 zNWWz5CLc?gm_p25I<$vcwvb0Ic}|%utGFBpMRho~6Tb-D_RK~u7(zgkgKG38wm1>+ zZ7|;i*$~7F!gQtO|A+e?qfmaFG(++^qj-J|S_YQeuxr;7bIzT0($~;RQh7u|xO6M2GUQUKMfWwL(ZIIfd^ug##hJCIZ zk+UZ?FZtZ<81)+>Iy~~*42G12qhGdaD?%Oyt0B=xgwTOH29_q-m!fa-#Cj+p>%xi^ zIP+>fP7C)T(+(ttlkwpGZ^<%AGMhAy&Mq$9?Y@tbO|F(*SY_+WenDXOvNPS>KM95% z-G_@~V(+11_jbq|lymrQj=wY2pT;3l9q&AjuCn`wAYUXXWs_(e&!d8OZb>8)Y+BlV zL$=2aa88eR_Rof=QTP-hWH!cGH_en3GhNuO^vqv2RP;TJgRa z-m4QV!B!w@r`6WJM=z+wfphye^K}q~$8u3_ETHfJkgvE`_%nFSx^MO4_44x!!#ETn%w;BQMNK1+O0PSfOav~f z9*^sz=SjP3VSL0K$4-8kq(3KZaJ9A#ah%-%2KAe=C##$d(q;iahD3K{+TW`!_(0s= z377(inWlv&B>!DifF0DO1iG3flR|WGOB0HYEuI-rz2O#0G zhmKM_Xk&jb*wg(_2y4syQ3xpiZn~nu2%o1Bym!bz_IvHf$N6xNzSc}c9y-~0;=Aw$ z1+M2NdO@1$K^Zd03I+1-pzJ9`#nb9Xajp~op25-(jD~r_VXc+^s&i$91h{j7KGy7w z+3u`;&MqW2s(9T?k*jpS{wD90vX*y$wyveqwl!_aD|c^klL{&uJ1|#1NlXdaQH>|g z8KPxV%n0`^zEwbE8I4u40V*69UZxD)zSC7V@#GW6t+ zH|yEz+6f(l49|gyeaaGil)McNt$_AGb_pEh z!Q1LHz?N=$;5+cX_A}&DfF&CZ0c%5(iD>?AUJ)SnDD!`LhQR!6-dDeT$n@RVw}PH9 zzG!MMNV~zY%sCJOe7!@)w=HmYR34Uyo#nl-0n>MM#QD2sat0BMR4LUTl8QZV-miTq z_|gj^evjN z>$yT8O7(?mVseeH51C~34~_^O+{K}wAAJKPSo%**;XHPWY(Q%iGBA&Q+IWPuz9d9( zg3}0eIDWtBSTeyy*7t5MsYS=TgjDJ0-$|sLbOAQ2Er0zi$KU(61Og!?&?@x<0A;`c zsn79qf1YI*XhIe9w1tEaE{~3twST_B0?|RZBKi>K@k@d>!r>>+F?msEbAFdCJY$-f zsT`Ov@ro<9ca}#8A5Pgq;Nz2%XjKItNVhVip{N>E-$n1Sx-G?qzBWhn&Uxhc%P%d9-4Q93r zvo{q_*9M(ycWid|f2*U^b#|_*9OD!``U=ix$T_zwd~&q?z1Aj0@ZYN$l{On^Sf|HF zZuP%`AXPR~j@z$EF;5p#EUb%wa>xIjXQnf%`gd{3kUEb2UwNix;i&M$TOQ3V%kC)n zSrvwBmOpQ-&ZZZ|!yws^*?nBR_Hm2_{$MAzmHV*FC?r=arjpSoW zQokhE?##TA4mxCv0FIfLAwPohFLGhVYquB>5b(+*)9~;4+AN#h%mwdQ6-HXsjg4}T z01Q#yyV!1N>v5u+$y`N*Ep%b5&nqF#K|2V13J6-=TTz+!#O!OS{@+q4DF%t5&gD=a znLpb*K+8QRMp-?5ALu}NhFh{HOxD+>47V0yu>^W;CG-M?DST(YPjqT5=&!$@XaWNR zv=rHNJOZ+;pXCxC&4GPdo7~ytx<6TCsr3Ej>E$CM{ejRRQ%w|1S%(KFa(#*4@0m!b0fhc2B^k<4FWhtE@eohICb*JtoQua0S6p-0%Jg zDf8!HboDtU&V{QZgz-*xXM$fceFpdbl}>0_+?xGX+Pm@n&s<{{@Va**dVj zz0ffcR;FeEjGQRN1rHBTBIu3+2t_f|t!vEP-0F#n|njp7-u;o zoB+|yX|bOE^r#C6b=Q=DCf$)?{bI_0|H7|VP}QA`BFRMr2( zm#b$|lI>V6*Ovjs1YOr(E%%QdC&|7ff7!^yXORCFysW z$CT5yvTHsn2cG+g8Dk)M^T-Y)0t3$0sCNoEPbpDWqF$%cVv>4a41*KF1V8@Enu?4n z3jM!R2E1XXn@RV1Z??z_Rc!+#7LsU-pDK&#M&%Ax>M{_q1*NhJAEp{~zVR?l-uj}M z@qx;#_h|jg%@pKU2*#f&GRqyns;?%|%=(0pz=$rKb8y z#KJM)GWJTM)z(tk=_4&|;?WPKKVHeWy2R_zsnmMiDC6nf4g_PmU-=JaWb2uQst7wEx(Osx zP0Kmis4HV=fd-Z_aS(`7jw5yQjR>5c5<7j_1sA0@66l5belRjhOg|Z?Xq24W2l-_M zhStMHMF?Ynu^NMrFbt56f+FF0K0K{e!`N)k|JDNiZ1t)!g)NsZ*XA&bFiqF|w>kij z!3v1Dg{^XCNg>}2Y*~`Py-4c?N27tF^nzgLj|)e!%;^o;=^hOhaYrlf*Q9Y)NHW+a zdpOgZGnOD$ z%y%qRW}m5^6rDEr`RwW#UGKM(W*zCoPu^3sK1GO}x)X+iyr*IST;iq#lR-Zyr8usF zBPSWfPCw2WGiKiRX)lMl13Xr<2GWF?_TMlCv-dxSnbbG9+AE=$m~N`J7)H!2-8)5O zp=q(NeNPg8s9BCA9-UAd82J-!O_>G&X_am>n&4}0Y>%nGNC&V-I$>xb${!gy)W6tG zcmNF+1j1Zp%^k@-*Sa=>=zhXwD2*L-@mCA^&1QJHE!f9%zW9qprWGTREMbLyvPiAA zvJFhL82>KbN*jjKWX(9WK8pCgsI#ByCws8l?r4?%<|q7@m!7vB=iLed)eTg7Lw}C; z#?ryuws@W-s0_fM3&}!jG!93PVC*8(((-zGq6B}g!^A77ny4#92m|-e|J0?A%`rju zvq|Ew$&K%=;M*%s3o(>WBA0y=HU6PHM*wPbEWNNlC zLg0$F#%*B4aHmIUDL=ttc}iQi1=qPGW8g!ehB5q^;wVh~ov`rQ?&d&stqs+a&ICCu63A_5V5^#moWyE0n z51A1>G%qs%m4V1x@J5=kTU)9Ywl?AHu9TQ5;6BsBvRA76Tj|gakqi>j!DOYGt*O+y zY}1#l~VgB& zAl1Yae6u6H8to698%Zj@#v(3Q+BQ)CNqby3bpujCGErzGNNv+ScNt|=lelz30RVM< z@909LI7AvkhTNDa?8<`Wf~3=uH~2mrHtK6Adz=R#I+Vmb=~HCwF3iO2O?Py;R2MV@ zl#6MSqMG37qAZF*mGF&Yt|kg1fi#iVtT*?du|G+9Zwi^esv(S7Zr!R0t7J9cwRtaX zg}GE=RRXyY50D$F{2%mf6VD`(p!l`x(@)0^;aWh7Dv8XB3@?BF>elb{?!yS}nE#>* zFI=tke{WwM*hFy>-XkFa@V~(d575MRa(3>@7W>>OMLWcN|E8*_h<+<~`+WL{b(Nim zB_DN=AVBdZ^R&>`GmPHi?HkF!2Kd&@uQ*|)#=L_wBwlZwm41-vqw2-Ix+*TaG4h8+ z)(O5CUbzIBno;}Fv^HMktLMa}n5TwN zk&i{(@l>f~K&YuuT8xGitkqj2r4b6<^m~7n>%tiG$31b&dFtNdFd$dCt?u)=8zM!m z!ED@K@HPD`_2gjw8QQtJmvLQuXT`x&;1}nrN`>M}kC3o>g5%3PrqI9K=t#lM(@*s$ z%xpU9UF9|zW*(rQ?uq1r31-x>q}OOTEdKrl@oTl&7qTTXBbJ*Goj$}?VYb9;ifBZm zxJtV$Ce;04L6n zogT=G@N3iru{6i7i8sWJkF<-nA#1q^$lfYWGd%W|kp~ttm|B?dDxmFeM>3=)3-eXo zt3qC0*K=B5JUG##SeakPMI=*=VBWNdY{phpcZe^P8+@-QU=6o^`Yz+j$%Gs;h5YYW z#Rvqhmz2y*gPC5lgUYbP07=TO-U#fFsh8^oBZX!tA6b>DN{G||3%bE!HK?OQ1mK(V ze^mjNQ0~9iP}@IM<`M0FMULk%87xdk9ku1AI47DL%_;p6)b#m4uX9}bY8}URj&(#n zN~Me01ZC{JKYA#LZg2uaitGINo8uExxb-v<{qf0(GcWte-T5{Fzsp;cE7bUNx7S$! zs8Lna!fM`h#^qwB=rG)eFTTlw3;~A=Og<%5d$y3LR1vcG!s>n)QkL?O*G7-_N3LNeXEHf6#rCc2uzNr*~u0)9o*ijB*@ ztz#%h_*EjBje>{3vRKo&BAsxr*-lbti%X0=;e%A?G9$^+B_qZTO2`?*)lWz%Z;&=I zk!mp`1%1l*$cQC89?eKT3)A1Q-}Q7+&1?m=Q|EOIE*=i$5aou!&E@}LV!t1c7cu2T zT#w<-ynDVg18gthSDW1=6cy1Pu1u284aDLOlS(2yY84J{#5!{IR<3%y>e9jyR z=2ECsEl%Ds9aw|z6kU_roa@Eh{-h80MM^U*%KrV(AmfMfqR&|a$JR{RZJt(MP+Rl; z6t^VCOFfut3ACo+ls&=T3$j%a^}5CBt~#44U6D?IuBPgs!4%KJnSEr@nGv~tBWJzQ z#-+d9_MddpoEfQvQR4W5wlN>*?kHb@Wadh{InH%;xLJ-;e=g3ARme?TgUU{wj5PQq zH)F>f!9-Y7cjqi`bX2iM=(;#g92~N8fLidQwKhS(F7Lir8Xq8z$H&o({O+ms4RoFc zrI8}6VgkOe0#Pgm6y;ebf^u~reCSuAzeEiNlmZQ)XG%>^H-i{x08$XmNUIDt1qD=x z_4cyPwf4*hk)s^&yfQSj1|~BX7eocARl7Qy++dc@VWYP zwa2yr3FfA8V6a+7+6s5|f+(Kr(UqI~p&^KH5jl!8GDJYZQ-yb5V<7~yTYczHETI!3 zQ0+?OIUdoaO6Ifo-1J&ZO{ZSYBIb3<+!>bs+iTB≫Xf5+o8Nr89f~x`cHU7 z<#Xe0{?GTcp{T?(y^@>`mZgx2MXnsM#p1X!CZ_^-dO>x#g^3A8V-;}?_+y&Bx)cQB z3Gi1c!k*&t5}M`x7otwSfj)fMp3iiimi@Q(Hz;V?fF9ofm7is`0ay}tgW1h$(~vxz zx13jj7MCH;y@s)US++j2u|*W@F_v(<5R`_NHEY6a3uw@-_EXU;=^_7kZ-tlW9{)#S zQE}%NW)r5YB^;*ZVWv7F=6z*~l9EK659B{OFoi7oYx1Wtcg^7O*Nxppg)*c=$XO`R zD6;Aib}0%+562-+1&!$ra&mI3l%a^o`U@$O14$fw zN(dM%3G{=h*#vdXt0mEmt^~YI?%{4LL~>?#9!%Jyiof;&ShV9h3|p^H?59MjSYhDa z+{GbfNw0w%WeSG+Epp<<+133O{P8xFqj8)hF6SYe=w`Q&sw(>2PB}j$7o?;$3_C4c z*tx`oc}8<{bMe*+h{;SrF9e?CEp2v2T+ZWYTHyX5JU>X>5E_|;;H<3T2zrI2{Vex%CaGkZ< zQyCS{A#V>!Pt5WQ1yzs^HNO|9e68`1eIrph`($ZhK{DAWG|JH%CNy(W(|d#*Or}DX z3u%DIa9H63H2>AX5!lP3poGl@GtknJbpR(^6Vvl7B*hL@&9Ed&ABntYDhg`t;F<8| z)IpVf+4P{HY@&3Mf(?yuC}DccRz|s=Dztq|^|4ml&dASGudO)KwJVVgcOYp^$z_`h z|LUYDZhFhKlFh8r`CBx&QF6y%@B{tZYRs0~D}%5eDK3Y+Ud~aMO{tV#pwLVr9GVyY zd~+B5iyyNAb3$luEzr0YfqkkMJ7r8SbShUSDxIiF9S_izlA!-+bJ>;q2v7@DkMf!S zPL{y1I3PDeU(c8c{|nimQwIXNJt|gKrON)vk16?kij#jLt+%dGP*E@)Fk&qignb;l znoItZorzby$J1yN-~3`}w;ZNJ>>Fu@Rn-h(nzj<6s2e}3|DRDm3v$xVFU?Q!-l&@q z^s*q3$IJ+ z(ZBN|TM7}PRkoSB`B{DLgF7Qlyr5Gs%J~g`ed3b|ce7%r%{=&jYcvyens6c699V+p zY`E}UB^iH21jGGVT1vpyunQiHJ(LAUAiaPS9df@>?CfsTy0+7a` zD1x0-%0E$EsqVy{j{Z|nJ|4~i=*X0TflnTgaGXx)Sx7ln#q zSwnNOi$yCzKbu?&Fa#CsDypdC(zSpdbng>rxZP#9A5q_6CTOW?IXcAQf6gZ-e}_Z+ z{8caZzIl0mP9hY!uIti#H;?W!oZYc62H|#3tPa#VIVUKbKe&PSLT#t~%XsVspP5_v zmkfG9x434RrTn?&#P`qj!mpL48s&uxR*!U7+l+!1IPndYh?C0GovC;dIXx6RbUhRS(-X6MwAVyvIDYqtg5{?eD?;}V z0DN@HK;jEzXhFnmBAQ0@6wo3@YLUP84giB5z}QOD>2k;S^7L(7%%ZZ$BjMHO&+RbJ z9+&=X>^xmrj<=n&>d$`GzdRLw-VQTXSm~w|iu?<2L}76Nj9nox=zri-fP_&2wdg?y zUC!;F8!q`6Q=D&_{t5yVE5h+zM@uLsL8`dJEQ#4a3SrHsiIS71RHo|3;`v8)OJJp> zES1$&Lwm_Ryxw5vZ1ukS!!T0HUw%wL42l|&(0PwJox5Kd6mt$CVZ+ zjCx#_X$p1Ek}ds$8Wg(nC1Q@B4ld;@Ef-x*#B7iNJog9(fZqv~ukC;H zkG;R#j~cVuQN@|L^GQr;=>z>#t`ftTx~uxebO+mP7c(*-`yZ4Ik_G`}8n^XK-M=FU zP=!N7@W+moC%CS!#_IFxIksWti_OlA`tO{({vwdJ=G4Co1xw&v8HpDc0W#DCiLY-@5jkZwLOl_+2LM6v2B}QJ!n}GX4K!?<=FCTKh*8FgZwv(juUwNH>av zQX(lez|b)CP{S}{(g;YGf`GI`H>iYkcS`p##0)vyjf$`DIsbL0QwP)C~ zpXd2?ZvAH663%p* z-Ky~1T$|$gY}M%_RddKkIA_GFYrhsS{8HUT9&Pj8B0kXiXaG~GMy>X~oXYdN?>C@= z*mHHfQpXD6wlM7RdG@7zK6D48^)fyqDTzUITmnevPJ9EpOKy8^&dQH!=%~ja$*xQo zNRTTr-3iqG+8dJXAE~InN;r9w?aifzbNZXaCI$TNrJQp2-Uz$ko3q)NqCvs@$?d+9 z(#1fFe4F6o?;9-!EQ1|IO=Hc@Ck*aHSf#oq&a4pk+^^swFa=?KeO<+QYu2=XsW5Qk(!C0{*i3;~dWCex=G=V!^Lx4IhkFgw7 z%|YJ8&??a?T5gP*=5}t)4rQt0jOobJptIPh^1=x%Jqs|fC6BFk;g^jUu3q_aC(yjh zp=rxU$lw}zFJR;wQ=O4!V@v-oA6HRYK^E?OmE8IaA#JvJPJRmz=3&l#h1^LT(78u1 zt~MFdpj7xc*ix&RYq+IOwIP%p@uhF%=O~$mc|U6dzqn7iGI&0g&R)!2Rf<7I+DmXAo&}AXI!rXXQ-{)UfT| zO1bhQhRtmwLv$-|ZJ+oH{i)n1g}^+W^E(*ZuL~7MudRO- zD_y$*lrulq)q}D(DB5a`x$^zA;F<7p6;9e5yWZmiqt;l zk|ZsV)kivFOomOWG%#1PY-~~wguBvh+KN7Xy8Y4*qd2JW$+nXxn7Z3TNHys%Um&NO zHmLNyW4O+duiolnlpYsEzyg@zf>|x>G~HS5ClXBKJJ`pfc+1@N^z^2u*8(qwy?9qt zp$R#D3r=4fEmdPq8D1J?X1;Zob7Cc!Xrr)GDw{pU>3a}+i!LyXg&r6jB}*$c7#tjl z`B6CdKbU$z;)x0Gq(LqDGg$fMix(2_IE&!?SU^x#3wQC$v`kRw>gxwq3b^e7jVn(B zA5x{gV$v9NH2_}pAA6|iFCcWI_lmRlM^z;gkh*Ta z2$WAr5VB4JWh`WKM zpgp<+I1`u1*#Ku^*MBS$>auF#2()mwR0Aa`&98I^UgtN}UBz!u$<+rYK><6Vl$ykb zL$p`UtI_|cPy;yq-C2TVTz1#fjz`p|DhbFDpY&#_BumkYii!rTs$vH#ERx3Q_}1!! z$%g>lM)Qpm4C)pmOuE3Fo(6PSvu%#QbAMlYpIE{tN+=TGzsU!+UoqLEb1wF{%kso@ zxhk@`gE~NS0Qo?!T3r5;|~l zaeaQAkDD6)t2o!s^ct}Mf;Y-~+5>cLk8G^gO`*2 zwNE2WL&Tg2?@+Dx`O|BXwStc4*M0?sAncz^zMl7h!DcE^HD7^PUol*yeYsYQFSv;u zCcAk)Z>;7S(9FTY%}=D}v-x2lfWts}CQ1>=)iDTr(50`boJ;4q!gKNFUnKVhvb%{a z9UxSPLSnj!F;Pz3vWkjKKm-@dQjOeNN&LhB<__8eilJ>I{k#q^5W7j9(yW#7O5rPv zQhz1EqklC~02uGuCm1oev>YtLnJ)!!t4_@``q4?^PZkgcCsdy&G;)1))_%GaCB$O2 zM^%EomGeOVoaNqeuT3S(uL`c1W8kfuw*?`GHMNlN=y_PhK0gI}0K+(L%p zi>X5;*(4rQV611Vw#tE4U%m+#N>W!>_a<@j&mm^_WfU&8#S5cC!nCxtFF%R%YyfJD z>c}6eTN~n;g$-0x>95(8a`2|S0j(j2Jv0Ez?bX@+D~T_L)zCZSyKCO8p!cV|E0b~2 zv9$;=_z1ybT;f4coqj=fxq2ir=GeOUJ_vC{Zs|{W57sUt1SU(F3@$UI{>_Z zaLK(lQ*A`wB3!qZS-%4y} zAno%HeeCy?ima>mim8e-#u_ZTsticAy(R9ifT}B#bN6}(sde93+h1Gps6B*x^Bx${=)Ch}O1F>U z$Z=7UU|>|C&grSyU*>%Irtw14xvz_hRwpkHq_LId9g#v21^Z)us6M}^_M}$dW>$c% zmdk+xZf~`-sF69&rx?1x4R7+`64m=tg?=#2$3Dri6AOtMig!?2!3&ZM!G$ zNjl2>(t(aU>h8v1+SgfG93}}sxVr&JAWhB9^~Y(*4F2LiK%p^^;mwZhy;0uIZABOl zW?hY8u2;ZMLcrW2rc)(vcv!{!ba&q2pEO(1($dmNcsyDF^zw394DgOEDIh2D;M@Ej zIx_SeY5TXXi%J#TpGP3Jlzp{c4y|Zc0uk zUcNKq@~JK!kq2}!?H#j=ZtG+I_T#RcR52Bv->IHRXsi_5LQ`luH_BjGdyyCsff(hx zwBVQie*uOR(B4D|yoTrXdy|=C7XO8w(-M#qd5eAc^u&Z4)Lv-jG3mrgP%j)8x=R~t zC+<`)BE(*p`GIfWCskHVFhG;2NIut*Fe##$(F~w>ael_2x^%90NHxBTj&-7kwygcb zGqnteyu(9N^b1pG-fGPzHVY0izJ{W!Ior2Vsfj3wQ%wmTJPVhNc)-@SWC#P&L@*1? zZQCLKw&bH>sfZ`AT&&>=TpBeph1<+4?%2)>p_QuPcrY$DJu%yySWT)2|C+9QluXr> zqg*#H_w>wJJc4w;x;hf>txvvfl5%&Vp;4`zt)o$#`)K~Y^KVUjs{8km(yk(P6qZv?fyY>xgWiwari zU$4Dxp_vq5NbAh3*WOa*aFnIXbXPnzE|>uL)osypNgmVwtCHP!_`^!w8v2bqjl8v^~EVGGu2KWAoH}2fP&IWrm^uv@Y!xvn+LoeXAQ~B~uhd3_(*rM-)TN}twI2UdQbSwz?9aOO} zv&ddsV?RumPBjOE$CU4?U2f*;Mu&S1|DD2~iNkqeg?S?nqn}Hi@pS<3{e6Mg)qBpBH+^CbYIYqbo5AO~N;?Vh6zC+kB#_UNsLclIcs_zTf)dVeG+s zZrxAs>Lt}emIsYsYqM4#waI=$PUB&nhImGwBL+I((6Lb9kd++)dVa5{Egd6*8B-d* zgOmM%t&7T1zHMAp&o2;L(E0!&kS3s6rG@Q&ymx9^!!K}XlHO*4WChJW_0i<5X2IN$ z{=Lr4PUe5LneTWqwQ?eLpAODJaPRP`>-zMREFtA^}hB;pT2V zyA)qIN6W9h5?p-CG}Y5VwxP-T1M9zl0}#_ilADZY@9OOZ6({|^_ZMlL{D?`B^Q@ZI zuVcG0S}l@jj2=1>5Q*4HDFa8Tzz-^r+xO*+RP*%u+EI3xc*P`?SHqB zv0m2W%PNn3*G0C6FjgaF++qciA45j=S<%T+lO%)-S}HwK)pTk*ws=8>sI<~vAuCpNN#L01nIWY@i9M-&Pm+o7>khRsoMC$yfW^&kSJ?kXySTK zHBl^b{JVylq+o@3i7=jq3;?Oif2HsQ#Xjw9is>FfQ;=-z#o#^vNC;+|+Tyb)l3m}; zt6aZUw6?d9%vjVB;KI2P089@#94(jpkeW#7yFLS(fpZR65*N>}(>u+48SWnUyq+|K zTe2EUdSMF*v(drGU{av0sJ!n} z*au2u<+O&pg%I;fjd4ei(EZk`%#R6_sQpNHjO*60yabM7RJN%NK4+7Ms|@wTSYiPDXv7H0!bF z{X52G$}eB+NniY?)0V&}?BnWF;%?YPe1C=f_4HnHaZ^cD5-dFzj?z-+`%A;U&EW!psE`_GvwK(Mz{xfhEALHywiw;UF%gyxp2wqy5jOc{L+J7GF;T?!J<^CtNXl#G8Z+7Cug%%K8$n-`4)7B*wFL?tac;u4Qy-J6o{4}PCR z=B9FAIm+SBTF|;u$Qys@AA7t5I1dlp7?ksks6r*S*B7P0m6bF4h!P20=&YL+5q2$z zlJN4^f-H~mgb&VlZmsQ6t0g<}0<&uW&KKL4F81tmu{A%I(`-)=wbQ4r%I3KqTV68v zzG()ZFiKu4uQ5|LR9*w)i7&{9P(_WHxM%$1bnjNr0r1_Xh`};O$wOB|LdI3lpDmMv ziff&zoUl=k3r}*RTjBfGCQ#(17e_yX84lTIY^f6 z5JhdNsgqBX-ef zx-ED@Wnj9UXMca5jByYEAn;#ySc1lfQ;Rsm?kW?A0c@>UQd-!-Zvd{)xa^2gGJbs7 zHF=2IHlkB;D=*~1uum}CqpZ5-5V+HE#y}%5Xer^S(iE~44GhxFglQ$$^^tA3H}qXo ztG-Hqc2ZEjsoN+${V5Z~CM4TP#mP4{bdr|S!69Iv1oI)Izxg;^M?qg8U!R^t`Gf=n|w=RJ6HFl-Q21dw9U;}+7Gn|Oxw=)J}$JKN=fG4jy$;cNO> ziEH}TbSwso1&`(ELv&jG(_*vBh!Tnvjnf!_0 zPVx-2m&^cUsuJKuD8E~&M}PAc&N6h<(o`uZvE}h~VRoF`N%nuu;B}K>cu#R`{7s^d z3(u7m8UnVrb%gAQFo>GUS8=eLNo&>wMe*>BH5mG>|V6opahtF$Gvn9>rM$qirB)YNWhTiuTL zi>2GJd~=rD_ZIIID&UrM zy_iQ7UX5)6Yl4kA5CP&m*+AyaE_eY81FlsCSzf##R?{p6pmd=9Mpe#*bY<=9V8s0NSctcA(U^&Ww@ z4Zp%+bME9kZ&RP5uFZ)o?@QnT>s{mv!=s~k3xm@|0!u$jXTln|%YZ-vd1oLi|Wr7DKf{tm_F z6MaqQ(Yu4uMpU&YqWzr?(_S80p~iOGW>@813_#T}0Z@WQxp&vIEIQ~g-wYMpW0z;-Q=!bUq;LO`^_HU;HyvsLN?1GodyRk}4yx=xMmW>cE zFDqQQihu4xmcNQPPJ{|Zrtj+Mrv;Y|Pti)7?LZy~kE+^~*T#2H^Si3f74eAOiT zpW#56&^j#_o_lxStrAd;l}W|fXQ}Lula9v4194WUfB+(`Wk2>f*a z@>^d3Vdl5LRp=JfaRL(pwj5Yd4OdR}1OeNJy384caD=faeJ@WLlhMI`xjL(C{F|8% z>OUoX-7JZt^};L-d+v>23Nj&5rhnB_UGN|$E9?HL7P&1`mzIbxQ}xf_Ak!mdnK#w>p52%HIfAj+o=|mnV?o;hw_km(DK=z>KYLjX*HF~oZU4d{{;fj&0_D~ZG zt)GBP2SrEZM!Mh(x#FZNJw$DPTS5CLHj9(TuedQuOb%Y}Od7)T_O(*zw(Gn7bgSVC z4hy1XRUHY;Qa{qk-2qv7&iZ*!&Z+T3LG5U5=|uheff1@x%UTFBVXcL@aU{3YQJU=Y zeMi&D=X)gF;h)wtvZ=0cfPct^6EjT432<2c{P81W?VLk1aLqU*o^~3UYPg5(_Pi}s zZ4bl2XV`UK1d46Z{mvqM1?<>YtdD=wlP3YkJdgi{f&K;BD)jLfX;UGpVjA5!K?W)3 zhT4`khL_8Z$#R26@lIhT^B#~6kG|gd_1XCH#WwwYePoHyVAw2Kr?i741^bml6@ilZ zf&LtS{bgAG*+e}0fpfp8vQCuDqXEdQ;MVBqTTOnmM@M8`^AQsBYBUwzh0aRNoO3~1 zX*ay%);>8efH#7R2~X;RiuezP9{$>SOadN1bz_AJ=Y?Xp=M4KxtLK`(q5FZHA|6@`yafYs#ljxe16L|2t$OcYl3BWA00SP)i2yTx?sd zVy!7a)!$@nSU=t891EbI03o9B^EB@g2viyTR7xFa`(_|3QuG&8iS!`=glppoy1#1N^$n8M0Q{Rh~+PiCbW6-Io?lQ1bZ}Y7`fF0%Ydcr@S=DSI7-G^(MPqqgT!wdLAyGC{qjWy)sd)VK^LGTK2Yl$2_5S$`6{9=ONntM)X zDReJgt^P~wUa2~t`vbct8u9wwq$@&_bnFL0`loA?(8vQj|LaZrFPqi3`MRTVQwCUH zA+H#^G7Sv3r0?6_6UG^h^}+lt59Z4pbR1sZmCHRvF$oQj#GpMheafDIHJUKSI9f96 z5JZI?D-hrabTf0yDRYtaXszDVv<$eL`2a z0P%b44F&T*AI7BUa^>{Lf9y?>nMqN}_rt~i;bC{|DT6y5{UUyS_c`JM?Z{Q2?MYQ7WzZ&4@) zoUZuC`A9y9UijnV>1zL?;%~tH7Zra}?!R2|Co}-2`By;wDMbDiP=7+>MA!VQD^4{F zpo9KZ+kaQY|Eldj46^@6EvVkdZ`FU9rZgpe#b4ZARba|^cCViG4N$f6gqx%R+ zi))Y2x_2CM#J%ree3rtjj^ze7zCKbs=utw)#nDbTHnNJ@Y|m}ntG#){(}3F+*nWF9b?2fW<3ifn7&=>p@2fp*e6yg zukpjPb4svL__CG6UMe{$uR@zwuNGga&7oyoWKTMHjT>2=Cdt}TVpsku{eo#jis!8r zq(rYrR&HLoM2#O(-G2I>@+yFrWAoPSXH2Gp%W*@=y^%W} zcJEnDxFGT{T@JlZ*crHhz|445r3n^9^f{j^aj^5t0G08se*|^uq}8W=1sxQYL0r^k zyB#N4kfwLF{9#quzT#z?~*NtRY zdF4}q*n82i9sTjj9#l8wq68gcz)?x|`P=s5X&>ZlxyVyXnb6+VM8D>;9G629)E5FTX3_$0}t^vtPhAf^uv-I^#y%4njO82OWaT$?M;} z*5N29I$)8&~$DgAOOXk2{PSM zA7u;ApTRVRCar~(7X}8&GKQrUlEip6w|(SL_S0Ag@8|3x&EE2wt9mYOa%rp6BqlBt zbZklxZpWRtI`TIk?UpWF%;r%2>5P3K6+tID+_$~^8A|M3<>9CO$YOu20sY#2l0tYE zzYKy7mTguUUa+)4xi)^TKA2u?%>e7onLD+o>>XJYo4fVGHy=r$JEfb~k(&hw#=!Z` zfQZ+a25H8XAj>3)1jFP#wcZ82nf_Lb#v-LV<`#`%_{Vh8E6VoK0$VM*F_|v2G7^~L z@owC#Mp*fxeIu(*W90-k8$EHBf2lD~C6$zZzLK_gfjdA@kW^Bq#H}8=5N_1I9#0A_ z03%lSD-DsTX2O&bOh3|POBk#=E~u_k%N1oX?jn0wQ%%VOZG2xH+O+Pl8Vcq$OPih+ zjcy#x_tfFi)1}9Eh(&NXjbooDTe>?A*VeMlcV3GTgUU74I5ZMRApLqH(WuG7n-)YZ z2X_`e56+-)Xc-A=>Kn)6((^S-xyW5frD>v<&1NS2TdlE2$A;cJVfhU>aOd2B5rso} zy3lge(L)2t(#{(o^BaU;Rpe+*L9ttFkLC%K(?1wG_m8G}JsL_Vf*niOlTRM1O2kXN zVd_9Y4SeQ!8!Dr?0){D@W~Q*os@kf7fq5~Y*$D&-ZR(;F=|K&8ZMK(6-E(-fFgY}} zIj6DK;h<@8x4W}2`E}qzTWBdO_;B4U2&V99YteTIwz0qm8CeoRO`k)0_-&~6{Nr%IfW(9p|OCbLm+o~e0uTGOlUaM z+VYLUef7iAonSatkb~?4R1(f~P1wF;Q%iU(ny4p9>Jy95{CaSdJ7kn@C1SKhYx3_u`WSss%a;d`3CXo|1cTmn`ENob95PgL7Y*;(`(lc>XT4L5% zBP0X?O`1ms;xhy^BaTm$1|AGfEEuH(=x!TER_8`d~~Q8#{`am6*YLEPSP{r4rn!Ky9Iv zsHN|6fC2zfR_=80E*0v!o$yt?`c8>G&T2DZN_o;5@rvT@L{FKNrw%=KJITS)GmirI zq|kXSZdZM4{#&v=uR(mZYw6S{svn`dx({R?b02JS8-J?tX11=K)LrenRQ($hVG;ty z7KfYpgA1@R+?uQt$5!4(jXX7x6Dajre0yi}_(jObOWDaQK2O?L3YyAq$Jc6FMjzZO zS~io{hWBA|I_v{WxRIO0H5BZyiD2t^FJeB9S~dEh{JGk0ohDZ59fJ|CS!5@Mu-j%i zNEQXnfne&vZq4rVv&?$w>zn%{&zvDhsd8oaWB(d!J?r&L?nY7@ z@!iMamFuZa#|6M$P?f%c?JJKIie{6%pLF>pHQ0^gnDy2+<4xP+UVT)vSl+1_Dai>n z$F2wGO;EDzOR`*6ttIWJ>q%Qt9E;yB2l+*9m$OaI&)zC6C*gLf`vN;oT%$4Xf=Tny zkH%jHPW3T2TV*=^nT{>k8!gkBVS*8?rGoCMoY*?Cy+RvMiM@+Etj}npss;6@az^9& zq&9W8o|5E>El2Z)R#d4!{s@MR9S%38X1{a5*QY9OHMM6e;{I-Mi>_(yfuCCr$x3;l zc=q|?-pfC^P!$Ly29sBPJ;95N=;gk~(Y%{6CK1o3!i9|@-?p^d$MW5`iZYdsz22{c z@{?>VavF zQ8bL?-od+HB9PAMKgu(Osw zpFR}c))OyXXc;%*kSj$JA3YgGTa77$x00Wf>1;-y!4!L%DZO@^poY7Igum7vJ8CX- zGU+tWNNip$>%4Lia~V6)oo}yV=QWk5E5SQBNGnZI8`l^*Vx?^0=AaMRi>Z+|#;jd8e^ zMSCLYV64PPwB{hb-Ki-_(n(?A{cc~uD~8IrVziY3;2s@bFx%ovER77oek-iwWw?9J;Q z9F4^A)h)jSLz>&(;fkXivQXu-d0V>PoYJC;Egmy-)y#Todrt_X8gn^oX-X~gQ@xpw zZ?trTl#HcC$2#wz>lpI`JgcKC*lg1~E5rt*T-_z0vOHCz<5bf7yWs{G2MbZ8$HJbD z4U<9(8GQ|L2U6t1Jd3stbE_|#`A`o+JoQ0!gRnPd+Jj$TLQ5%LSPkEyL>e#UI(0UK zo%Gi$I@9T4Gu7?}!e2J>SxN*QRwb9OA_fKQu1YB?jU1K6AM97T**eGC-7OP#nV2q1 zHP@gss*$2~qhfFrX!_{$6(8Nry0z~H66Mt%FXt)Aw{OW?@z`Wsn&wQfYeUAO}=dc>bD}ja4gwZ_%EU?-U_a!s3Hu!bMlpQ*Y^)Hm&Mc(c? zt*>BfPK!gCx-sQ;Zgq@`d%WjUz49~_X7J;|4_cH)wr>B-dDIb5ddv|(qAf^Iyb#|^ z=0{E@2I?M9dT%UieRY5vcOBK;X$O6>Grz-r6Dx*st$p{&ea8c5uZEXhQi7Almo2Ig zbZ}O90ct3Y8VETHMX_HjWxD+DJ_fRA7V))Zlh1P4C)28$hCV4Cl)w z$`^D^{Guku+g>6~%~6*=iPs!QOgusCk+JA49-`g77TDSDF2wCdJ|p3-O^wxQa+nQ` z1;vl)<-fBhD=~XC$5NRm_VD0P zFiTLq^rL01<>Wxev#XJ$oV3dUKKA*zeywKKre^b|=Cmq@c$axm6M!)^^Q*M{qNJ(0 zeo($9$+zRLy_$az~! zlTJ62R14mJR(s1e5buJ_6DxNZkdwQzZ6FaX@nxgsB@|iYW6j0oeb2OEbvNyX$4FA8 z3!-VAyZgL$3uvi)MDa~A=3imDN%ieQIh)6ys zhEjs`U3NXG#L;6AlLl<9f3J1pyj$6RB9wI0C3bX%5u*-U zZD9svlz2LJ_KlqEmoc9U#}9d`m6McV{V$s#CFdPhOVdrgChaPGXfIDt_?)+y6h-(A zhf+E>v1ejTjnM-m$MX zH|JU{Cx_jLDBd?(X0yk!nST@m{BSg#1Gz&?fCG; zad82C=xD9$lMg41uJX~IwWk~`-Q6m2ID)tsgq&xMs|kAAL2h=4NQuS3dPEtITmvHVUVh`E)Z7}3;i zwOOkq&V|IuEtyd74DO#a_khPS{g_&?xiWq4N3`4+@1mv=#a4tt2Df;121`v1wh4I? z+U0UR$)k*y=9@$$(owZ92FqjqG`wIaMpzj3l%Igx(!Q=u*UQ$0>q$51+Hnyt4($)` z9?>SzV~rFaDQQ}Rs5OogA6YviGI6l%yxhQ21Mj=Xk@yqUyIa&V;e_(&V=v0FP)ObT z^Ar0Vm3!N*_)0`OHJ^fAroR+6o6Dm=#6)e9zH6!hk%&<*jg&K_2F&_EZNu3uFa$ZH z)1wMHjZhcFwaIUtj|>rgFE-bCDLwjd)^JUp#2QS2jSR z1G>T^(j#aYBRL4=!eBd|W@h?}#=7P^y7k2rTYrGlc5$y=#Nwt&2{#d{2@kopvNDI`W3;d|=}y0;V#$6$RMB?Do`cFVcB*zGF#6Tl zV5iUS^Sh%)(#I-$&`v37*Kv^@JGyo5Sc9J35hO|*)g6FK;@c9G8zeUHtgxLbLe4I| znYk?wU&{)x%>yM7SW9YL!yY1}gBy4;_@($ZgVz5p{#`=?{ z&Rctu)t^C=BWBZ&agm_S3?+N1lEy}XufH^nM*5y78P@=o{kY7#u()~ zTiSY+36oZbHLL+;3*vuV8(X;%&Enr(m|rscAQ6PKnOxY!uk|+n?DNppcGB$YY+-rr z?i2K^PFxBS$#<~D%S*K1M^(gHeiK8N#CddOmFNe(#Sj32y9c)w<9--G5&G{;F@owb zUyFc>5viTDBXt3dX4m2sEAOR_NM!Z9(G{O9I6NB4c1unw+Xf+mp6c7 zh%@wgpUvWCw@Uc^n!WDZ%GWwI_p@FG@Gl5tV!HJ|ZuXS!?39C*>=^sj*JpjNBrn0@ ztQ{I=-EV<4KgAb zPHjE3wnNP!H>A`^`!Kv`!bCiJ;bG19Uk1tzpnX#%+_y*h#QTV8!|Me%qwJRm@x$|0 zC%j=?m2Qx+<1`?AQ}=#Rq=)qNl^fVOuTP?R@Z**V*jr0+!&-Y=u3VlnalN^O+2*&^ z4D{V)n}EAM)h*(5sJJ@1V|=*299&KN`Uq%B4sG;(eqnkl%yi3JZ)ScH5>kC3v_7*# z!>I}k5;%4v+nD1O&JCPH#ZS5fxeTP6R0>SvMMhH-2<8T(FJLXcQ_P*t0w`$=i;(2A zbniihz;n?GrPzkJ!1-DpSWue=$>1D&T5(gw+%ar*1aU*GvLa=x&wr?P1-CoJ-M)wE z%he~X)*aervo4ifdt+{N zp&he_lFr6Dc4gl&6PC~K_Z@jv73+o!)UfMe8T{~@lNHz_0y{g~%X>l_X@B~dw+I3Z zly2V|$qoCd!<;W^c7zs-Gx{kFa2A`}}1}J1ZsiBeO|{M<6Yyafh+D zW3Wu&+tz`N`T@#!mqE9V?K|Qp6SzQb)G@-DV9{7kG z&i~f=TtTh2&^X=J*4~6}apsUP?u)ir6fpnzyuBi}#?&as%77s^SQ1-c<6s@q->lZH zGhR`KTb>ZoYV3hx8*s!E;k|aD>7zOd8pr^s?F{yEBhm&3OL_0*V%N-93b73;nfT82 zxYjXmv?{H%e&nb!wcPR|7#p?ItQPiAPP@Hwz0`EPR4?_GPHFKTMp4!GdVZYM2=e;D z+ve-E7xFn0!eZ!(x$3-4PkJ^R!yDP{P1*R^*EkHs&P<$u61$9pQPY^5zB1d=1MyF@3R9 zC->k^`<(LuyI+y46L+t&38er$n^7o{Fxw(vg zA`q3Y__lZ=v-Y~n@{#WsHuLz4Dk|*F1td2fk)_O8coW?ecln+*RA4gqv{=$1du5=0 zAS%~Mg9j1sWO9rU&aT>&Y+lVw3a;iKfJvcPoJ&)Eyx)yWg3BH?M}O7_7Qgc3asmhEkVUJr1uRn`l+Jk2`i)Ra03 zUfZm>YJnfBUz>i*Z9-kz_}`Flz?!XA^G*yWbG}S&e)he+@X1fb9`R&~5{6mb zD{US-3ptq?a27A(@w-a?5C$9y{XF?l0N2z?%HJW*YpQ2nH4n`XZRM>Nu+U07+>e!b zvbo!tVi(YAGK;V1%FnSsh6DP>c}UqxRE<^ zHsIDl;l{&RReObSH>p9mNdjWu%oVizs;t1vZ89hdSGnpboCV=0~+e;sF;%Iev%I;VY^Rp$ghtwma4+}*i`~0n`>uoKJ$Xs|}vTZr5(a(r% zyce2hoK}|>u8TjTuHrB{cvY?ro#)ttn}v98Y-ogC#aUBpHmpoK%C*HO+8`&ZgN_!-@2qmmZFoum>&;J#<KOT-s6w$j6~CjH=PtR`(^_9r}&YXZ11b#WoxUpUR}cKCXrN$G^qKUh(vvsMpX_P!yyI}hlAfE>jo*s>px(FwUevRWj_DSu8w*F~QvlS_SJqg7h#lPY~Hi)Abphw}MoN;=YR4Fa%ms)=DsIjc0vDN0*c2 zzPE9jJ=(8)*C+&uNz&pu<1f@2^LKB=Sf6Pbu zV|^eOZMlz=1UCki?F{I@p7bY6Ia_c+pr9+2-SuPd>{lbls!o>}c z(UBS2Nt`uTjlEG=Vp`6@x^18M=>0 zLM9X4Cq&Hy1>B`6v&?oc%w?M%N{&w*NSckGUaBtJFZi`pK{1BqNl&v8vC+^u)}_l z7ielN+wM^Uj4r@>x}KHgW$KVjY>Dst8BdDwxk?Um7w6PgH|KZZ-(ScWAD%N95#<4I zm>x>FjA2ks{Duv*UAS4b0{))!4k|j>*)T(70g_!pzh-^i%G1nJq92jCd-gF^U6qhX3t9z&YK zERIwgZD`+Qg_M0(hn-C@if+BqePqaOXyUX2{5MZTfB+U_0PBi!PQ(!vhBOq`emkjE zpgWi-EvQ||^8%w`F=;t4qdXA38u6;JR+WirXGPw7aFuW5k109-MFDTCfz~_&Ro^E_ z$0zek3G~RCrSBrcqpu<^m@2q*_MVy8Q_AnmT?-0%e!=wE7f36`m#1ZmLmJOX?tm8k zX3jk^#QLKelUA8+OmPW!L+>m?{7=Q;AM4-~7$JeT$0KMzb&LY7Xv@ZSfLR~bT$Ht9 zA3)5VzyPuxAzO}&E5nWLUJg}7|LtxKl} z+3OwTd9-i6PW4n;e~-Tt9}ec?ky_#IeFcmh?vx;B#BIDTwDsAHNcI?#@eTf&DE-aX zj(G%0Udx$fs|BqEuMfUZ*O27H*K9lc-_x_R8_SP)Bwp4coF*=Hcz8X}%xGjC(%na! z(x2K9Sy9 z(Qc26mp~SNN;$2p`C+5}R3%}T%Zgk+n6{Ar656K<2k0P^nclf!u}obna8*J$VSw?hiT-kzvd#;)I;VW*25Kf9aPQF6V&+2f2b1QODuql zesT3O(LeiUV!i?A@|O@J{{Z1ltSSX5sUEMeprsMGS~&|LJqwKPJbPinljz}r1F=!( z>Yby12odEQCmED$!Iws&CJ(>m&*={w9j^N@>!DuxnvB$uKt9ZQwH~PatnK=b;S2Pa z|L7JYFQX@<;`&y)j!3HFmL}(}y;J&#CY`cFSA5hNf5r13(u_v!EHN#O+R+1#)!Nx0@z@9PM3{m?1)#=6UlW8OoHe~`)r`S3Dj|}iDV0H*As_gHW2mEAU z?uk(5D<$whh>)OB3HaB)OP$D`{|f+2SIHW%v3qjSd#@D~(ge({4uTsNm_2ZV#W@3S zz&W~#73>MN9u*#GqPG^0{EzqglXm2F0Mt|@VT3N@#X7DY?;}wD*3s0`l4{4huDF}l zEx~T@B1_<@@Uhcm*&nKU=K3RvOq%F)k(<6!fD5C&ha)`Drr?4~?R(=4N1XRArxv}L zFH4nl;asbAx${2OzovH_-p;6w52T_EEk%e=I4P&#F*sD?vT$S9~+ z{J*V++cNBFr~-irKaBd=Q%7~5LP!S;NP3i!uJ_i?C>rWTQ5MtQmC z$E=wOpOnsz4@Yad@6$CaS9eR~{dD|K0SUN~wu5=}nu~pz8qWuwIN1XgdkX`VQ&t%% zI~}ws<++wxQ0RRH1(=;tshCfdy={(RTn^2qp!awN_E@W^gzfv-RinwuS)B+b$mgpM zhXf+ewY`XMNzHcnFCM*31pp5DNCnyXfA*8LD8@D7Dg>d)LAilOt2&60BEsng#&y3} zjRR(I$k@Iw=K(}*YraDq6{G~jzf`)zTM}^SrJLSjmfryhZR*q+S2TBdCA4wVgWg?W z#HcJ39dk*?!@KprcJ#>L-4G>6O>#$b3~_waz^MyU1%%7O3(u*$VqnyyU2C2}jkV#) z;GS6MPOwv_3dCu&1hd;%intb-&*|oGxnI#=c%A`!e|1-x?S7!H%VdSFi0ytI9{Kc9 zIIB{}Sjdh0v?a0VfCgBA4%Xl?I|`dBl3J0dVU=Ib zGYW8?pnADGkpC+CK1QjF7CyeS6Yj!00%JoLI%ktAWlq+sW-d91V zy%X!Pobsp%!-Zxo_Fh{6Z$oFo)We%s-_XKT`rnp2%e`Vv_*MUaaR?V2<^Dj_Y`zgz zTG?%xEK(N^Tn%=V39f!lBJvN*_ZM!L$4YHw*(1Y5GY2%+81*>qCTvf)+jcfGFI5b) z|GEXSwq#Rr{#C0HeqjDZMHm>rC|nIDF4ns*3~km&IcksTIfro2_H(G1ZhDnjC0{wy z4_`Dh2vc=tb$fg~_L$T)DwWcI-xb+-zXzfcl$EVD^1W3{uS{eDsmR$|n(aZoh_yA^ zW##IkZKwGY#^v42L9ForBP$gNnkWbJt)0u=*47h=s6IQ^Oya7r+0Zp(01hhiL?jXP zP=_!ww1pwV9!bA!1vk8rpT}i{$D-vhQG z+o&SrGk#TjB%5X8lZEQ}!|(jAl{m-hLS_kl@;r69q##dpUglR!gyk3E9t4c=me+wuZA|t%BD#TGd19uPqc6BMyyAYwvce2h0W0`dyx)3$JFR4q9x$UrNA~f{VYH z^Y7og)U*4?0c*`T^Ac!IzjRQ;yi)wczO5#&3TOnU0s7yc9`JxKZ>}=`^R)kd09rXY zfEN2RzgnqptD>>12*ZD0#xK$iG>raNGyc(+ zZy)|^rhYM#|1ULDcj7oyK4#LKmt&;bSPGS&*tzDuxu$e~KHaXD!E!7#n5W*dwc-g6 zbkbxAn&#+i1Mk`yDqFH>=5a0z4hk&dpd23f=JJJCo=D&0ux^*-f^M|l`K#Qt-Mzi6 zjvXn1DN@Tpf_WuJ0#jZ)5|RU<&UKu}V?HXuU|Q}FNOPxS!enC>Rhq}%dZ5A0+$!ME zt+cEx?i^WT6BF76-Tf|M?l5ppfhb>bQIWH5+~YR>&)uDE3ozmXo>K~UHPZT#Tyzm>k+CSgNU4$1@L*Js*XLvpk)qs(iquY~>?X{ooBiZnC2z0Wv>lHT(#_i11Wt4b4WMSo zdU0;QL#j3QhbM2zdM+BuZGC=HRH42k^!P6&wDMJfhTvYjt8<|q%;(`;?VlYJ_D{J2I>9IhRn-vr5U?enEZ)T&`3L8IC?(WB$kp!>@ zlvq5K^2`<(VunGbT0P*BICVfGi2iF>8Qe_B(GIDHj~=C)UjYC4K0Rt(o!+X~0qk?P zLLRnuQw_6tJITakvQ4m~`=#(w$jtf#a23tP=fCI?86yM&z=fNp`qH}1qyfUW#*az( zR0zx;m%`Ffbd%XEp>QcMW~D34Izg&wFT4!36D5TzVKzXH5j;`xKG{q7sV}4IJwCtK z?{7}O2Y^PUGoUzn6jrvTqyDrRZ*9n|u_|j`V z;*SHK)fL!5gGtC8GiIqNH={ayPa9SYy!~p8xYWD7{ETZkWKt#JgKBJvV~1VQ5n@c6PMu_A(Y zys*0>6Yrz*)+oW>c>qUB9-4AZ8i2XqxY*r4Mh}6_G2@58P|LQ0gmf{fza&N`f2R&w z=-x>QtpG_jnb$1A|vgjM&^y>2~vW4YMM}B<2?&GNQP9W zo<1sHX0YioI({uiZP>Gzq{QO4ZK~Te{36~SY~B&;Y@uF@3l^?FUpeM!Qcna9*MvX( z^1jKCZl6##W}cM>6ji{u?DE9+jnney22I^MG7&G$X^-&H+UuIK`3J=tni{CAsI4E;%jLIbZN4zVo>7-&$|#v04cy}X$a-HXejRIpaMmpA$Ud(ol8|J{fm zYW3fZ`0qyi8p{4ZDG_f4D&OgbDiZsiz=_Eli%Swsoy()F9@(c+118Hsg=JE;mgrK7{Tjr?-7MJo9hi+tf*()n4FZC=wa%+zj;Y8*)&=l#`%Ud6Sv%Z~vb*U$4RUf*y zV(0;Dxim0<4BG4n^v{)2b=1^TL(YC6M)FL$>spbpl-}!#-Vr%;+?CQNU5ae>%!ADh zgpJZ;XOWYbiSYhPNfnIQL})#pv;S9?2HEawSpLS}bBnu(wB?3hEb3BHp3awk^m$}E zUWVi@?(NNo8AWtEU(IWuXO+J*6n3H1pHCSW)p6An=DS)a5}!Mxb~-?uOpvDZl)P|h&`x(}YL=MIppu?vwiY|( zAIagr@Je5{l1@)sXEfE#dTG+IH0@%vkC$4Xt{&97w6BH29o}O)elvrPXK#DiPeO8c z^Mxa6WfMGnGVZNWv0R0Ng~CZc8YhZ*=vVIv0Lino);Pb#4xGRzx{>P58FeN}ab)gk zV8q7#nc=JV@-4zd_ph%}8~|DBv0pnkDn9Nh_W@8w@x|$hG6B(52cDOWnX&s3#g*wJ zfY9h98m%ISR&gmRn~{EWe)Ul2wZxPioUL=)o)z^xbK2bLfhNtiak|x@<+~vEnO{|s zj6v_+qQ04tlro+h8)L+j=jxY2G-X+4VS72$2zCV#|-Q2kv z^zWqLzcH+i-c=6A_P8Wz;HqIZM8l{}$1}n1uidAcdXR3-i<%cHU`FE<0{|u-o@&0C z%jzYYEcc1cY%Bo9x>V4Of8y`uK3M&gzWms=Bl}~n!`zvGtQqy{-^?wDb<=BqVN)7d zR!CidHI;V%XYTMn$)YRktq1^qx}DTha)}!$0%acBze@P3+2&3QiacZ=Pl5{^!~lUkIhg4 zf*HDN^kXXlwhhzqhIdTlAJ5)R+E8w;t5mT!NF`}-r)6h@(D912iq7M7Ci+&?G>UWR zQ^_)HP!I@(o1UNW(+eyk;A(^)Ja~Y@iJ73$=y=2I{qg=yo?_s9PHle}43^${&eWW& zco2`J$tqLcvQaKA7N3k*r96N=JaM*-!Js+t#%Wc|OvSA2@c9Np{W5X|9VFs+(r#pf zP2sh(A0f;Pk6$PZDt+<|T>_9vpjF&e7O;>R;7pzyNtTa-ntCYj;+F5HZE)00E&T|%XPR;QP1NU} zqao`}gK%*3`utRNcqS4VC)x78tp7BElLu_qS@tji{^6ctw>&6DqU z;5{$P90?fqYPYV?&ajv;mY{4lKRGt&B3}AyyPL=zbP{;KsripT*~4X8R)4&6*1_Gd%eQC%uyZ#$nq zm!WiJ*~HhL73ItIdDA(_W+QKUKU zp{c#f#J=#{shMR(#@7L@sLNU}yrYo)fh zfzZt+@blzg;=3!;?wx&DU4}y;0TFPK3qbphr-nW1YS3f@n z%1*+DCqPVck1qN8Txlq`uBZhWOZsseKT{5oHiC0)I9kvigHnYY4SP$NeP-A_KxZ}# zi5gqa)WNq{>EXwslZN@N6zwaJRL{oE21pKNV{jRh0Iv%;f`pP0^00A<77GZl*vwPfqHK19RpKo&BClb=!9wdskCQRVh zqYWcXbO(DG9R&rkk+#eg3THa!VfZ@|~p>D&QQ62GangVYN;LbYI;G3;H*!ERVS zj36~b)2Jxb_GGXy%CT}Rx?#RMT^0Q0eZURQulw0xrriGE<}hTMa`1`)^zR~~hJk0? z#oxj`PpO4$6~-n6CK#AM(=(xyfL9d)4r@+Y2RPvyu?o1q9?!6Sx&eqO$cHxQ<;1W@Bh6v(-i8Kc6<6WI5U$bBhHBrJ0B)MQ)t3@84qs&)n z&;gF1DNgR}Ql?tj=<%{=JBD5zdM@)wFuwQqrS4%f+gb=A>n&`76YiTM^@__i_a7hr z_XiDE;7>d~Cg<|`)sR`f{b)5m(Zdtt|L5t4g(L?b;`S4K`ALYfs-zRvfGu3FX5dkd z-{Wk{{TtADpTP@rzrkN!f=H{H(g8`bb6`wT;x{j|epj+31@%<&b?!>W%`P?Qrgsbf zs=>fQ8l!++R*&^>%^lv3-wrNftEX{P@Vc${`zh8NFHYC}0xJ)wHVf386q7S%sufRF5 zb&;uzA#As6D_CBzTYD5yO5Bimm2+!Wg92s)gJ=Las2=}5SHs8G#?9U7L5r&U(ikJ6 z0j|hAIZlt--|JDN2BrXCucb6=9mXn^(GSCK*>7s0@i;0c~<`LO|{rhbLRrvl! zsQ*|1ookIjWiHn6z@SzmmzBoKRS7rZobB40 zE(++!xy3>KJj$ppo$5H$(>l5%JT9H%90l+?`c3vD;^cOw3?{=WCM6S*E4Ym%IK1xC z+5$2$?#*Z&}v##an)WdU=f4=vwc54__utE z$9(=T$78*}?FTzR9{s0yOvg+5*MLm-z7&l#L#h1(%gYu66ES1P7ZR?vEurTQh=13= z#lOvZ)X~k+ODW-%yEmXdvz+X~<@U>=ANFp%+k-Dt+k@9t70wkv>)Cn!c-{)5ps*g* z(EDaFmV(0y?12VsXG^8RG%&WpcK4$J2I24WL6;)zJ^I5!EJ9~SfI3rtdw!tfrW6vC z`}}bpCK_ryx$#T`e$UxwI-%=?-p&DUoE$71i$oJ5*%=;)RyHS_nM}@P#!08Bwhi=I zJB4UX?4@9cn*1v+W1yNzuj>{;lvbF#0SJfQvjA_bb1;9aNmE4lB6o?&X@R{DTB;YL zHXL^6cR_tL%y0j-4u1znu^GVKX*Eoj@sSwSM0w`y=I}cQP;!p|5flwK+*KttEqkxf zYwJ_s)3-#g`34wQwOe^ywU_-|Y05Fda|i6_ZU5F5nMHNQd8Cw-$_;fJcWdGP3WP}Y zz$hDgLyp4uBS**fATfMf%ATX#M(DuoB5z1$eX7C(;k{zhe~wD{$54Gfe(Yuj*E;~l zv7UAa>;xIFo*iSY;oOu0*wei6gTkz(w+;w0`_6o|#Y(SAZ|+Fo>r_2ebijXt{JT@* zhCc0113~7Fi{z#AV`rZ50_~3B{4(pCU z>D2{ksXWPS zV{5nlDBrk2GTLh9c3-eJrHN4!0zByW;bggU7S$eZz90O1ifzJEg5Cw}bq^yzq`^<_ ze!xj(O!V1Nd@>YLrXdK>gwzweBJp4SE~};WpZ3!-Mz%{SXAM=9I=U??Jmi4$?5`yN zKn`f;;t>4v6ah+x-I~)&tTu9C{{U1KEqy1K!mLM{0tp-c$Z`;UJUEawsYi z92?{Jxm|`|sMQ`z=$|O-%8VQYeKBa+GpN!X7yKe63p! zJmQ8meiZ%rfOJ(q081jQw@l$_+5DbPT|_Ib1!l=6_WbRK0wxiGOw81mT6p)<#E)vI zk>xjbU>-7hPOu{(?V{VoH#Md56^`TQ?G;y2ZTC=MeETYou&(R@kh=$Psa%@Q{k2st z{X4r8*znD!R2*=eTFP=*Ve<7SVZe-OI0%|J_V5ikHo?0E{D$W4SDF7{u zyV%;CJG!WK&+$cM7keQ4UI&TUjL^WIG&Y=XtYuT?2s~8$WVhKu8n?3F0n|C}6ms&) zP5-PLLmyJpxiB<-So>zi$?_C2_s{z{seoI*#|PvdGZXvkX}geOBuviMI0Rx%Yn#qo zV1)0Cm8mwQh%5hUJa-m=QHdYl%>wzS#hUI@O|-{a#A>{_Cvzw~fQAfN1$dX1hZ*dp zD#8j2x}1c*x+GACJk#D!1UM6m1|DB+667*~W>f9$7YhwI`A}e|IUObTb z;nHwNI`^aEpN(L$P~lBI)kn0G&SW3(WpDDA+uiyfPs68i)5ppW@NroQ0O{bcn8FZ* zBaDUcC~=W_BOhC=kL^qf&YM^VX&abG7R@~?x>MMaHzKF=(5rh|c zgsNUuzsW-3&a)yKnX`L3OxvN1UO4!gTG?&@7006vG?FE*p2FN5E_@0wGLFqR`?`Mh z>3CMKsft>x>Ciczg7;XPEv0yU0i0MX~K$sT{pkoC-Crc6VP>=+=My?KlA4 zYzkCJDvEPk>JL0iiD3_Cg_GUBX_ub?tUjy8(U}FOb(9jLU%28(_TE{*gbxn~v&g)~ z5Y0`w^Usr)gWo+hMDDjhza7%CXax1p%y@l9{ElA=v$TRh9mYSIOSjzTh?Jk7<8>UO zJ!!)BqGr(4!g%(8S9$!HA&@VLdtzfAgoDqQFqS3z&G4He$n%M7YuZ7YTg;3-O z?%g^{zA99&Ts;kekg;x$eRW3Lc=uNA-R+H7Bjc$9?sJQb1T<`vm)7AAqEsbly(rN~ zRuUjGN;(Q&qNn8XUc3FhV=hxSo!xEN7e+soNICft!qcq=yTJWBBPVw^fcUlVi2>jb6uN|%+oqrUMeh$tCNT~8CeYvwiX@s^{!4Y*cR*F zF?5ppgjXm^b0mp%yQ-R`_WI*@^i}UYI6g&VMVg>8Kswj&C232=^0y6!0_l$ao4@@M zuxfySWCFLwqvbC*0^*+);1N+C+ZeH;kfVj>EX?tcrp>{L$PRxGz~y_mz}8>HKJBHY zVhp~iX7QG0lB!fzZ9{F1ZNh}32gCM?ZMT8!CY%l>+8Gptg@nldm!h+!<@seY9&&AW>VIhH#`tKQn&}fBxkS=k#LfoWT>E!qQuOn zJ_~LSp~dad+qoz-)B6 zkoVj8$9)`6>`iRKd(FWTxbWDCb_Vg&5jHnk%1jSA8qyTR8YVbwrihQbnnvv0`87xX0k}IePTSQ})^F5f z)<&e5J2FuN1m!d5t6&nalP7+e^Fl~5XtT4M$~qg)g2rQD3?sP3Q>lIsfjCWF;h{0SA+N{?q1g?ty}}o z^s}QtAMyGgp$0MTNqBUKft0@AbahC7^T!fA>WcZ;N8iMsbmR)W(NN2W733+v=JWMk zAaiileX^_l)akcLpq5A)!Ptj0n0G4LZ{+(7FXkPe5b=_eCHBr-uD!LyDq>TcX`eYb z=XdcG*5ZjIh)*>`l;4J4c21^;IY8Ik?!%&sj3ZS_xU<7h^76d}QoTYU|0$?a`0ZmF z-u^vPL(|4~(dsOjX2)Xw*s|P=shL5#BhF#eF&y9LP5y|7i+IT1ZTM~HD2*mE^zeq$ z#5v}MwSKP)l_mUyIfr59a4X2j=+>+CZeySOt{oNq@C!?&NpuEC6Ia&$!RGf*A-Vm+ zM(0Z1L^u_5Moyvp$4Cn3(j`ESk*!D0{*Ddm$_rcUmGN&Ac&Xlc35Lp3e-OzptjQGW zbK2Ohrl(m(ogFyHFw)W|lfT{-b_#Zj(1py{d)+(XrNX(;Q>`K1>57ulwwN9Wug&h+ z!1PCuXgxnI5?i|AK-#;yqWM&XsvFevDSt&1*1+mnxWG~?;!8o|*{aKj5KYZU(-ytX zH_x~bCCf_unxZGUKD07UeB9t92-`RyhQ#-x#Ohs(6t#Wg@tS?Ah;@S%U*Mp2ub)t0 zRy|-}z1&Q77$7)5aoj{TFF~?YL?P~(EzuRaSiL%P)6h^-CEcNWJ0>S7%nhunBv31B z9i|tPONPfObdq8(3J0pBYI>Vdixcyq0jv@=#wM1S=aQ%o&u@o#9I-kql+&1LJgghv z#AKgoq;OHJ3}n-SI`kpp#_%Nz1cu)+rHn=hvXs&Fu{v;vAL2ucm_kXop2!5V8cB6; z-5R;=E?2VLvz1!yx?EXyy7^~cS2c8qbNrJBye8%NUVl?h6p%aq2lig)&P#jX*;_`xkx@5bL>b}L@q-s+wpiITnPj=pv0Bm4FSPxHPEFZY$NA56eH zCI%bwcAEH`3)Qg+HLWc-@R=r3sDQik%qw5_D zd#O{1$zi*>Wjkb!?ASU3AGfqO9?5{I7;$P-WtI2tD@>yG%DF9k<@Ib!`FZ~C*IC)F zG@^Dp{Gn=M^{KI=TW%?En#Y+Uh>SZpG5z6>E2G zv9S~Ymg$IoB0Q;DD11*6xqVbkz^MvmGK!x|pl*RbeOn#w^nz|!$=Cp5GWt`UA%OMj zphNKKsYu&l)Lxg$2rDA#ni#&Z-pmE&dDe8kg0slXC(PZ`)bKYc(Iy%H7VbNpw@QOy z&Nl*+dNOX~yzZ~?F9qe6?PVguT}=y{wp=$~ZdBXh3{L#)B88s+1E>`9> z>DGv6lXRZ%g64L_u--~Gr){G7Z~i?Q-FH@a4fJEd?cqTSIA=D4$;M0XNweGm%GHJh(I7@$5kHs!lFl?GWKq|Ep}xBMkT z_WACyk8dx1)ED;Ea%ja}DHbDn!=DX#lNy4i)#;TygyJNm&+u4yOA@`8Nvs7>>Sgwk z>V;QTE}e=hgPazD(HjoKg|(Nv zp4n)fpWmo_t@zz?*6u9JcO9=_uOUh~I)`E`-iY%?sAo(m08TWu-)veg-V45bp<~N6 z@U0Yg?TA1M%dNNSO!hK+nT!@a9_EgZWH8Y3!J7{tTsSE^f!3dL46);6XVp%++NFcM zgbu1Ab=QEWdn{=L87#Ev+_3$~cn~Rk2%cJ;g7w>L`=G5;BdrhfRCO@ZF)a6-edv6B znG5-?Od?dnd)3;Vrr?r8`-j0E&-PPvG*xSd>doaYGCHPUCU76L5Nd`TFNUATC>f&W6nKWuYYp5 zSMSLI{_HR?2X%3IRU+=B-+6Ua0h*dm8GOFj5A3Ze8@p4sYt!Mmm?YXR&IV>b?A4J7Fd(xM)R07idj$U^AL-FSCK;}(51(UK)btiEiJSqArNGpZuRbaVmc zv)yFi8PhW=AHF-q>gs`1GGU5QcWK~ko=*H3y4#T}3>lq?yL$Cm*npU3*x>Nz;h0qy z{_)k$ShK~vk_WSwQ{9kWFpoWjSS-5nHE?r)%I$mS6+cHFWJ{t_f(}~)Zu9l zYDECBA)eK<94ucyy54N;A|+rK9m!*fsdfnYLL;m~%|T^!7gSO2rwu_$$s`kpBp zA%97MKz;YBbAxp>zJ+T4BpLr|T%#!-zWUy$YtN=F?V`ybqkOk_bAgNPhv==3L6GM6 zmFV?3C1gDlK^(iaPk>b0LnxUq8h*ObnsR8Aj2HWnh!Yw!%vGjf_C-a; zW{X|D#1_Y}|Ms7SXf8|m&yGHwwt2crT z&s^lRqgGdss}c{CEQ@%UUotrOHuPU$*SKKuc?b@MlkNUhcAy=yY9`*^ob)8wl5@@0 zruA6C+hk^Hdh9^bHZVdAaX2Ev7O@a+070Ozvgt>9@-O*-omK-}Je^p_@@Rg7Ge>I` zfIvBD_2F&~6?jh`5%Xq0UvRbT7rtQQxpN=6K1R@3ZJ#`PsXOa)eht^Zf38;*2&04C z_7=ud&V`!?y<6;Vq*hc1C?e&HtS?4&QWO9*5{o;8DUbx7wz>x4_wS{L%}Ic1qQk!Z zA8CXF_5b&1gt>YiTt^P*638kAYV+#We)sNO_~Q@P)1FzR6KhE?uqldcuxSZ(zu&$O zbNyE{0PQcL229YTI1pOc`z(UlgTwcnCxYtF>WDp@M0m}75_9nhaTqS07(2@>nIB#* z7~b6Y!tQq3rK~qsr_a@94|=NC zQKe(xFY;XLYH38kC~A0J>8I(`hQ$|*&sQa|MfmscII-cZ2UG#gfVcI^ZG_BDK(-IR zs0L_-=WDo`rJ+j2KaK?dX{-Esxqzh8q6^d6eK0&=0Vu7a(MqMC5VihD)x`qzN6~FWM|7eX~ z5i~b@??^+ueu9!cc|n-W7&!gs*ZVAsKH|j1E{G~bOYPy$i~+&}eR$N`^vj=p(ziQ* z_8?!}dG2Jg^m)NhNNC-=t6BzSmD5BHrb=80$Y3C}H}aq=E>L2LmVUyMRwo`9ZQ?(0 zhn>0z!FRjPOTgFrjl66Pb?7D#K$bhhE)57LB@88eKVIkWQX2U={aXadY#~3MQ*tyl z!5%=t1leF77x+K21p;)954}C80_%mj#7^R3CI+T=%@8`pD?)Ck(G?k&-ZR=wkOy%@ zf|lpP#)*~z!ao^}@6zMX3seH4uVhB)m99#xGFZs_aG_sp1xZ;j57M~=fs>($DEE0E$0f|6Pxrij3Uy$8OD@%AagmQuRJQXZs4-$ty|T?z(2r_ zY5$JbB@CwVTYx~;#u3qc-!yRhh=b(IE6=GFz6RQUbmII3xYZInN4eoD-T~h& zlo!Edf+lKM!d94NI|a`GHBL*fad8NB{9x1Kx48cz|Bq)^6v_>}X#^ z_$<0c%+(L}9X3gSN!yHJPnx@kfw z8vG7tJqD7zwC$G2QH^|r8#JGo9_doM86YiYf4f~4DE;Zk@XRzgn(IP^(T5-B{)8~U zOTG>nt4z!3>vjRN0ujjBN4zG-C|Pr{H8LBjXX*e>wqb$|W*Eyb;=ds^7zrwrE*J-MQM z*g^76;>Wx~=J;+6;)Tc@cD~Eool1K_blMh2d3nR%TC8OKiM@S*If@2$VtYG3Ob@c$ zMU2wGeO|?fDvqD2am(uywYOAVeYTZU{YYf(>&(>&{MHv*D7l1Q-yOQ#@a{nwjRIv0 zT@E*yHxaVIzJ2XtApJ>rsiV4puv*%5yl#JcF!SIj;}S@63Oa(<7;vjCKk1=?sU1&-6>q2DvU>Iy?L_%A2lm9pnpk;m(oK1O^k`( zUJe_rPeNQ8#?7qbKS(54?vYC5#uaR=Pn=2)wIJ?=Pr{wcRBW8Y9xw8g{^@8c)yerL z;4+D#E^$;tAF$*ZaS1iz^;=hUM!i){6w+$IHd90t(cuMD5e7DVC0P-6cfYze>V7MJ zmjcCa4}4M`C_RVFjsf?Ec}+o6LXmob7Y`~d{6&^{4NE;ov-}!2sLy2hZ4`c-VRF!h zYE_WW8sNoHGbn`~{-4F&g^dc+6BZ`ZwS za$+bLsg37oq*LhQuW!0*J6XN4u@HB1(3~*H;$<5v7kbcBdFCgO{GvYGNG;IbRIIxapwp+%%jEKJ@Q=l*qZ&g%<7ZV5n@zTNlfYH|$^n*kqGmq{M8C&)Qd)p(rg?3_zw|l5jO> z&kxh65ASu6B+O=?rj*UTa{#x&K6daO5!1cBZl>*}j^2$0H~0^}$>H7q?a%Aqyp!Ud zd^mDGRB*W8I?gg}@%laP%EJZOHeTC73zPHI1RB$$KMsoh*#LZFFhpeGM{|5SUbq{j z<`*9>$e0e=S|Op&&}T%wlqQeP{^f=Mzl+SAD@e9rH%W+}bno|?gEQi@Dy=3fo!k58`psEB-j>GX`xbi*uopUt3hEbbD=n( z)cFscVt9Sku-w;CpshXWEyFtH_L7+bjxfhpE6Rc(SQ1k_ zE_}vA{e$`MoNp2-bTR8CP$T?QXF6pf!+4tXJmQk^UeKR=2JJpNo{C-Xv%fPcEDh^u z_bVg7b>brX&XhfQKUT-FfYzUiH0MrG6lv0g|F<^pG&^6PxI<`HL4F&1$|EbKhwGC= zpK8;KMGI;`Ve8>ZyqguYAJ>Rt-?sW=vD4QPkMS&wn=n zI9r~H+p$Hj1sFmwlOi9U(`7vXoqx{m93H(?vx*6JXmv)$Ctyx9JBL~Y+duQ5Y-YH6 z&xg#gh21`gkD=1P{;dtb%^nK$RJGJSEco6CP!K-dZ-U23u9xuMl}iD*95&(A9Aq}fc*A8}yTZ5(xd5?rU;%=& z&~5TFXLtOPLYji74@_0S!jzTglsrpl!bn=W6>yv?p`M}5zS$fTowY>hl4pf1wOd?> zudn495KLfrvd1&ovOzjOg04yP=W?=i3Flbw_XBTi(!wKl6mjIBdeUH6k)?-<)&L9uphUoysJc+OQ?00bK{k^GmMEKpVwwwbX<{ z>72h0jUeQ%HqZ(V6OXyj?h8ze>!c6crQ~_NybdfxDI?wxGSRs;jn;%qG57W&Dz}p1 zrcVG4ijh~8-6*}E>+0jCWEF$Nu$a|Hg*#iHdmeqr{nq10yWm+uxKWa=&sV&2Gj0hK9R4s`{G+(iY7&W0Uyz?CFxqi~be*NM;SWbo z-7tVG8yA4R0S#MI!3jF1v!!VaZ!C^Z!*QhnV=7c(>_K5Xr<-#uJezTK0nPH2K3*;7 z)v6<)Zr=|`{CP=NH0~)e*%&58)?U>VFpGmdb(`i%C*8M5WD{`Uy`wDwniYsj$6p*i78PT^HXySx!HVs02Wh5o0DyTwjhlgn7 zj}5ZHG`T3PoPLJpSLXAJPz7K2+2O4M$lmR~{u#C!mKQc`1Qnm98sVmh1rGBU5M|6W zEv2zhQGG8u`f@9REHHMUu;J@`tc<={ZiRE8pLZdTdAp~A4RIvBqMC1JptFr6SWh5@ z_;ZHBYqI%IkxCRrhsUnNe%`q6{(zevpv|$QW2iQoR0o`1)Zh?2r|RzvVM?%Y#ZZtj zI=!Yw487^8f*r3Ia$jI?*6E6Y*YejuXHISh{CU9Gg7+gmDkrY!jUe%n!1Optr_u9C zU12N+nzbIjsa-lYG`(J&Li3an4gkUDq&`6|8^=SEi;}conXr4x5z{hD4fWY8ozZ6k zsJNh$$y7wqoi0@^_dCpO`Zf@3JMMSy?@bX~EjQrV<;9GR6g?3#mEw*JNX3Snixa+B z&WZ5)RAT7_?0EyN;I?|7>e#Nf3FB)eSqk3>r04*SZH`I{-pz~^X(nsX6t*#ASpSCa zk7R<&q9DqlnpAJtP{ut#+(^I-fJ%f{mc2!jmn`T8vwiNC;L@2bl}!xXJt~oA7P53H zruuXfj=Z=N)z&6H7oc^!*`#6H8nBzEK%CdTi9#Qe9a_J`Jj$_~Q688oxsjRD6df)U zsH2)3l7ut^fj{gRa)j%PVkpx?7$C}z{_KW}x~>fK>>pk^5)jfWOd_fxXC>gZu>tW= zGuXKSU**^M=&~2_3z*;~5J!Vhfg7ak1Xnjxd|KVy40X7NzHU?hc3wTG)441zYiZO9 zKbcK^tadav}uDER4#RKKM>PR$#wK-H%Sb9bi>i-=j8o&d{oN^z@ozyKrr!fz+G8u zW6##4%jv~1RVm1W9y|g4FjAtr=d$}X)z*1fUm%Iy4K-?AYPdw%R&(zzwU?^#`?Y>k z65dv*emCFV8S1`q?so(6=FnBXG}2c_>1Vsq*=&v}U$Qe2c3T9R;kOrK4dP--fo|29d=#}R#}%6Xp?B{Ro)fOOTWo44r3Zp*W704Kn- zR7Y3i%JF+xs!r8|Iy23IC>y!0;PSP5NTw|HYW|CY&E=q?AIT^P>M$t5f{usD=Wd&Mi_l zDnD0kzQ;ii$*lRp|6cXq-S|H&8~&t*I)Z;mZvAOxlIHnqSCq~LUiXiMa~aT#gxTH1 zTq;LAYl;oa2{7E54Z^>At&+nJ=}{Qhacj*!qPseH>6a`4xn$AU1Q5v+yuexe>Xc+FM+fBI^8QoJ zP3e5NlL7Wo6SJ5+W%ljuUw~IR@=4-=bP=9h1{#l@qs%;9C95mnUxjLvekPF2uE(P^ z!YNn52TUo||IJtd^%c0qLN(lP@!&i03z+Y0r=*+Am3F>DlzGT&k(mUP^hjzvbCk#o;jdtsw@>>FR=Qx86>UbZ<0hyP zs%${?&9@t3VV0G(>`E=HT0B?U=KB1^4kiqQd$GNWJ&k@4weLAEXE#2C_^?VpyimV| z6pk)s^oY8APsKR!-ZZbToVo1lfxa^h9>W2@=hki|g(BAL6lKqR=u&I*blB|1Ml#EA zG8GxIa8A6Cd1J>zuS_vIN`cEa=flE&4Di}W3=cL^_7B*_-**CnqVJ8=thPlof5PG{h1?=vPQ zOWz5zq#N;$hP4;eN>8pbV&HYnvY6BcGj?XQD`Tx~HB8gh1bbzDgksLYg)8}hRVx<2 z)ZwvsOSDpRIOd_2=t90T4f-z?Q1W<6`%M#3cNOE;UJlFfA%vq_zN7s1MyCXXq%2V5 zGyc~`KIQTf$?&gF0aIE@`eFkl`EClfH(ba%Sy}WN94Q6y{qiI)?d5fQ<2nVWtvY_o zjVpmQU*bzbe|<#YuYS(>E9PWknY5dFuTAsFLhsBev;^*m7c0toEd(qP`*2=4PgbUu(5!aP30Q1SrFT_nliIe>0bZB zQ~UOIP?tUrbc(=}h}VJ@iba{E%2c7v&$imJ=7Hhn3KI3*B_giEeGbM9%SkZ}q!L2h z(8FHAmW`b%$VgU|_#>HX(co~o3aN>f0+$c$TvCvYI`Esfn3HhdHoriAA@>xUvqAe# z?gHG{3-aM~H-Ymxt<9NI#%dK@S9>ZZ7z6{zng@T>5yj)YkSNL}^wDG5QN;09y%uBG z-lNSn#QXI*!HgRbwsVDH(rYVu2vH)1gq=BW_SpmExwwU#MZP{S>lLujTK0lm*Z(dI7_idGr2M zY%gToW?rWod%V5?Y#B{4Ds%RA9Q>j7v6j-w+lifUN=T88q{Kq9_3KM!5@M|(u2mJUx;QQPD$~btIIF>>{fq?J?05KR}o};Yzk!Ojs zP;Tpmm)C$$|Mv(EN=pHNfy9Hxc8db5wYqwjVsOt@0CU8fNW|L?qvH$pcUe+Y;_ z-?^=U%6d8r(#{~&+TsUx;zLu9e!br-jDC8%Z{&a-nRd~o{4z)`=)uX;#Lqxl zn#x*f%&4fXe9yQ`I4x(68o%jBy6Q8<$u5(=*M0tLh>5pk{P2b zVz|4V>ee57X<^!bgYTh0HLe6Sx*{%4@+#`Me=n_c@8I^_`at)0v1#-IKwD@`zBgIX z-f$o06A=@rOIrP1-R6W^TIOdv%cbAd!?8}=?!h*iiH{%4{eCE_s$x z@(=OnWKn2rqQ5-Y9&xMl1jzou#8Tpi(59S?lLqyDj?*jrWhVCPbdw6b6aBcv{`66o zkO3!jGUJM9igN%qvfs;4BmJ%1^7>2%z%V3 z!rf2CX6CiA1%d#{THp^Ev|Wm>G0o`=*ihXODzA4%n&f!uJbyRn3h(#~Gr|ZKe)w_p z(-jvo@a{C6`&N}q%(}{D6Ma8PEi!ZTx#I4Z(Gi(m2I>>O!;KJh%_Q?kxax=?d#9aM zj<8^CrYU0YR!-6AUY-)nD5=e?@Eo>msaqklF9s>&D6EdSRSt?*#S;Zw&c#&zfZD*= z(x0F==h6Fd$3(q0Zo23_*|f@W{<>H+xpSQSbZX%pvY3Ts64Bpucnx_dz&D#bD8$`j zC(YlrGGU|WlYX__mYA0XeZ#xvbAv`h2DexOz(EVV0t_2^CG`TBgpJD4>Emz)fuIJe z<-DdUt5C>8i|4`Vukjsdr!U4rPDRD0LowxM!oTr;-2QGvX>|aoZW)$fPKN4@7yMnj z4!7L37>N7R$fhUbq7{r8s|plQG$IH=2y{lVROaKY1IdrwgYz=Q+OhvA1o#K? ze%%QicC7pU>^w(QQv5|ZrQG|o{YN#V~^WWV{h#{1+0 zCq+ZE3<24{^WjibqEUpEwc$XVxKW!G>wNyY6|Vsyr|mQQD>U68!evCsj7vn4J}X?5fK&86XQlwRajPwe74A_S^w}O+YnO#QCgY1m zIn{8QTF`WdT<>{)bPjwiv;R+Sxk#mG zpDSXNgS@_{C)yJ9la8~#Fgc5SLOGG?0Bq^Ycd44MOZL~0i}M@#Hb1T|24zZ837_LT zP@0xvksV-7*Tcf&`XuTt2gv1D)Wf!fSRD5>AJFg zt)B-saM5i~zJ+ONR+z9~`R{lC_2TvxprCqHZ}{)VUT^OQKDvMDP{nTb{LiocZHiC- zGaLW=XCsFDFT6Fna30=S<#@kp0lENRLpny}&ke^RH3WHwAYOJ-Le51hT?xp*%zq!v z&e}b2sE5(0(Br`ViuC!2+4SP-{AMckc}GV?{|9o2FL*x7zD8l-(r9*%HG~MUB(U&k zLM1KaC9IIf;9jZmwQ3;ARUnC{`@Q zZy|Z0OCJMRkY{i(KU1f{BK{J5;knXUq|JS$z@pOHxv#-_7~a`o4rfymV#u&s6^D<@ z>imz4g`JT%T?9II8jb7DmXUlnbJxZljcNj&&CqWCE`DU2*3ne1A!4k+^y&`GQS26P zP{6nGsUy6z0$o+=fp^F7NWf^D52-dM{0*b z!CFmmOFaUk^{mT;bF@^f(oM~qy|z%Iv}R4?pcO^lVE|D*#t-9RQlUIhs+#{+2JyD6 z-pg6I+pdo^2H=^tdp08dTUY0rbxtNRdq2JL+GViEOg#m{)VTP_;dA%rTk>FPTYRtV zfT(SXWX6j<@4|lfnQlV>m-W6?TXyn^$?SO>rmVr71lJK)S|1LtExe2bYqb7(VOS1OH!uXVIyHD&|P$qajG1x&=o%0$L}*2Y+{UoHge$pi4hc-x zfNAJ2)eodw(6+XhF-wHO@5G7o&+3Bq9<=)SWPf=MukY8Ocu6~5`JDS2hZ(#DU)}do z+LzV#A{c(_W-9C2H?Ig$)b2*FkL%tsxXg3M-282W{MdfXfIG{uOYr~ixU>IY3%}tZ z{{TJzvD*Lt1$zD$cg975@6n1dmTCGp^&SK|9#(i;59eD5cp9dtg}ix#9JBIKk)CMj zh^y@$r5l*q*FUv?B)Rv>7G9fCwK8as!OGKxaOLPD59o+_wjxi z$bhlQy8EwOA~lE2B|oJwZT+ zN(2yxDf}B$JO5Pep+UV{;g=&DUA9Lg+dCj@ep|@YnRxwq#qQM~&mH*q(Hplmw?$h~ z!*|wFalqV^m~KDrXA?0H_c%%%R0+(+yd~fQ4r zk5A0le`yqVACa^@z|cGXC^)y)ro_p=>&(;%oIIaub3Z=iP8G5z5{^)g zUbS(S1(n|=@*)GaMZ|@yzO{hm)Br|6yx;@C6z#ys=zwGTZ=!p&GjnguoB_%Ak}2@i zU&hhvCk&0XBC#13wi77wQJGLq=4U?Ot@ZSiTnoMU^=geu!i2nJ9xN{u;uM zV>3squJyo0RbIeC3p}>++er#o5)T*vC|c+7m9AZph(Cf_fQS0O{Z~0zVtBwi=Nr>;Zz`&PN&I^&{Ib!G$M*xK=n2T?%HM0`Z)toz z24KG_XYAW|n|9hxK^37RTZi!*fNSMBgzI1Wa-y?yfa4C~ILH8|5G!gd=l_5Mp^Ph+ zEO8lN(VK(H1A$@AL|j9*YOx5LE?@Ow!QI2Rrbz9LNf>RCGA5)dZ-ph55hIw_2Ch7~ zeALU^d;F84+t<2Bf9gB!17c{@Qzw$1I$o<=y~Dp(v`yl0nMzZ`#A{6fHy~R5XXr$X zp#?`ldx)g)_X?m4(B^2H%=_YdU#FtiFiY6sI1%Xvs(3y^IO>9xh7VLQmQZl!U{T<# zEHP|A-=p;@{WEwDT?Xr2Dicqb!hP_7=FuBiC92ofDJ?w0R_cjWrg#=Ym7nG@!lybm zCO#GJ?SArg;ke)CBq;2#()YT(5_TQQVa1lYjF1~9*KgD)^3oj>uW2nA&-P6Kp|z#C z*4Rd6S(&%+?T~fs1}{~k#JW^ET=ao4zfCLVIRfhKGo=^(o(anS!u*C3>pf#9_(~5S8Y18#*$hz5gjw_4IGxEWg zbSb9)zj#(TfM@mI%H{%n*;fLYO~kmD*-?&c2dI-?%6Pc~j|wE-Q3mSOb3g3%1YbV_ zK(nak7B@AzIt_sdeVLO4yK=1Nf~27o5v}jv#ih<4Q7cmv+~)x~E9P7OL3C{Gzm!rx zzg>D9C#T=K6OL~B6^_RJ9gb$HuQ_mSuPDSu$g{V@6M_ZI$+53?jkc!>Saj`a41esJ&3y%j zdyIyFn`DhM;pCfo9dtxj)SaegF2l7-dmA3 zCC!gkW2`1ikm&I@Lh1bLJSsC>R_!x-n1U;{rN`b6heYjH;zvFzXojivrIwU|y+JBp zsP7ORW3?;YC%H1V|#oWH5#T#0@O_v|CSss-*ZTvMcVb1Oe7{?42) z+WmW@RHE7>e;WM0Xz)JI#7k2fI{eBgKVp4n(L3>k3F;}GE?P7H7cRvGdZ7`-QsWp6 za*xxx1oTFD(4|(!WN72wOthA8duC=@7ITy*6tX4H2C$tXt~=l5^OWLsn$}8}&NJg1 z0y~B25QbSmoZYX7tX2u67kEvc+OGhh@BeRnr|IGk#LoUQ7aNosWR}o?LV0ycN3$>A zFJu^5{A|g}a^NYm8Q;_Ed3_;u(e&@@W>)g+;pa<~DlV9q`ZP?$a(RCAadJXTdTOB% zgK}KggTILQ5)aJm+rG0*i^ANcWAF5HmEcb{*_B8EeM3cQ{Y`R*+Y$@lfmr zk$isvP%ap87$za%)ePD!iA(uu-#0$beaWJk8LDIek30_)KKmjg6U!6>;Fz@5UH9U)Jn;nw(2AlP+e4{-P ziW2P8P<sQ$ z0Q;-ykgPG2V&Gh+nLT4Jft8)@p!~97x`YoS?NXrs+iYKi4MX(Z*OODr&=}tKlyn(4VQs$en%nB2*~t2R$F@mYj3{+GV*gaQyMp# z3-qv?3ka~A8}t;wLzd0a?^h?@>CrN~R`2W;dU{-gZOR@Y#6)ga);c;rO$oFdIBMHa z^r(5Y6Y~w*Eb7**K~vbSYUt}bRPVkvGc5W*ouLCP$_r~~)TcxEby|uB?M0W!!e;!4jeX{LlpCa*dYyJ(oYCan~TKfUZ zVQtHvSHmt}qy*ACFtjr=W2v6q$KewlFER*j?Z>VZyph@G&(yRfywK9X=T{7rZq)9N=%%!{t&*Jcs#!1yb7*LYS6S4 zQxVg5&ANL@j^2oD)=HtKTUG{WlEnV#)6hv~l;#_IH7#jvKw|sw+!$Kx#3a$|C8|U( z?2k_BmEoT`VBiqk*AL_OnTl5wDHJGLd!7p6SHq1MhkSPTb@TgAia5QW_tm{2tHWZyu2! zNy%pn?Zm$NA-cMkTjdA$j`8c@d;B)SjHH$ueIGdkh$_vra#ROxgAw|)BeZ9%zV%$# zc>*49;QL;mkTq?$w&g!Vof0azO=L(w1z(tKSbwjtuHJ?fnmAA>En;yTT06~zw}y&r zV~}tYM($;z9aEp$tE&Q$S58C2y@uFFxB8GRLY*_#?ITGU>p9dt2P zbA!6=)TCuFX~zPuLuh`t8~33QR*bFD=3fbjF;c$fCo|49rG8FflnG0mc)5bhNQs0W z-H$rJO&IW8ik0cp__L8u`s|m&g_kthbaB1Zk5#MRoEN|2bULf57UXqjT__9a{3*Y- zZA~EYmwZm$`}|--Vq;NA1Rv%bbLyv_#TDA{i=sPRcTYzH(Sw9|$Luemsa~J#mdu{? zbzuVj+&nqKY4$zjV=44c5xelZx6C*3!|TCJsIX79F)~OCKG{t?t3>7bJHVkXl_470 z#?ZXinT}crDrv5kch^{s@RgS@DdhLTdKSDFG!PVr)l8()Ipfw7qKxl27S;bg-pvgpjKF4GKU*T-*G}Odd2#2cAyw zi4J+x)xR~}?D;b?4+rJar_`;*k>SFliB=!?2Osq=06oty*9?fN{MQB$L$5VgeK#wQ(^-s4M25kGbyo}=~^tUy~Y>Ik63$!Lq*?Ul98 z-ZbPX+sYlrFV3nnt^MUsfbu9`u}yh|&>kQaVijFNi#*`E->J6d)Q&Nm`R2AK$kImj z%1Ua+K#WrDYjFfQ!naV#q!u3BEc~S~U)jV%XlseGWEpr?Of)sb;A9CX>rlN__#tgX zuItFlsu$M!Kve6(x^A3LLy1m%IfV;1oMc)H8b}Au<!Yru{VMaKA0pA z>F~~)H_9@@E7QK2>lqTOLWJkx&d}1iJX4@5;pO)XY{!DW?T^^>at*}b%Dy3@k`$Mt z59!9bJYO}XX_>jd0ZqIO%}iwbOwhWNLCO9r?SeG)R*%znNj;jGkx#vJrUHu#H3vmmA3oZw?AeamG>!RC$PtW-3D{~DzUlzS03gSE zED-LxO#`))(SAAOAg=qvDkl|z?tFa95?@FeK~q+-4YFy}INzC2G6kPH8AX|~_HG3# zozQP@en4olmOMCg?6}-!hHF#(d`dGC#4OGTu)eOR!5g*m5sih}!64f|Qerq+eYbM4 zja!ym;LT@iX6y^lS--6#C`90qqtXJG>=q%TJcTe``btALdh)$c`c|m<;$0ScSFfo1 zP*^K5uPS&pBSiPe8}i{~X+`|<8_V#tg{>P(Tf?;B1x~$yh3Vm%keE_Th-%Gc+lhq) z52&1jqvqF4WjZ&LYA1&wbc~6(wBPmAasMpP!@*=<>_N6hy;>^0vDQl^Mj*E~ZnGpR zAQp%=rO|GW(Y9>WkOsOMU%i{>Pu0q$9N|{<+h37NV1H(z!t8CA#^WiL$Sx^}7Fu`! z&l6!lB@|rBc@zbJFz2T0j?BJds2CY zsb>fpWa(N_g>QY*L{pV<_p|VxRS6Mzxb*qRX_MdMRbZDeJ4VNF*Uu~t#MxZ%f57M4 zR(z1=*{?QxqaU3PzVbIF^XtW>8o+PZMdiP(VeEoE+w*A~znl#aDPPP>$~SW0;7EdPMl0hQ{`@@2)vZc?Gu z#~~O$TRU%>K^kGzsK}Xav1!WESw%2o7DmO(dN_r)M-$escr_X?#nqCG}%w5zMjDqb_! zOt426eP(fjGNIHNfH+`vhmO8q8u@}q;#1-pkx-t2&3DC1JCbJKuGypWs zJr^}{-;Jq67`)lqS)pph+lcPQq3InWS9~M`fd6eQ**lBR1X%0m`C;=LhXq2trv?@< zAsd5Bii)GC-oD1ga3+oJ^n~jNU${#&N0>z_dNo}NjoN%sCShPc^Ki0lO3{jBoSjm@ zku{y#KVoq+y%)2@71tyP@8HQY>&&{Kza+yf8PYa*@uJ~=_y{&Y(S_R9JT{qktQq^n zC#vEFHKgIJ?B1o}o9EP;S~W*YBD+Qc`g9f>5K#h??S|(ZD$uZ|E)2SK;^xlAea#85 zKyx0RT-j;uqkTC@*_nz;i|kaySVU=pj+nOu9cmR z+O*V6Ynmu_H|&0ZNF6g6DlsaS(doGESFH@4<Ub;qo4xGpj-In~&U)h1!}GeT|Bj&5dm1xJmH#vQ%20 z^L(iU7GCdZ%!A{(*~OEg1*D)xEz{Zgu#RjWjGpDxEk~4A>g;OP8|>=Ld}t~j22TxW zpU3jRNsoqBF4V926#ePt!vi;joCtL&w6cw|R|0>KB&+%egyxIcTDaXYyQ}K@z)90| z1aqRYf}sgS)IYwS`(CnLNk{qM>`4K}+P*)hSA2ag_`ddK>6HHGaDSx>TMT{Gz0 z!F_^q*yX4_^ygPOk?3{AGrjax4GBq|@@ct(yT8S+TUCRccNif-5OMbn_aLW>B%FaT_=v*cfvfy9oCvtmQRW8rByScW&ChMz1fR- zrPh!dpf58OH7wai&21MysoIwdaWz#*-RwYuh>2rj*evg)4VITrnN-I8eVjH{ywPW> z7u=_3jG$z4zuZ_kSjT;#zTMq{)-gmQ&&FW}K^Eb$jKM=*M;WD?Ds(`^7HZAjinXF~ zjHI8-NTgj)iWAJQi=Gr$#hq3|&e~(`-pAN?T?!TM>|+i!DYeR~PEuG1qjdK&x_39N zXu3NCz&L%4(r>dqxeRbsN!1x-^fG$Gy0>D5Dxwwn9Xbn(-*NLSFX!Oe8m-kz9q zwqeostBjqImFoGDB+$rl^(C{dRUT}6Rm=U}T2U)&7zX0lMaQe77qlCg)Rh-@RTT7{Sb*l)yNgPeU1Zr(9 zMdtg$J8NLhh2Uu$S~X=_8?JbMb>mqVO)rhBwz8-;{p!J}g}$&@Jy-Y6jo38|xhIu< zDJJGpO~%S%KPms%&2(uQQN*}TaI{jo^{n~gdWmv2`(GJ96=~R^_?HIE;P4+wqe0CV zzp$!8E~ZRJ2eM(VuXAVv{V2EOmGNX$CiDEXQVFryvn?c$(-6-K^BPec4R5Y5ux;vm zWMBk>y^;2Djj)LmD(sKqUsD2G=IPo>r_4xBiPg~3b4_cw`OLd~hU_OxYmVoVW+0LD zXg$8?x$NgcQ%LGBmGV{A9Lzj@0>A^~>!v4NBc3XjHnbFTMMLM&Aef=Z#r4d?8pMyZ z_?hU^c$)VTTenJ8Osnci`kJ?_pl5c}zP1o7rh`C(MHvSgUYm`07R|0;mIPrlCnkow z8s6YZ10Z=4AycT^ha?{aDc-*3)V{jzE0q{K&C=|YeR>D(zl3MV z=UX_?^Rv}xPn6n{cK%h{^uoygU#HrEt4&nN?=A7-Orc09V6o}t2YZF|z z@-Xl%dqr8MA&(-Xmlc`sI5mnBqloZ+e6Tj+|a1$b$k;qd361=R#Qhv%7ET?zJkRb8Rjjk6@- zwnLm{u9V^sdj&r|tD4V6@%L{Ynkd!IuT_wlFHtiUWfLtaR)(i;c0f5p*Py!_Ip$x!ZE> z%U=f8n-lq15`j>9NidM8Udz?YC3L_!Kdbi*Y`++RqKn8)log~4Et0)3os&#Wt>S#) z{2CT7CfBx3^A5(>F8Y3dx|tKIVfj;0Fz*kciCQ(JgMWo>ZymR%YjW#GsT|dSUU{gu z>1F6|YDZ0A?mqAc#F-K3LJ)ll-GI~0xLr9XYRcYw5)zYmV)YD9@?HAm(Pi6I%gi&f zh3u!SS6V>=Zwwz|jjdTP-aNw?j(!jye-?jgxqC7`rgG$pa*8|dkIgHPXQnA!uBRp< zqnp2weW?kkGT%Wr3|mPxXMXbZ+pa}^_qSgHI(-J3O4sa_O8l(zD)Zee*rlyr{xoR6 zFmTWZ<=$BL`0dE<>x;wM2&w?b$$1CPT+tfXSAFz{z_-_oYV^%jBl6jo7#ZUlhwq$l zT_?^EE`36}3w6tlA5(;K)GZ~2*%}Lq%NbEi)+J#0QObCEeuMLDSLut0#cmj4qGwOx zf-cS==uvlblZJ(3)uUHKK3t=sp3(22;t^)oy=uSE=gRe8U7STtrqSYPvKcZWBuMww z^`=tcL6T7_1Z@7Mn=xD~`r6c{(I(k4!f? zgY92u@bndj5nK5Hafp>29T~=mCgX=jXSCoh4s-J&?rg(#2D;pIKSD^vSP*SQN1ZQ^ z;CXq#D@(|gac=r`@tA1tGk+AXhvHN&#-fEinc4N!rN&E|IgF_vWim-)d+7)7tE%K` zSYxu<*K@e6IyYDLT_jM*bD#yi^j`5=W!kJze%NN^0S{-=tOY09{G4+@lL7LF?fzhM zf0fgN&G6zMoX35Rw;teZw1`btfqAx4A0mAkj9egfXjt6`oq7L3DsVC^SiR*K{*sS- z;yW!>9`si8Qz(23M!ta|J8yx;O{c<%!N+i2Oqi}w0h6xJoeR*(q9`9v0@%1F7ii76 zxEblSP7jhx6)O|@$_IC zCHs8xo~IWT$`$0+!5-M}TKXE)(ovG{9DLT=n#MLjje(tt8)re=oFWA9LG-F{glvSx z+R)x8duA-AlrF(H|Jv^RD&M1@4Ljd$w^hvHo(*uIU~ImqKtxmoW7~;wi2qFfv}>RSOgqW7Qkr)F zT10=0f+8AuWLaIf%y}1ToebzTjz;~yt&_w>Xz(KwL#04E)qXlD@)RRT2|^$x#l`<0}wy8dtuU#JM4AB>^Xn|(J; zEkm)$igP$;v8v}*4teH&>vem(izUJrzC#0+AbM-HTgwuM!<`ai>vS%c3Kfr6xIfS%(LQ)NuzIdpx?AD44NwlQ;oH0|9SwbMsnW zxt+~1sNQts;0=7_K9hJOL$mWB)n`7Ah#{Aom+hJ*i8G3D<=CdgEG^G=?eedqzCH!7 zxKhcC=%vJ~Rv*gP`9*yMRxP>ED`ejlT&}KPlG~e*juKqid*lb1VITFpjV0K{Tf-4H z0dQdg^V+eqH{%gj*pMYyFJzIh+uSgUIwAZ#SI&RGds$-!~}+Zp5&5e z0R}(EL)=c5@H^!oKf?f#66bOu0~s8GJ3w7OuhIu3Wm#>nmT7)Uf=R)laqXk)PShD_i)*cWY+HN2sHon zd^#j8G|p0)Ilr20lDrxMAB|&gZ>|38&7(AT1)#5|Xt}5?^yze0|4O?lvE3Yt!LwVu zz~fb-gF~CYr{Sj*`~@s-9TdrG^kX+aB?)KoujlFZ7O58Wy``;H0JWCkL5(t(gxdr> zTd?++b+l86&|Jt)iB4XqJz1DFi47gAg3>a_v=hip@iAt5t(Ej|Jg>9enUDuc+u=!l z!;zn`V36i{x63bar`B95K~ttTgdVnC{ncRfEf7-j;{9bl9C~3~=aeJSE&r218vhu8 zoKCoX@%mNR{`Q@q6Tt@|JbbNMw}c11!+$YCbv5ct74`g>P_ zamjLoS4Eskd;i)=Y;`oOlk9TznY+HLT*v9#PXIQLR^8p-8T>#(G8lNK|LKpenH)9> z+g-e-2L(_q<|C><{}<9CkS{H~N9)ecjQzq|x1rcURv<-g1kJv?;FsEAU^)aJT=<=X z1Y~Mf0-qEslF#mjTh9Zz`N(}Dzn3on>;y+37Xt~#Sd9G?(~c?eL%j+DoXD8RnLF0R z-vmD3h1xtooa-g^!~dg&&5sMb-UsL_mJcTdcNFx_!Sdsp@3z|~4Dvu;{u0-}K6|xY z_=h1jefJQG0RY*c(pubO_YAQB#?GyOg!GS)er%gt|LmmgUG~pT`f=9$A3W}Y=zQ|_ XYtp7GhlKwC{xq-aUPE1d@Z^60?sBS` diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 09b56576699e0..6e46a47338398 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -270,23 +270,92 @@ for i in range(2): ## Power iteration clustering (PIC) -Power iteration clustering (PIC) is a scalable and efficient algorithm for clustering points given pointwise mutual affinity values. Internally the algorithm: +Power iteration clustering (PIC) is a scalable and efficient algorithm for clustering vertices of a +graph given pairwise similarties as edge properties, +described in [Lin and Cohen, Power Iteration Clustering](http://www.icml2010.org/papers/387.pdf). +It computes a pseudo-eigenvector of the normalized affinity matrix of the graph via +[power iteration](http://en.wikipedia.org/wiki/Power_iteration) and uses it to cluster vertices. +MLlib includes an implementation of PIC using GraphX as its backend. +It takes an `RDD` of `(srcId, dstId, similarity)` tuples and outputs a model with the clustering assignments. +The similarities must be nonnegative. +PIC assumes that the similarity measure is symmetric. +A pair `(srcId, dstId)` regardless of the ordering should appear at most once in the input data. +If a pair is missing from input, their similarity is treated as zero. +MLlib's PIC implementation takes the following (hyper-)parameters: + +* `k`: number of clusters +* `maxIterations`: maximum number of power iterations +* `initializationMode`: initialization model. This can be either "random", which is the default, + to use a random vector as vertex properties, or "degree" to use normalized sum similarities. -* accepts a [Graph](api/graphx/index.html#org.apache.spark.graphx.Graph) that represents a normalized pairwise affinity between all input points. -* calculates the principal eigenvalue and eigenvector -* Clusters each of the input points according to their principal eigenvector component value +**Examples** + +In the following, we show code snippets to demonstrate how to use PIC in MLlib. + +

    +
    + +[`PowerIterationClustering`](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClustering) +implements the PIC algorithm. +It takes an `RDD` of `(srcId: Long, dstId: Long, similarity: Double)` tuples representing the +affinity matrix. +Calling `PowerIterationClustering.run` returns a +[`PowerIterationClusteringModel`](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClusteringModel), +which contains the computed clustering assignments. -Details of this algorithm are found within [Power Iteration Clustering, Lin and Cohen]{www.icml2010.org/papers/387.pdf} +{% highlight scala %} +import org.apache.spark.mllib.clustering.PowerIterationClustering +import org.apache.spark.mllib.linalg.Vectors -Example outputs for a dataset inspired by the paper - but with five clusters instead of three- have he following output from our implementation: +val similarities: RDD[(Long, Long, Double)] = ... + +val pic = new PowerIteartionClustering() + .setK(3) + .setMaxIterations(20) +val model = pic.run(similarities) + +model.assignments.foreach { case (vertexId, clusterId) => + println(s"$vertexId -> $clusterId") +} +{% endhighlight %} + +A full example that produces the experiment described in the PIC paper can be found under +[`examples/`](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala). + +
    -

    - The Property Graph - -

    +
    + +[`PowerIterationClustering`](api/java/org/apache/spark/mllib/clustering/PowerIterationClustering.html) +implements the PIC algorithm. +It takes an `JavaRDD` of `(srcId: Long, dstId: Long, similarity: Double)` tuples representing the +affinity matrix. +Calling `PowerIterationClustering.run` returns a +[`PowerIterationClusteringModel`](api/java/org/apache/spark/mllib/clustering/PowerIterationClusteringModel.html) +which contains the computed clustering assignments. + +{% highlight java %} +import scala.Tuple2; +import scala.Tuple3; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.clustering.PowerIterationClustering; +import org.apache.spark.mllib.clustering.PowerIterationClusteringModel; + +JavaRDD> similarities = ... + +PowerIterationClustering pic = new PowerIterationClustering() + .setK(2) + .setMaxIterations(10); +PowerIterationClusteringModel model = pic.run(similarities); + +for (Tuple2 assignment: model.assignments().toJavaRDD().collect()) { + System.out.println(assignment._1() + " -> " + assignment._2()); +} +{% endhighlight %} +
    + +
    ## Latent Dirichlet allocation (LDA) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java new file mode 100644 index 0000000000000..e9371de39f284 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import scala.Tuple2; +import scala.Tuple3; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.clustering.PowerIterationClustering; +import org.apache.spark.mllib.clustering.PowerIterationClusteringModel; + +/** + * Java example for graph clustering using power iteration clustering (PIC). + */ +public class JavaPowerIterationClusteringExample { + public static void main(String[] args) { + SparkConf sparkConf = new SparkConf().setAppName("JavaPowerIterationClusteringExample"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + @SuppressWarnings("unchecked") + JavaRDD> similarities = sc.parallelize(Lists.newArrayList( + new Tuple3(0L, 1L, 0.9), + new Tuple3(1L, 2L, 0.9), + new Tuple3(2L, 3L, 0.9), + new Tuple3(3L, 4L, 0.1), + new Tuple3(4L, 5L, 0.9))); + + PowerIterationClustering pic = new PowerIterationClustering() + .setK(2) + .setMaxIterations(10); + PowerIterationClusteringModel model = pic.run(similarities); + + for (Tuple2 assignment: model.assignments().toJavaRDD().collect()) { + System.out.println(assignment._1() + " -> " + assignment._2()); + } + + sc.stop(); + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 3b1caf0c679ef..63d03347f4572 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.clustering +import org.apache.spark.api.java.JavaRDD import org.apache.spark.{Logging, SparkException} import org.apache.spark.annotation.Experimental import org.apache.spark.graphx._ @@ -115,6 +116,14 @@ class PowerIterationClustering private[clustering] ( pic(w0) } + /** + * A Java-friendly version of [[PowerIterationClustering.run]]. + */ + def run(similarities: JavaRDD[(java.lang.Long, java.lang.Long, java.lang.Double)]) + : PowerIterationClusteringModel = { + run(similarities.rdd.asInstanceOf[RDD[(Long, Long, Double)]]) + } + /** * Runs the PIC algorithm. * From e945aa6139e022d13ac793f46819cfee07b782fc Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 19 Feb 2015 09:49:34 +0800 Subject: [PATCH 195/817] [SPARK-5846] Correctly set job description and pool for SQL jobs marmbrus am I missing something obvious here? I verified that this fixes the problem for me (on 1.2.1) on EC2, but I'm confused about how others wouldn't have noticed this? Author: Kay Ousterhout Closes #4630 from kayousterhout/SPARK-5846_1.3 and squashes the following commits: 2022ad4 [Kay Ousterhout] [SPARK-5846] Correctly set job description and pool for SQL jobs --- .../org/apache/spark/sql/hive/thriftserver/Shim12.scala | 8 ++++---- .../org/apache/spark/sql/hive/thriftserver/Shim13.scala | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index ea9d61d8d0f5e..13116b40bb259 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -185,6 +185,10 @@ private[hive] class SparkExecuteStatementOperation( def run(): Unit = { logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) + hiveContext.sparkContext.setJobDescription(statement) + sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } try { result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) @@ -194,10 +198,6 @@ private[hive] class SparkExecuteStatementOperation( logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } - hiveContext.sparkContext.setJobDescription(statement) - sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => - hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) - } iter = { val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 71e3954b2c7ac..9b8faeff94eab 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -156,6 +156,10 @@ private[hive] class SparkExecuteStatementOperation( def run(): Unit = { logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) + hiveContext.sparkContext.setJobDescription(statement) + sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } try { result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) @@ -165,10 +169,6 @@ private[hive] class SparkExecuteStatementOperation( logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } - hiveContext.sparkContext.setJobDescription(statement) - sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => - hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) - } iter = { val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean From fb87f449219c673a16bc46f85c1ef7a6e3f22736 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Thu, 19 Feb 2015 09:53:36 -0800 Subject: [PATCH 196/817] SPARK-5548: Fix for AkkaUtilsSuite failure - attempt 2 Author: Jacek Lewandowski Closes #4653 from jacek-lewandowski/SPARK-5548-2-master and squashes the following commits: 0e199b6 [Jacek Lewandowski] SPARK-5548: applied reviewer's comments 843eafb [Jacek Lewandowski] SPARK-5548: Fix for AkkaUtilsSuite failure - attempt 2 --- .../scala/org/apache/spark/util/AkkaUtilsSuite.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 2cc5817758cf7..6250d50fb7036 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import java.util.concurrent.TimeoutException import scala.concurrent.Await +import scala.util.{Failure, Try} import akka.actor._ @@ -370,8 +371,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val selection = slaveSystem.actorSelection( AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) - intercept[TimeoutException] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout * 2), timeout) + val result = Try(Await.result(selection.resolveOne(timeout * 2), timeout)) + + result match { + case Failure(ex: ActorNotFound) => + case Failure(ex: TimeoutException) => + case r => fail(s"$r is neither Failure(ActorNotFound) nor Failure(TimeoutException)") } actorSystem.shutdown() From 38e624a732b18e01ad2e7a499ce0bb0d7acdcdf6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 Feb 2015 09:56:25 -0800 Subject: [PATCH 197/817] [SPARK-5816] Add huge compatibility warning in DriverWrapper The stability of the new submission gateway assumes that the arguments in `DriverWrapper` are consistent across multiple Spark versions. However, this is not at all clear from the code itself. In fact, this was broken in 20a6013106b56a1a1cc3e8cda092330ffbe77cc3, which is fortunately OK because both that commit and the original commit that added this gateway are part of the same release. To prevent this from happening again we should at the very least add a huge warning where appropriate. Author: Andrew Or Closes #4687 from andrewor14/driver-wrapper-warning and squashes the following commits: 7989b56 [Andrew Or] Add huge compatibility warning --- .../org/apache/spark/deploy/worker/DriverWrapper.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index ab467a5ee8c6c..deef6ef9043c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -26,10 +26,17 @@ import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLCla /** * Utility object for launching driver programs such that they share fate with the Worker process. + * This is used in standalone cluster mode only. */ object DriverWrapper { def main(args: Array[String]) { args.toList match { + /* + * IMPORTANT: Spark 1.3 provides a stable application submission gateway that is both + * backward and forward compatible across future Spark versions. Because this gateway + * uses this class to launch the driver, the ordering and semantics of the arguments + * here must also remain consistent across versions. + */ case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", From 90095bf3ce9304d09a32ceffaa99069079071b59 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 19 Feb 2015 18:37:31 +0000 Subject: [PATCH 198/817] [SPARK-5423][Core] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file This PR adds a `finalize` method in DiskMapIterator to clean up the resources even if some exception happens during processing data. Author: zsxwing Closes #4219 from zsxwing/SPARK-5423 and squashes the following commits: d4b2ca6 [zsxwing] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file --- .../collection/ExternalAppendOnlyMap.scala | 52 +++++++++++++++---- 1 file changed, 43 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 8a0f5a602de12..fc7e86e297540 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -387,6 +387,15 @@ class ExternalAppendOnlyMap[K, V, C]( private var batchIndex = 0 // Which batch we're in private var fileStream: FileInputStream = null + @volatile private var closed = false + + // A volatile variable to remember which DeserializationStream is using. Need to set it when we + // open a DeserializationStream. But we should use `deserializeStream` rather than + // `deserializeStreamToBeClosed` to read the content because touching a volatile variable will + // reduce the performance. It must be volatile so that we can see its correct value in the + // `finalize` method, which could run in any thread. + @volatile private var deserializeStreamToBeClosed: DeserializationStream = null + // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams private var deserializeStream = nextBatchStream() @@ -401,6 +410,7 @@ class ExternalAppendOnlyMap[K, V, C]( // we're still in a valid batch. if (batchIndex < batchOffsets.length - 1) { if (deserializeStream != null) { + deserializeStreamToBeClosed = null deserializeStream.close() fileStream.close() deserializeStream = null @@ -419,7 +429,11 @@ class ExternalAppendOnlyMap[K, V, C]( val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) val compressedStream = blockManager.wrapForCompression(blockId, bufferedStream) - ser.deserializeStream(compressedStream) + // Before returning the stream, assign it to `deserializeStreamToBeClosed` so that we can + // close it in `finalize` and also avoid to touch the volatile `deserializeStreamToBeClosed` + // during reading the (K, C) pairs. + deserializeStreamToBeClosed = ser.deserializeStream(compressedStream) + deserializeStreamToBeClosed } else { // No more batches left cleanup() @@ -468,14 +482,34 @@ class ExternalAppendOnlyMap[K, V, C]( item } - // TODO: Ensure this gets called even if the iterator isn't drained. - private def cleanup() { - batchIndex = batchOffsets.length // Prevent reading any other batch - val ds = deserializeStream - deserializeStream = null - fileStream = null - ds.close() - file.delete() + // TODO: Now only use `finalize` to ensure `close` gets called to clean up the resources. In the + // future, we need some mechanism to ensure this gets called once the resources are not used. + private def cleanup(): Unit = { + if (!closed) { + closed = true + batchIndex = batchOffsets.length // Prevent reading any other batch + fileStream = null + try { + val ds = deserializeStreamToBeClosed + deserializeStreamToBeClosed = null + deserializeStream = null + if (ds != null) { + ds.close() + } + } finally { + if (file.exists()) { + file.delete() + } + } + } + } + + override def finalize(): Unit = { + try { + cleanup() + } finally { + super.finalize() + } } } From 94cdb05ff7e6b8fc5b3a574202ba8bc8e5bbe689 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 19 Feb 2015 12:07:51 -0800 Subject: [PATCH 199/817] [SPARK-5825] [Spark Submit] Remove the double checking instance name when stopping the service `spark-daemon.sh` will confirm the process id by fuzzy matching the class name while stopping the service, however, it will fail if the java process arguments is very long (greater than 4096 characters). This PR looses the check for the service process. Author: Cheng Hao Closes #4611 from chenghao-intel/stopping_service and squashes the following commits: a0051f6 [Cheng Hao] loosen the process checking while stopping a service --- sbin/spark-daemon.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index ec6d0b5a40ef2..e1bcc7d64254a 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -166,7 +166,7 @@ case $option in if [ -f $pid ]; then TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then echo "stopping $command" kill "$TARGET_ID" else From 8ca3418e1b3e2687e75a08c185d17045a97279fb Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Feb 2015 12:09:44 -0800 Subject: [PATCH 200/817] [SPARK-5904][SQL] DataFrame API fixes. 1. Column is no longer a DataFrame to simplify class hierarchy. 2. Don't use varargs on abstract methods (see Scala compiler bug SI-9013). Author: Reynold Xin Closes #4686 from rxin/SPARK-5904 and squashes the following commits: fd9b199 [Reynold Xin] Fixed Python tests. df25cef [Reynold Xin] Non final. 5221530 [Reynold Xin] [SPARK-5904][SQL] DataFrame API fixes. --- python/pyspark/sql/dataframe.py | 56 +- .../scala/org/apache/spark/sql/Column.scala | 223 +++----- .../apache/spark/sql/ComputableColumn.scala | 33 -- .../org/apache/spark/sql/DataFrame.scala | 420 ++++++++++++--- .../org/apache/spark/sql/DataFrameImpl.scala | 483 ------------------ .../org/apache/spark/sql/GroupedData.scala | 2 +- .../apache/spark/sql/IncomputableColumn.scala | 183 ------- .../spark/sql/ColumnExpressionSuite.scala | 44 +- .../org/apache/spark/sql/DataFrameSuite.scala | 7 +- 9 files changed, 427 insertions(+), 1024 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c68c97e9260e2..010c38f93b9cf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -546,7 +546,7 @@ def first(self): def __getitem__(self, item): """ Return the column by given name - >>> df['age'].collect() + >>> df.select(df['age']).collect() [Row(age=2), Row(age=5)] >>> df[ ["name", "age"]].collect() [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] @@ -555,7 +555,7 @@ def __getitem__(self, item): """ if isinstance(item, basestring): jc = self._jdf.apply(item) - return Column(jc, self.sql_ctx) + return Column(jc) elif isinstance(item, Column): return self.filter(item) elif isinstance(item, list): @@ -566,13 +566,13 @@ def __getitem__(self, item): def __getattr__(self, name): """ Return the column by given name - >>> df.age.collect() + >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] """ if name.startswith("__"): raise AttributeError(name) jc = self._jdf.apply(name) - return Column(jc, self.sql_ctx) + return Column(jc) def select(self, *cols): """ Selecting a set of expressions. @@ -698,7 +698,7 @@ def withColumnRenamed(self, existing, new): >>> df.withColumnRenamed('age', 'age2').collect() [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] """ - cols = [Column(_to_java_column(c), self.sql_ctx).alias(new) + cols = [Column(_to_java_column(c)).alias(new) if c == existing else c for c in self.columns] return self.select(*cols) @@ -873,15 +873,16 @@ def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): jc = getattr(self._jc, name)() - return Column(jc, self.sql_ctx) + return Column(jc) _.__doc__ = doc return _ def _func_op(name, doc=''): def _(self): - jc = getattr(self._sc._jvm.functions, name)(self._jc) - return Column(jc, self.sql_ctx) + sc = SparkContext._active_spark_context + jc = getattr(sc._jvm.functions, name)(self._jc) + return Column(jc) _.__doc__ = doc return _ @@ -892,7 +893,7 @@ def _bin_op(name, doc="binary operator"): def _(self, other): jc = other._jc if isinstance(other, Column) else other njc = getattr(self._jc, name)(jc) - return Column(njc, self.sql_ctx) + return Column(njc) _.__doc__ = doc return _ @@ -903,12 +904,12 @@ def _reverse_op(name, doc="binary operator"): def _(self, other): jother = _create_column_from_literal(other) jc = getattr(jother, name)(self._jc) - return Column(jc, self.sql_ctx) + return Column(jc) _.__doc__ = doc return _ -class Column(DataFrame): +class Column(object): """ A column in a DataFrame. @@ -924,9 +925,8 @@ class Column(DataFrame): 1 / df.colName """ - def __init__(self, jc, sql_ctx=None): + def __init__(self, jc): self._jc = jc - super(Column, self).__init__(jc, sql_ctx) # arithmetic operators __neg__ = _func_op("negate") @@ -975,7 +975,7 @@ def substr(self, startPos, length): :param startPos: start position (int or Column) :param length: length of the substring (int or Column) - >>> df.name.substr(1, 3).collect() + >>> df.select(df.name.substr(1, 3).alias("col")).collect() [Row(col=u'Ali'), Row(col=u'Bob')] """ if type(startPos) != type(length): @@ -986,7 +986,7 @@ def substr(self, startPos, length): jc = self._jc.substr(startPos._jc, length._jc) else: raise TypeError("Unexpected type: %s" % type(startPos)) - return Column(jc, self.sql_ctx) + return Column(jc) __getslice__ = substr @@ -1000,10 +1000,10 @@ def substr(self, startPos, length): def alias(self, alias): """Return a alias for this column - >>> df.age.alias("age2").collect() + >>> df.select(df.age.alias("age2")).collect() [Row(age2=2), Row(age2=5)] """ - return Column(getattr(self._jc, "as")(alias), self.sql_ctx) + return Column(getattr(self._jc, "as")(alias)) def cast(self, dataType): """ Convert the column into type `dataType` @@ -1013,34 +1013,18 @@ def cast(self, dataType): >>> df.select(df.age.cast(StringType()).alias('ages')).collect() [Row(ages=u'2'), Row(ages=u'5')] """ - if self.sql_ctx is None: - sc = SparkContext._active_spark_context - ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) - else: - ssql_ctx = self.sql_ctx._ssql_ctx if isinstance(dataType, basestring): jc = self._jc.cast(dataType) elif isinstance(dataType, DataType): + sc = SparkContext._active_spark_context + ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) jdt = ssql_ctx.parseDataType(dataType.json()) jc = self._jc.cast(jdt) - return Column(jc, self.sql_ctx) + return Column(jc) def __repr__(self): return 'Column<%s>' % self._jdf.toString().encode('utf8') - def toPandas(self): - """ - Return a pandas.Series from the column - - >>> df.age.toPandas() # doctest: +SKIP - 0 2 - 1 5 - dtype: int64 - """ - import pandas as pd - data = [c for c, in self.collect()] - return pd.Series(data) - def _test(): import doctest diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 8b6241c213c87..980754322e6c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -22,20 +22,15 @@ import scala.language.implicitConversions import org.apache.spark.annotation.Experimental import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} -import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar, UnresolvedGetField} import org.apache.spark.sql.types._ private[sql] object Column { - def apply(colName: String): Column = new IncomputableColumn(colName) + def apply(colName: String): Column = new Column(colName) - def apply(expr: Expression): Column = new IncomputableColumn(expr) - - def apply(sqlContext: SQLContext, plan: LogicalPlan, expr: Expression): Column = { - new ComputableColumn(sqlContext, plan, expr) - } + def apply(expr: Expression): Column = new Column(expr) def unapply(col: Column): Option[Expression] = Some(col.expr) } @@ -51,68 +46,18 @@ private[sql] object Column { * @groupname Ungrouped Support functions for DataFrames. */ @Experimental -trait Column extends DataFrame { - - protected[sql] def expr: Expression - - /** - * Returns true iff the [[Column]] is computable. - */ - def isComputable: Boolean - - /** Removes the top project so we can get to the underlying plan. */ - private def stripProject(p: LogicalPlan): LogicalPlan = p match { - case Project(_, child) => child - case p => sys.error("Unexpected logical plan (expected Project): " + p) - } - - private def computableCol(baseCol: ComputableColumn, expr: Expression) = { - val namedExpr = expr match { - case named: NamedExpression => named - case unnamed: Expression => Alias(unnamed, "col")() - } - val plan = Project(Seq(namedExpr), stripProject(baseCol.plan)) - Column(baseCol.sqlContext, plan, expr) - } +class Column(protected[sql] val expr: Expression) { - /** - * Construct a new column based on the expression and the other column value. - * - * There are two cases that can happen here: - * If otherValue is a constant, it is first turned into a Column. - * If otherValue is a Column, then: - * - If this column and otherValue are both computable and come from the same logical plan, - * then we can construct a ComputableColumn by applying a Project on top of the base plan. - * - If this column is not computable, but otherValue is computable, then we can construct - * a ComputableColumn based on otherValue's base plan. - * - If this column is computable, but otherValue is not, then we can construct a - * ComputableColumn based on this column's base plan. - * - If neither columns are computable, then we create an IncomputableColumn. - */ - private def constructColumn(otherValue: Any)(newExpr: Column => Expression): Column = { - // lit(otherValue) returns a Column always. - (this, lit(otherValue)) match { - case (left: ComputableColumn, right: ComputableColumn) => - if (stripProject(left.plan).sameResult(stripProject(right.plan))) { - computableCol(right, newExpr(right)) - } else { - // We don't want to throw an exception here because "df1("a") === df2("b")" can be - // a valid expression for join conditions, even though standalone they are not valid. - Column(newExpr(right)) - } - case (left: ComputableColumn, right) => computableCol(left, newExpr(right)) - case (_, right: ComputableColumn) => computableCol(right, newExpr(right)) - case (_, right) => Column(newExpr(right)) - } - } + def this(name: String) = this(name match { + case "*" => UnresolvedStar(None) + case _ if name.endsWith(".*") => UnresolvedStar(Some(name.substring(0, name.length - 2))) + case _ => UnresolvedAttribute(name) + }) /** Creates a column based on the given expression. */ - private def exprToColumn(newExpr: Expression, computable: Boolean = true): Column = { - this match { - case c: ComputableColumn if computable => computableCol(c, newExpr) - case _ => Column(newExpr) - } - } + implicit private def exprToColumn(newExpr: Expression): Column = new Column(newExpr) + + override def toString: String = expr.prettyString /** * Unary minus, i.e. negate the expression. @@ -127,7 +72,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def unary_- : Column = exprToColumn(UnaryMinus(expr)) + def unary_- : Column = UnaryMinus(expr) /** * Inversion of boolean expression, i.e. NOT. @@ -142,7 +87,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def unary_! : Column = exprToColumn(Not(expr)) + def unary_! : Column = Not(expr) /** * Equality test. @@ -157,9 +102,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def === (other: Any): Column = constructColumn(other) { o => - EqualTo(expr, o.expr) - } + def === (other: Any): Column = EqualTo(expr, lit(other).expr) /** * Equality test. @@ -190,9 +133,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def !== (other: Any): Column = constructColumn(other) { o => - Not(EqualTo(expr, o.expr)) - } + def !== (other: Any): Column = Not(EqualTo(expr, lit(other).expr)) /** * Inequality test. @@ -208,9 +149,7 @@ trait Column extends DataFrame { * * @group java_expr_ops */ - def notEqual(other: Any): Column = constructColumn(other) { o => - Not(EqualTo(expr, o.expr)) - } + def notEqual(other: Any): Column = Not(EqualTo(expr, lit(other).expr)) /** * Greater than. @@ -225,9 +164,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def > (other: Any): Column = constructColumn(other) { o => - GreaterThan(expr, o.expr) - } + def > (other: Any): Column = GreaterThan(expr, lit(other).expr) /** * Greater than. @@ -256,9 +193,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def < (other: Any): Column = constructColumn(other) { o => - LessThan(expr, o.expr) - } + def < (other: Any): Column = LessThan(expr, lit(other).expr) /** * Less than. @@ -286,9 +221,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def <= (other: Any): Column = constructColumn(other) { o => - LessThanOrEqual(expr, o.expr) - } + def <= (other: Any): Column = LessThanOrEqual(expr, lit(other).expr) /** * Less than or equal to. @@ -316,9 +249,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def >= (other: Any): Column = constructColumn(other) { o => - GreaterThanOrEqual(expr, o.expr) - } + def >= (other: Any): Column = GreaterThanOrEqual(expr, lit(other).expr) /** * Greater than or equal to an expression. @@ -339,9 +270,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def <=> (other: Any): Column = constructColumn(other) { o => - EqualNullSafe(expr, o.expr) - } + def <=> (other: Any): Column = EqualNullSafe(expr, lit(other).expr) /** * Equality test that is safe for null values. @@ -355,14 +284,14 @@ trait Column extends DataFrame { * * @group expr_ops */ - def isNull: Column = exprToColumn(IsNull(expr)) + def isNull: Column = IsNull(expr) /** * True if the current expression is NOT null. * * @group expr_ops */ - def isNotNull: Column = exprToColumn(IsNotNull(expr)) + def isNotNull: Column = IsNotNull(expr) /** * Boolean OR. @@ -376,9 +305,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def || (other: Any): Column = constructColumn(other) { o => - Or(expr, o.expr) - } + def || (other: Any): Column = Or(expr, lit(other).expr) /** * Boolean OR. @@ -406,9 +333,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def && (other: Any): Column = constructColumn(other) { o => - And(expr, o.expr) - } + def && (other: Any): Column = And(expr, lit(other).expr) /** * Boolean AND. @@ -436,9 +361,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def + (other: Any): Column = constructColumn(other) { o => - Add(expr, o.expr) - } + def + (other: Any): Column = Add(expr, lit(other).expr) /** * Sum of this expression and another expression. @@ -466,9 +389,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def - (other: Any): Column = constructColumn(other) { o => - Subtract(expr, o.expr) - } + def - (other: Any): Column = Subtract(expr, lit(other).expr) /** * Subtraction. Subtract the other expression from this expression. @@ -496,9 +417,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def * (other: Any): Column = constructColumn(other) { o => - Multiply(expr, o.expr) - } + def * (other: Any): Column = Multiply(expr, lit(other).expr) /** * Multiplication of this expression and another expression. @@ -526,9 +445,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def / (other: Any): Column = constructColumn(other) { o => - Divide(expr, o.expr) - } + def / (other: Any): Column = Divide(expr, lit(other).expr) /** * Division this expression by another expression. @@ -549,9 +466,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def % (other: Any): Column = constructColumn(other) { o => - Remainder(expr, o.expr) - } + def % (other: Any): Column = Remainder(expr, lit(other).expr) /** * Modulo (a.k.a. remainder) expression. @@ -567,37 +482,35 @@ trait Column extends DataFrame { * @group expr_ops */ @scala.annotation.varargs - def in(list: Column*): Column = { - new IncomputableColumn(In(expr, list.map(_.expr))) - } + def in(list: Column*): Column = In(expr, list.map(_.expr)) /** * SQL like expression. * * @group expr_ops */ - def like(literal: String): Column = exprToColumn(Like(expr, lit(literal).expr)) + def like(literal: String): Column = Like(expr, lit(literal).expr) /** * SQL RLIKE expression (LIKE with Regex). * * @group expr_ops */ - def rlike(literal: String): Column = exprToColumn(RLike(expr, lit(literal).expr)) + def rlike(literal: String): Column = RLike(expr, lit(literal).expr) /** * An expression that gets an item at position `ordinal` out of an array. * * @group expr_ops */ - def getItem(ordinal: Int): Column = exprToColumn(GetItem(expr, Literal(ordinal))) + def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) /** * An expression that gets a field by name in a [[StructField]]. * * @group expr_ops */ - def getField(fieldName: String): Column = exprToColumn(UnresolvedGetField(expr, fieldName)) + def getField(fieldName: String): Column = UnresolvedGetField(expr, fieldName) /** * An expression that returns a substring. @@ -606,8 +519,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def substr(startPos: Column, len: Column): Column = - exprToColumn(Substring(expr, startPos.expr, len.expr), computable = false) + def substr(startPos: Column, len: Column): Column = Substring(expr, startPos.expr, len.expr) /** * An expression that returns a substring. @@ -616,26 +528,21 @@ trait Column extends DataFrame { * * @group expr_ops */ - def substr(startPos: Int, len: Int): Column = - exprToColumn(Substring(expr, lit(startPos).expr, lit(len).expr)) + def substr(startPos: Int, len: Int): Column = Substring(expr, lit(startPos).expr, lit(len).expr) /** * Contains the other element. * * @group expr_ops */ - def contains(other: Any): Column = constructColumn(other) { o => - Contains(expr, o.expr) - } + def contains(other: Any): Column = Contains(expr, lit(other).expr) /** * String starts with. * * @group expr_ops */ - def startsWith(other: Column): Column = constructColumn(other) { o => - StartsWith(expr, o.expr) - } + def startsWith(other: Column): Column = StartsWith(expr, lit(other).expr) /** * String starts with another string literal. @@ -649,9 +556,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def endsWith(other: Column): Column = constructColumn(other) { o => - EndsWith(expr, o.expr) - } + def endsWith(other: Column): Column = EndsWith(expr, lit(other).expr) /** * String ends with another string literal. @@ -669,7 +574,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - override def as(alias: String): Column = exprToColumn(Alias(expr, alias)()) + def as(alias: String): Column = Alias(expr, alias)() /** * Gives the column an alias. @@ -680,7 +585,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - override def as(alias: Symbol): Column = exprToColumn(Alias(expr, alias.name)()) + def as(alias: Symbol): Column = Alias(expr, alias.name)() /** * Casts the column to a different data type. @@ -695,7 +600,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def cast(to: DataType): Column = exprToColumn(Cast(expr, to)) + def cast(to: DataType): Column = Cast(expr, to) /** * Casts the column to a different data type, using the canonical string representation @@ -708,22 +613,20 @@ trait Column extends DataFrame { * * @group expr_ops */ - def cast(to: String): Column = exprToColumn( - Cast(expr, to.toLowerCase match { - case "string" => StringType - case "boolean" => BooleanType - case "byte" => ByteType - case "short" => ShortType - case "int" => IntegerType - case "long" => LongType - case "float" => FloatType - case "double" => DoubleType - case "decimal" => DecimalType.Unlimited - case "date" => DateType - case "timestamp" => TimestampType - case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") - }) - ) + def cast(to: String): Column = Cast(expr, to.toLowerCase match { + case "string" | "str" => StringType + case "boolean" => BooleanType + case "byte" => ByteType + case "short" => ShortType + case "int" => IntegerType + case "long" => LongType + case "float" => FloatType + case "double" => DoubleType + case "decimal" => DecimalType.Unlimited + case "date" => DateType + case "timestamp" => TimestampType + case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") + }) /** * Returns an ordering used in sorting. @@ -737,7 +640,7 @@ trait Column extends DataFrame { * * @group expr_ops */ - def desc: Column = exprToColumn(SortOrder(expr, Descending), computable = false) + def desc: Column = SortOrder(expr, Descending) /** * Returns an ordering used in sorting. @@ -751,14 +654,14 @@ trait Column extends DataFrame { * * @group expr_ops */ - def asc: Column = exprToColumn(SortOrder(expr, Ascending), computable = false) + def asc: Column = SortOrder(expr, Ascending) /** - * Prints the plans (logical and physical) to the console for debugging purpose. + * Prints the expression to the console for debugging purpose. * * @group df_ops */ - override def explain(extended: Boolean): Unit = { + def explain(extended: Boolean): Unit = { if (extended) { println(expr) } else { @@ -768,7 +671,7 @@ trait Column extends DataFrame { } -class ColumnName(name: String) extends IncomputableColumn(name) { +class ColumnName(name: String) extends Column(name) { /** Creates a new AttributeReference of type boolean */ def boolean: StructField = StructField(name, BooleanType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala deleted file mode 100644 index ac479b26a7c6a..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import scala.language.implicitConversions - -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - - -private[sql] class ComputableColumn protected[sql]( - sqlContext: SQLContext, - protected[sql] val plan: LogicalPlan, - protected[sql] val expr: Expression) - extends DataFrameImpl(sqlContext, plan) with Column { - - override def isComputable: Boolean = true -} 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 5007a5a34de1a..810f7c77477bb 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 @@ -17,26 +17,38 @@ package org.apache.spark.sql +import java.io.CharArrayWriter import java.sql.DriverManager - import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal +import com.fasterxml.jackson.core.JsonFactory + import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils + private[sql] object DataFrame { def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = { - new DataFrameImpl(sqlContext, logicalPlan) + new DataFrame(sqlContext, logicalPlan) } } @@ -90,22 +102,100 @@ private[sql] object DataFrame { */ // TODO: Improve documentation. @Experimental -trait DataFrame extends RDDApi[Row] with Serializable { +class DataFrame protected[sql]( + @transient val sqlContext: SQLContext, + @DeveloperApi @transient val queryExecution: SQLContext#QueryExecution) + extends RDDApi[Row] with Serializable { + + /** + * A constructor that automatically analyzes the logical plan. + * + * This reports error eagerly as the [[DataFrame]] is constructed, unless + * [[SQLConf.dataFrameEagerAnalysis]] is turned off. + */ + def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { + this(sqlContext, { + val qe = sqlContext.executePlan(logicalPlan) + if (sqlContext.conf.dataFrameEagerAnalysis) { + qe.analyzed // This should force analysis and throw errors if there are any + } + qe + }) + } + + @transient protected[sql] val logicalPlan: LogicalPlan = queryExecution.logical match { + // For various commands (like DDL) and queries with side effects, we force query optimization to + // happen right away to let these side effects take place eagerly. + case _: Command | + _: InsertIntoTable | + _: CreateTableAsSelect[_] | + _: CreateTableUsingAsSelect | + _: WriteToFile => + LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) + case _ => + queryExecution.logical + } + + /** + * An implicit conversion function internal to this class for us to avoid doing + * "new DataFrameImpl(...)" everywhere. + */ + @inline private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { + new DataFrame(sqlContext, logicalPlan) + } - val sqlContext: SQLContext + protected[sql] def resolve(colName: String): NamedExpression = { + queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + throw new AnalysisException( + s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") + } + } - @DeveloperApi - def queryExecution: SQLContext#QueryExecution + protected[sql] def numericColumns: Seq[Expression] = { + schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => + queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + } + } - protected[sql] def logicalPlan: LogicalPlan + /** + * Internal API for Python + */ + private[sql] def showString(): String = { + val data = take(20) + val numCols = schema.fieldNames.length - override def toString = + // For cells that are beyond 20 characters, replace it with the first 17 and "..." + val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row => + row.toSeq.map { cell => + val str = if (cell == null) "null" else cell.toString + if (str.length > 20) str.substring(0, 17) + "..." else str + }: Seq[String] + } + + // Compute the width of each column + val colWidths = Array.fill(numCols)(0) + for (row <- rows) { + for ((cell, i) <- row.zipWithIndex) { + colWidths(i) = math.max(colWidths(i), cell.length) + } + } + + // Pad the cells + rows.map { row => + row.zipWithIndex.map { case (cell, i) => + String.format(s"%-${colWidths(i)}s", cell) + }.mkString(" ") + }.mkString("\n") + } + + override def toString: String = { try { schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]") } catch { case NonFatal(e) => s"Invalid tree; ${e.getMessage}:\n$queryExecution" } + } /** Left here for backward compatibility. */ @deprecated("1.3.0", "use toDF") @@ -130,19 +220,31 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group basic */ @scala.annotation.varargs - def toDF(colNames: String*): DataFrame + def toDF(colNames: String*): DataFrame = { + require(schema.size == colNames.size, + "The number of columns doesn't match.\n" + + "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + + "New column names: " + colNames.mkString(", ")) + + val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => + apply(oldName).as(newName) + } + select(newCols :_*) + } /** * Returns the schema of this [[DataFrame]]. * @group basic */ - def schema: StructType + def schema: StructType = queryExecution.analyzed.schema /** * Returns all column names and their data types as an array. * @group basic */ - def dtypes: Array[(String, String)] + def dtypes: Array[(String, String)] = schema.fields.map { field => + (field.name, field.dataType.toString) + } /** * Returns all column names as an array. @@ -154,13 +256,19 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Prints the schema to the console in a nice tree format. * @group basic */ - def printSchema(): Unit + def printSchema(): Unit = println(schema.treeString) /** * Prints the plans (logical and physical) to the console for debugging purpose. * @group basic */ - def explain(extended: Boolean): Unit + def explain(extended: Boolean): Unit = { + ExplainCommand( + logicalPlan, + extended = extended).queryExecution.executedPlan.executeCollect().map { + r => println(r.getString(0)) + } + } /** * Only prints the physical plan to the console for debugging purpose. @@ -173,7 +281,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * (without any Spark executors). * @group basic */ - def isLocal: Boolean + def isLocal: Boolean = logicalPlan.isInstanceOf[LocalRelation] /** * Displays the [[DataFrame]] in a tabular form. For example: @@ -187,7 +295,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group basic */ - def show(): Unit + def show(): Unit = println(showString()) /** * Cartesian join with another [[DataFrame]]. @@ -197,7 +305,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @param right Right side of the join operation. * @group dfops */ - def join(right: DataFrame): DataFrame + def join(right: DataFrame): DataFrame = { + Join(logicalPlan, right.logicalPlan, joinType = Inner, None) + } /** * Inner join with another [[DataFrame]], using the given join expression. @@ -209,7 +319,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def join(right: DataFrame, joinExprs: Column): DataFrame + def join(right: DataFrame, joinExprs: Column): DataFrame = { + Join(logicalPlan, right.logicalPlan, joinType = Inner, Some(joinExprs.expr)) + } /** * Join with another [[DataFrame]], using the given join expression. The following performs @@ -230,7 +342,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. * @group dfops */ - def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame + def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { + Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + } /** * Returns a new [[DataFrame]] sorted by the specified column, all in ascending order. @@ -243,7 +357,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def sort(sortCol: String, sortCols: String*): DataFrame + def sort(sortCol: String, sortCols: String*): DataFrame = { + sort((sortCol +: sortCols).map(apply) :_*) + } /** * Returns a new [[DataFrame]] sorted by the given expressions. For example: @@ -253,7 +369,17 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def sort(sortExprs: Column*): DataFrame + def sort(sortExprs: Column*): DataFrame = { + val sortOrder: Seq[SortOrder] = sortExprs.map { col => + col.expr match { + case expr: SortOrder => + expr + case expr: Expression => + SortOrder(expr, Ascending) + } + } + Sort(sortOrder, global = true, logicalPlan) + } /** * Returns a new [[DataFrame]] sorted by the given expressions. @@ -261,7 +387,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def orderBy(sortCol: String, sortCols: String*): DataFrame + def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols :_*) /** * Returns a new [[DataFrame]] sorted by the given expressions. @@ -269,7 +395,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def orderBy(sortExprs: Column*): DataFrame + def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs :_*) /** * Selects column based on the column name and return it as a [[Column]]. @@ -281,19 +407,25 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Selects column based on the column name and return it as a [[Column]]. * @group dfops */ - def col(colName: String): Column + def col(colName: String): Column = colName match { + case "*" => + Column(ResolvedStar(schema.fieldNames.map(resolve))) + case _ => + val expr = resolve(colName) + Column(expr) + } /** * Returns a new [[DataFrame]] with an alias set. * @group dfops */ - def as(alias: String): DataFrame + def as(alias: String): DataFrame = Subquery(alias, logicalPlan) /** * (Scala-specific) Returns a new [[DataFrame]] with an alias set. * @group dfops */ - def as(alias: Symbol): DataFrame + def as(alias: Symbol): DataFrame = as(alias.name) /** * Selects a set of expressions. @@ -303,7 +435,13 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def select(cols: Column*): DataFrame + def select(cols: Column*): DataFrame = { + val namedExpressions = cols.map { + case Column(expr: NamedExpression) => expr + case Column(expr: Expression) => Alias(expr, expr.prettyString)() + } + Project(namedExpressions.toSeq, logicalPlan) + } /** * Selects a set of columns. This is a variant of `select` that can only select @@ -317,7 +455,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def select(col: String, cols: String*): DataFrame + def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)) :_*) /** * Selects a set of SQL expressions. This is a variant of `select` that accepts @@ -329,7 +467,11 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def selectExpr(exprs: String*): DataFrame + def selectExpr(exprs: String*): DataFrame = { + select(exprs.map { expr => + Column(new SqlParser().parseExpression(expr)) + }: _*) + } /** * Filters rows using the given condition. @@ -341,7 +483,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def filter(condition: Column): DataFrame + def filter(condition: Column): DataFrame = Filter(condition.expr, logicalPlan) /** * Filters rows using the given SQL expression. @@ -350,7 +492,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def filter(conditionExpr: String): DataFrame + def filter(conditionExpr: String): DataFrame = { + filter(Column(new SqlParser().parseExpression(conditionExpr))) + } /** * Filters rows using the given condition. This is an alias for `filter`. @@ -362,7 +506,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def where(condition: Column): DataFrame + def where(condition: Column): DataFrame = filter(condition) /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -381,7 +525,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedData + 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. @@ -403,7 +547,10 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group dfops */ @scala.annotation.varargs - def groupBy(col1: String, cols: String*): GroupedData + def groupBy(col1: String, cols: String*): GroupedData = { + val colNames: Seq[String] = col1 +: cols + new GroupedData(this, colNames.map(colName => resolve(colName))) + } /** * (Scala-specific) Compute aggregates by specifying a map from column name to @@ -462,28 +609,28 @@ trait DataFrame extends RDDApi[Row] with Serializable { * and `head` is that `head` returns an array while `limit` returns a new [[DataFrame]]. * @group dfops */ - def limit(n: Int): DataFrame + def limit(n: Int): DataFrame = Limit(Literal(n), logicalPlan) /** * Returns a new [[DataFrame]] containing union of rows in this frame and another frame. * This is equivalent to `UNION ALL` in SQL. * @group dfops */ - def unionAll(other: DataFrame): DataFrame + def unionAll(other: DataFrame): DataFrame = Union(logicalPlan, other.logicalPlan) /** * Returns a new [[DataFrame]] containing rows only in both this frame and another frame. * This is equivalent to `INTERSECT` in SQL. * @group dfops */ - def intersect(other: DataFrame): DataFrame + def intersect(other: DataFrame): DataFrame = Intersect(logicalPlan, other.logicalPlan) /** * Returns a new [[DataFrame]] containing rows in this frame but not in another frame. * This is equivalent to `EXCEPT` in SQL. * @group dfops */ - def except(other: DataFrame): DataFrame + def except(other: DataFrame): DataFrame = Except(logicalPlan, other.logicalPlan) /** * Returns a new [[DataFrame]] by sampling a fraction of rows. @@ -493,7 +640,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @param seed Seed for sampling. * @group dfops */ - def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame + def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { + Sample(fraction, withReplacement, seed, logicalPlan) + } /** * Returns a new [[DataFrame]] by sampling a fraction of rows, using a random seed. @@ -527,8 +676,15 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame + def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { + val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + val attributes = schema.toAttributes + val rowFunction = + f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) + val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) + Generate(generator, join = true, outer = false, None, logicalPlan) + } /** * (Scala-specific) Returns a new [[DataFrame]] where a single column has been expanded to zero @@ -540,10 +696,17 @@ trait DataFrame extends RDDApi[Row] with Serializable { * }}} * @group dfops */ - def explode[A, B : TypeTag]( - inputColumn: String, - outputColumn: String)( - f: A => TraversableOnce[B]): DataFrame + def explode[A, B : TypeTag](inputColumn: String, outputColumn: String)(f: A => TraversableOnce[B]) + : DataFrame = { + val dataType = ScalaReflection.schemaFor[B].dataType + val attributes = AttributeReference(outputColumn, dataType)() :: Nil + def rowFunction(row: Row) = { + f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) + } + val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) + + Generate(generator, join = true, outer = false, None, logicalPlan) + } ///////////////////////////////////////////////////////////////////////////// @@ -551,110 +714,130 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Returns a new [[DataFrame]] by adding a column. * @group dfops */ - def withColumn(colName: String, col: Column): DataFrame + def withColumn(colName: String, col: Column): DataFrame = select(Column("*"), col.as(colName)) /** * Returns a new [[DataFrame]] with a column renamed. * @group dfops */ - def withColumnRenamed(existingName: String, newName: String): DataFrame + def withColumnRenamed(existingName: String, newName: String): DataFrame = { + val resolver = sqlContext.analyzer.resolver + val colNames = schema.map { field => + val name = field.name + if (resolver(name, existingName)) Column(name).as(newName) else Column(name) + } + select(colNames :_*) + } /** * Returns the first `n` rows. */ - def head(n: Int): Array[Row] + def head(n: Int): Array[Row] = limit(n).collect() /** * Returns the first row. */ - def head(): Row + def head(): Row = head(1).head /** * Returns the first row. Alias for head(). */ - override def first(): Row + override def first(): Row = head() /** * Returns a new RDD by applying a function to all rows of this DataFrame. * @group rdd */ - override def map[R: ClassTag](f: Row => R): RDD[R] + override def map[R: ClassTag](f: Row => R): RDD[R] = rdd.map(f) /** * Returns a new RDD by first applying a function to all rows of this [[DataFrame]], * and then flattening the results. * @group rdd */ - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] + override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) /** * Returns a new RDD by applying a function to each partition of this DataFrame. * @group rdd */ - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] + override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { + rdd.mapPartitions(f) + } /** * Applies a function `f` to all rows. * @group rdd */ - override def foreach(f: Row => Unit): Unit + override def foreach(f: Row => Unit): Unit = rdd.foreach(f) /** * Applies a function f to each partition of this [[DataFrame]]. * @group rdd */ - override def foreachPartition(f: Iterator[Row] => Unit): Unit + override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) /** * Returns the first `n` rows in the [[DataFrame]]. * @group action */ - override def take(n: Int): Array[Row] + override def take(n: Int): Array[Row] = head(n) /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. * @group action */ - override def collect(): Array[Row] + override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. * @group action */ - override def collectAsList(): java.util.List[Row] + override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*) /** * Returns the number of rows in the [[DataFrame]]. * @group action */ - override def count(): Long + override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. * @group rdd */ - override def repartition(numPartitions: Int): DataFrame + override def repartition(numPartitions: Int): DataFrame = { + sqlContext.createDataFrame(rdd.repartition(numPartitions), schema) + } /** * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. * @group dfops */ - override def distinct: DataFrame + override def distinct: DataFrame = Distinct(logicalPlan) /** * @group basic */ - override def persist(): this.type + override def persist(): this.type = { + sqlContext.cacheManager.cacheQuery(this) + this + } /** * @group basic */ - override def persist(newLevel: StorageLevel): this.type + override def persist(newLevel: StorageLevel): this.type = { + sqlContext.cacheManager.cacheQuery(this, None, newLevel) + this + } /** * @group basic */ - override def unpersist(blocking: Boolean): this.type + override def unpersist(blocking: Boolean): this.type = { + sqlContext.cacheManager.tryUncacheQuery(this, blocking) + this + } ///////////////////////////////////////////////////////////////////////////// // I/O @@ -664,7 +847,11 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. * @group rdd */ - def rdd: RDD[Row] + def rdd: RDD[Row] = { + // use a local variable to make sure the map closure doesn't capture the whole DataFrame + val schema = this.schema + queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + } /** * Returns the content of the [[DataFrame]] as a [[JavaRDD]] of [[Row]]s. @@ -684,7 +871,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { * * @group basic */ - def registerTempTable(tableName: String): Unit + def registerTempTable(tableName: String): Unit = { + sqlContext.registerDataFrameAsTable(this, tableName) + } /** * Saves the contents of this [[DataFrame]] as a parquet file, preserving the schema. @@ -692,7 +881,13 @@ trait DataFrame extends RDDApi[Row] with Serializable { * using the `parquetFile` function in [[SQLContext]]. * @group output */ - def saveAsParquetFile(path: String): Unit + def saveAsParquetFile(path: String): Unit = { + if (sqlContext.conf.parquetUseDataSourceApi) { + save("org.apache.spark.sql.parquet", SaveMode.ErrorIfExists, Map("path" -> path)) + } else { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } + } /** * :: Experimental :: @@ -747,9 +942,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group output */ @Experimental - def saveAsTable( - tableName: String, - source: String): Unit = { + def saveAsTable(tableName: String, source: String): Unit = { saveAsTable(tableName, source, SaveMode.ErrorIfExists) } @@ -765,10 +958,7 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group output */ @Experimental - def saveAsTable( - tableName: String, - source: String, - mode: SaveMode): Unit = { + def saveAsTable(tableName: String, source: String, mode: SaveMode): Unit = { saveAsTable(tableName, source, mode, Map.empty[String, String]) } @@ -809,7 +999,18 @@ trait DataFrame extends RDDApi[Row] with Serializable { tableName: String, source: String, mode: SaveMode, - options: Map[String, String]): Unit + options: Map[String, String]): Unit = { + val cmd = + CreateTableUsingAsSelect( + tableName, + source, + temporary = false, + mode, + options, + logicalPlan) + + sqlContext.executePlan(cmd).toRdd + } /** * :: Experimental :: @@ -882,7 +1083,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { def save( source: String, mode: SaveMode, - options: Map[String, String]): Unit + options: Map[String, String]): Unit = { + ResolvedDataSource(sqlContext, source, mode, options, this) + } /** * :: Experimental :: @@ -890,7 +1093,10 @@ trait DataFrame extends RDDApi[Row] with Serializable { * @group output */ @Experimental - def insertInto(tableName: String, overwrite: Boolean): Unit + def insertInto(tableName: String, overwrite: Boolean): Unit = { + sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), + Map.empty, logicalPlan, overwrite)).toRdd + } /** * :: Experimental :: @@ -905,7 +1111,31 @@ trait DataFrame extends RDDApi[Row] with Serializable { * Returns the content of the [[DataFrame]] as a RDD of JSON strings. * @group rdd */ - def toJSON: RDD[String] + def toJSON: RDD[String] = { + val rowSchema = this.schema + this.mapPartitions { iter => + val writer = new CharArrayWriter() + // create the Generator without separator inserted between 2 records + val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) + + new Iterator[String] { + override def hasNext = iter.hasNext + override def next(): String = { + JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) + gen.flush() + + val json = writer.toString + if (hasNext) { + writer.reset() + } else { + gen.close() + } + + json + } + } + } + } //////////////////////////////////////////////////////////////////////////// // JDBC Write Support @@ -919,7 +1149,21 @@ trait DataFrame extends RDDApi[Row] with Serializable { * exists. * @group output */ - def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit + def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit = { + val conn = DriverManager.getConnection(url) + try { + if (allowExisting) { + val sql = s"DROP TABLE IF EXISTS $table" + conn.prepareStatement(sql).executeUpdate() + } + val schema = JDBCWriteDetails.schemaString(this, url) + val sql = s"CREATE TABLE $table ($schema)" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + JDBCWriteDetails.saveTable(this, url, table) + } /** * Save this RDD to a JDBC database at `url` under the table name `table`. @@ -933,8 +1177,18 @@ trait DataFrame extends RDDApi[Row] with Serializable { * `INSERT INTO table VALUES (?, ?, ..., ?)` should not fail. * @group output */ - def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit - + def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit = { + if (overwrite) { + val conn = DriverManager.getConnection(url) + try { + val sql = s"TRUNCATE TABLE $table" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + } + JDBCWriteDetails.saveTable(this, url, table) + } //////////////////////////////////////////////////////////////////////////// // for Python API @@ -943,5 +1197,9 @@ trait DataFrame extends RDDApi[Row] with Serializable { /** * Converts a JavaRDD to a PythonRDD. */ - protected[sql] def javaToPython: JavaRDD[Array[Byte]] + protected[sql] def javaToPython: JavaRDD[Array[Byte]] = { + val fieldTypes = schema.fields.map(_.dataType) + val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() + SerDeUtil.javaToPython(jrdd) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala deleted file mode 100644 index 25bc9d929237d..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala +++ /dev/null @@ -1,483 +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 - -import java.io.CharArrayWriter -import java.sql.DriverManager - -import scala.language.implicitConversions -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.TypeTag -import scala.collection.JavaConversions._ - -import com.fasterxml.jackson.core.JsonFactory - -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.api.python.SerDeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{expressions, SqlParser, ScalaReflection} -import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{ExplainCommand, LogicalRDD, EvaluatePython} -import org.apache.spark.sql.jdbc.JDBCWriteDetails -import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{NumericType, StructType} - -/** - * Internal implementation of [[DataFrame]]. Users of the API should use [[DataFrame]] directly. - */ -private[sql] class DataFrameImpl protected[sql]( - @transient override val sqlContext: SQLContext, - @transient val queryExecution: SQLContext#QueryExecution) - extends DataFrame { - - /** - * A constructor that automatically analyzes the logical plan. - * - * This reports error eagerly as the [[DataFrame]] is constructed, unless - * [[SQLConf.dataFrameEagerAnalysis]] is turned off. - */ - def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { - this(sqlContext, { - val qe = sqlContext.executePlan(logicalPlan) - if (sqlContext.conf.dataFrameEagerAnalysis) { - qe.analyzed // This should force analysis and throw errors if there are any - } - qe - }) - } - - @transient protected[sql] override val logicalPlan: LogicalPlan = queryExecution.logical match { - // For various commands (like DDL) and queries with side effects, we force query optimization to - // happen right away to let these side effects take place eagerly. - case _: Command | - _: InsertIntoTable | - _: CreateTableAsSelect[_] | - _: CreateTableUsingAsSelect | - _: WriteToFile => - LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) - case _ => - queryExecution.logical - } - - /** - * An implicit conversion function internal to this class for us to avoid doing - * "new DataFrameImpl(...)" everywhere. - */ - @inline private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { - new DataFrameImpl(sqlContext, logicalPlan) - } - - protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { - throw new AnalysisException( - s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") - } - } - - protected[sql] def numericColumns: Seq[Expression] = { - schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get - } - } - - override def toDF(colNames: String*): DataFrame = { - require(schema.size == colNames.size, - "The number of columns doesn't match.\n" + - "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + colNames.mkString(", ")) - - val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => - apply(oldName).as(newName) - } - select(newCols :_*) - } - - override def schema: StructType = queryExecution.analyzed.schema - - override def dtypes: Array[(String, String)] = schema.fields.map { field => - (field.name, field.dataType.toString) - } - - override def columns: Array[String] = schema.fields.map(_.name) - - override def printSchema(): Unit = println(schema.treeString) - - override def explain(extended: Boolean): Unit = { - ExplainCommand( - logicalPlan, - extended = extended).queryExecution.executedPlan.executeCollect().map { - r => println(r.getString(0)) - } - } - - override def isLocal: Boolean = { - logicalPlan.isInstanceOf[LocalRelation] - } - - /** - * Internal API for Python - */ - private[sql] def showString(): String = { - val data = take(20) - val numCols = schema.fieldNames.length - - // For cells that are beyond 20 characters, replace it with the first 17 and "..." - val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row => - row.toSeq.map { cell => - val str = if (cell == null) "null" else cell.toString - if (str.length > 20) str.substring(0, 17) + "..." else str - } : Seq[String] - } - - // Compute the width of each column - val colWidths = Array.fill(numCols)(0) - for (row <- rows) { - for ((cell, i) <- row.zipWithIndex) { - colWidths(i) = math.max(colWidths(i), cell.length) - } - } - - // Pad the cells - rows.map { row => - row.zipWithIndex.map { case (cell, i) => - String.format(s"%-${colWidths(i)}s", cell) - }.mkString(" ") - }.mkString("\n") - } - - override def show(): Unit = { - println(showString()) - } - - override def join(right: DataFrame): DataFrame = { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None) - } - - override def join(right: DataFrame, joinExprs: Column): DataFrame = { - Join(logicalPlan, right.logicalPlan, Inner, Some(joinExprs.expr)) - } - - override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) - } - - override def sort(sortCol: String, sortCols: String*): DataFrame = { - sort((sortCol +: sortCols).map(apply) :_*) - } - - override def sort(sortExprs: Column*): DataFrame = { - val sortOrder: Seq[SortOrder] = sortExprs.map { col => - col.expr match { - case expr: SortOrder => - expr - case expr: Expression => - SortOrder(expr, Ascending) - } - } - Sort(sortOrder, global = true, logicalPlan) - } - - override def orderBy(sortCol: String, sortCols: String*): DataFrame = { - sort(sortCol, sortCols :_*) - } - - override def orderBy(sortExprs: Column*): DataFrame = { - sort(sortExprs :_*) - } - - override def col(colName: String): Column = colName match { - case "*" => - Column(ResolvedStar(schema.fieldNames.map(resolve))) - case _ => - val expr = resolve(colName) - Column(sqlContext, Project(Seq(expr), logicalPlan), expr) - } - - override def as(alias: String): DataFrame = Subquery(alias, logicalPlan) - - override def as(alias: Symbol): DataFrame = Subquery(alias.name, logicalPlan) - - override def select(cols: Column*): DataFrame = { - val namedExpressions = cols.map { - case Column(expr: NamedExpression) => expr - case Column(expr: Expression) => Alias(expr, expr.prettyString)() - } - Project(namedExpressions.toSeq, logicalPlan) - } - - override def select(col: String, cols: String*): DataFrame = { - select((col +: cols).map(Column(_)) :_*) - } - - override def selectExpr(exprs: String*): DataFrame = { - select(exprs.map { expr => - Column(new SqlParser().parseExpression(expr)) - }: _*) - } - - override def withColumn(colName: String, col: Column): DataFrame = { - select(Column("*"), col.as(colName)) - } - - override def withColumnRenamed(existingName: String, newName: String): DataFrame = { - val resolver = sqlContext.analyzer.resolver - val colNames = schema.map { field => - val name = field.name - if (resolver(name, existingName)) Column(name).as(newName) else Column(name) - } - select(colNames :_*) - } - - override def filter(condition: Column): DataFrame = { - Filter(condition.expr, logicalPlan) - } - - override def filter(conditionExpr: String): DataFrame = { - filter(Column(new SqlParser().parseExpression(conditionExpr))) - } - - override def where(condition: Column): DataFrame = { - filter(condition) - } - - override def groupBy(cols: Column*): GroupedData = { - new GroupedData(this, cols.map(_.expr)) - } - - override def groupBy(col1: String, cols: String*): GroupedData = { - val colNames: Seq[String] = col1 +: cols - new GroupedData(this, colNames.map(colName => resolve(colName))) - } - - override def limit(n: Int): DataFrame = { - Limit(Literal(n), logicalPlan) - } - - override def unionAll(other: DataFrame): DataFrame = { - Union(logicalPlan, other.logicalPlan) - } - - override def intersect(other: DataFrame): DataFrame = { - Intersect(logicalPlan, other.logicalPlan) - } - - override def except(other: DataFrame): DataFrame = { - Except(logicalPlan, other.logicalPlan) - } - - override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { - Sample(fraction, withReplacement, seed, logicalPlan) - } - - override def explode[A <: Product : TypeTag] - (input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { - val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] - val attributes = schema.toAttributes - val rowFunction = - f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) - val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) - - Generate(generator, join = true, outer = false, None, logicalPlan) - } - - override def explode[A, B : TypeTag]( - inputColumn: String, - outputColumn: String)( - f: A => TraversableOnce[B]): DataFrame = { - val dataType = ScalaReflection.schemaFor[B].dataType - val attributes = AttributeReference(outputColumn, dataType)() :: Nil - def rowFunction(row: Row) = { - f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) - } - val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) - - Generate(generator, join = true, outer = false, None, logicalPlan) - - } - - ///////////////////////////////////////////////////////////////////////////// - // RDD API - ///////////////////////////////////////////////////////////////////////////// - - override def head(n: Int): Array[Row] = limit(n).collect() - - override def head(): Row = head(1).head - - override def first(): Row = head() - - override def map[R: ClassTag](f: Row => R): RDD[R] = rdd.map(f) - - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) - - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { - rdd.mapPartitions(f) - } - - override def foreach(f: Row => Unit): Unit = rdd.foreach(f) - - override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) - - override def take(n: Int): Array[Row] = head(n) - - override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() - - override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*) - - override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) - - override def repartition(numPartitions: Int): DataFrame = { - sqlContext.createDataFrame(rdd.repartition(numPartitions), schema) - } - - override def distinct: DataFrame = Distinct(logicalPlan) - - override def persist(): this.type = { - sqlContext.cacheManager.cacheQuery(this) - this - } - - override def persist(newLevel: StorageLevel): this.type = { - sqlContext.cacheManager.cacheQuery(this, None, newLevel) - this - } - - override def unpersist(blocking: Boolean): this.type = { - sqlContext.cacheManager.tryUncacheQuery(this, blocking) - this - } - - ///////////////////////////////////////////////////////////////////////////// - // I/O - ///////////////////////////////////////////////////////////////////////////// - - override def rdd: RDD[Row] = { - // use a local variable to make sure the map closure doesn't capture the whole DataFrame - val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) - } - - override def registerTempTable(tableName: String): Unit = { - sqlContext.registerDataFrameAsTable(this, tableName) - } - - override def saveAsParquetFile(path: String): Unit = { - if (sqlContext.conf.parquetUseDataSourceApi) { - save("org.apache.spark.sql.parquet", SaveMode.ErrorIfExists, Map("path" -> path)) - } else { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd - } - } - - override def saveAsTable( - tableName: String, - source: String, - mode: SaveMode, - options: Map[String, String]): Unit = { - val cmd = - CreateTableUsingAsSelect( - tableName, - source, - temporary = false, - mode, - options, - logicalPlan) - - sqlContext.executePlan(cmd).toRdd - } - - override def save( - source: String, - mode: SaveMode, - options: Map[String, String]): Unit = { - ResolvedDataSource(sqlContext, source, mode, options, this) - } - - override def insertInto(tableName: String, overwrite: Boolean): Unit = { - sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd - } - - override def toJSON: RDD[String] = { - val rowSchema = this.schema - this.mapPartitions { iter => - val writer = new CharArrayWriter() - // create the Generator without separator inserted between 2 records - val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) - - new Iterator[String] { - override def hasNext = iter.hasNext - override def next(): String = { - JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) - gen.flush() - - val json = writer.toString - if (hasNext) { - writer.reset() - } else { - gen.close() - } - - json - } - } - } - } - - def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit = { - val conn = DriverManager.getConnection(url) - try { - if (allowExisting) { - val sql = s"DROP TABLE IF EXISTS $table" - conn.prepareStatement(sql).executeUpdate() - } - val schema = JDBCWriteDetails.schemaString(this, url) - val sql = s"CREATE TABLE $table ($schema)" - conn.prepareStatement(sql).executeUpdate() - } finally { - conn.close() - } - JDBCWriteDetails.saveTable(this, url, table) - } - - def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit = { - if (overwrite) { - val conn = DriverManager.getConnection(url) - try { - val sql = s"TRUNCATE TABLE $table" - conn.prepareStatement(sql).executeUpdate() - } finally { - conn.close() - } - } - JDBCWriteDetails.saveTable(this, url, table) - } - - //////////////////////////////////////////////////////////////////////////// - // for Python API - //////////////////////////////////////////////////////////////////////////// - protected[sql] override def javaToPython: JavaRDD[Array[Byte]] = { - val fieldTypes = schema.fields.map(_.dataType) - val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() - SerDeUtil.javaToPython(jrdd) - } -} 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 17158303b889a..d00175265924c 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 @@ -32,7 +32,7 @@ import org.apache.spark.sql.types.NumericType * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ @Experimental -class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { +class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala deleted file mode 100644 index b48b682b36e1f..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala +++ /dev/null @@ -1,183 +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 - -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.TypeTag - -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.types.StructType - -private[sql] class IncomputableColumn(protected[sql] val expr: Expression) extends Column { - - def this(name: String) = this(name match { - case "*" => UnresolvedStar(None) - case _ if name.endsWith(".*") => UnresolvedStar(Some(name.substring(0, name.length - 2))) - case _ => UnresolvedAttribute(name) - }) - - private def err[T](): T = { - throw new UnsupportedOperationException("Cannot run this method on an UncomputableColumn") - } - - override def toString = expr.prettyString - - override def isComputable: Boolean = false - - override val sqlContext: SQLContext = null - - override def queryExecution = err() - - protected[sql] override def logicalPlan: LogicalPlan = err() - - override def toDF(colNames: String*): DataFrame = err() - - override def schema: StructType = err() - - override def dtypes: Array[(String, String)] = err() - - override def columns: Array[String] = err() - - override def printSchema(): Unit = err() - - override def show(): Unit = err() - - override def isLocal: Boolean = false - - override def join(right: DataFrame): DataFrame = err() - - override def join(right: DataFrame, joinExprs: Column): DataFrame = err() - - override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = err() - - override def sort(sortCol: String, sortCols: String*): DataFrame = err() - - override def sort(sortExprs: Column*): DataFrame = err() - - override def orderBy(sortCol: String, sortCols: String*): DataFrame = err() - - override def orderBy(sortExprs: Column*): DataFrame = err() - - override def col(colName: String): Column = err() - - override def select(cols: Column*): DataFrame = err() - - override def select(col: String, cols: String*): DataFrame = err() - - override def selectExpr(exprs: String*): DataFrame = err() - - override def withColumn(colName: String, col: Column): DataFrame = err() - - override def withColumnRenamed(existingName: String, newName: String): DataFrame = err() - - override def filter(condition: Column): DataFrame = err() - - override def filter(conditionExpr: String): DataFrame = err() - - override def where(condition: Column): DataFrame = err() - - override def groupBy(cols: Column*): GroupedData = err() - - override def groupBy(col1: String, cols: String*): GroupedData = err() - - override def limit(n: Int): DataFrame = err() - - override def unionAll(other: DataFrame): DataFrame = err() - - override def intersect(other: DataFrame): DataFrame = err() - - override def except(other: DataFrame): DataFrame = err() - - override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = err() - - override def explode[A <: Product : TypeTag] - (input: Column*)(f: Row => TraversableOnce[A]): DataFrame = err() - - override def explode[A, B : TypeTag]( - inputColumn: String, - outputColumn: String)( - f: A => TraversableOnce[B]): DataFrame = err() - - ///////////////////////////////////////////////////////////////////////////// - - override def head(n: Int): Array[Row] = err() - - override def head(): Row = err() - - override def first(): Row = err() - - override def map[R: ClassTag](f: Row => R): RDD[R] = err() - - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = err() - - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = err() - - override def foreach(f: Row => Unit): Unit = err() - - override def foreachPartition(f: Iterator[Row] => Unit): Unit = err() - - override def take(n: Int): Array[Row] = err() - - override def collect(): Array[Row] = err() - - override def collectAsList(): java.util.List[Row] = err() - - override def count(): Long = err() - - override def repartition(numPartitions: Int): DataFrame = err() - - override def distinct: DataFrame = err() - - override def persist(): this.type = err() - - override def persist(newLevel: StorageLevel): this.type = err() - - override def unpersist(blocking: Boolean): this.type = err() - - override def rdd: RDD[Row] = err() - - override def registerTempTable(tableName: String): Unit = err() - - override def saveAsParquetFile(path: String): Unit = err() - - override def saveAsTable( - tableName: String, - source: String, - mode: SaveMode, - options: Map[String, String]): Unit = err() - - override def save( - source: String, - mode: SaveMode, - options: Map[String, String]): Unit = err() - - override def insertInto(tableName: String, overwrite: Boolean): Unit = err() - - def createJDBCTable(url: String, table: String, allowExisting: Boolean): Unit = err() - - def insertIntoJDBC(url: String, table: String, overwrite: Boolean): Unit = err() - - override def toJSON: RDD[String] = err() - - protected[sql] override def javaToPython: JavaRDD[Array[Byte]] = err() -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a63d733ece627..928b0deb61921 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -28,49 +28,10 @@ class ColumnExpressionSuite extends QueryTest { // TODO: Add test cases for bitwise operations. - test("computability check") { - def shouldBeComputable(c: Column): Unit = assert(c.isComputable === true) - - def shouldNotBeComputable(c: Column): Unit = { - assert(c.isComputable === false) - intercept[UnsupportedOperationException] { c.head() } - } - - shouldBeComputable(testData2("a")) - shouldBeComputable(testData2("b")) - - shouldBeComputable(testData2("a") + testData2("b")) - shouldBeComputable(testData2("a") + testData2("b") + 1) - - shouldBeComputable(-testData2("a")) - shouldBeComputable(!testData2("a")) - - shouldNotBeComputable(testData2.select(($"a" + 1).as("c"))("c") + testData2("b")) - shouldNotBeComputable( - testData2.select(($"a" + 1).as("c"))("c") + testData2.select(($"b" / 2).as("d"))("d")) - shouldNotBeComputable( - testData2.select(($"a" + 1).as("c")).select(($"c" + 2).as("d"))("d") + testData2("b")) - - // Literals and unresolved columns should not be computable. - shouldNotBeComputable(col("1")) - shouldNotBeComputable(col("1") + 2) - shouldNotBeComputable(lit(100)) - shouldNotBeComputable(lit(100) + 10) - shouldNotBeComputable(-col("1")) - shouldNotBeComputable(!col("1")) - - // Getting data from different frames should not be computable. - shouldNotBeComputable(testData2("a") + testData("key")) - shouldNotBeComputable(testData2("a") + 1 + testData("key")) - - // Aggregate functions alone should not be computable. - shouldNotBeComputable(sum(testData2("a"))) - } - test("collect on column produced by a binary operator") { val df = Seq((1, 2, 3)).toDF("a", "b", "c") - checkAnswer(df("a") + df("b"), Seq(Row(3))) - checkAnswer(df("a") + df("b").as("c"), Seq(Row(3))) + checkAnswer(df.select(df("a") + df("b")), Seq(Row(3))) + checkAnswer(df.select(df("a") + df("b").as("c")), Seq(Row(3))) } test("star") { @@ -78,7 +39,6 @@ class ColumnExpressionSuite extends QueryTest { } test("star qualified by data frame object") { - // This is not yet supported. val df = testData.toDF val goldAnswer = df.collect().toSeq checkAnswer(df.select(df("*")), goldAnswer) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f31bc38922d4e..6b9b3a8425964 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -56,10 +56,7 @@ class DataFrameSuite extends QueryTest { test("dataframe toString") { assert(testData.toString === "[key: int, value: string]") - assert(testData("key").toString === "[key: int]") - } - - test("incomputable toString") { + assert(testData("key").toString === "key") assert($"test".toString === "test") } @@ -431,7 +428,7 @@ class DataFrameSuite extends QueryTest { test("apply on query results (SPARK-5462)") { val df = testData.sqlContext.sql("select key from testData") - checkAnswer(df("key"), testData.select('key).collect().toSeq) + checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) } } From a5fed34355b403188ad50b567ab62ee54597b493 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 19 Feb 2015 12:46:27 -0800 Subject: [PATCH 201/817] [SPARK-5902] [ml] Made PipelineStage.transformSchema public instead of private to ml For users to implement their own PipelineStages, we need to make PipelineStage.transformSchema be public instead of private to ml. This would be nice to include in Spark 1.3 CC: mengxr Author: Joseph K. Bradley Closes #4682 from jkbradley/SPARK-5902 and squashes the following commits: 6f02357 [Joseph K. Bradley] Made transformSchema public 0e6d0a0 [Joseph K. Bradley] made implementations of transformSchema protected as well fdaf26a [Joseph K. Bradley] Made PipelineStage.transformSchema protected instead of private[ml] --- .../scala/org/apache/spark/ml/Pipeline.scala | 16 ++++++++++++---- .../apache/spark/ml/feature/StandardScaler.scala | 4 ++-- .../spark/ml/impl/estimator/Predictor.scala | 4 ++-- .../org/apache/spark/ml/recommendation/ALS.scala | 4 ++-- .../apache/spark/ml/tuning/CrossValidator.scala | 4 ++-- 5 files changed, 20 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index 5607ed21afe18..5bbcd2e080e07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml import scala.collection.mutable.ListBuffer import org.apache.spark.Logging -import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.param.{Param, ParamMap} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.StructType @@ -33,9 +33,17 @@ import org.apache.spark.sql.types.StructType abstract class PipelineStage extends Serializable with Logging { /** + * :: DeveloperAPI :: + * * Derives the output schema from the input schema and parameters. + * The schema describes the columns and types of the data. + * + * @param schema Input schema to this stage + * @param paramMap Parameters passed to this stage + * @return Output schema from this stage */ - private[ml] def transformSchema(schema: StructType, paramMap: ParamMap): StructType + @DeveloperApi + def transformSchema(schema: StructType, paramMap: ParamMap): StructType /** * Derives the output schema from the input schema and parameters, optionally with logging. @@ -126,7 +134,7 @@ class Pipeline extends Estimator[PipelineModel] { new PipelineModel(this, map, transformers.toArray) } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { val map = this.paramMap ++ paramMap val theStages = map(stages) require(theStages.toSet.size == theStages.size, @@ -171,7 +179,7 @@ class PipelineModel private[ml] ( stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap val map = (fittingParamMap ++ this.paramMap) ++ paramMap stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index ddbd648d64f23..1142aa4f8e73d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -55,7 +55,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP model } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { val map = this.paramMap ++ paramMap val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], @@ -91,7 +91,7 @@ class StandardScalerModel private[ml] ( dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { val map = this.paramMap ++ paramMap val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index 7daeff980f0ea..dfb89cc8d4af3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -132,7 +132,7 @@ private[spark] abstract class Predictor[ @DeveloperApi protected def featuresDataType: DataType = new VectorUDT - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap, fitting = true, featuresDataType) } @@ -184,7 +184,7 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel @DeveloperApi protected def featuresDataType: DataType = new VectorUDT - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap, fitting = false, featuresDataType) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 8d70e4347c4c9..c2ec716f08b7c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -188,7 +188,7 @@ class ALSModel private[ml] ( .select(dataset("*"), predict(users("features"), items("features")).as(map(predictionCol))) } - override private[ml] def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap) } } @@ -292,7 +292,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { model } - override private[ml] def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { validateAndTransformSchema(schema, paramMap) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index b07a68269cc2b..2eb1dac56f1e9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -129,7 +129,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP cvModel } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { val map = this.paramMap ++ paramMap map(estimator).transformSchema(schema, paramMap) } @@ -150,7 +150,7 @@ class CrossValidatorModel private[ml] ( bestModel.transform(dataset, paramMap) } - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { bestModel.transformSchema(schema, paramMap) } } From ad6b169dee84df175b51933b7a3ad7f0bbc52cf3 Mon Sep 17 00:00:00 2001 From: Zhan Zhang Date: Thu, 19 Feb 2015 23:13:02 +0000 Subject: [PATCH 202/817] [Spark-5889] Remove pid file after stopping service. Currently the pid file is not deleted, and potentially may cause some problem after service is stopped. The fix remove the pid file after service stopped. Author: Zhan Zhang Closes #4676 from zhzhan/spark-5889 and squashes the following commits: eb01be1 [Zhan Zhang] solve review comments b4c009e [Zhan Zhang] solve review comments 018110a [Zhan Zhang] spark-5889: remove pid file after stopping service 088d2a2 [Zhan Zhang] squash all commits c1f1fa5 [Zhan Zhang] test --- sbin/spark-daemon.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index e1bcc7d64254a..5e812a1d91c6b 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -168,7 +168,7 @@ case $option in TARGET_ID="$(cat "$pid")" if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then echo "stopping $command" - kill "$TARGET_ID" + kill "$TARGET_ID" && rm -f "$pid" else echo "no $command to stop" fi From 34b7c35380c88569a1396fb4ed991a0bed4288e7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 19 Feb 2015 15:35:23 -0800 Subject: [PATCH 203/817] SPARK-4682 [CORE] Consolidate various 'Clock' classes Another one from JoshRosen 's wish list. The first commit is much smaller and removes 2 of the 4 Clock classes. The second is much larger, necessary for consolidating the streaming one. I put together implementations in the way that seemed simplest. Almost all the change is standardizing class and method names. Author: Sean Owen Closes #4514 from srowen/SPARK-4682 and squashes the following commits: 5ed3a03 [Sean Owen] Javadoc Clock classes; make ManualClock private[spark] 169dd13 [Sean Owen] Add support for legacy org.apache.spark.streaming clock class names 277785a [Sean Owen] Reduce the net change in this patch by reversing some unnecessary syntax changes along the way b5e53df [Sean Owen] FakeClock -> ManualClock; getTime() -> getTimeMillis() 160863a [Sean Owen] Consolidate Streaming Clock class into common util Clock 7c956b2 [Sean Owen] Consolidate Clocks except for Streaming Clock --- .../spark/ExecutorAllocationManager.scala | 28 +----- .../spark/deploy/worker/DriverRunner.scala | 17 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 20 ++--- .../spark/scheduler/TaskSetManager.scala | 16 ++-- .../scala/org/apache/spark/util/Clock.scala | 44 ++++++++- .../org/apache/spark/util/ManualClock.scala | 69 ++++++++++++++ .../ExecutorAllocationManagerSuite.scala | 65 +++++++------- .../deploy/worker/DriverRunnerTest.scala | 3 +- .../spark/scheduler/TaskSetManagerSuite.scala | 36 ++++---- .../org/apache/spark/util/FakeClock.scala | 26 ------ .../streaming/LocalJavaStreamingContext.java | 2 +- .../flume/FlumePollingStreamSuite.scala | 7 +- .../streaming/LocalJavaStreamingContext.java | 2 +- .../streaming/LocalJavaStreamingContext.java | 2 +- .../streaming/LocalJavaStreamingContext.java | 2 +- .../kinesis/KinesisCheckpointState.scala | 10 +-- .../kinesis/KinesisRecordProcessor.scala | 2 +- .../kinesis/KinesisReceiverSuite.scala | 25 +++--- .../JavaStreamingLogisticRegressionSuite.java | 2 +- .../JavaStreamingLinearRegressionSuite.java | 2 +- project/MimaExcludes.scala | 5 ++ .../streaming/dstream/FileInputDStream.scala | 6 +- .../streaming/receiver/BlockGenerator.scala | 3 +- .../receiver/ReceivedBlockHandler.scala | 4 +- .../streaming/scheduler/JobGenerator.scala | 13 ++- .../scheduler/ReceivedBlockTracker.scala | 6 +- .../apache/spark/streaming/util/Clock.scala | 89 ------------------- .../spark/streaming/util/RecurringTimer.scala | 5 +- .../streaming/util/WriteAheadLogManager.scala | 5 +- .../streaming/LocalJavaStreamingContext.java | 2 +- .../streaming/BasicOperationsSuite.scala | 9 +- .../spark/streaming/CheckpointSuite.scala | 33 ++++--- .../spark/streaming/InputStreamsSuite.scala | 37 ++++---- .../streaming/ReceivedBlockHandlerSuite.scala | 6 +- .../streaming/ReceivedBlockTrackerSuite.scala | 10 +-- .../spark/streaming/TestSuiteBase.scala | 15 ++-- .../streaming/util/WriteAheadLogSuite.scala | 10 +-- 37 files changed, 301 insertions(+), 337 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/ManualClock.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/FakeClock.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 998695b6ac8ab..21c6e6ffa6666 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.collection.mutable import org.apache.spark.scheduler._ +import org.apache.spark.util.{SystemClock, Clock} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -123,7 +124,7 @@ private[spark] class ExecutorAllocationManager( private val intervalMillis: Long = 100 // Clock used to schedule when executors should be added and removed - private var clock: Clock = new RealClock + private var clock: Clock = new SystemClock() // Listener for Spark events that impact the allocation policy private val listener = new ExecutorAllocationListener @@ -588,28 +589,3 @@ private[spark] class ExecutorAllocationManager( private object ExecutorAllocationManager { val NOT_SET = Long.MaxValue } - -/** - * An abstract clock for measuring elapsed time. - */ -private trait Clock { - def getTimeMillis: Long -} - -/** - * A clock backed by a monotonically increasing time source. - * The time returned by this clock does not correspond to any notion of wall-clock time. - */ -private class RealClock extends Clock { - override def getTimeMillis: Long = System.nanoTime / (1000 * 1000) -} - -/** - * A clock that allows the caller to customize the time. - * This is used mainly for testing. - */ -private class TestClock(startTimeMillis: Long) extends Clock { - private var time: Long = startTimeMillis - override def getTimeMillis: Long = time - def tick(ms: Long): Unit = { time += ms } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index b964a09bdb218..e16bccb24d2c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -20,19 +20,18 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileUtil, Path} import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.{Command, DriverDescription, SparkHadoopUtil} +import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState +import org.apache.spark.util.{Clock, SystemClock} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -59,9 +58,7 @@ private[spark] class DriverRunner( // Decoupled for testing private[deploy] def setClock(_clock: Clock) = clock = _clock private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper - private var clock = new Clock { - def currentTimeMillis(): Long = System.currentTimeMillis() - } + private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) } @@ -190,9 +187,9 @@ private[spark] class DriverRunner( initialize(process.get) } - val processStart = clock.currentTimeMillis() + val processStart = clock.getTimeMillis() val exitCode = process.get.waitFor() - if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) { + if (clock.getTimeMillis() - processStart > successfulRunDuration * 1000) { waitSeconds = 1 } @@ -208,10 +205,6 @@ private[spark] class DriverRunner( } } -private[deploy] trait Clock { - def currentTimeMillis(): Long -} - private[deploy] trait Sleeper { def sleep(seconds: Int) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8b62d2405ecb7..c58721c2c82b7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -63,7 +63,7 @@ class DAGScheduler( mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv, - clock: org.apache.spark.util.Clock = SystemClock) + clock: Clock = new SystemClock()) extends Logging { def this(sc: SparkContext, taskScheduler: TaskScheduler) = { @@ -657,7 +657,7 @@ class DAGScheduler( // completion events or stage abort stageIdToStage -= s.id jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), jobResult)) } } @@ -706,7 +706,7 @@ class DAGScheduler( stage.latestInfo.stageFailed(stageFailedMessage) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } - listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error))) } } @@ -745,7 +745,7 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) val shouldRunLocally = localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 - val jobSubmissionTime = clock.getTime() + val jobSubmissionTime = clock.getTimeMillis() if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. listenerBus.post( @@ -871,7 +871,7 @@ class DAGScheduler( logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stage.latestInfo.submissionTime = Some(clock.getTime()) + stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } else { // Because we posted SparkListenerStageSubmitted earlier, we should post // SparkListenerStageCompleted here in case there are no tasks to run. @@ -940,12 +940,12 @@ class DAGScheduler( def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { val serviceTime = stage.latestInfo.submissionTime match { - case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) + case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) case _ => "Unknown" } if (errorMessage.isEmpty) { logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stage.latestInfo.completionTime = Some(clock.getTime()) + stage.latestInfo.completionTime = Some(clock.getTimeMillis()) } else { stage.latestInfo.stageFailed(errorMessage.get) logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) @@ -971,7 +971,7 @@ class DAGScheduler( markStageAsFinished(stage) cleanupStateForJobAndIndependentStages(job) listenerBus.post( - SparkListenerJobEnd(job.jobId, clock.getTime(), JobSucceeded)) + SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) } // taskSucceeded runs some user code that might throw an exception. Make sure @@ -1187,7 +1187,7 @@ class DAGScheduler( } val dependentJobs: Seq[ActiveJob] = activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq - failedStage.latestInfo.completionTime = Some(clock.getTime()) + failedStage.latestInfo.completionTime = Some(clock.getTimeMillis()) for (job <- dependentJobs) { failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } @@ -1242,7 +1242,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) cleanupStateForJobAndIndependentStages(job) - listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error))) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 99a5f7117790d..529237f0d35dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -51,7 +51,7 @@ private[spark] class TaskSetManager( sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - clock: Clock = SystemClock) + clock: Clock = new SystemClock()) extends Schedulable with Logging { val conf = sched.sc.conf @@ -166,7 +166,7 @@ private[spark] class TaskSetManager( // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. // We then move down if we manage to launch a "more local" task. var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels - var lastLaunchTime = clock.getTime() // Time we last launched a task at this level + var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level override def schedulableQueue = null @@ -281,7 +281,7 @@ private[spark] class TaskSetManager( val failed = failedExecutors.get(taskId).get return failed.contains(execId) && - clock.getTime() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT + clock.getTimeMillis() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT } false @@ -428,7 +428,7 @@ private[spark] class TaskSetManager( : Option[TaskDescription] = { if (!isZombie) { - val curTime = clock.getTime() + val curTime = clock.getTimeMillis() var allowedLocality = maxLocality @@ -459,7 +459,7 @@ private[spark] class TaskSetManager( lastLaunchTime = curTime } // Serialize and return the task - val startTime = clock.getTime() + val startTime = clock.getTimeMillis() val serializedTask: ByteBuffer = try { Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) } catch { @@ -674,7 +674,7 @@ private[spark] class TaskSetManager( return } val key = ef.description - val now = clock.getTime() + val now = clock.getTimeMillis() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { val (dupCount, printTime) = recentExceptions(key) @@ -706,7 +706,7 @@ private[spark] class TaskSetManager( } // always add to failed executors failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). - put(info.executorId, clock.getTime()) + put(info.executorId, clock.getTimeMillis()) sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics) addPendingTask(index) if (!isZombie && state != TaskState.KILLED && !reason.isInstanceOf[TaskCommitDenied]) { @@ -821,7 +821,7 @@ private[spark] class TaskSetManager( val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) { - val time = clock.getTime() + val time = clock.getTimeMillis() val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray Arrays.sort(durations) val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1)) diff --git a/core/src/main/scala/org/apache/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala index 97c2b45aabf28..e92ed11bd165b 100644 --- a/core/src/main/scala/org/apache/spark/util/Clock.scala +++ b/core/src/main/scala/org/apache/spark/util/Clock.scala @@ -21,9 +21,47 @@ package org.apache.spark.util * An interface to represent clocks, so that they can be mocked out in unit tests. */ private[spark] trait Clock { - def getTime(): Long + def getTimeMillis(): Long + def waitTillTime(targetTime: Long): Long } -private[spark] object SystemClock extends Clock { - def getTime(): Long = System.currentTimeMillis() +/** + * A clock backed by the actual time from the OS as reported by the `System` API. + */ +private[spark] class SystemClock extends Clock { + + val minPollTime = 25L + + /** + * @return the same time (milliseconds since the epoch) + * as is reported by `System.currentTimeMillis()` + */ + def getTimeMillis(): Long = System.currentTimeMillis() + + /** + * @param targetTime block until the current time is at least this value + * @return current system time when wait has completed + */ + def waitTillTime(targetTime: Long): Long = { + var currentTime = 0L + currentTime = System.currentTimeMillis() + + var waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong + + while (true) { + currentTime = System.currentTimeMillis() + waitTime = targetTime - currentTime + if (waitTime <= 0) { + return currentTime + } + val sleepTime = math.min(waitTime, pollTime) + Thread.sleep(sleepTime) + } + -1 + } } diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala new file mode 100644 index 0000000000000..cf89c1782fd67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * A `Clock` whose time can be manually set and modified. Its reported time does not change + * as time elapses, but only as its time is modified by callers. This is mainly useful for + * testing. + * + * @param time initial time (in milliseconds since the epoch) + */ +private[spark] class ManualClock(private var time: Long) extends Clock { + + /** + * @return `ManualClock` with initial time 0 + */ + def this() = this(0L) + + def getTimeMillis(): Long = + synchronized { + time + } + + /** + * @param timeToSet new time (in milliseconds) that the clock should represent + */ + def setTime(timeToSet: Long) = + synchronized { + time = timeToSet + notifyAll() + } + + /** + * @param timeToAdd time (in milliseconds) to add to the clock's time + */ + def advance(timeToAdd: Long) = + synchronized { + time += timeToAdd + notifyAll() + } + + /** + * @param targetTime block until the clock time is set or advanced to at least this time + * @return current time reported by the clock when waiting finishes + */ + def waitTillTime(targetTime: Long): Long = + synchronized { + while (time < targetTime) { + wait(100) + } + getTimeMillis() + } + +} diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index d3123e854016b..abfcee75728dc 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.util.ManualClock /** * Test add and remove behavior of ExecutorAllocationManager. @@ -321,7 +322,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("starting/canceling add timer") { sc = createSparkContext(2, 10) - val clock = new TestClock(8888L) + val clock = new ManualClock(8888L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -330,21 +331,21 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onSchedulerBacklogged(manager) val firstAddTime = addTime(manager) assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === firstAddTime) // timer is already started - clock.tick(200L) + clock.advance(200L) onSchedulerBacklogged(manager) assert(addTime(manager) === firstAddTime) onSchedulerQueueEmpty(manager) // Restart add timer - clock.tick(1000L) + clock.advance(1000L) assert(addTime(manager) === NOT_SET) onSchedulerBacklogged(manager) val secondAddTime = addTime(manager) assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === secondAddTime) // timer is already started assert(addTime(manager) !== firstAddTime) @@ -353,7 +354,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("starting/canceling remove timers") { sc = createSparkContext(2, 10) - val clock = new TestClock(14444L) + val clock = new ManualClock(14444L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -366,17 +367,17 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("1")) val firstRemoveTime = removeTimes(manager)("1") assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.tick(100L) + clock.advance(100L) onExecutorIdle(manager, "1") assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started - clock.tick(200L) + clock.advance(200L) onExecutorIdle(manager, "1") assert(removeTimes(manager)("1") === firstRemoveTime) - clock.tick(300L) + clock.advance(300L) onExecutorIdle(manager, "2") assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.tick(400L) + clock.advance(400L) onExecutorIdle(manager, "3") assert(removeTimes(manager)("3") !== firstRemoveTime) assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000) @@ -385,7 +386,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("3")) // Restart remove timer - clock.tick(1000L) + clock.advance(1000L) onExecutorBusy(manager, "1") assert(removeTimes(manager).size === 2) onExecutorIdle(manager, "1") @@ -401,7 +402,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("mock polling loop with no events") { sc = createSparkContext(1, 20) val manager = sc.executorAllocationManager.get - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) manager.setClock(clock) // No events - we should not be adding or removing @@ -410,15 +411,15 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(100L) + clock.advance(100L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(1000L) + clock.advance(1000L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(10000L) + clock.advance(10000L) schedule(manager) assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) @@ -426,57 +427,57 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("mock polling loop add behavior") { sc = createSparkContext(1, 20) - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Scheduler queue backlogged onSchedulerBacklogged(manager) - clock.tick(schedulerBacklogTimeout * 1000 / 2) + clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) assert(numExecutorsPending(manager) === 0) // timer not exceeded yet - clock.tick(schedulerBacklogTimeout * 1000) + clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1) // first timer exceeded - clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2) + clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7) // timer is canceled - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) - clock.tick(schedulerBacklogTimeout * 1000) + clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1) // timer restarted - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1 + 2) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4) - clock.tick(sustainedSchedulerBacklogTimeout * 1000) + clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) assert(numExecutorsPending(manager) === 20) // limit reached } test("mock polling loop remove behavior") { sc = createSparkContext(1, 20) - val clock = new TestClock(2020L) + val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -486,11 +487,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onExecutorAdded(manager, "executor-3") assert(removeTimes(manager).size === 3) assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(executorIdleTimeout * 1000 / 2) + clock.advance(executorIdleTimeout * 1000 / 2) schedule(manager) assert(removeTimes(manager).size === 3) // idle threshold not reached yet assert(executorsPendingToRemove(manager).isEmpty) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) // idle threshold exceeded assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) @@ -511,7 +512,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(!removeTimes(manager).contains("executor-5")) assert(!removeTimes(manager).contains("executor-6")) assert(executorsPendingToRemove(manager).size === 2) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) // idle executors are removed assert(executorsPendingToRemove(manager).size === 4) @@ -529,7 +530,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("executor-5")) assert(removeTimes(manager).contains("executor-6")) assert(executorsPendingToRemove(manager).size === 4) - clock.tick(executorIdleTimeout * 1000) + clock.advance(executorIdleTimeout * 1000) schedule(manager) assert(removeTimes(manager).isEmpty) assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index b6f4411e0587a..aa6e4874cecde 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -27,6 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf import org.apache.spark.deploy.{Command, DriverDescription} +import org.apache.spark.util.Clock class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { @@ -129,7 +130,7 @@ class DriverRunnerTest extends FunSuite { .thenReturn(-1) // fail 3 .thenReturn(-1) // fail 4 .thenReturn(0) // success - when(clock.currentTimeMillis()) + when(clock.getTimeMillis()) .thenReturn(0).thenReturn(1000) // fail 1 (short) .thenReturn(1000).thenReturn(2000) // fail 2 (short) .thenReturn(2000).thenReturn(10000) // fail 3 (long) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 59580561cb45a..12330d8f63c40 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.FakeClock +import org.apache.spark.util.ManualClock class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -164,7 +164,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Offer a host with NO_PREF as the constraint, @@ -213,7 +213,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // An executor that is not NODE_LOCAL should be rejected. @@ -234,7 +234,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")), Seq() // Last task has no locality prefs ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -263,7 +263,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2", "exec3")), Seq() // Last task has no locality prefs ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) @@ -283,7 +283,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host3")), Seq(TaskLocation("host2")) ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen @@ -321,7 +321,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2")), Seq(TaskLocation("host3")) ) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen @@ -353,7 +353,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -370,7 +370,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted @@ -402,7 +402,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { ("exec1.1", "host1"), ("exec2", "host2")) // affinity to exec1 on host1 - which we will fail. val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, 4, clock) { @@ -486,7 +486,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) @@ -522,7 +522,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execA"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) @@ -611,7 +611,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2"), TaskLocation("host1")), Seq(), Seq(TaskLocation("host3", "execC"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) @@ -637,7 +637,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2")), Seq(), Seq(TaskLocation("host3"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // node-local tasks are scheduled without delay @@ -658,7 +658,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(TaskLocation("host2")), Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // process-local tasks are scheduled first @@ -678,7 +678,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(), Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // process-local tasks are scheduled first @@ -698,7 +698,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2", "execB.1"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(ANY))) @@ -732,7 +732,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq(HostTaskLocation("host1")), Seq(HostTaskLocation("host2")), Seq(HDFSCacheTaskLocation("host3"))) - val clock = new FakeClock + val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) sched.removeExecutor("execA") diff --git a/core/src/test/scala/org/apache/spark/util/FakeClock.scala b/core/src/test/scala/org/apache/spark/util/FakeClock.scala deleted file mode 100644 index 0a45917b08dd2..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/FakeClock.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -class FakeClock extends Clock { - private var time = 0L - - def advance(millis: Long): Unit = time += millis - - def getTime(): Long = time -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 1e24da7f5f60c..cfedb5a042a35 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -31,7 +31,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index b57a1c71e35b9..e04d4088df7dc 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -34,10 +34,9 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{Seconds, TestOutputStream, StreamingContext} import org.apache.spark.streaming.flume.sink._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ManualClock, Utils} class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging { @@ -54,7 +53,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging def beforeFunction() { logInfo("Using manual clock") - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.util.ManualClock") } before(beforeFunction()) @@ -236,7 +235,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging tx.commit() tx.close() Thread.sleep(500) // Allow some time for the events to reach - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) } null } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 1e24da7f5f60c..cfedb5a042a35 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -31,7 +31,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 1e24da7f5f60c..cfedb5a042a35 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -31,7 +31,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 1e24da7f5f60c..cfedb5a042a35 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -31,7 +31,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala index 0b80b611cdce7..588e86a1887ec 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala @@ -18,9 +18,7 @@ package org.apache.spark.streaming.kinesis import org.apache.spark.Logging import org.apache.spark.streaming.Duration -import org.apache.spark.streaming.util.Clock -import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.streaming.util.SystemClock +import org.apache.spark.util.{Clock, ManualClock, SystemClock} /** * This is a helper class for managing checkpoint clocks. @@ -35,7 +33,7 @@ private[kinesis] class KinesisCheckpointState( /* Initialize the checkpoint clock using the given currentClock + checkpointInterval millis */ val checkpointClock = new ManualClock() - checkpointClock.setTime(currentClock.currentTime() + checkpointInterval.milliseconds) + checkpointClock.setTime(currentClock.getTimeMillis() + checkpointInterval.milliseconds) /** * Check if it's time to checkpoint based on the current time and the derived time @@ -44,13 +42,13 @@ private[kinesis] class KinesisCheckpointState( * @return true if it's time to checkpoint */ def shouldCheckpoint(): Boolean = { - new SystemClock().currentTime() > checkpointClock.currentTime() + new SystemClock().getTimeMillis() > checkpointClock.getTimeMillis() } /** * Advance the checkpoint clock by the checkpoint interval. */ def advanceCheckpoint() = { - checkpointClock.addToTime(checkpointInterval.milliseconds) + checkpointClock.advance(checkpointInterval.milliseconds) } } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index 8ecc2d90160b1..af8cd875b4541 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -104,7 +104,7 @@ private[kinesis] class KinesisRecordProcessor( logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint of ${batch.size}" + s" records for shardId $shardId") logDebug(s"Checkpoint: Next checkpoint is at " + - s" ${checkpointState.checkpointClock.currentTime()} for shardId $shardId") + s" ${checkpointState.checkpointClock.getTimeMillis()} for shardId $shardId") } } catch { case e: Throwable => { diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index f56898af029c1..255fe65819608 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -25,8 +25,7 @@ import org.apache.spark.streaming.Milliseconds import org.apache.spark.streaming.Seconds import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.TestSuiteBase -import org.apache.spark.streaming.util.Clock -import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.util.{ManualClock, Clock} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter @@ -129,45 +128,45 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft } test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") { - when(currentClockMock.currentTime()).thenReturn(0) + when(currentClockMock.getTimeMillis()).thenReturn(0) val checkpointIntervalMillis = 10 val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) - assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis) - verify(currentClockMock, times(1)).currentTime() + verify(currentClockMock, times(1)).getTimeMillis() } test("should checkpoint if we have exceeded the checkpoint interval") { - when(currentClockMock.currentTime()).thenReturn(0) + when(currentClockMock.getTimeMillis()).thenReturn(0) val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) assert(checkpointState.shouldCheckpoint()) - verify(currentClockMock, times(1)).currentTime() + verify(currentClockMock, times(1)).getTimeMillis() } test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { - when(currentClockMock.currentTime()).thenReturn(0) + when(currentClockMock.getTimeMillis()).thenReturn(0) val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) assert(!checkpointState.shouldCheckpoint()) - verify(currentClockMock, times(1)).currentTime() + verify(currentClockMock, times(1)).getTimeMillis() } test("should add to time when advancing checkpoint") { - when(currentClockMock.currentTime()).thenReturn(0) + when(currentClockMock.getTimeMillis()).thenReturn(0) val checkpointIntervalMillis = 10 val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) - assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis) checkpointState.advanceCheckpoint() - assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis)) + assert(checkpointState.checkpointClock.getTimeMillis() == (2 * checkpointIntervalMillis)) - verify(currentClockMock, times(1)).currentTime() + verify(currentClockMock, times(1)).getTimeMillis() } test("shutdown should checkpoint if the reason is TERMINATE") { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java index ac945ba6f23c1..640d2ec55e4e7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java @@ -47,7 +47,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index a4dd1ac39a3c8..899c4ea607869 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -45,7 +45,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4065a562a1a18..ee6229aa6bbe1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -148,6 +148,11 @@ object MimaExcludes { ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.linalg.VectorUDT"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.serialize"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.linalg.VectorUDT.sqlType") + ) ++ Seq( + // SPARK-4682 + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.RealClock"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Clock"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.TestClock") ) case v if v.startsWith("1.2") => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 4f7db41abe76f..22de8c02e63c8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -88,7 +88,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( // Initial ignore threshold based on which old, existing files in the directory (at the time of // starting the streaming application) will be ignored or considered - private val initialModTimeIgnoreThreshold = if (newFilesOnly) clock.currentTime() else 0L + private val initialModTimeIgnoreThreshold = if (newFilesOnly) clock.getTimeMillis() else 0L /* * Make sure that the information of files selected in the last few batches are remembered. @@ -161,7 +161,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( */ private def findNewFiles(currentTime: Long): Array[String] = { try { - lastNewFileFindingTime = clock.currentTime() + lastNewFileFindingTime = clock.getTimeMillis() // Calculate ignore threshold val modTimeIgnoreThreshold = math.max( @@ -174,7 +174,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) } val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) - val timeTaken = clock.currentTime() - lastNewFileFindingTime + val timeTaken = clock.getTimeMillis() - lastNewFileFindingTime logInfo("Finding new files took " + timeTaken + " ms") logDebug("# cached file times = " + fileToModTime.size) if (timeTaken > slideDuration.milliseconds) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 79263a7183977..ee5e639b26d91 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.streaming.util.RecurringTimer +import org.apache.spark.util.SystemClock /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index f7a8ebee8a544..dcdc27d29c270 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage._ -import org.apache.spark.streaming.util.{Clock, SystemClock, WriteAheadLogFileSegment, WriteAheadLogManager} -import org.apache.spark.util.Utils +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogManager} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** Trait that represents the metadata related to storage of blocks */ private[streaming] trait ReceivedBlockStoreResult { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 8632c94349bf9..ac92774a38273 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -23,7 +23,8 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} -import org.apache.spark.streaming.util.{Clock, ManualClock, RecurringTimer} +import org.apache.spark.streaming.util.RecurringTimer +import org.apache.spark.util.{Clock, ManualClock} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -45,8 +46,14 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val clock = { val clockClass = ssc.sc.conf.get( - "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - Class.forName(clockClass).newInstance().asInstanceOf[Clock] + "spark.streaming.clock", "org.apache.spark.util.SystemClock") + try { + Class.forName(clockClass).newInstance().asInstanceOf[Clock] + } catch { + case e: ClassNotFoundException if clockClass.startsWith("org.apache.spark.streaming") => + val newClockClass = clockClass.replace("org.apache.spark.streaming", "org.apache.spark") + Class.forName(newClockClass).newInstance().asInstanceOf[Clock] + } } private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index e19ac939f9ac5..200cf4ef4b0f1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, Logging, SparkConf} import org.apache.spark.streaming.Time -import org.apache.spark.streaming.util.{Clock, WriteAheadLogManager} -import org.apache.spark.util.Utils +import org.apache.spark.streaming.util.WriteAheadLogManager +import org.apache.spark.util.{Clock, Utils} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ private[streaming] sealed trait ReceivedBlockTrackerLogEvent @@ -150,7 +150,7 @@ private[streaming] class ReceivedBlockTracker( * returns only after the files are cleaned up. */ def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { - assert(cleanupThreshTime.milliseconds < clock.currentTime()) + assert(cleanupThreshTime.milliseconds < clock.getTimeMillis()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq logInfo("Deleting batches " + timesToCleanup) writeToLog(BatchCleanupEvent(timesToCleanup)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala deleted file mode 100644 index d6d96d7ba00fd..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.util - -private[streaming] -trait Clock { - def currentTime(): Long - def waitTillTime(targetTime: Long): Long -} - -private[streaming] -class SystemClock() extends Clock { - - val minPollTime = 25L - - def currentTime(): Long = { - System.currentTimeMillis() - } - - def waitTillTime(targetTime: Long): Long = { - var currentTime = 0L - currentTime = System.currentTimeMillis() - - var waitTime = targetTime - currentTime - if (waitTime <= 0) { - return currentTime - } - - val pollTime = math.max(waitTime / 10.0, minPollTime).toLong - - while (true) { - currentTime = System.currentTimeMillis() - waitTime = targetTime - currentTime - if (waitTime <= 0) { - return currentTime - } - val sleepTime = math.min(waitTime, pollTime) - Thread.sleep(sleepTime) - } - -1 - } -} - -private[streaming] -class ManualClock() extends Clock { - - private var time = 0L - - def currentTime() = this.synchronized { - time - } - - def setTime(timeToSet: Long) = { - this.synchronized { - time = timeToSet - this.notifyAll() - } - } - - def addToTime(timeToAdd: Long) = { - this.synchronized { - time += timeToAdd - this.notifyAll() - } - } - def waitTillTime(targetTime: Long): Long = { - this.synchronized { - while (time < targetTime) { - this.wait(100) - } - } - currentTime() - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 1a616a0434f2c..c8eef833eb431 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.util import org.apache.spark.Logging +import org.apache.spark.util.{Clock, SystemClock} private[streaming] class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) @@ -38,7 +39,7 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: * current system time. */ def getStartTime(): Long = { - (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period + (math.floor(clock.getTimeMillis().toDouble / period) + 1).toLong * period } /** @@ -48,7 +49,7 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: * more than current time. */ def getRestartTime(originalStartTime: Long): Long = { - val gap = clock.currentTime - originalStartTime + val gap = clock.getTimeMillis() - originalStartTime (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala index 166661b7496df..985ded9111f74 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -19,13 +19,12 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.Logging -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, SystemClock, Utils} import WriteAheadLogManager._ /** @@ -82,7 +81,7 @@ private[streaming] class WriteAheadLogManager( var succeeded = false while (!succeeded && failures < maxFailures) { try { - fileSegment = getLogWriter(clock.currentTime).write(byteBuffer) + fileSegment = getLogWriter(clock.getTimeMillis()).write(byteBuffer) succeeded = true } catch { case ex: Exception => diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 1e24da7f5f60c..cfedb5a042a35 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -31,7 +31,7 @@ public void setUp() { SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index e8f4a7779ec21..cf191715d29d6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -22,13 +22,12 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.language.existentials import scala.reflect.ClassTag -import util.ManualClock - import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} +import org.apache.spark.util.{Clock, ManualClock} import org.apache.spark.HashPartitioner class BasicOperationsSuite extends TestSuiteBase { @@ -586,7 +585,7 @@ class BasicOperationsSuite extends TestSuiteBase { for (i <- 0 until input.size) { testServer.send(input(i).toString + "\n") Thread.sleep(200) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) collectRddInfo() } @@ -637,8 +636,8 @@ class BasicOperationsSuite extends TestSuiteBase { ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]] if (rememberDuration != null) ssc.remember(rememberDuration) val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - assert(clock.currentTime() === Seconds(10).milliseconds) + val clock = ssc.scheduler.clock.asInstanceOf[Clock] + assert(clock.getTimeMillis() === Seconds(10).milliseconds) assert(output.size === numExpectedOutput) operatedStream } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 8f8bc61437ba5..03c448f1df5f1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -32,8 +32,7 @@ import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutput import org.scalatest.concurrent.Eventually._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} -import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, ManualClock, Utils} /** * This test suites tests the checkpointing functionality of DStreams - @@ -61,7 +60,7 @@ class CheckpointSuite extends TestSuiteBase { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) val fs = FileSystem.getLocal(new Configuration()) @@ -324,13 +323,13 @@ class CheckpointSuite extends TestSuiteBase { * Writes a file named `i` (which contains the number `i`) to the test directory and sets its * modification time to `clock`'s current time. */ - def writeFile(i: Int, clock: ManualClock): Unit = { + def writeFile(i: Int, clock: Clock): Unit = { val file = new File(testDir, i.toString) Files.write(i + "\n", file, Charsets.UTF_8) - assert(file.setLastModified(clock.currentTime())) + assert(file.setLastModified(clock.getTimeMillis())) // Check that the file's modification date is actually the value we wrote, since rounding or // truncation will break the test: - assert(file.lastModified() === clock.currentTime()) + assert(file.lastModified() === clock.getTimeMillis()) } /** @@ -372,13 +371,13 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() // Advance half a batch so that the first file is created after the StreamingContext starts - clock.addToTime(batchDuration.milliseconds / 2) + clock.advance(batchDuration.milliseconds / 2) // Create files and advance manual clock to process them for (i <- Seq(1, 2, 3)) { writeFile(i, clock) // Advance the clock after creating the file to avoid a race when // setting its modification time - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) if (i != 3) { // Since we want to shut down while the 3rd batch is processing eventually(eventuallyTimeout) { @@ -386,7 +385,7 @@ class CheckpointSuite extends TestSuiteBase { } } } - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) eventually(eventuallyTimeout) { // Wait until all files have been recorded and all batches have started assert(recordedFiles(ssc) === Seq(1, 2, 3) && batchCounter.getNumStartedBatches === 3) @@ -410,7 +409,7 @@ class CheckpointSuite extends TestSuiteBase { writeFile(i, clock) // Advance the clock after creating the file to avoid a race when // setting its modification time - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) } // Recover context from checkpoint file and verify whether the files that were @@ -419,7 +418,7 @@ class CheckpointSuite extends TestSuiteBase { withStreamingContext(new StreamingContext(checkpointDir)) { ssc => // So that the restarted StreamingContext's clock has gone forward in time since failure ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration * 3).milliseconds.toString) - val oldClockTime = clock.currentTime() + val oldClockTime = clock.getTimeMillis() clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val batchCounter = new BatchCounter(ssc) val outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]] @@ -430,7 +429,7 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() // Verify that the clock has traveled forward to the expected time eventually(eventuallyTimeout) { - clock.currentTime() === oldClockTime + clock.getTimeMillis() === oldClockTime } // Wait for pre-failure batch to be recomputed (3 while SSC was down plus last batch) val numBatchesAfterRestart = 4 @@ -441,12 +440,12 @@ class CheckpointSuite extends TestSuiteBase { writeFile(i, clock) // Advance the clock after creating the file to avoid a race when // setting its modification time - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) eventually(eventuallyTimeout) { assert(batchCounter.getNumCompletedBatches === index + numBatchesAfterRestart + 1) } } - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) logInfo("Output after restart = " + outputStream.output.mkString("[", ", ", "]")) assert(outputStream.output.size > 0, "No files processed after restart") ssc.stop() @@ -521,12 +520,12 @@ class CheckpointSuite extends TestSuiteBase { */ def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.currentTime()) + logInfo("Manual clock before advancing = " + clock.getTimeMillis()) for (i <- 1 to numBatches.toInt) { - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) Thread.sleep(batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.currentTime()) + logInfo("Manual clock after advancing = " + clock.getTimeMillis()) Thread.sleep(batchDuration.milliseconds) val outputStream = ssc.graph.getOutputStreams.filter { dstream => diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 01084a457db4f..7ed6320a3d0bc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -17,12 +17,8 @@ package org.apache.spark.streaming -import akka.actor.Actor -import akka.actor.Props -import akka.util.ByteString - import java.io.{File, BufferedWriter, OutputStreamWriter} -import java.net.{InetSocketAddress, SocketException, ServerSocket} +import java.net.{SocketException, ServerSocket} import java.nio.charset.Charset import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger @@ -36,9 +32,8 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.util.Utils -import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} +import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.rdd.RDD import org.apache.hadoop.io.{Text, LongWritable} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat @@ -69,7 +64,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { for (i <- 0 until input.size) { testServer.send(input(i).toString + "\n") Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) } Thread.sleep(1000) logInfo("Stopping server") @@ -120,19 +115,19 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Advance the clock so that the files are created after StreamingContext starts, but // not enough to trigger a batch - clock.addToTime(batchDuration.milliseconds / 2) + clock.advance(batchDuration.milliseconds / 2) val input = Seq(1, 2, 3, 4, 5) input.foreach { i => Thread.sleep(batchDuration.milliseconds) val file = new File(testDir, i.toString) Files.write(Array[Byte](i.toByte), file) - assert(file.setLastModified(clock.currentTime())) - assert(file.lastModified === clock.currentTime) + assert(file.setLastModified(clock.getTimeMillis())) + assert(file.lastModified === clock.getTimeMillis()) logInfo("Created file " + file) // Advance the clock after creating the file to avoid a race when // setting its modification time - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) eventually(eventuallyTimeout) { assert(batchCounter.getNumCompletedBatches === i) } @@ -179,7 +174,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { while((!MultiThreadTestReceiver.haveAllThreadsFinished || output.sum < numTotalRecords) && System.currentTimeMillis() - startTime < 5000) { Thread.sleep(100) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) } Thread.sleep(1000) logInfo("Stopping context") @@ -214,7 +209,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { for (i <- 0 until input.size) { // Enqueue more than 1 item per tick but they should dequeue one at a time inputIterator.take(2).foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) } Thread.sleep(1000) logInfo("Stopping context") @@ -256,12 +251,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Enqueue the first 3 items (one by one), they should be merged in the next batch val inputIterator = input.toIterator inputIterator.take(3).foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) Thread.sleep(1000) // Enqueue the remaining items (again one by one), merged in the final batch inputIterator.foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) Thread.sleep(1000) logInfo("Stopping context") ssc.stop() @@ -308,19 +303,19 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Advance the clock so that the files are created after StreamingContext starts, but // not enough to trigger a batch - clock.addToTime(batchDuration.milliseconds / 2) + clock.advance(batchDuration.milliseconds / 2) // Over time, create files in the directory val input = Seq(1, 2, 3, 4, 5) input.foreach { i => val file = new File(testDir, i.toString) Files.write(i + "\n", file, Charset.forName("UTF-8")) - assert(file.setLastModified(clock.currentTime())) - assert(file.lastModified === clock.currentTime) + assert(file.setLastModified(clock.getTimeMillis())) + assert(file.lastModified === clock.getTimeMillis()) logInfo("Created file " + file) // Advance the clock after creating the file to avoid a race when // setting its modification time - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) eventually(eventuallyTimeout) { assert(batchCounter.getNumCompletedBatches === i) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 132ff2443fc0f..818f551dbe996 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{AkkaUtils, ManualClock} import WriteAheadLogBasedBlockHandler._ import WriteAheadLogSuite._ @@ -165,7 +165,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche preCleanupLogFiles.size should be > 1 // this depends on the number of blocks inserted using generateAndStoreData() - manualClock.currentTime() shouldEqual 5000L + manualClock.getTimeMillis() shouldEqual 5000L val cleanupThreshTime = 3000L handler.cleanupOldBlocks(cleanupThreshTime) @@ -243,7 +243,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche val blockIds = Seq.fill(blocks.size)(generateBlockId()) val storeResults = blocks.zip(blockIds).map { case (block, id) => - manualClock.addToTime(500) // log rolling interval set to 1000 ms through SparkConf + manualClock.advance(500) // log rolling interval set to 1000 ms through SparkConf logDebug("Inserting block " + id) receivedBlockHandler.storeBlock(id, block) }.toList diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index fbb7b0bfebafc..a3a0fd5187403 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -34,9 +34,9 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.util.{Clock, ManualClock, SystemClock, WriteAheadLogReader} +import org.apache.spark.streaming.util.WriteAheadLogReader import org.apache.spark.streaming.util.WriteAheadLogSuite._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} class ReceivedBlockTrackerSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { @@ -100,7 +100,7 @@ class ReceivedBlockTrackerSuite def incrementTime() { val timeIncrementMillis = 2000L - manualClock.addToTime(timeIncrementMillis) + manualClock.advance(timeIncrementMillis) } // Generate and add blocks to the given tracker @@ -138,13 +138,13 @@ class ReceivedBlockTrackerSuite tracker2.getUnallocatedBlocks(streamId).toList shouldEqual blockInfos1 // Allocate blocks to batch and verify whether the unallocated blocks got allocated - val batchTime1 = manualClock.currentTime + val batchTime1 = manualClock.getTimeMillis() tracker2.allocateBlocksToBatch(batchTime1) tracker2.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 // Add more blocks and allocate to another batch incrementTime() - val batchTime2 = manualClock.currentTime + val batchTime2 = manualClock.getTimeMillis() val blockInfos2 = addBlockInfos(tracker2) tracker2.allocateBlocksToBatch(batchTime2) tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 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 7d82c3e4aadcf..c2375ff65edac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -31,10 +31,9 @@ import org.scalatest.concurrent.PatienceConfiguration import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} import org.apache.spark.streaming.scheduler.{StreamingListenerBatchStarted, StreamingListenerBatchCompleted, StreamingListener} -import org.apache.spark.streaming.util.ManualClock import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils +import org.apache.spark.util.{ManualClock, Utils} /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, @@ -189,10 +188,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def beforeFunction() { if (useManualClock) { logInfo("Using manual clock") - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + conf.set("spark.streaming.clock", "org.apache.spark.util.ManualClock") } else { logInfo("Using real clock") - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") + conf.set("spark.streaming.clock", "org.apache.spark.util.SystemClock") } } @@ -333,17 +332,17 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Advance manual clock val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - logInfo("Manual clock before advancing = " + clock.currentTime()) + logInfo("Manual clock before advancing = " + clock.getTimeMillis()) if (actuallyWait) { for (i <- 1 to numBatches) { logInfo("Actually waiting for " + batchDuration) - clock.addToTime(batchDuration.milliseconds) + clock.advance(batchDuration.milliseconds) Thread.sleep(batchDuration.milliseconds) } } else { - clock.addToTime(numBatches * batchDuration.milliseconds) + clock.advance(numBatches * batchDuration.milliseconds) } - logInfo("Manual clock after advancing = " + clock.currentTime()) + logInfo("Manual clock after advancing = " + clock.getTimeMillis()) // Wait until expected number of output items have been generated val startTime = System.currentTimeMillis() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 7ce9499dc614d..8335659667f22 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -26,7 +26,7 @@ import scala.language.{implicitConversions, postfixOps} import WriteAheadLogSuite._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.util.Utils +import org.apache.spark.util.{ManualClock, Utils} import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Eventually._ @@ -197,7 +197,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - manager.cleanupOldLogs(manualClock.currentTime() / 2, waitForCompletion) + manager.cleanupOldLogs(manualClock.getTimeMillis() / 2, waitForCompletion) if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) @@ -219,7 +219,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() - manualClock.addToTime(100000) + manualClock.advance(100000) writeDataUsingManager(testDir, dataToWrite2, manualClock) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) @@ -279,12 +279,12 @@ object WriteAheadLogSuite { manualClock: ManualClock = new ManualClock, stopManager: Boolean = true ): WriteAheadLogManager = { - if (manualClock.currentTime < 100000) manualClock.setTime(10000) + if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) val manager = new WriteAheadLogManager(logDirectory, hadoopConf, rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = manualClock) // Ensure that 500 does not get sorted after 2000, so put a high base value. data.foreach { item => - manualClock.addToTime(500) + manualClock.advance(500) manager.writeToLog(item) } if (stopManager) manager.stop() From 6bddc40353057a562c78e75c5549c79a0d7d5f8b Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Thu, 19 Feb 2015 15:50:58 -0800 Subject: [PATCH 204/817] SPARK-5570: No docs stating that `new SparkConf().set("spark.driver.memory", ...) will not work I've updated documentation to reflect true behavior of this setting in client vs. cluster mode. Author: Ilya Ganelin Closes #4665 from ilganeli/SPARK-5570 and squashes the following commits: 5d1c8dd [Ilya Ganelin] Added example configuration code a51700a [Ilya Ganelin] Getting rid of extra spaces 85f7a08 [Ilya Ganelin] Reworded note 5889d43 [Ilya Ganelin] Formatting adjustment f149ba1 [Ilya Ganelin] Minor updates 1fec7a5 [Ilya Ganelin] Updated to add clarification for other driver properties db47595 [Ilya Ganelin] Slight formatting update c899564 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5570 17b751d [Ilya Ganelin] Updated documentation for driver-memory to reflect its true behavior in client vs cluster mode --- docs/configuration.md | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index eb0d6d33c97d9..541695c83ae36 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -115,7 +115,11 @@ of the most common options to set are: Amount of memory to use for the driver process, i.e. where SparkContext is initialized. (e.g. 512m, 2g). - + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-memory command line option + or in your default properties file. spark.executor.memory @@ -214,6 +218,11 @@ Apart from these, the following properties are also available, and may be useful (none) A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-java-options command line option or in + your default properties file. @@ -221,6 +230,11 @@ Apart from these, the following properties are also available, and may be useful (none) Extra classpath entries to append to the classpath of the driver. + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-class-path command line option or in + your default properties file. @@ -228,6 +242,11 @@ Apart from these, the following properties are also available, and may be useful (none) Set a special library path to use when launching the driver JVM. + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-library-path command line option or in + your default properties file. @@ -237,6 +256,8 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading classes in the the driver. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. + + This is used in cluster mode only. From 0cfd2cebde0b7fac3779eda80d6e42223f8a3d9f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 19 Feb 2015 18:06:16 -0800 Subject: [PATCH 205/817] [SPARK-5900][MLLIB] make PIC and FPGrowth Java-friendly In the previous version, PIC stores clustering assignments as an `RDD[(Long, Int)]`. This is mapped to `RDD>` in Java and hence Java users have to cast types manually. We should either create a new method called `javaAssignments` that returns `JavaRDD[(java.lang.Long, java.lang.Int)]` or wrap the result pair in a class. I chose the latter approach in this PR. Now assignments are stored as an `RDD[Assignment]`, where `Assignment` is a class with `id` and `cluster`. Similarly, in FPGrowth, the frequent itemsets are stored as an `RDD[(Array[Item], Long)]`, which is mapped to `RDD>`. Though we provide a "Java-friendly" method `javaFreqItemsets` that returns `JavaRDD[(Array[Item], java.lang.Long)]`. It doesn't really work because `Array[Item]` is mapped to `Object` in Java. So in this PR I created a class `FreqItemset` to wrap the results. It has `items` and `freq`, as well as a `javaItems` method that returns `List` in Java. I'm not certain that the names I chose are proper: `Assignment`/`id`/`cluster` and `FreqItemset`/`items`/`freq`. Please let me know if there are better suggestions. CC: jkbradley Author: Xiangrui Meng Closes #4695 from mengxr/SPARK-5900 and squashes the following commits: 865b5ca [Xiangrui Meng] make Assignment serializable cffa96e [Xiangrui Meng] fix test 9c0e590 [Xiangrui Meng] remove unused Tuple2 1b9db3d [Xiangrui Meng] make PIC and FPGrowth Java-friendly --- docs/mllib-clustering.md | 8 ++-- docs/mllib-frequent-pattern-mining.md | 12 +++--- .../examples/mllib/JavaFPGrowthExample.java | 8 ++-- .../JavaPowerIterationClusteringExample.java | 5 +-- .../examples/mllib/FPGrowthExample.scala | 4 +- .../PowerIterationClusteringExample.scala | 8 +--- .../clustering/PowerIterationClustering.scala | 33 ++++++++++++--- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 41 +++++++++++++------ .../spark/mllib/fpm/JavaFPGrowthSuite.java | 30 +++++--------- .../PowerIterationClusteringSuite.scala | 8 ++-- .../spark/mllib/fpm/FPGrowthSuite.scala | 10 ++--- 11 files changed, 93 insertions(+), 74 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 6e46a47338398..0b6db4fcb7b1f 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -314,8 +314,8 @@ val pic = new PowerIteartionClustering() .setMaxIterations(20) val model = pic.run(similarities) -model.assignments.foreach { case (vertexId, clusterId) => - println(s"$vertexId -> $clusterId") +model.assignments.foreach { a => + println(s"${a.id} -> ${a.cluster}") } {% endhighlight %} @@ -349,8 +349,8 @@ PowerIterationClustering pic = new PowerIterationClustering() .setMaxIterations(10); PowerIterationClusteringModel model = pic.run(similarities); -for (Tuple2 assignment: model.assignments().toJavaRDD().collect()) { - System.out.println(assignment._1() + " -> " + assignment._2()); +for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) { + System.out.println(a.id() + " -> " + a.cluster()); } {% endhighlight %}
    diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md index 0ff9738768aca..9fd9be0dd01b1 100644 --- a/docs/mllib-frequent-pattern-mining.md +++ b/docs/mllib-frequent-pattern-mining.md @@ -57,8 +57,8 @@ val fpg = new FPGrowth() .setNumPartitions(10) val model = fpg.run(transactions) -model.freqItemsets.collect().foreach { case (itemset, freq) => - println(itemset.mkString("[", ",", "]") + ", " + freq) +model.freqItemsets.collect().foreach { itemset => + println(itemset.items.mkString("[", ",", "]") + ", " + itemset.freq) } {% endhighlight %} @@ -74,10 +74,9 @@ Calling `FPGrowth.run` with transactions returns an that stores the frequent itemsets with their frequencies. {% highlight java %} -import java.util.Arrays; import java.util.List; -import scala.Tuple2; +import com.google.common.base.Joiner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.fpm.FPGrowth; @@ -88,11 +87,10 @@ JavaRDD> transactions = ... FPGrowth fpg = new FPGrowth() .setMinSupport(0.2) .setNumPartitions(10); - FPGrowthModel model = fpg.run(transactions); -for (Tuple2 s: model.javaFreqItemsets().collect()) { - System.out.println("(" + Arrays.toString((Object[]) s._1()) + "): " + s._2()); +for (FPGrowth.FreqItemset itemset: model.freqItemsets().toJavaRDD().collect()) { + System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq()); } {% endhighlight %} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java index 0db572d7607a9..f50e802cf683c 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java @@ -18,10 +18,8 @@ package org.apache.spark.examples.mllib; import java.util.ArrayList; -import java.util.Arrays; - -import scala.Tuple2; +import com.google.common.base.Joiner; import com.google.common.collect.Lists; import org.apache.spark.SparkConf; @@ -54,8 +52,8 @@ public static void main(String[] args) { .setMinSupport(0.3); FPGrowthModel model = fpg.run(transactions); - for (Tuple2 s: model.javaFreqItemsets().collect()) { - System.out.println(Arrays.toString((Object[]) s._1()) + ", " + s._2()); + for (FPGrowth.FreqItemset s: model.freqItemsets().toJavaRDD().collect()) { + System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq()); } sc.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java index e9371de39f284..6c6f9768f015e 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java @@ -17,7 +17,6 @@ package org.apache.spark.examples.mllib; -import scala.Tuple2; import scala.Tuple3; import com.google.common.collect.Lists; @@ -49,8 +48,8 @@ public static void main(String[] args) { .setMaxIterations(10); PowerIterationClusteringModel model = pic.run(similarities); - for (Tuple2 assignment: model.assignments().toJavaRDD().collect()) { - System.out.println(assignment._1() + " -> " + assignment._2()); + for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) { + System.out.println(a.id() + " -> " + a.cluster()); } sc.stop(); diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala index ae66107d7015b..aaae275ec5524 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -42,8 +42,8 @@ object FPGrowthExample { .setMinSupport(0.3) val model = fpg.run(transactions) - model.freqItemsets.collect().foreach { case (itemset, freq) => - println(itemset.mkString("[", ",", "]") + ", " + freq) + model.freqItemsets.collect().foreach { itemset => + println(itemset.items.mkString("[", ",", "]") + ", " + itemset.freq) } sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index b2373adba1fd4..91c9772744f18 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -44,8 +44,7 @@ import org.apache.spark.{SparkConf, SparkContext} * * Here is a sample run and output: * - * ./bin/run-example mllib.PowerIterationClusteringExample - * -k 3 --n 30 --maxIterations 15 + * ./bin/run-example mllib.PowerIterationClusteringExample -k 3 --n 30 --maxIterations 15 * * Cluster assignments: 1 -> [0,1,2,3,4],2 -> [5,6,7,8,9,10,11,12,13,14], * 0 -> [15,16,17,18,19,20,21,22,23,24,25,26,27,28,29] @@ -103,7 +102,7 @@ object PowerIterationClusteringExample { .setMaxIterations(params.maxIterations) .run(circlesRdd) - val clusters = model.assignments.collect.groupBy(_._2).mapValues(_.map(_._1)) + val clusters = model.assignments.collect().groupBy(_.cluster).mapValues(_.map(_.id)) val assignments = clusters.toList.sortBy { case (k, v) => v.length} val assignmentsStr = assignments .map { case (k, v) => @@ -153,8 +152,5 @@ object PowerIterationClusteringExample { val expCoeff = -1.0 / 2.0 * math.pow(sigma, 2.0) val ssquares = (p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2) coeff * math.exp(expCoeff * ssquares) - // math.exp((p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2)) } - - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 63d03347f4572..180023922a9b0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.api.java.JavaRDD import org.apache.spark.{Logging, SparkException} import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.linalg.Vectors @@ -33,12 +33,12 @@ import org.apache.spark.util.random.XORShiftRandom * Model produced by [[PowerIterationClustering]]. * * @param k number of clusters - * @param assignments an RDD of (vertexID, clusterID) pairs + * @param assignments an RDD of clustering [[PowerIterationClustering#Assignment]]s */ @Experimental class PowerIterationClusteringModel( val k: Int, - val assignments: RDD[(Long, Int)]) extends Serializable + val assignments: RDD[PowerIterationClustering.Assignment]) extends Serializable /** * :: Experimental :: @@ -133,16 +133,33 @@ class PowerIterationClustering private[clustering] ( */ private def pic(w: Graph[Double, Double]): PowerIterationClusteringModel = { val v = powerIter(w, maxIterations) - val assignments = kMeans(v, k) + val assignments = kMeans(v, k).mapPartitions({ iter => + iter.map { case (id, cluster) => + new Assignment(id, cluster) + } + }, preservesPartitioning = true) new PowerIterationClusteringModel(k, assignments) } } -private[clustering] object PowerIterationClustering extends Logging { +@Experimental +object PowerIterationClustering extends Logging { + + /** + * :: Experimental :: + * Cluster assignment. + * @param id node id + * @param cluster assigned cluster id + */ + @Experimental + class Assignment(val id: Long, val cluster: Int) extends Serializable + /** * Normalizes the affinity matrix (A) by row sums and returns the normalized affinity matrix (W). */ - def normalize(similarities: RDD[(Long, Long, Double)]): Graph[Double, Double] = { + private[clustering] + def normalize(similarities: RDD[(Long, Long, Double)]) + : Graph[Double, Double] = { val edges = similarities.flatMap { case (i, j, s) => if (s < 0.0) { throw new SparkException("Similarity must be nonnegative but found s($i, $j) = $s.") @@ -173,6 +190,7 @@ private[clustering] object PowerIterationClustering extends Logging { * @return a graph with edges representing W and vertices representing a random vector * with unit 1-norm */ + private[clustering] def randomInit(g: Graph[Double, Double]): Graph[Double, Double] = { val r = g.vertices.mapPartitionsWithIndex( (part, iter) => { @@ -194,6 +212,7 @@ private[clustering] object PowerIterationClustering extends Logging { * @param g a graph representing the normalized affinity matrix (W) * @return a graph with edges representing W and vertices representing the degree vector */ + private[clustering] def initDegreeVector(g: Graph[Double, Double]): Graph[Double, Double] = { val sum = g.vertices.values.sum() val v0 = g.vertices.mapValues(_ / sum) @@ -207,6 +226,7 @@ private[clustering] object PowerIterationClustering extends Logging { * @param maxIterations maximum number of iterations * @return a [[VertexRDD]] representing the pseudo-eigenvector */ + private[clustering] def powerIter( g: Graph[Double, Double], maxIterations: Int): VertexRDD[Double] = { @@ -246,6 +266,7 @@ private[clustering] object PowerIterationClustering extends Logging { * @param k number of clusters * @return a [[VertexRDD]] representing the clustering assignments */ + private[clustering] def kMeans(v: VertexRDD[Double], k: Int): VertexRDD[Int] = { val points = v.mapValues(x => Vectors.dense(x)).cache() val model = new KMeans() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 3168d608c9556..efa8459d3cdba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -26,8 +26,9 @@ import scala.reflect.ClassTag import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} import org.apache.spark.annotation.Experimental -import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} +import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -35,18 +36,11 @@ import org.apache.spark.storage.StorageLevel * :: Experimental :: * * Model trained by [[FPGrowth]], which holds frequent itemsets. - * @param freqItemsets frequent itemset, which is an RDD of (itemset, frequency) pairs + * @param freqItemsets frequent itemset, which is an RDD of [[FreqItemset]] * @tparam Item item type */ @Experimental -class FPGrowthModel[Item: ClassTag]( - val freqItemsets: RDD[(Array[Item], Long)]) extends Serializable { - - /** Returns frequent itemsets as a [[org.apache.spark.api.java.JavaPairRDD]]. */ - def javaFreqItemsets(): JavaPairRDD[Array[Item], java.lang.Long] = { - JavaPairRDD.fromRDD(freqItemsets).asInstanceOf[JavaPairRDD[Array[Item], java.lang.Long]] - } -} +class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable /** * :: Experimental :: @@ -151,7 +145,7 @@ class FPGrowth private ( data: RDD[Array[Item]], minCount: Long, freqItems: Array[Item], - partitioner: Partitioner): RDD[(Array[Item], Long)] = { + partitioner: Partitioner): RDD[FreqItemset[Item]] = { val itemToRank = freqItems.zipWithIndex.toMap data.flatMap { transaction => genCondTransactions(transaction, itemToRank, partitioner) @@ -161,7 +155,7 @@ class FPGrowth private ( .flatMap { case (part, tree) => tree.extract(minCount, x => partitioner.getPartition(x) == part) }.map { case (ranks, count) => - (ranks.map(i => freqItems(i)).toArray, count) + new FreqItemset(ranks.map(i => freqItems(i)).toArray, count) } } @@ -193,3 +187,26 @@ class FPGrowth private ( output } } + +/** + * :: Experimental :: + */ +@Experimental +object FPGrowth { + + /** + * Frequent itemset. + * @param items items in this itemset. Java users should call [[FreqItemset#javaItems]] instead. + * @param freq frequency + * @tparam Item item type + */ + class FreqItemset[Item](val items: Array[Item], val freq: Long) extends Serializable { + + /** + * Returns items in a Java List. + */ + def javaItems: java.util.List[Item] = { + items.toList.asJava + } + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index 851707c8a19c4..bd0edf2b9ea62 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; import org.junit.After; import org.junit.Before; @@ -28,6 +29,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; public class JavaFPGrowthSuite implements Serializable { private transient JavaSparkContext sc; @@ -55,30 +57,18 @@ public void runFPGrowth() { Lists.newArrayList("z".split(" ")), Lists.newArrayList("x z y r q t p".split(" "))), 2); - FPGrowth fpg = new FPGrowth(); - - FPGrowthModel model6 = fpg - .setMinSupport(0.9) - .setNumPartitions(1) - .run(rdd); - assertEquals(0, model6.javaFreqItemsets().count()); - - FPGrowthModel model3 = fpg + FPGrowthModel model = new FPGrowth() .setMinSupport(0.5) .setNumPartitions(2) .run(rdd); - assertEquals(18, model3.javaFreqItemsets().count()); - FPGrowthModel model2 = fpg - .setMinSupport(0.3) - .setNumPartitions(4) - .run(rdd); - assertEquals(54, model2.javaFreqItemsets().count()); + List> freqItemsets = model.freqItemsets().toJavaRDD().collect(); + assertEquals(18, freqItemsets.size()); - FPGrowthModel model1 = fpg - .setMinSupport(0.1) - .setNumPartitions(8) - .run(rdd); - assertEquals(625, model1.javaFreqItemsets().count()); + for (FreqItemset itemset: freqItemsets) { + // Test return types. + List items = itemset.javaItems(); + long freq = itemset.freq(); + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 03ecd9ca730be..6315c03a700f1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -51,8 +51,8 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext .setK(2) .run(sc.parallelize(similarities, 2)) val predictions = Array.fill(2)(mutable.Set.empty[Long]) - model.assignments.collect().foreach { case (i, c) => - predictions(c) += i + model.assignments.collect().foreach { a => + predictions(a.cluster) += a.id } assert(predictions.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) @@ -61,8 +61,8 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext .setInitializationMode("degree") .run(sc.parallelize(similarities, 2)) val predictions2 = Array.fill(2)(mutable.Set.empty[Long]) - model2.assignments.collect().foreach { case (i, c) => - predictions2(c) += i + model2.assignments.collect().foreach { a => + predictions2(a.cluster) += a.id } assert(predictions2.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index 68128284b8608..bd5b9cc3afa10 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -46,8 +46,8 @@ class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { .setMinSupport(0.5) .setNumPartitions(2) .run(rdd) - val freqItemsets3 = model3.freqItemsets.collect().map { case (items, count) => - (items.toSet, count) + val freqItemsets3 = model3.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) } val expected = Set( (Set("s"), 3L), (Set("z"), 5L), (Set("x"), 4L), (Set("t"), 3L), (Set("y"), 3L), @@ -96,10 +96,10 @@ class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { .setMinSupport(0.5) .setNumPartitions(2) .run(rdd) - assert(model3.freqItemsets.first()._1.getClass === Array(1).getClass, + assert(model3.freqItemsets.first().items.getClass === Array(1).getClass, "frequent itemsets should use primitive arrays") - val freqItemsets3 = model3.freqItemsets.collect().map { case (items, count) => - (items.toSet, count) + val freqItemsets3 = model3.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) } val expected = Set( (Set(1), 6L), (Set(2), 5L), (Set(3), 5L), (Set(4), 4L), From 3be92cdac30cf488e09dbdaaa70e5c4cdaa9a099 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 19 Feb 2015 18:09:22 -0800 Subject: [PATCH 206/817] [SPARK-4808] Removing minimum number of elements read before spill check In the general case, Spillable's heuristic of checking for memory stress on every 32nd item after 1000 items are read is good enough. In general, we do not want to be enacting the spilling checks until later on in the job; checking for disk-spilling too early can produce unacceptable performance impact in trivial cases. However, there are non-trivial cases, particularly if each serialized object is large, where checking for the necessity to spill too late would allow the memory to overflow. Consider if every item is 1.5 MB in size, and the heap size is 1000 MB. Then clearly if we only try to spill the in-memory contents to disk after 1000 items are read, we would have already accumulated 1500 MB of RAM and overflowed the heap. Patch #3656 attempted to circumvent this by checking the need to spill on every single item read, but that would cause unacceptable performance in the general case. However, the convoluted cases above should not be forced to be refactored to shrink the data items. Therefore it makes sense that the memory spilling thresholds be configurable. Author: mcheah Closes #4420 from mingyukim/memory-spill-configurable and squashes the following commits: 6e2509f [mcheah] [SPARK-4808] Removing minimum number of elements read before spill check --- .../scala/org/apache/spark/util/collection/Spillable.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 9f54312074856..747ecf075a397 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -42,9 +42,6 @@ private[spark] trait Spillable[C] extends Logging { // Memory manager that can be used to acquire/release memory private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager - // Threshold for `elementsRead` before we start tracking this collection's memory usage - private[this] val trackMemoryThreshold = 1000 - // Initial threshold for the size of a collection before we start tracking its memory usage // Exposed for testing private[this] val initialMemoryThreshold: Long = @@ -72,8 +69,7 @@ private[spark] trait Spillable[C] extends Logging { * @return true if `collection` was spilled to disk; false otherwise */ protected def maybeSpill(collection: C, currentMemory: Long): Boolean = { - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - currentMemory >= myMemoryThreshold) { + if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) From 70bfb5c7282df84e76eba01f59bf1b8551583c33 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 20 Feb 2015 16:20:02 +0800 Subject: [PATCH 207/817] [SPARK-5909][SQL] Add a clearCache command to Spark SQL's cache manager JIRA: https://issues.apache.org/jira/browse/SPARK-5909 Author: Yin Huai Closes #4694 from yhuai/clearCache and squashes the following commits: 397ecc4 [Yin Huai] Address comments. a2702fc [Yin Huai] Update parser. 3a54506 [Yin Huai] add isEmpty to CacheManager. 6d14460 [Yin Huai] Python clearCache. f7b8dbd [Yin Huai] Add clear cache command. --- python/pyspark/sql/context.py | 4 ++++ .../org/apache/spark/sql/CacheManager.scala | 6 ++++++ .../scala/org/apache/spark/sql/SQLContext.scala | 5 +++++ .../org/apache/spark/sql/SparkSQLParser.scala | 11 +++++++---- .../apache/spark/sql/execution/commands.scala | 15 +++++++++++++++ .../org/apache/spark/sql/CachedTableSuite.scala | 16 ++++++++++++++++ 6 files changed, 53 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 2e2309f10375d..3f168f718bd7f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -687,6 +687,10 @@ def uncacheTable(self, tableName): """Removes the specified table from the in-memory cache.""" self._ssql_ctx.uncacheTable(tableName) + def clearCache(self): + """Removes all cached tables from the in-memory cache. """ + self._ssql_ctx.clearCache() + class HiveContext(SQLContext): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index f1949aa5dd74b..ca4a127120b37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -71,11 +71,17 @@ private[sql] class CacheManager(sqlContext: SQLContext) extends Logging { } } + /** Clears all cached tables. */ private[sql] def clearCache(): Unit = writeLock { cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) cachedData.clear() } + /** Checks if the cache is empty. */ + private[sql] def isEmpty: Boolean = readLock { + cachedData.isEmpty + } + /** * Caches the data produced by the logical representation of the given schema rdd. Unlike * `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because recomputing diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a6cf3cd9ddd4f..4bdaa023914b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -217,6 +217,11 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def uncacheTable(tableName: String): Unit = cacheManager.uncacheTable(tableName) + /** + * Removes all cached tables from the in-memory cache. + */ + def clearCache(): Unit = cacheManager.clearCache() + // scalastyle:off // Disable style checker so "implicits" object can start with lowercase i /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala index 00e19da4374a8..5921eaf5e63f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala @@ -23,7 +23,7 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{ShowTablesCommand, UncacheTableCommand, CacheTableCommand, SetCommand} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.types.StringType @@ -57,6 +57,7 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr protected val AS = Keyword("AS") protected val CACHE = Keyword("CACHE") + protected val CLEAR = Keyword("CLEAR") protected val IN = Keyword("IN") protected val LAZY = Keyword("LAZY") protected val SET = Keyword("SET") @@ -74,9 +75,11 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr } private lazy val uncache: Parser[LogicalPlan] = - UNCACHE ~ TABLE ~> ident ^^ { - case tableName => UncacheTableCommand(tableName) - } + ( UNCACHE ~ TABLE ~> ident ^^ { + case tableName => UncacheTableCommand(tableName) + } + | CLEAR ~ CACHE ^^^ ClearCacheCommand + ) private lazy val set: Parser[LogicalPlan] = SET ~> restInput ^^ { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 7c92e9fc88168..a11232142d0fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -174,6 +174,21 @@ case class UncacheTableCommand(tableName: String) extends RunnableCommand { override def output: Seq[Attribute] = Seq.empty } +/** + * :: DeveloperApi :: + * Clear all cached data from the in-memory cache. + */ +@DeveloperApi +case object ClearCacheCommand extends RunnableCommand { + + override def run(sqlContext: SQLContext) = { + sqlContext.clearCache() + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty +} + /** * :: DeveloperApi :: */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index e70e866fdbf14..c240f2be955ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -280,4 +280,20 @@ class CachedTableSuite extends QueryTest { assert(intercept[RuntimeException](table("t1")).getMessage.startsWith("Table Not Found")) assert(!isCached("t2")) } + + test("Clear all cache") { + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + cacheTable("t1") + cacheTable("t2") + clearCache() + assert(cacheManager.isEmpty) + + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + cacheTable("t1") + cacheTable("t2") + sql("Clear CACHE") + assert(cacheManager.isEmpty) + } } From d3dfebebce9f76e4433e16d4d6d29fb8fa4d4193 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Feb 2015 10:21:39 +0000 Subject: [PATCH 208/817] SPARK-5744 [CORE] Take 2. RDD.isEmpty / take fails for (empty) RDD of Nothing Follow-on to https://github.com/apache/spark/pull/4591 Document isEmpty / take / parallelize and their interaction with (an empty) RDD[Nothing] and RDD[Null]. Also, fix a marginally related minor issue with histogram() and EmptyRDD. CC rxin since you reviewed the last one although I imagine this is an uncontroversial resolution. Author: Sean Owen Closes #4698 from srowen/SPARK-5744.2 and squashes the following commits: 9b2a811 [Sean Owen] 2 extra javadoc fixes d1b9fba [Sean Owen] Document isEmpty / take / parallelize and their interaction with (an empty) RDD[Nothing] and RDD[Null]. Also, fix a marginally related minor issue with histogram() and EmptyRDD. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 ++ .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 9 ++++++++- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 7 +++++++ core/src/test/java/org/apache/spark/JavaAPISuite.java | 4 ++++ .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 3 +++ 5 files changed, 24 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d59b466830fdc..85ec5ea11357e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -548,6 +548,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @note Parallelize acts lazily. If `seq` is a mutable collection and is altered after the call * to parallelize and before the first action on the RDD, the resultant RDD will reflect the * modified collection. Pass a copy of the argument to avoid this. + * @note avoid using `parallelize(Seq())` to create an empty `RDD`. Consider `emptyRDD` for an + * RDD with no partitions, or `parallelize(Seq[T]())` for an RDD of `T` with empty partitions. */ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { assertNotStopped() diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index e66f83bb34e30..03afc289736bb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -213,7 +213,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } else { basicBucketFunction _ } - self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters) + if (self.partitions.length == 0) { + new Array[Long](buckets.length - 1) + } else { + // reduce() requires a non-empty RDD. This works because the mapPartitions will make + // non-empty partitions out of empty ones. But it doesn't handle the no-partitions case, + // which is below + self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters) + } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3ab9e54f0ec56..cf0433010aa03 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1146,6 +1146,9 @@ abstract class RDD[T: ClassTag]( * Take the first num elements of the RDD. It works by first scanning one partition, and use the * results from that partition to estimate the number of additional partitions needed to satisfy * the limit. + * + * @note due to complications in the internal implementation, this method will raise + * an exception if called on an RDD of `Nothing` or `Null`. */ def take(num: Int): Array[T] = { if (num == 0) { @@ -1258,6 +1261,10 @@ abstract class RDD[T: ClassTag]( def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min) /** + * @note due to complications in the internal implementation, this method will raise an + * exception if called on an RDD of `Nothing` or `Null`. This may be come up in practice + * because, for example, the type of `parallelize(Seq())` is `RDD[Nothing]`. + * (`parallelize(Seq())` should be avoided anyway in favor of `parallelize(Seq[T]())`.) * @return true if and only if the RDD contains no elements at all. Note that an RDD * may be empty even when it has at least 1 partition. */ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b16a1e9460286..74e88c767ee07 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -708,6 +708,10 @@ public void javaDoubleRDDHistoGram() { // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); Assert.assertArrayEquals(expected_counts, histogram); + // SPARK-5744 + Assert.assertArrayEquals( + new long[] {0}, + sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); } @Test diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index de306533752c1..4cd0f97368ca3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -33,6 +33,9 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { val expectedHistogramResults = Array(0) assert(histogramResults === expectedHistogramResults) assert(histogramResults2 === expectedHistogramResults) + val emptyRDD: RDD[Double] = sc.emptyRDD + assert(emptyRDD.histogram(buckets) === expectedHistogramResults) + assert(emptyRDD.histogram(buckets, true) === expectedHistogramResults) } test("WorksWithOutOfRangeWithOneBucket") { From 4a17eedb16343413e5b6f8bb58c6da8952ee7ab6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 20 Feb 2015 02:31:32 -0800 Subject: [PATCH 209/817] [SPARK-5867] [SPARK-5892] [doc] [ml] [mllib] Doc cleanups for 1.3 release For SPARK-5867: * The spark.ml programming guide needs to be updated to use the new SQL DataFrame API instead of the old SchemaRDD API. * It should also include Python examples now. For SPARK-5892: * Fix Python docs * Various other cleanups BTW, I accidentally merged this with master. If you want to compile it on your own, use this branch which is based on spark/branch-1.3 and cherry-picks the commits from this PR: [https://github.com/jkbradley/spark/tree/doc-review-1.3-check] CC: mengxr (ML), davies (Python docs) Author: Joseph K. Bradley Closes #4675 from jkbradley/doc-review-1.3 and squashes the following commits: f191bb0 [Joseph K. Bradley] small cleanups e786efa [Joseph K. Bradley] small doc corrections 6b1ab4a [Joseph K. Bradley] fixed python lint test 946affa [Joseph K. Bradley] Added sample data for ml.MovieLensALS example. Changed spark.ml Java examples to use DataFrames API instead of sql() da81558 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into doc-review-1.3 629dbf5 [Joseph K. Bradley] Updated based on code review: * made new page for old migration guides * small fixes * moved inherit_doc in python b9df7c4 [Joseph K. Bradley] Small cleanups: toDF to toDF(), adding s for string interpolation 34b067f [Joseph K. Bradley] small doc correction da16aef [Joseph K. Bradley] Fixed python mllib docs 8cce91c [Joseph K. Bradley] GMM: removed old imports, added some doc 695f3f6 [Joseph K. Bradley] partly done trying to fix inherit_doc for class hierarchies in python docs a72c018 [Joseph K. Bradley] made ChiSqTestResult appear in python docs b05a80d [Joseph K. Bradley] organize imports. doc cleanups e572827 [Joseph K. Bradley] updated programming guide for ml and mllib --- data/mllib/als/sample_movielens_movies.txt | 100 ++ data/mllib/als/sample_movielens_ratings.txt | 1501 +++++++++++++++++ docs/ml-guide.md | 249 +-- docs/mllib-guide.md | 133 +- docs/mllib-migration-guides.md | 67 + .../ml/JavaCrossValidatorExample.java | 7 +- .../examples/ml/JavaSimpleParamsExample.java | 8 +- .../JavaSimpleTextClassificationPipeline.java | 7 +- .../ml/simple_text_classification_pipeline.py | 4 +- .../examples/ml/CrossValidatorExample.scala | 2 +- .../spark/examples/ml/MovieLensALS.scala | 32 +- .../examples/ml/SimpleParamsExample.scala | 6 +- .../ml/SimpleTextClassificationPipeline.scala | 2 +- .../mllib/clustering/GaussianMixture.scala | 12 +- .../apache/spark/mllib/linalg/Matrices.scala | 6 +- .../apache/spark/mllib/linalg/Vectors.scala | 2 +- .../spark/mllib/optimization/Gradient.scala | 14 +- python/docs/pyspark.mllib.rst | 11 +- python/pyspark/ml/classification.py | 3 +- python/pyspark/ml/feature.py | 3 +- python/pyspark/ml/pipeline.py | 3 +- python/pyspark/ml/util.py | 14 - python/pyspark/ml/wrapper.py | 2 +- python/pyspark/mllib/__init__.py | 2 +- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/common.py | 17 + python/pyspark/mllib/regression.py | 8 +- python/pyspark/mllib/stat/__init__.py | 4 +- python/pyspark/mllib/stat/distribution.py | 3 +- python/pyspark/mllib/tree.py | 6 +- 30 files changed, 1940 insertions(+), 290 deletions(-) create mode 100644 data/mllib/als/sample_movielens_movies.txt create mode 100644 data/mllib/als/sample_movielens_ratings.txt create mode 100644 docs/mllib-migration-guides.md diff --git a/data/mllib/als/sample_movielens_movies.txt b/data/mllib/als/sample_movielens_movies.txt new file mode 100644 index 0000000000000..934a0253849e1 --- /dev/null +++ b/data/mllib/als/sample_movielens_movies.txt @@ -0,0 +1,100 @@ +0::Movie 0::Romance|Comedy +1::Movie 1::Action|Anime +2::Movie 2::Romance|Thriller +3::Movie 3::Action|Romance +4::Movie 4::Anime|Comedy +5::Movie 5::Action|Action +6::Movie 6::Action|Comedy +7::Movie 7::Anime|Comedy +8::Movie 8::Comedy|Action +9::Movie 9::Anime|Thriller +10::Movie 10::Action|Anime +11::Movie 11::Action|Anime +12::Movie 12::Anime|Comedy +13::Movie 13::Thriller|Action +14::Movie 14::Anime|Comedy +15::Movie 15::Comedy|Thriller +16::Movie 16::Anime|Romance +17::Movie 17::Thriller|Action +18::Movie 18::Action|Comedy +19::Movie 19::Anime|Romance +20::Movie 20::Action|Anime +21::Movie 21::Romance|Thriller +22::Movie 22::Romance|Romance +23::Movie 23::Comedy|Comedy +24::Movie 24::Anime|Action +25::Movie 25::Comedy|Comedy +26::Movie 26::Anime|Romance +27::Movie 27::Anime|Anime +28::Movie 28::Thriller|Anime +29::Movie 29::Anime|Romance +30::Movie 30::Thriller|Romance +31::Movie 31::Thriller|Romance +32::Movie 32::Comedy|Anime +33::Movie 33::Comedy|Comedy +34::Movie 34::Anime|Anime +35::Movie 35::Action|Thriller +36::Movie 36::Anime|Romance +37::Movie 37::Romance|Anime +38::Movie 38::Thriller|Romance +39::Movie 39::Romance|Comedy +40::Movie 40::Action|Anime +41::Movie 41::Comedy|Thriller +42::Movie 42::Comedy|Action +43::Movie 43::Thriller|Anime +44::Movie 44::Anime|Action +45::Movie 45::Comedy|Romance +46::Movie 46::Comedy|Action +47::Movie 47::Romance|Comedy +48::Movie 48::Action|Comedy +49::Movie 49::Romance|Romance +50::Movie 50::Comedy|Romance +51::Movie 51::Action|Action +52::Movie 52::Thriller|Action +53::Movie 53::Action|Action +54::Movie 54::Romance|Thriller +55::Movie 55::Anime|Romance +56::Movie 56::Comedy|Action +57::Movie 57::Action|Anime +58::Movie 58::Thriller|Romance +59::Movie 59::Thriller|Comedy +60::Movie 60::Anime|Comedy +61::Movie 61::Comedy|Action +62::Movie 62::Comedy|Romance +63::Movie 63::Romance|Thriller +64::Movie 64::Romance|Action +65::Movie 65::Anime|Romance +66::Movie 66::Comedy|Action +67::Movie 67::Thriller|Anime +68::Movie 68::Thriller|Romance +69::Movie 69::Action|Comedy +70::Movie 70::Thriller|Thriller +71::Movie 71::Action|Comedy +72::Movie 72::Thriller|Romance +73::Movie 73::Comedy|Action +74::Movie 74::Action|Action +75::Movie 75::Action|Action +76::Movie 76::Comedy|Comedy +77::Movie 77::Comedy|Comedy +78::Movie 78::Comedy|Comedy +79::Movie 79::Thriller|Thriller +80::Movie 80::Comedy|Anime +81::Movie 81::Comedy|Anime +82::Movie 82::Romance|Anime +83::Movie 83::Comedy|Thriller +84::Movie 84::Anime|Action +85::Movie 85::Thriller|Anime +86::Movie 86::Romance|Anime +87::Movie 87::Thriller|Thriller +88::Movie 88::Romance|Thriller +89::Movie 89::Action|Anime +90::Movie 90::Anime|Romance +91::Movie 91::Anime|Thriller +92::Movie 92::Action|Comedy +93::Movie 93::Romance|Thriller +94::Movie 94::Thriller|Comedy +95::Movie 95::Action|Action +96::Movie 96::Thriller|Romance +97::Movie 97::Thriller|Thriller +98::Movie 98::Thriller|Comedy +99::Movie 99::Thriller|Romance diff --git a/data/mllib/als/sample_movielens_ratings.txt b/data/mllib/als/sample_movielens_ratings.txt new file mode 100644 index 0000000000000..0889142950797 --- /dev/null +++ b/data/mllib/als/sample_movielens_ratings.txt @@ -0,0 +1,1501 @@ +0::2::3::1424380312 +0::3::1::1424380312 +0::5::2::1424380312 +0::9::4::1424380312 +0::11::1::1424380312 +0::12::2::1424380312 +0::15::1::1424380312 +0::17::1::1424380312 +0::19::1::1424380312 +0::21::1::1424380312 +0::23::1::1424380312 +0::26::3::1424380312 +0::27::1::1424380312 +0::28::1::1424380312 +0::29::1::1424380312 +0::30::1::1424380312 +0::31::1::1424380312 +0::34::1::1424380312 +0::37::1::1424380312 +0::41::2::1424380312 +0::44::1::1424380312 +0::45::2::1424380312 +0::46::1::1424380312 +0::47::1::1424380312 +0::48::1::1424380312 +0::50::1::1424380312 +0::51::1::1424380312 +0::54::1::1424380312 +0::55::1::1424380312 +0::59::2::1424380312 +0::61::2::1424380312 +0::64::1::1424380312 +0::67::1::1424380312 +0::68::1::1424380312 +0::69::1::1424380312 +0::71::1::1424380312 +0::72::1::1424380312 +0::77::2::1424380312 +0::79::1::1424380312 +0::83::1::1424380312 +0::87::1::1424380312 +0::89::2::1424380312 +0::91::3::1424380312 +0::92::4::1424380312 +0::94::1::1424380312 +0::95::2::1424380312 +0::96::1::1424380312 +0::98::1::1424380312 +0::99::1::1424380312 +1::2::2::1424380312 +1::3::1::1424380312 +1::4::2::1424380312 +1::6::1::1424380312 +1::9::3::1424380312 +1::12::1::1424380312 +1::13::1::1424380312 +1::14::1::1424380312 +1::16::1::1424380312 +1::19::1::1424380312 +1::21::3::1424380312 +1::27::1::1424380312 +1::28::3::1424380312 +1::33::1::1424380312 +1::36::2::1424380312 +1::37::1::1424380312 +1::40::1::1424380312 +1::41::2::1424380312 +1::43::1::1424380312 +1::44::1::1424380312 +1::47::1::1424380312 +1::50::1::1424380312 +1::54::1::1424380312 +1::56::2::1424380312 +1::57::1::1424380312 +1::58::1::1424380312 +1::60::1::1424380312 +1::62::4::1424380312 +1::63::1::1424380312 +1::67::1::1424380312 +1::68::4::1424380312 +1::70::2::1424380312 +1::72::1::1424380312 +1::73::1::1424380312 +1::74::2::1424380312 +1::76::1::1424380312 +1::77::3::1424380312 +1::78::1::1424380312 +1::81::1::1424380312 +1::82::1::1424380312 +1::85::3::1424380312 +1::86::2::1424380312 +1::88::2::1424380312 +1::91::1::1424380312 +1::92::2::1424380312 +1::93::1::1424380312 +1::94::2::1424380312 +1::96::1::1424380312 +1::97::1::1424380312 +2::4::3::1424380312 +2::6::1::1424380312 +2::8::5::1424380312 +2::9::1::1424380312 +2::10::1::1424380312 +2::12::3::1424380312 +2::13::1::1424380312 +2::15::2::1424380312 +2::18::2::1424380312 +2::19::4::1424380312 +2::22::1::1424380312 +2::26::1::1424380312 +2::28::1::1424380312 +2::34::4::1424380312 +2::35::1::1424380312 +2::37::5::1424380312 +2::38::1::1424380312 +2::39::5::1424380312 +2::40::4::1424380312 +2::47::1::1424380312 +2::50::1::1424380312 +2::52::2::1424380312 +2::54::1::1424380312 +2::55::1::1424380312 +2::57::2::1424380312 +2::58::2::1424380312 +2::59::1::1424380312 +2::61::1::1424380312 +2::62::1::1424380312 +2::64::1::1424380312 +2::65::1::1424380312 +2::66::3::1424380312 +2::68::1::1424380312 +2::71::3::1424380312 +2::76::1::1424380312 +2::77::1::1424380312 +2::78::1::1424380312 +2::80::1::1424380312 +2::83::5::1424380312 +2::85::1::1424380312 +2::87::2::1424380312 +2::88::1::1424380312 +2::89::4::1424380312 +2::90::1::1424380312 +2::92::4::1424380312 +2::93::5::1424380312 +3::0::1::1424380312 +3::1::1::1424380312 +3::2::1::1424380312 +3::7::3::1424380312 +3::8::3::1424380312 +3::9::1::1424380312 +3::14::1::1424380312 +3::15::1::1424380312 +3::16::1::1424380312 +3::18::4::1424380312 +3::19::1::1424380312 +3::24::3::1424380312 +3::26::1::1424380312 +3::29::3::1424380312 +3::33::1::1424380312 +3::34::3::1424380312 +3::35::1::1424380312 +3::36::3::1424380312 +3::37::1::1424380312 +3::38::2::1424380312 +3::43::1::1424380312 +3::44::1::1424380312 +3::46::1::1424380312 +3::47::1::1424380312 +3::51::5::1424380312 +3::52::3::1424380312 +3::56::1::1424380312 +3::58::1::1424380312 +3::60::3::1424380312 +3::62::1::1424380312 +3::65::2::1424380312 +3::66::1::1424380312 +3::67::1::1424380312 +3::68::2::1424380312 +3::70::1::1424380312 +3::72::2::1424380312 +3::76::3::1424380312 +3::79::3::1424380312 +3::80::4::1424380312 +3::81::1::1424380312 +3::83::1::1424380312 +3::84::1::1424380312 +3::86::1::1424380312 +3::87::2::1424380312 +3::88::4::1424380312 +3::89::1::1424380312 +3::91::1::1424380312 +3::94::3::1424380312 +4::1::1::1424380312 +4::6::1::1424380312 +4::8::1::1424380312 +4::9::1::1424380312 +4::10::1::1424380312 +4::11::1::1424380312 +4::12::1::1424380312 +4::13::1::1424380312 +4::14::2::1424380312 +4::15::1::1424380312 +4::17::1::1424380312 +4::20::1::1424380312 +4::22::1::1424380312 +4::23::1::1424380312 +4::24::1::1424380312 +4::29::4::1424380312 +4::30::1::1424380312 +4::31::1::1424380312 +4::34::1::1424380312 +4::35::1::1424380312 +4::36::1::1424380312 +4::39::2::1424380312 +4::40::3::1424380312 +4::41::4::1424380312 +4::43::2::1424380312 +4::44::1::1424380312 +4::45::1::1424380312 +4::46::1::1424380312 +4::47::1::1424380312 +4::49::2::1424380312 +4::50::1::1424380312 +4::51::1::1424380312 +4::52::4::1424380312 +4::54::1::1424380312 +4::55::1::1424380312 +4::60::3::1424380312 +4::61::1::1424380312 +4::62::4::1424380312 +4::63::3::1424380312 +4::65::1::1424380312 +4::67::2::1424380312 +4::69::1::1424380312 +4::70::4::1424380312 +4::71::1::1424380312 +4::73::1::1424380312 +4::78::1::1424380312 +4::84::1::1424380312 +4::85::1::1424380312 +4::87::3::1424380312 +4::88::3::1424380312 +4::89::2::1424380312 +4::96::1::1424380312 +4::97::1::1424380312 +4::98::1::1424380312 +4::99::1::1424380312 +5::0::1::1424380312 +5::1::1::1424380312 +5::4::1::1424380312 +5::5::1::1424380312 +5::8::1::1424380312 +5::9::3::1424380312 +5::10::2::1424380312 +5::13::3::1424380312 +5::15::1::1424380312 +5::19::1::1424380312 +5::20::3::1424380312 +5::21::2::1424380312 +5::23::3::1424380312 +5::27::1::1424380312 +5::28::1::1424380312 +5::29::1::1424380312 +5::31::1::1424380312 +5::36::3::1424380312 +5::38::2::1424380312 +5::39::1::1424380312 +5::42::1::1424380312 +5::48::3::1424380312 +5::49::4::1424380312 +5::50::3::1424380312 +5::51::1::1424380312 +5::52::1::1424380312 +5::54::1::1424380312 +5::55::5::1424380312 +5::56::3::1424380312 +5::58::1::1424380312 +5::60::1::1424380312 +5::61::1::1424380312 +5::64::3::1424380312 +5::65::2::1424380312 +5::68::4::1424380312 +5::70::1::1424380312 +5::71::1::1424380312 +5::72::1::1424380312 +5::74::1::1424380312 +5::79::1::1424380312 +5::81::2::1424380312 +5::84::1::1424380312 +5::85::1::1424380312 +5::86::1::1424380312 +5::88::1::1424380312 +5::90::4::1424380312 +5::91::2::1424380312 +5::95::2::1424380312 +5::99::1::1424380312 +6::0::1::1424380312 +6::1::1::1424380312 +6::2::3::1424380312 +6::5::1::1424380312 +6::6::1::1424380312 +6::9::1::1424380312 +6::10::1::1424380312 +6::15::2::1424380312 +6::16::2::1424380312 +6::17::1::1424380312 +6::18::1::1424380312 +6::20::1::1424380312 +6::21::1::1424380312 +6::22::1::1424380312 +6::24::1::1424380312 +6::25::5::1424380312 +6::26::1::1424380312 +6::28::1::1424380312 +6::30::1::1424380312 +6::33::1::1424380312 +6::38::1::1424380312 +6::39::1::1424380312 +6::43::4::1424380312 +6::44::1::1424380312 +6::45::1::1424380312 +6::48::1::1424380312 +6::49::1::1424380312 +6::50::1::1424380312 +6::53::1::1424380312 +6::54::1::1424380312 +6::55::1::1424380312 +6::56::1::1424380312 +6::58::4::1424380312 +6::59::1::1424380312 +6::60::1::1424380312 +6::61::3::1424380312 +6::63::3::1424380312 +6::66::1::1424380312 +6::67::3::1424380312 +6::68::1::1424380312 +6::69::1::1424380312 +6::71::2::1424380312 +6::73::1::1424380312 +6::75::1::1424380312 +6::77::1::1424380312 +6::79::1::1424380312 +6::81::1::1424380312 +6::84::1::1424380312 +6::85::3::1424380312 +6::86::1::1424380312 +6::87::1::1424380312 +6::88::1::1424380312 +6::89::1::1424380312 +6::91::2::1424380312 +6::94::1::1424380312 +6::95::2::1424380312 +6::96::1::1424380312 +7::1::1::1424380312 +7::2::2::1424380312 +7::3::1::1424380312 +7::4::1::1424380312 +7::7::1::1424380312 +7::10::1::1424380312 +7::11::2::1424380312 +7::14::2::1424380312 +7::15::1::1424380312 +7::16::1::1424380312 +7::18::1::1424380312 +7::21::1::1424380312 +7::22::1::1424380312 +7::23::1::1424380312 +7::25::5::1424380312 +7::26::1::1424380312 +7::29::4::1424380312 +7::30::1::1424380312 +7::31::3::1424380312 +7::32::1::1424380312 +7::33::1::1424380312 +7::35::1::1424380312 +7::37::2::1424380312 +7::39::3::1424380312 +7::40::2::1424380312 +7::42::2::1424380312 +7::44::1::1424380312 +7::45::2::1424380312 +7::47::4::1424380312 +7::48::1::1424380312 +7::49::1::1424380312 +7::53::1::1424380312 +7::54::1::1424380312 +7::55::1::1424380312 +7::56::1::1424380312 +7::59::1::1424380312 +7::61::2::1424380312 +7::62::3::1424380312 +7::63::2::1424380312 +7::66::1::1424380312 +7::67::3::1424380312 +7::74::1::1424380312 +7::75::1::1424380312 +7::76::3::1424380312 +7::77::1::1424380312 +7::81::1::1424380312 +7::82::1::1424380312 +7::84::2::1424380312 +7::85::4::1424380312 +7::86::1::1424380312 +7::92::2::1424380312 +7::96::1::1424380312 +7::97::1::1424380312 +7::98::1::1424380312 +8::0::1::1424380312 +8::2::4::1424380312 +8::3::2::1424380312 +8::4::2::1424380312 +8::5::1::1424380312 +8::7::1::1424380312 +8::9::1::1424380312 +8::11::1::1424380312 +8::15::1::1424380312 +8::18::1::1424380312 +8::19::1::1424380312 +8::21::1::1424380312 +8::29::5::1424380312 +8::31::3::1424380312 +8::33::1::1424380312 +8::35::1::1424380312 +8::36::1::1424380312 +8::40::2::1424380312 +8::44::1::1424380312 +8::45::1::1424380312 +8::50::1::1424380312 +8::51::1::1424380312 +8::52::5::1424380312 +8::53::5::1424380312 +8::54::1::1424380312 +8::55::1::1424380312 +8::56::1::1424380312 +8::58::4::1424380312 +8::60::3::1424380312 +8::62::4::1424380312 +8::64::1::1424380312 +8::67::3::1424380312 +8::69::1::1424380312 +8::71::1::1424380312 +8::72::3::1424380312 +8::77::3::1424380312 +8::78::1::1424380312 +8::79::1::1424380312 +8::83::1::1424380312 +8::85::5::1424380312 +8::86::1::1424380312 +8::88::1::1424380312 +8::90::1::1424380312 +8::92::2::1424380312 +8::95::4::1424380312 +8::96::3::1424380312 +8::97::1::1424380312 +8::98::1::1424380312 +8::99::1::1424380312 +9::2::3::1424380312 +9::3::1::1424380312 +9::4::1::1424380312 +9::5::1::1424380312 +9::6::1::1424380312 +9::7::5::1424380312 +9::9::1::1424380312 +9::12::1::1424380312 +9::14::3::1424380312 +9::15::1::1424380312 +9::19::1::1424380312 +9::21::1::1424380312 +9::22::1::1424380312 +9::24::1::1424380312 +9::25::1::1424380312 +9::26::1::1424380312 +9::30::3::1424380312 +9::32::4::1424380312 +9::35::2::1424380312 +9::36::2::1424380312 +9::37::2::1424380312 +9::38::1::1424380312 +9::39::1::1424380312 +9::43::3::1424380312 +9::49::5::1424380312 +9::50::3::1424380312 +9::53::1::1424380312 +9::54::1::1424380312 +9::58::1::1424380312 +9::59::1::1424380312 +9::60::1::1424380312 +9::61::1::1424380312 +9::63::3::1424380312 +9::64::3::1424380312 +9::68::1::1424380312 +9::69::1::1424380312 +9::70::3::1424380312 +9::71::1::1424380312 +9::73::2::1424380312 +9::75::1::1424380312 +9::77::2::1424380312 +9::81::2::1424380312 +9::82::1::1424380312 +9::83::1::1424380312 +9::84::1::1424380312 +9::86::1::1424380312 +9::87::4::1424380312 +9::88::1::1424380312 +9::90::3::1424380312 +9::94::2::1424380312 +9::95::3::1424380312 +9::97::2::1424380312 +9::98::1::1424380312 +10::0::3::1424380312 +10::2::4::1424380312 +10::4::3::1424380312 +10::7::1::1424380312 +10::8::1::1424380312 +10::10::1::1424380312 +10::13::2::1424380312 +10::14::1::1424380312 +10::16::2::1424380312 +10::17::1::1424380312 +10::18::1::1424380312 +10::21::1::1424380312 +10::22::1::1424380312 +10::24::1::1424380312 +10::25::3::1424380312 +10::28::1::1424380312 +10::35::1::1424380312 +10::36::1::1424380312 +10::37::1::1424380312 +10::38::1::1424380312 +10::39::1::1424380312 +10::40::4::1424380312 +10::41::2::1424380312 +10::42::3::1424380312 +10::43::1::1424380312 +10::49::3::1424380312 +10::50::1::1424380312 +10::51::1::1424380312 +10::52::1::1424380312 +10::55::2::1424380312 +10::56::1::1424380312 +10::58::1::1424380312 +10::63::1::1424380312 +10::66::1::1424380312 +10::67::2::1424380312 +10::68::1::1424380312 +10::75::1::1424380312 +10::77::1::1424380312 +10::79::1::1424380312 +10::86::1::1424380312 +10::89::3::1424380312 +10::90::1::1424380312 +10::97::1::1424380312 +10::98::1::1424380312 +11::0::1::1424380312 +11::6::2::1424380312 +11::9::1::1424380312 +11::10::1::1424380312 +11::11::1::1424380312 +11::12::1::1424380312 +11::13::4::1424380312 +11::16::1::1424380312 +11::18::5::1424380312 +11::19::4::1424380312 +11::20::1::1424380312 +11::21::1::1424380312 +11::22::1::1424380312 +11::23::5::1424380312 +11::25::1::1424380312 +11::27::5::1424380312 +11::30::5::1424380312 +11::32::5::1424380312 +11::35::3::1424380312 +11::36::2::1424380312 +11::37::2::1424380312 +11::38::4::1424380312 +11::39::1::1424380312 +11::40::1::1424380312 +11::41::1::1424380312 +11::43::2::1424380312 +11::45::1::1424380312 +11::47::1::1424380312 +11::48::5::1424380312 +11::50::4::1424380312 +11::51::3::1424380312 +11::59::1::1424380312 +11::61::1::1424380312 +11::62::1::1424380312 +11::64::1::1424380312 +11::66::4::1424380312 +11::67::1::1424380312 +11::69::5::1424380312 +11::70::1::1424380312 +11::71::3::1424380312 +11::72::3::1424380312 +11::75::3::1424380312 +11::76::1::1424380312 +11::77::1::1424380312 +11::78::1::1424380312 +11::79::5::1424380312 +11::80::3::1424380312 +11::81::4::1424380312 +11::82::1::1424380312 +11::86::1::1424380312 +11::88::1::1424380312 +11::89::1::1424380312 +11::90::4::1424380312 +11::94::2::1424380312 +11::97::3::1424380312 +11::99::1::1424380312 +12::2::1::1424380312 +12::4::1::1424380312 +12::6::1::1424380312 +12::7::3::1424380312 +12::8::1::1424380312 +12::14::1::1424380312 +12::15::2::1424380312 +12::16::4::1424380312 +12::17::5::1424380312 +12::18::2::1424380312 +12::21::1::1424380312 +12::22::2::1424380312 +12::23::3::1424380312 +12::24::1::1424380312 +12::25::1::1424380312 +12::27::5::1424380312 +12::30::2::1424380312 +12::31::4::1424380312 +12::35::5::1424380312 +12::38::1::1424380312 +12::41::1::1424380312 +12::44::2::1424380312 +12::45::1::1424380312 +12::50::4::1424380312 +12::51::1::1424380312 +12::52::1::1424380312 +12::53::1::1424380312 +12::54::1::1424380312 +12::56::2::1424380312 +12::57::1::1424380312 +12::60::1::1424380312 +12::63::1::1424380312 +12::64::5::1424380312 +12::66::3::1424380312 +12::67::1::1424380312 +12::70::1::1424380312 +12::72::1::1424380312 +12::74::1::1424380312 +12::75::1::1424380312 +12::77::1::1424380312 +12::78::1::1424380312 +12::79::3::1424380312 +12::82::2::1424380312 +12::83::1::1424380312 +12::84::1::1424380312 +12::85::1::1424380312 +12::86::1::1424380312 +12::87::1::1424380312 +12::88::1::1424380312 +12::91::3::1424380312 +12::92::1::1424380312 +12::94::4::1424380312 +12::95::2::1424380312 +12::96::1::1424380312 +12::98::2::1424380312 +13::0::1::1424380312 +13::3::1::1424380312 +13::4::2::1424380312 +13::5::1::1424380312 +13::6::1::1424380312 +13::12::1::1424380312 +13::14::2::1424380312 +13::15::1::1424380312 +13::17::1::1424380312 +13::18::3::1424380312 +13::20::1::1424380312 +13::21::1::1424380312 +13::22::1::1424380312 +13::26::1::1424380312 +13::27::1::1424380312 +13::29::3::1424380312 +13::31::1::1424380312 +13::33::1::1424380312 +13::40::2::1424380312 +13::43::2::1424380312 +13::44::1::1424380312 +13::45::1::1424380312 +13::49::1::1424380312 +13::51::1::1424380312 +13::52::2::1424380312 +13::53::3::1424380312 +13::54::1::1424380312 +13::62::1::1424380312 +13::63::2::1424380312 +13::64::1::1424380312 +13::68::1::1424380312 +13::71::1::1424380312 +13::72::3::1424380312 +13::73::1::1424380312 +13::74::3::1424380312 +13::77::2::1424380312 +13::78::1::1424380312 +13::79::2::1424380312 +13::83::3::1424380312 +13::85::1::1424380312 +13::86::1::1424380312 +13::87::2::1424380312 +13::88::2::1424380312 +13::90::1::1424380312 +13::93::4::1424380312 +13::94::1::1424380312 +13::98::1::1424380312 +13::99::1::1424380312 +14::1::1::1424380312 +14::3::3::1424380312 +14::4::1::1424380312 +14::5::1::1424380312 +14::6::1::1424380312 +14::7::1::1424380312 +14::9::1::1424380312 +14::10::1::1424380312 +14::11::1::1424380312 +14::12::1::1424380312 +14::13::1::1424380312 +14::14::3::1424380312 +14::15::1::1424380312 +14::16::1::1424380312 +14::17::1::1424380312 +14::20::1::1424380312 +14::21::1::1424380312 +14::24::1::1424380312 +14::25::2::1424380312 +14::27::1::1424380312 +14::28::1::1424380312 +14::29::5::1424380312 +14::31::3::1424380312 +14::34::1::1424380312 +14::36::1::1424380312 +14::37::2::1424380312 +14::39::2::1424380312 +14::40::1::1424380312 +14::44::1::1424380312 +14::45::1::1424380312 +14::47::3::1424380312 +14::48::1::1424380312 +14::49::1::1424380312 +14::51::1::1424380312 +14::52::5::1424380312 +14::53::3::1424380312 +14::54::1::1424380312 +14::55::1::1424380312 +14::56::1::1424380312 +14::62::4::1424380312 +14::63::5::1424380312 +14::67::3::1424380312 +14::68::1::1424380312 +14::69::3::1424380312 +14::71::1::1424380312 +14::72::4::1424380312 +14::73::1::1424380312 +14::76::5::1424380312 +14::79::1::1424380312 +14::82::1::1424380312 +14::83::1::1424380312 +14::88::1::1424380312 +14::93::3::1424380312 +14::94::1::1424380312 +14::95::2::1424380312 +14::96::4::1424380312 +14::98::1::1424380312 +15::0::1::1424380312 +15::1::4::1424380312 +15::2::1::1424380312 +15::5::2::1424380312 +15::6::1::1424380312 +15::7::1::1424380312 +15::13::1::1424380312 +15::14::1::1424380312 +15::15::1::1424380312 +15::17::2::1424380312 +15::19::2::1424380312 +15::22::2::1424380312 +15::23::2::1424380312 +15::25::1::1424380312 +15::26::3::1424380312 +15::27::1::1424380312 +15::28::2::1424380312 +15::29::1::1424380312 +15::32::1::1424380312 +15::33::2::1424380312 +15::34::1::1424380312 +15::35::2::1424380312 +15::36::1::1424380312 +15::37::1::1424380312 +15::39::1::1424380312 +15::42::1::1424380312 +15::46::5::1424380312 +15::48::2::1424380312 +15::50::2::1424380312 +15::51::1::1424380312 +15::52::1::1424380312 +15::58::1::1424380312 +15::62::1::1424380312 +15::64::3::1424380312 +15::65::2::1424380312 +15::72::1::1424380312 +15::73::1::1424380312 +15::74::1::1424380312 +15::79::1::1424380312 +15::80::1::1424380312 +15::81::1::1424380312 +15::82::2::1424380312 +15::85::1::1424380312 +15::87::1::1424380312 +15::91::2::1424380312 +15::96::1::1424380312 +15::97::1::1424380312 +15::98::3::1424380312 +16::2::1::1424380312 +16::5::3::1424380312 +16::6::2::1424380312 +16::7::1::1424380312 +16::9::1::1424380312 +16::12::1::1424380312 +16::14::1::1424380312 +16::15::1::1424380312 +16::19::1::1424380312 +16::21::2::1424380312 +16::29::4::1424380312 +16::30::2::1424380312 +16::32::1::1424380312 +16::34::1::1424380312 +16::36::1::1424380312 +16::38::1::1424380312 +16::46::1::1424380312 +16::47::3::1424380312 +16::48::1::1424380312 +16::49::1::1424380312 +16::50::1::1424380312 +16::51::5::1424380312 +16::54::5::1424380312 +16::55::1::1424380312 +16::56::2::1424380312 +16::57::1::1424380312 +16::60::1::1424380312 +16::63::2::1424380312 +16::65::1::1424380312 +16::67::1::1424380312 +16::72::1::1424380312 +16::74::1::1424380312 +16::80::1::1424380312 +16::81::1::1424380312 +16::82::1::1424380312 +16::85::5::1424380312 +16::86::1::1424380312 +16::90::5::1424380312 +16::91::1::1424380312 +16::93::1::1424380312 +16::94::3::1424380312 +16::95::2::1424380312 +16::96::3::1424380312 +16::98::3::1424380312 +16::99::1::1424380312 +17::2::1::1424380312 +17::3::1::1424380312 +17::6::1::1424380312 +17::10::4::1424380312 +17::11::1::1424380312 +17::13::2::1424380312 +17::17::5::1424380312 +17::19::1::1424380312 +17::20::5::1424380312 +17::22::4::1424380312 +17::28::1::1424380312 +17::29::1::1424380312 +17::33::1::1424380312 +17::34::1::1424380312 +17::35::2::1424380312 +17::37::1::1424380312 +17::38::1::1424380312 +17::45::1::1424380312 +17::46::5::1424380312 +17::47::1::1424380312 +17::49::3::1424380312 +17::51::1::1424380312 +17::55::5::1424380312 +17::56::3::1424380312 +17::57::1::1424380312 +17::58::1::1424380312 +17::59::1::1424380312 +17::60::1::1424380312 +17::63::1::1424380312 +17::66::1::1424380312 +17::68::4::1424380312 +17::69::1::1424380312 +17::70::1::1424380312 +17::72::1::1424380312 +17::73::3::1424380312 +17::78::1::1424380312 +17::79::1::1424380312 +17::82::2::1424380312 +17::84::1::1424380312 +17::90::5::1424380312 +17::91::3::1424380312 +17::92::1::1424380312 +17::93::1::1424380312 +17::94::4::1424380312 +17::95::2::1424380312 +17::97::1::1424380312 +18::1::1::1424380312 +18::4::3::1424380312 +18::5::2::1424380312 +18::6::1::1424380312 +18::7::1::1424380312 +18::10::1::1424380312 +18::11::4::1424380312 +18::12::2::1424380312 +18::13::1::1424380312 +18::15::1::1424380312 +18::18::1::1424380312 +18::20::1::1424380312 +18::21::2::1424380312 +18::22::1::1424380312 +18::23::2::1424380312 +18::25::1::1424380312 +18::26::1::1424380312 +18::27::1::1424380312 +18::28::5::1424380312 +18::29::1::1424380312 +18::31::1::1424380312 +18::32::1::1424380312 +18::36::1::1424380312 +18::38::5::1424380312 +18::39::5::1424380312 +18::40::1::1424380312 +18::42::1::1424380312 +18::43::1::1424380312 +18::44::4::1424380312 +18::46::1::1424380312 +18::47::1::1424380312 +18::48::1::1424380312 +18::51::2::1424380312 +18::55::1::1424380312 +18::56::1::1424380312 +18::57::1::1424380312 +18::62::1::1424380312 +18::63::1::1424380312 +18::66::3::1424380312 +18::67::1::1424380312 +18::70::1::1424380312 +18::75::1::1424380312 +18::76::3::1424380312 +18::77::1::1424380312 +18::80::3::1424380312 +18::81::3::1424380312 +18::82::1::1424380312 +18::83::5::1424380312 +18::84::1::1424380312 +18::97::1::1424380312 +18::98::1::1424380312 +18::99::2::1424380312 +19::0::1::1424380312 +19::1::1::1424380312 +19::2::1::1424380312 +19::4::1::1424380312 +19::6::2::1424380312 +19::11::1::1424380312 +19::12::1::1424380312 +19::14::1::1424380312 +19::23::1::1424380312 +19::26::1::1424380312 +19::31::1::1424380312 +19::32::4::1424380312 +19::33::1::1424380312 +19::34::1::1424380312 +19::37::1::1424380312 +19::38::1::1424380312 +19::41::1::1424380312 +19::43::1::1424380312 +19::45::1::1424380312 +19::48::1::1424380312 +19::49::1::1424380312 +19::50::2::1424380312 +19::53::2::1424380312 +19::54::3::1424380312 +19::55::1::1424380312 +19::56::2::1424380312 +19::58::1::1424380312 +19::61::1::1424380312 +19::62::1::1424380312 +19::63::1::1424380312 +19::64::1::1424380312 +19::65::1::1424380312 +19::69::2::1424380312 +19::72::1::1424380312 +19::74::3::1424380312 +19::76::1::1424380312 +19::78::1::1424380312 +19::79::1::1424380312 +19::81::1::1424380312 +19::82::1::1424380312 +19::84::1::1424380312 +19::86::1::1424380312 +19::87::2::1424380312 +19::90::4::1424380312 +19::93::1::1424380312 +19::94::4::1424380312 +19::95::2::1424380312 +19::96::1::1424380312 +19::98::4::1424380312 +20::0::1::1424380312 +20::1::1::1424380312 +20::2::2::1424380312 +20::4::2::1424380312 +20::6::1::1424380312 +20::8::1::1424380312 +20::12::1::1424380312 +20::21::2::1424380312 +20::22::5::1424380312 +20::24::2::1424380312 +20::25::1::1424380312 +20::26::1::1424380312 +20::29::2::1424380312 +20::30::2::1424380312 +20::32::2::1424380312 +20::39::1::1424380312 +20::40::1::1424380312 +20::41::2::1424380312 +20::45::2::1424380312 +20::48::1::1424380312 +20::50::1::1424380312 +20::51::3::1424380312 +20::53::3::1424380312 +20::55::1::1424380312 +20::57::2::1424380312 +20::60::1::1424380312 +20::61::1::1424380312 +20::64::1::1424380312 +20::66::1::1424380312 +20::70::2::1424380312 +20::72::1::1424380312 +20::73::2::1424380312 +20::75::4::1424380312 +20::76::1::1424380312 +20::77::4::1424380312 +20::78::1::1424380312 +20::79::1::1424380312 +20::84::2::1424380312 +20::85::2::1424380312 +20::88::3::1424380312 +20::89::1::1424380312 +20::90::3::1424380312 +20::91::1::1424380312 +20::92::2::1424380312 +20::93::1::1424380312 +20::94::4::1424380312 +20::97::1::1424380312 +21::0::1::1424380312 +21::2::4::1424380312 +21::3::1::1424380312 +21::7::2::1424380312 +21::11::1::1424380312 +21::12::1::1424380312 +21::13::1::1424380312 +21::14::3::1424380312 +21::17::1::1424380312 +21::19::1::1424380312 +21::20::1::1424380312 +21::21::1::1424380312 +21::22::1::1424380312 +21::23::1::1424380312 +21::24::1::1424380312 +21::27::1::1424380312 +21::29::5::1424380312 +21::30::2::1424380312 +21::38::1::1424380312 +21::40::2::1424380312 +21::43::3::1424380312 +21::44::1::1424380312 +21::45::1::1424380312 +21::46::1::1424380312 +21::48::1::1424380312 +21::51::1::1424380312 +21::53::5::1424380312 +21::54::1::1424380312 +21::55::1::1424380312 +21::56::1::1424380312 +21::58::3::1424380312 +21::59::3::1424380312 +21::64::1::1424380312 +21::66::1::1424380312 +21::68::1::1424380312 +21::71::1::1424380312 +21::73::1::1424380312 +21::74::4::1424380312 +21::80::1::1424380312 +21::81::1::1424380312 +21::83::1::1424380312 +21::84::1::1424380312 +21::85::3::1424380312 +21::87::4::1424380312 +21::89::2::1424380312 +21::92::2::1424380312 +21::96::3::1424380312 +21::99::1::1424380312 +22::0::1::1424380312 +22::3::2::1424380312 +22::5::2::1424380312 +22::6::2::1424380312 +22::9::1::1424380312 +22::10::1::1424380312 +22::11::1::1424380312 +22::13::1::1424380312 +22::14::1::1424380312 +22::16::1::1424380312 +22::18::3::1424380312 +22::19::1::1424380312 +22::22::5::1424380312 +22::25::1::1424380312 +22::26::1::1424380312 +22::29::3::1424380312 +22::30::5::1424380312 +22::32::4::1424380312 +22::33::1::1424380312 +22::35::1::1424380312 +22::36::3::1424380312 +22::37::1::1424380312 +22::40::1::1424380312 +22::41::3::1424380312 +22::44::1::1424380312 +22::45::2::1424380312 +22::48::1::1424380312 +22::51::5::1424380312 +22::55::1::1424380312 +22::56::2::1424380312 +22::60::3::1424380312 +22::61::1::1424380312 +22::62::4::1424380312 +22::63::1::1424380312 +22::65::1::1424380312 +22::66::1::1424380312 +22::68::4::1424380312 +22::69::4::1424380312 +22::70::3::1424380312 +22::71::1::1424380312 +22::74::5::1424380312 +22::75::5::1424380312 +22::78::1::1424380312 +22::80::3::1424380312 +22::81::1::1424380312 +22::82::1::1424380312 +22::84::1::1424380312 +22::86::1::1424380312 +22::87::3::1424380312 +22::88::5::1424380312 +22::90::2::1424380312 +22::92::3::1424380312 +22::95::2::1424380312 +22::96::2::1424380312 +22::98::4::1424380312 +22::99::1::1424380312 +23::0::1::1424380312 +23::2::1::1424380312 +23::4::1::1424380312 +23::6::2::1424380312 +23::10::4::1424380312 +23::12::1::1424380312 +23::13::4::1424380312 +23::14::1::1424380312 +23::15::1::1424380312 +23::18::4::1424380312 +23::22::2::1424380312 +23::23::4::1424380312 +23::24::1::1424380312 +23::25::1::1424380312 +23::26::1::1424380312 +23::27::5::1424380312 +23::28::1::1424380312 +23::29::1::1424380312 +23::30::4::1424380312 +23::32::5::1424380312 +23::33::2::1424380312 +23::36::3::1424380312 +23::37::1::1424380312 +23::38::1::1424380312 +23::39::1::1424380312 +23::43::1::1424380312 +23::48::5::1424380312 +23::49::5::1424380312 +23::50::4::1424380312 +23::53::1::1424380312 +23::55::5::1424380312 +23::57::1::1424380312 +23::59::1::1424380312 +23::60::1::1424380312 +23::61::1::1424380312 +23::64::4::1424380312 +23::65::5::1424380312 +23::66::2::1424380312 +23::67::1::1424380312 +23::68::3::1424380312 +23::69::1::1424380312 +23::72::1::1424380312 +23::73::3::1424380312 +23::77::1::1424380312 +23::82::2::1424380312 +23::83::1::1424380312 +23::84::1::1424380312 +23::85::1::1424380312 +23::87::3::1424380312 +23::88::1::1424380312 +23::95::2::1424380312 +23::97::1::1424380312 +24::4::1::1424380312 +24::6::3::1424380312 +24::7::1::1424380312 +24::10::2::1424380312 +24::12::1::1424380312 +24::15::1::1424380312 +24::19::1::1424380312 +24::24::1::1424380312 +24::27::3::1424380312 +24::30::5::1424380312 +24::31::1::1424380312 +24::32::3::1424380312 +24::33::1::1424380312 +24::37::1::1424380312 +24::39::1::1424380312 +24::40::1::1424380312 +24::42::1::1424380312 +24::43::3::1424380312 +24::45::2::1424380312 +24::46::1::1424380312 +24::47::1::1424380312 +24::48::1::1424380312 +24::49::1::1424380312 +24::50::1::1424380312 +24::52::5::1424380312 +24::57::1::1424380312 +24::59::4::1424380312 +24::63::4::1424380312 +24::65::1::1424380312 +24::66::1::1424380312 +24::67::1::1424380312 +24::68::3::1424380312 +24::69::5::1424380312 +24::71::1::1424380312 +24::72::4::1424380312 +24::77::4::1424380312 +24::78::1::1424380312 +24::80::1::1424380312 +24::82::1::1424380312 +24::84::1::1424380312 +24::86::1::1424380312 +24::87::1::1424380312 +24::88::2::1424380312 +24::89::1::1424380312 +24::90::5::1424380312 +24::91::1::1424380312 +24::92::1::1424380312 +24::94::2::1424380312 +24::95::1::1424380312 +24::96::5::1424380312 +24::98::1::1424380312 +24::99::1::1424380312 +25::1::3::1424380312 +25::2::1::1424380312 +25::7::1::1424380312 +25::9::1::1424380312 +25::12::3::1424380312 +25::16::3::1424380312 +25::17::1::1424380312 +25::18::1::1424380312 +25::20::1::1424380312 +25::22::1::1424380312 +25::23::1::1424380312 +25::26::2::1424380312 +25::29::1::1424380312 +25::30::1::1424380312 +25::31::2::1424380312 +25::33::4::1424380312 +25::34::3::1424380312 +25::35::2::1424380312 +25::36::1::1424380312 +25::37::1::1424380312 +25::40::1::1424380312 +25::41::1::1424380312 +25::43::1::1424380312 +25::47::4::1424380312 +25::50::1::1424380312 +25::51::1::1424380312 +25::53::1::1424380312 +25::56::1::1424380312 +25::58::2::1424380312 +25::64::2::1424380312 +25::67::2::1424380312 +25::68::1::1424380312 +25::70::1::1424380312 +25::71::4::1424380312 +25::73::1::1424380312 +25::74::1::1424380312 +25::76::1::1424380312 +25::79::1::1424380312 +25::82::1::1424380312 +25::84::2::1424380312 +25::85::1::1424380312 +25::91::3::1424380312 +25::92::1::1424380312 +25::94::1::1424380312 +25::95::1::1424380312 +25::97::2::1424380312 +26::0::1::1424380312 +26::1::1::1424380312 +26::2::1::1424380312 +26::3::1::1424380312 +26::4::4::1424380312 +26::5::2::1424380312 +26::6::3::1424380312 +26::7::5::1424380312 +26::13::3::1424380312 +26::14::1::1424380312 +26::16::1::1424380312 +26::18::3::1424380312 +26::20::1::1424380312 +26::21::3::1424380312 +26::22::5::1424380312 +26::23::5::1424380312 +26::24::5::1424380312 +26::27::1::1424380312 +26::31::1::1424380312 +26::35::1::1424380312 +26::36::4::1424380312 +26::40::1::1424380312 +26::44::1::1424380312 +26::45::2::1424380312 +26::47::1::1424380312 +26::48::1::1424380312 +26::49::3::1424380312 +26::50::2::1424380312 +26::52::1::1424380312 +26::54::4::1424380312 +26::55::1::1424380312 +26::57::3::1424380312 +26::58::1::1424380312 +26::61::1::1424380312 +26::62::2::1424380312 +26::66::1::1424380312 +26::68::4::1424380312 +26::71::1::1424380312 +26::73::4::1424380312 +26::76::1::1424380312 +26::81::3::1424380312 +26::85::1::1424380312 +26::86::3::1424380312 +26::88::5::1424380312 +26::91::1::1424380312 +26::94::5::1424380312 +26::95::1::1424380312 +26::96::1::1424380312 +26::97::1::1424380312 +27::0::1::1424380312 +27::9::1::1424380312 +27::10::1::1424380312 +27::18::4::1424380312 +27::19::3::1424380312 +27::20::1::1424380312 +27::22::2::1424380312 +27::24::2::1424380312 +27::25::1::1424380312 +27::27::3::1424380312 +27::28::1::1424380312 +27::29::1::1424380312 +27::31::1::1424380312 +27::33::3::1424380312 +27::40::1::1424380312 +27::42::1::1424380312 +27::43::1::1424380312 +27::44::3::1424380312 +27::45::1::1424380312 +27::51::3::1424380312 +27::52::1::1424380312 +27::55::3::1424380312 +27::57::1::1424380312 +27::59::1::1424380312 +27::60::1::1424380312 +27::61::1::1424380312 +27::64::1::1424380312 +27::66::3::1424380312 +27::68::1::1424380312 +27::70::1::1424380312 +27::71::2::1424380312 +27::72::1::1424380312 +27::75::3::1424380312 +27::78::1::1424380312 +27::80::3::1424380312 +27::82::1::1424380312 +27::83::3::1424380312 +27::86::1::1424380312 +27::87::2::1424380312 +27::90::1::1424380312 +27::91::1::1424380312 +27::92::1::1424380312 +27::93::1::1424380312 +27::94::2::1424380312 +27::95::1::1424380312 +27::98::1::1424380312 +28::0::3::1424380312 +28::1::1::1424380312 +28::2::4::1424380312 +28::3::1::1424380312 +28::6::1::1424380312 +28::7::1::1424380312 +28::12::5::1424380312 +28::13::2::1424380312 +28::14::1::1424380312 +28::15::1::1424380312 +28::17::1::1424380312 +28::19::3::1424380312 +28::20::1::1424380312 +28::23::3::1424380312 +28::24::3::1424380312 +28::27::1::1424380312 +28::29::1::1424380312 +28::33::1::1424380312 +28::34::1::1424380312 +28::36::1::1424380312 +28::38::2::1424380312 +28::39::2::1424380312 +28::44::1::1424380312 +28::45::1::1424380312 +28::49::4::1424380312 +28::50::1::1424380312 +28::52::1::1424380312 +28::54::1::1424380312 +28::56::1::1424380312 +28::57::3::1424380312 +28::58::1::1424380312 +28::59::1::1424380312 +28::60::1::1424380312 +28::62::3::1424380312 +28::63::1::1424380312 +28::65::1::1424380312 +28::75::1::1424380312 +28::78::1::1424380312 +28::81::5::1424380312 +28::82::4::1424380312 +28::83::1::1424380312 +28::85::1::1424380312 +28::88::2::1424380312 +28::89::4::1424380312 +28::90::1::1424380312 +28::92::5::1424380312 +28::94::1::1424380312 +28::95::2::1424380312 +28::98::1::1424380312 +28::99::1::1424380312 +29::3::1::1424380312 +29::4::1::1424380312 +29::5::1::1424380312 +29::7::2::1424380312 +29::9::1::1424380312 +29::10::3::1424380312 +29::11::1::1424380312 +29::13::3::1424380312 +29::14::1::1424380312 +29::15::1::1424380312 +29::17::3::1424380312 +29::19::3::1424380312 +29::22::3::1424380312 +29::23::4::1424380312 +29::25::1::1424380312 +29::29::1::1424380312 +29::31::1::1424380312 +29::32::4::1424380312 +29::33::2::1424380312 +29::36::2::1424380312 +29::38::3::1424380312 +29::39::1::1424380312 +29::42::1::1424380312 +29::46::5::1424380312 +29::49::3::1424380312 +29::51::2::1424380312 +29::59::1::1424380312 +29::61::1::1424380312 +29::62::1::1424380312 +29::67::1::1424380312 +29::68::3::1424380312 +29::69::1::1424380312 +29::70::1::1424380312 +29::74::1::1424380312 +29::75::1::1424380312 +29::79::2::1424380312 +29::80::1::1424380312 +29::81::2::1424380312 +29::83::1::1424380312 +29::85::1::1424380312 +29::86::1::1424380312 +29::90::4::1424380312 +29::93::1::1424380312 +29::94::4::1424380312 +29::97::1::1424380312 +29::99::1::1424380312 diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 4bf14fba34eec..da6aef7f14c4c 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -23,13 +23,13 @@ to `spark.ml`. Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple algorithms into a single pipeline, or workflow. This section covers the key concepts introduced by the Spark ML API. -* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL as a dataset which can hold a variety of data types. +* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL as a dataset which can hold a variety of data types. E.g., a dataset could have different columns storing text, feature vectors, true labels, and predictions. -* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `SchemaRDD` into another `SchemaRDD`. +* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. E.g., an ML model is a `Transformer` which transforms an RDD with features into an RDD with predictions. -* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `SchemaRDD` to produce a `Transformer`. +* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. E.g., a learning algorithm is an `Estimator` which trains on a dataset and produces a model. * **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. @@ -39,20 +39,20 @@ E.g., a learning algorithm is an `Estimator` which trains on a dataset and produ ## ML Dataset Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. -Spark ML adopts the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL in order to support a variety of data types under a unified Dataset concept. +Spark ML adopts the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL in order to support a variety of data types under a unified Dataset concept. -`SchemaRDD` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. -In addition to the types listed in the Spark SQL guide, `SchemaRDD` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types. +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. +In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types. -A `SchemaRDD` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. +A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. -Columns in a `SchemaRDD` are named. The code examples below use names such as "text," "features," and "label." +Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." ## ML Algorithms ### Transformers -A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `SchemaRDD` into another, generally by appending one or more columns. +A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `DataFrame` into another, generally by appending one or more columns. For example: * A feature transformer might take a dataset, read a column (e.g., text), convert it into a new column (e.g., feature vectors), append the new column to the dataset, and output the updated dataset. @@ -60,7 +60,7 @@ For example: ### Estimators -An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `SchemaRDD` and produces a `Transformer`. +An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `DataFrame` and produces a `Transformer`. For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling `fit()` trains a `LogisticRegressionModel`, which is a `Transformer`. ### Properties of ML Algorithms @@ -101,7 +101,7 @@ We illustrate this for the simple text document workflow. The figure below is f Above, the top row represents a `Pipeline` with three stages. The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). -The bottom row represents data flowing through the pipeline, where cylinders indicate `SchemaRDD`s. +The bottom row represents data flowing through the pipeline, where cylinders indicate `DataFrame`s. The `Pipeline.fit()` method is called on the original dataset which has raw text documents and labels. The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words into the dataset. The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the dataset. @@ -130,7 +130,7 @@ Each stage's `transform()` method updates the dataset and passes it to the next *DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. -*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `SchemaRDD`. +*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `DataFrame`. ## Parameters @@ -171,12 +171,12 @@ import org.apache.spark.sql.{Row, SQLContext} val conf = new SparkConf().setAppName("SimpleParamsExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) -import sqlContext._ +import sqlContext.implicits._ // Prepare training data. // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes -// into SchemaRDDs, where it uses the case class metadata to infer the schema. -val training = sparkContext.parallelize(Seq( +// into DataFrames, where it uses the case class metadata to infer the schema. +val training = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), @@ -192,7 +192,7 @@ lr.setMaxIter(10) .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. -val model1 = lr.fit(training) +val model1 = lr.fit(training.toDF) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -203,33 +203,35 @@ println("Model 1 was fit using parameters: " + model1.fittingParamMap) // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. -paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.5) // Specify multiple Params. +paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. // One can also combine ParamMaps. -val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Changes output column name. +val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name val paramMapCombined = paramMap ++ paramMap2 // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. -val model2 = lr.fit(training, paramMapCombined) +val model2 = lr.fit(training.toDF, paramMapCombined) println("Model 2 was fit using parameters: " + model2.fittingParamMap) -// Prepare test documents. -val test = sparkContext.parallelize(Seq( +// Prepare test data. +val test = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) -// Make predictions on test documents using the Transformer.transform() method. +// Make predictions on test data using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. -// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' -// column since we renamed the lr.scoreCol parameter previously. -model2.transform(test) - .select('features, 'label, 'probability, 'prediction) +// Note that model2.transform() outputs a 'myProbability' column instead of the usual +// 'probability' column since we renamed the lr.probabilityCol parameter previously. +model2.transform(test.toDF) + .select("features", "label", "myProbability", "prediction") .collect() - .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => - println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => + println("($features, $label) -> prob=$prob, prediction=$prediction") } + +sc.stop() {% endhighlight %}
    @@ -244,23 +246,23 @@ import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.Row; SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); JavaSparkContext jsc = new JavaSparkContext(conf); -JavaSQLContext jsql = new JavaSQLContext(jsc); +SQLContext jsql = new SQLContext(jsc); // Prepare training data. -// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes -// into SchemaRDDs, where it uses the case class metadata to infer the schema. +// We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans +// into DataFrames, where it uses the bean metadata to infer the schema. List localTraining = Lists.newArrayList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); -JavaSchemaRDD training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); +DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -281,13 +283,13 @@ System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap // We may alternatively specify parameters using a ParamMap. ParamMap paramMap = new ParamMap(); -paramMap.put(lr.maxIter(), 20); // Specify 1 Param. +paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. -paramMap.put(lr.regParam(), 0.1); +paramMap.put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. // One can also combine ParamMaps. ParamMap paramMap2 = new ParamMap(); -paramMap2.put(lr.scoreCol(), "probability"); // Changes output column name. +paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); // Now learn a new model using the paramMapCombined parameters. @@ -300,19 +302,19 @@ List localTest = Lists.newArrayList( new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); -JavaSchemaRDD test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); +DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. -// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' -// column since we renamed the lr.scoreCol parameter previously. -model2.transform(test).registerAsTable("results"); -JavaSchemaRDD results = - jsql.sql("SELECT features, label, probability, prediction FROM results"); -for (Row r: results.collect()) { +// Note that model2.transform() outputs a 'myProbability' column instead of the usual +// 'probability' column since we renamed the lr.probabilityCol parameter previously. +DataFrame results = model2.transform(test); +for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + +jsc.stop(); {% endhighlight %}
    @@ -330,6 +332,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} // Labeled and unlabeled instance types. @@ -337,14 +340,14 @@ import org.apache.spark.sql.{Row, SQLContext} case class LabeledDocument(id: Long, text: String, label: Double) case class Document(id: Long, text: String) -// Set up contexts. Import implicit conversions to SchemaRDD from sqlContext. +// Set up contexts. Import implicit conversions to DataFrame from sqlContext. val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) -import sqlContext._ +import sqlContext.implicits._ // Prepare training documents, which are labeled. -val training = sparkContext.parallelize(Seq( +val training = sc.parallelize(Seq( LabeledDocument(0L, "a b c d e spark", 1.0), LabeledDocument(1L, "b d", 0.0), LabeledDocument(2L, "spark f g h", 1.0), @@ -365,30 +368,32 @@ val pipeline = new Pipeline() .setStages(Array(tokenizer, hashingTF, lr)) // Fit the pipeline to training documents. -val model = pipeline.fit(training) +val model = pipeline.fit(training.toDF) // Prepare test documents, which are unlabeled. -val test = sparkContext.parallelize(Seq( +val test = sc.parallelize(Seq( Document(4L, "spark i j k"), Document(5L, "l m n"), Document(6L, "mapreduce spark"), Document(7L, "apache hadoop"))) // Make predictions on test documents. -model.transform(test) - .select('id, 'text, 'score, 'prediction) +model.transform(test.toDF) + .select("id", "text", "probability", "prediction") .collect() - .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => - println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println("($id, $text) --> prob=$prob, prediction=$prediction") } + +sc.stop() {% endhighlight %}
    {% highlight java %} -import java.io.Serializable; import java.util.List; import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -396,10 +401,9 @@ import org.apache.spark.ml.PipelineStage; import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; -import org.apache.spark.SparkConf; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; // Labeled and unlabeled instance types. // Spark SQL can infer schema from Java Beans. @@ -434,7 +438,7 @@ public class LabeledDocument extends Document implements Serializable { // Set up contexts. SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); JavaSparkContext jsc = new JavaSparkContext(conf); -JavaSQLContext jsql = new JavaSQLContext(jsc); +SQLContext jsql = new SQLContext(jsc); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -442,8 +446,7 @@ List localTraining = Lists.newArrayList( new LabeledDocument(1L, "b d", 0.0), new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); -JavaSchemaRDD training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); +DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -468,16 +471,62 @@ List localTest = Lists.newArrayList( new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); -JavaSchemaRDD test = - jsql.createDataFrame(jsc.parallelize(localTest), Document.class); +DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. -model.transform(test).registerAsTable("prediction"); -JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); -for (Row r: predictions.collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) +DataFrame predictions = model.transform(test); +for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + +jsc.stop(); +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark import SparkContext +from pyspark.ml import Pipeline +from pyspark.ml.classification import LogisticRegression +from pyspark.ml.feature import HashingTF, Tokenizer +from pyspark.sql import Row, SQLContext + +sc = SparkContext(appName="SimpleTextClassificationPipeline") +sqlCtx = SQLContext(sc) + +# Prepare training documents, which are labeled. +LabeledDocument = Row("id", "text", "label") +training = sc.parallelize([(0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0)]) \ + .map(lambda x: LabeledDocument(*x)).toDF() + +# Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. +tokenizer = Tokenizer(inputCol="text", outputCol="words") +hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") +lr = LogisticRegression(maxIter=10, regParam=0.01) +pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) + +# Fit the pipeline to training documents. +model = pipeline.fit(training) + +# Prepare test documents, which are unlabeled. +Document = Row("id", "text") +test = sc.parallelize([(4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop")]) \ + .map(lambda x: Document(*x)).toDF() + +# Make predictions on test documents and print columns of interest. +prediction = model.transform(test) +selected = prediction.select("id", "text", "prediction") +for row in selected.collect(): + print row + +sc.stop() {% endhighlight %}
    @@ -508,21 +557,21 @@ However, it is also a well-established method for choosing parameters which is m
    {% highlight scala %} import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} val conf = new SparkConf().setAppName("CrossValidatorExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) -import sqlContext._ +import sqlContext.implicits._ // Prepare training documents, which are labeled. -val training = sparkContext.parallelize(Seq( +val training = sc.parallelize(Seq( LabeledDocument(0L, "a b c d e spark", 1.0), LabeledDocument(1L, "b d", 0.0), LabeledDocument(2L, "spark f g h", 1.0), @@ -565,24 +614,24 @@ crossval.setEstimatorParamMaps(paramGrid) crossval.setNumFolds(2) // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. -val cvModel = crossval.fit(training) -// Get the best LogisticRegression model (with the best set of parameters from paramGrid). -val lrModel = cvModel.bestModel +val cvModel = crossval.fit(training.toDF) // Prepare test documents, which are unlabeled. -val test = sparkContext.parallelize(Seq( +val test = sc.parallelize(Seq( Document(4L, "spark i j k"), Document(5L, "l m n"), Document(6L, "mapreduce spark"), Document(7L, "apache hadoop"))) // Make predictions on test documents. cvModel uses the best model found (lrModel). -cvModel.transform(test) - .select('id, 'text, 'score, 'prediction) +cvModel.transform(test.toDF) + .select("id", "text", "probability", "prediction") .collect() - .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => - println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println(s"($id, $text) --> prob=$prob, prediction=$prediction") } + +sc.stop() {% endhighlight %}
    @@ -592,7 +641,6 @@ import java.util.List; import com.google.common.collect.Lists; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.Model; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineStage; import org.apache.spark.ml.classification.LogisticRegression; @@ -603,13 +651,13 @@ import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.tuning.CrossValidator; import org.apache.spark.ml.tuning.CrossValidatorModel; import org.apache.spark.ml.tuning.ParamGridBuilder; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); JavaSparkContext jsc = new JavaSparkContext(conf); -JavaSQLContext jsql = new JavaSQLContext(jsc); +SQLContext jsql = new SQLContext(jsc); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -625,8 +673,7 @@ List localTraining = Lists.newArrayList( new LabeledDocument(9L, "a e c l", 0.0), new LabeledDocument(10L, "spark compile", 1.0), new LabeledDocument(11L, "hadoop software", 0.0)); -JavaSchemaRDD training = - jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); +DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -660,8 +707,6 @@ crossval.setNumFolds(2); // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. CrossValidatorModel cvModel = crossval.fit(training); -// Get the best LogisticRegression model (with the best set of parameters from paramGrid). -Model lrModel = cvModel.bestModel(); // Prepare test documents, which are unlabeled. List localTest = Lists.newArrayList( @@ -669,15 +714,16 @@ List localTest = Lists.newArrayList( new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); -JavaSchemaRDD test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); +DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). -cvModel.transform(test).registerAsTable("prediction"); -JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); -for (Row r: predictions.collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) +DataFrame predictions = cvModel.transform(test); +for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + +jsc.stop(); {% endhighlight %}
    @@ -686,6 +732,21 @@ for (Row r: predictions.collect()) { # Dependencies Spark ML currently depends on MLlib and has the same dependencies. -Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info. +Please see the [MLlib Dependencies guide](mllib-guide.html#dependencies) for more info. Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. + +# Migration Guide + +## From 1.2 to 1.3 + +The main API changes are from Spark SQL. We list the most important changes here: + +* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame. +* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. +* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. + +Other changes were in `LogisticRegression`: + +* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). +* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 0ca51f92d7a61..4c7a7d9115ca1 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -46,7 +46,7 @@ and the migration guide below will explain all changes between releases. # spark.ml: high-level APIs for ML pipelines -Spark 1.2 includes a new package called `spark.ml`, which aims to provide a uniform set of +Spark 1.2 introduced a new package called `spark.ml`, which aims to provide a uniform set of high-level APIs that help users create and tune practical machine learning pipelines. It is currently an alpha component, and we would like to hear back from the community about how it fits real-world use cases and how it could be improved. @@ -92,125 +92,22 @@ version 1.4 or newer. # Migration Guide -## From 1.1 to 1.2 +For the `spark.ml` package, please see the [spark.ml Migration Guide](ml-guide.html#migration-guide). -The only API changes in MLlib v1.2 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.2: +## From 1.2 to 1.3 -1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number -of classes. In MLlib v1.1, this argument was called `numClasses` in Python and -`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. -This `numClasses` parameter is specified either via -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. +In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. -2. *(Breaking change)* The API for -[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. -This should generally not affect user code, unless the user manually constructs decision trees -(instead of using the `trainClassifier` or `trainRegressor` methods). -The tree `Node` now includes more information, including the probability of the predicted label -(for classification). +* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed. +* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`. +* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes: + * The constructor taking arguments was removed in favor of a builder patten using the default constructor plus parameter setter methods. + * Variable `model` is no longer public. +* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes: + * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) + * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. +* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. -3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. +## Previous Spark Versions -Examples in the Spark distribution and examples in the -[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. - -## From 1.0 to 1.1 - -The only API changes in MLlib v1.1 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.1: - -1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match -the implementations of trees in -[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) -and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). -In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. -In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. -This depth is specified by the `maxDepth` parameter in -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. - -2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` -methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -rather than using the old parameter class `Strategy`. These new training methods explicitly -separate classification and regression, and they replace specialized parameter types with -simple `String` types. - -Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the -[Decision Trees Guide](mllib-decision-tree.html#examples). - -## From 0.9 to 1.0 - -In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few -breaking changes. If your data is sparse, please store it in a sparse format instead of dense to -take advantage of sparsity in both storage and computation. Details are described below. - -
    -
    - -We used to represent a feature vector by `Array[Double]`, which is replaced by -[`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used -to accept `RDD[Array[Double]]` now take -`RDD[Vector]`. [`LabeledPoint`](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) -is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting -`Array[Double]` to `Vector` is straightforward: - -{% highlight scala %} -import org.apache.spark.mllib.linalg.{Vector, Vectors} - -val array: Array[Double] = ... // a double array -val vector: Vector = Vectors.dense(array) // a dense vector -{% endhighlight %} - -[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. - -*Note*: Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. - -
    - -
    - -We used to represent a feature vector by `double[]`, which is replaced by -[`Vector`](api/java/index.html?org/apache/spark/mllib/linalg/Vector.html) in v1.0. Algorithms that used -to accept `RDD` now take -`RDD`. [`LabeledPoint`](api/java/index.html?org/apache/spark/mllib/regression/LabeledPoint.html) -is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to -`Vector` is straightforward: - -{% highlight java %} -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; - -double[] array = ... // a double array -Vector vector = Vectors.dense(array); // a dense vector -{% endhighlight %} - -[`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to -create sparse vectors. - -
    - -
    - -We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to -the label and the rest are features. This representation is replaced by class -[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html), which takes both -dense and sparse feature vectors. - -{% highlight python %} -from pyspark.mllib.linalg import SparseVector -from pyspark.mllib.regression import LabeledPoint - -# Create a labeled point with a positive label and a dense feature vector. -pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) - -# Create a labeled point with a negative label and a sparse feature vector. -neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) -{% endhighlight %} -
    -
    +Earlier migration guides are archived [on this page](mllib-migration-guides.html). diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md new file mode 100644 index 0000000000000..4de2d9491ac2b --- /dev/null +++ b/docs/mllib-migration-guides.md @@ -0,0 +1,67 @@ +--- +layout: global +title: Old Migration Guides - MLlib +displayTitle:
    MLlib - Old Migration Guides +description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +--- + +The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide). + +## From 1.1 to 1.2 + +The only API changes in MLlib v1.2 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.2: + +1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number +of classes. In MLlib v1.1, this argument was called `numClasses` in Python and +`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. +This `numClasses` parameter is specified either via +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Breaking change)* The API for +[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. +This should generally not affect user code, unless the user manually constructs decision trees +(instead of using the `trainClassifier` or `trainRegressor` methods). +The tree `Node` now includes more information, including the probability of the predicted label +(for classification). + +3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. + +Examples in the Spark distribution and examples in the +[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. + +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + +## From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. Details are described below. + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 5d8c5d0a92daa..9bbc14ea40875 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -34,8 +34,8 @@ import org.apache.spark.ml.tuning.CrossValidatorModel; import org.apache.spark.ml.tuning.ParamGridBuilder; import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; /** * A simple example demonstrating model selection using CrossValidator. @@ -115,9 +115,8 @@ public static void main(String[] args) { DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). - cvModel.transform(test).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT id, text, probability, prediction FROM prediction"); - for (Row r: predictions.collect()) { + DataFrame predictions = cvModel.transform(test); + for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index 4c4d532388781..4e02acce696e6 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -29,8 +29,8 @@ import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -100,10 +100,8 @@ public static void main(String[] args) { // LogisticRegression.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. - model2.transform(test).registerTempTable("results"); - DataFrame results = - jsql.sql("SELECT features, label, myProbability, prediction FROM results"); - for (Row r: results.collect()) { + DataFrame results = model2.transform(test); + for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index fdcfc888c235f..ef1ec103a879f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -30,8 +30,8 @@ import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; /** * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java @@ -82,9 +82,8 @@ public static void main(String[] args) { DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. - model.transform(test).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); - for (Row r: predictions.collect()) { + DataFrame predictions = model.transform(test); + for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index b4d9355b681f6..d281f4fa44282 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -16,10 +16,10 @@ # from pyspark import SparkContext -from pyspark.sql import SQLContext, Row from pyspark.ml import Pipeline -from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.classification import LogisticRegression +from pyspark.ml.feature import HashingTF, Tokenizer +from pyspark.sql import Row, SQLContext """ diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index 7ab892cd7560c..6c0af20461d3b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -100,7 +100,7 @@ object CrossValidatorExample { Document(7L, "apache hadoop"))) // Make predictions on test documents. cvModel uses the best model found (lrModel). - cvModel.transform(test.toDF) + cvModel.transform(test.toDF()) .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala index 96b2dd463e253..25f21113bf622 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala @@ -93,8 +93,8 @@ object MovieLensALS { | bin/spark-submit --class org.apache.spark.examples.ml.MovieLensALS \ | examples/target/scala-*/spark-examples-*.jar \ | --rank 10 --maxIter 15 --regParam 0.1 \ - | --movies path/to/movielens/movies.dat \ - | --ratings path/to/movielens/ratings.dat + | --movies data/mllib/als/sample_movielens_movies.txt \ + | --ratings data/mllib/als/sample_movielens_ratings.txt """.stripMargin) } @@ -157,17 +157,23 @@ object MovieLensALS { println(s"Test RMSE = $rmse.") // Inspect false positives. - predictions.registerTempTable("prediction") - sc.textFile(params.movies).map(Movie.parseMovie).toDF().registerTempTable("movie") - sqlContext.sql( - """ - |SELECT userId, prediction.movieId, title, rating, prediction - | FROM prediction JOIN movie ON prediction.movieId = movie.movieId - | WHERE rating <= 1 AND prediction >= 4 - | LIMIT 100 - """.stripMargin) - .collect() - .foreach(println) + // Note: We reference columns in 2 ways: + // (1) predictions("movieId") lets us specify the movieId column in the predictions + // DataFrame, rather than the movieId column in the movies DataFrame. + // (2) $"userId" specifies the userId column in the predictions DataFrame. + // We could also write predictions("userId") but do not have to since + // the movies DataFrame does not have a column "userId." + val movies = sc.textFile(params.movies).map(Movie.parseMovie).toDF() + val falsePositives = predictions.join(movies) + .where((predictions("movieId") === movies("movieId")) + && ($"rating" <= 1) && ($"prediction" >= 4)) + .select($"userId", predictions("movieId"), $"title", $"rating", $"prediction") + val numFalsePositives = falsePositives.count() + println(s"Found $numFalsePositives false positives") + if (numFalsePositives > 0) { + println(s"Example false positives:") + falsePositives.limit(100).collect().foreach(println) + } sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index e8af5c162586a..bf805149d0af6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -40,8 +40,8 @@ object SimpleParamsExample { import sqlContext.implicits._ // Prepare training data. - // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans - // into DataFrames, where it uses the bean metadata to infer the schema. + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes + // into DataFrames, where it uses the case class metadata to infer the schema. val training = sc.parallelize(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), @@ -94,7 +94,7 @@ object SimpleParamsExample { .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => - println("($features, $label) -> prob=$prob, prediction=$prediction") + println(s"($features, $label) -> prob=$prob, prediction=$prediction") } sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index a11db6fd5c382..6772efd2c581c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -83,7 +83,7 @@ object SimpleTextClassificationPipeline { .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println("($id, $text) --> prob=$prob, prediction=$prediction") + println(s"($id, $text) --> prob=$prob, prediction=$prediction") } sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 80584ef5e5979..568b65305649f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -19,12 +19,10 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.IndexedSeq -import breeze.linalg.{diag, DenseMatrix => BreezeMatrix, DenseVector => BDV, SparseVector => BSV, - Transpose, Vector => BV} +import breeze.linalg.{diag, DenseMatrix => BreezeMatrix, DenseVector => BDV, Vector => BV} import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{BLAS, DenseVector, DenseMatrix, Matrices, - SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, Matrices, Vector, Vectors} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD @@ -43,7 +41,11 @@ import org.apache.spark.util.Utils * less than convergenceTol, or until it has reached the max number of iterations. * While this process is generally guaranteed to converge, it is not guaranteed * to find a global optimum. - * + * + * Note: For high-dimensional data (with many features), this algorithm may perform poorly. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. + * * @param k The number of independent Gaussians in the mixture model * @param convergenceTol The maximum change in log-likelihood at which convergence * is considered to have occurred. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 89b38679b7494..0e4a4d0085895 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -706,7 +706,7 @@ object Matrices { } /** - * Generate a `DenseMatrix` consisting of zeros. + * Generate a `Matrix` consisting of zeros. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of zeros @@ -778,8 +778,8 @@ object Matrices { SparseMatrix.sprandn(numRows, numCols, density, rng) /** - * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. - * @param vector a `Vector` tat will form the values on the diagonal of the matrix + * Generate a diagonal matrix in `Matrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 480bbfb5fe94a..4bdcb283da09c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -247,7 +247,7 @@ object Vectors { } /** - * Creates a dense vector of all zeros. + * Creates a vector of all zeros. * * @param size vector size * @return a zero vector diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 0acdab797e8f3..8bfa0d2b64995 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -63,10 +63,12 @@ abstract class Gradient extends Serializable { * http://statweb.stanford.edu/~tibs/ElemStatLearn/ , Eq. (4.17) on page 119 gives the formula of * multinomial logistic regression model. A simple calculation shows that * + * {{{ * P(y=0|x, w) = 1 / (1 + \sum_i^{K-1} \exp(x w_i)) * P(y=1|x, w) = exp(x w_1) / (1 + \sum_i^{K-1} \exp(x w_i)) * ... * P(y=K-1|x, w) = exp(x w_{K-1}) / (1 + \sum_i^{K-1} \exp(x w_i)) + * }}} * * for K classes multiclass classification problem. * @@ -75,9 +77,11 @@ abstract class Gradient extends Serializable { * will be (K-1) * N. * * As a result, the loss of objective function for a single instance of data can be written as + * {{{ * l(w, x) = -log P(y|x, w) = -\alpha(y) log P(y=0|x, w) - (1-\alpha(y)) log P(y|x, w) * = log(1 + \sum_i^{K-1}\exp(x w_i)) - (1-\alpha(y)) x w_{y-1} * = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} + * }}} * * where \alpha(i) = 1 if i != 0, and * \alpha(i) = 0 if i == 0, @@ -86,14 +90,16 @@ abstract class Gradient extends Serializable { * For optimization, we have to calculate the first derivative of the loss function, and * a simple calculation shows that * + * {{{ * \frac{\partial l(w, x)}{\partial w_{ij}} * = (\exp(x w_i) / (1 + \sum_k^{K-1} \exp(x w_k)) - (1-\alpha(y)\delta_{y, i+1})) * x_j * = multiplier_i * x_j + * }}} * * where \delta_{i, j} = 1 if i == j, * \delta_{i, j} = 0 if i != j, and - * multiplier - * = \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) + * multiplier = + * \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) * * If any of margins is larger than 709.78, the numerical computation of multiplier and loss * function will be suffered from arithmetic overflow. This issue occurs when there are outliers @@ -103,10 +109,12 @@ abstract class Gradient extends Serializable { * Fortunately, when max(margins) = maxMargin > 0, the loss function and the multiplier can be * easily rewritten into the following equivalent numerically stable formula. * + * {{{ * l(w, x) = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} * = log(\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin)) + maxMargin * - (1-\alpha(y)) margins_{y-1} * = log(1 + sum) + maxMargin - (1-\alpha(y)) margins_{y-1} + * }}} * * where sum = \exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin) - 1. * @@ -115,8 +123,10 @@ abstract class Gradient extends Serializable { * * For multiplier, similar trick can be applied as the following, * + * {{{ * multiplier = \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) * = \exp(margins_i - maxMargin) / (1 + sum) - (1-\alpha(y)\delta_{y, i+1}) + * }}} * * where each term in \exp is also smaller than zero, so overflow is not a concern. * diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index 21f66ca344a3c..b706c5e376ef4 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -7,7 +7,7 @@ pyspark.mllib.classification module .. automodule:: pyspark.mllib.classification :members: :undoc-members: - :show-inheritance: + :inherited-members: pyspark.mllib.clustering module ------------------------------- @@ -15,7 +15,6 @@ pyspark.mllib.clustering module .. automodule:: pyspark.mllib.clustering :members: :undoc-members: - :show-inheritance: pyspark.mllib.feature module ------------------------------- @@ -39,7 +38,6 @@ pyspark.mllib.random module .. automodule:: pyspark.mllib.random :members: :undoc-members: - :show-inheritance: pyspark.mllib.recommendation module ----------------------------------- @@ -47,7 +45,6 @@ pyspark.mllib.recommendation module .. automodule:: pyspark.mllib.recommendation :members: :undoc-members: - :show-inheritance: pyspark.mllib.regression module ------------------------------- @@ -55,7 +52,7 @@ pyspark.mllib.regression module .. automodule:: pyspark.mllib.regression :members: :undoc-members: - :show-inheritance: + :inherited-members: pyspark.mllib.stat module ------------------------- @@ -63,7 +60,6 @@ pyspark.mllib.stat module .. automodule:: pyspark.mllib.stat :members: :undoc-members: - :show-inheritance: pyspark.mllib.tree module ------------------------- @@ -71,7 +67,7 @@ pyspark.mllib.tree module .. automodule:: pyspark.mllib.tree :members: :undoc-members: - :show-inheritance: + :inherited-members: pyspark.mllib.util module ------------------------- @@ -79,4 +75,3 @@ pyspark.mllib.util module .. automodule:: pyspark.mllib.util :members: :undoc-members: - :show-inheritance: diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index b6de7493d7523..4ff7463498cce 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,10 +15,11 @@ # limitations under the License. # -from pyspark.ml.util import inherit_doc, keyword_only +from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam +from pyspark.mllib.common import inherit_doc __all__ = ['LogisticRegression', 'LogisticRegressionModel'] diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index f1ddbb478dd9c..433b4fb5d22bf 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -16,8 +16,9 @@ # from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures -from pyspark.ml.util import inherit_doc, keyword_only +from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaTransformer +from pyspark.mllib.common import inherit_doc __all__ = ['Tokenizer', 'HashingTF'] diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 18d8a58f357bd..5233c5801e2e6 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -18,7 +18,8 @@ from abc import ABCMeta, abstractmethod from pyspark.ml.param import Param, Params -from pyspark.ml.util import inherit_doc, keyword_only +from pyspark.ml.util import keyword_only +from pyspark.mllib.common import inherit_doc __all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 81d3f0882b8a9..6f7f39c40eb5a 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -19,20 +19,6 @@ import uuid -def inherit_doc(cls): - for name, func in vars(cls).items(): - # only inherit docstring for public functions - if name.startswith("_"): - continue - if not func.__doc__: - for parent in cls.__bases__: - parent_func = getattr(parent, name, None) - if parent_func and getattr(parent_func, "__doc__", None): - func.__doc__ = parent_func.__doc__ - break - return cls - - def keyword_only(func): """ A decorator that forces keyword arguments in the wrapped method diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 9e12ddc3d9b8f..4bae96f678388 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -21,7 +21,7 @@ from pyspark.sql import DataFrame from pyspark.ml.param import Params from pyspark.ml.pipeline import Estimator, Transformer -from pyspark.ml.util import inherit_doc +from pyspark.mllib.common import inherit_doc def _jvm(): diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index c3217620e3c4e..6449800d9c120 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -19,7 +19,7 @@ Python bindings for MLlib. """ -# MLlib currently needs and NumPy 1.4+, so complain if lower +# MLlib currently needs NumPy 1.4+, so complain if lower import numpy if numpy.version.version < '1.4': diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f6b97abb1723c..949db5705abd7 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -152,7 +152,7 @@ def predictSoft(self, x): class GaussianMixture(object): """ - Estimate model parameters with the expectation-maximization algorithm. + Learning algorithm for Gaussian Mixtures using the expectation-maximization algorithm. :param data: RDD of data points :param k: Number of components diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 3c5ee66cd8b64..621591c26b77f 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -134,3 +134,20 @@ def __del__(self): def call(self, name, *a): """Call method of java_model""" return callJavaFunc(self._sc, getattr(self._java_model, name), *a) + + +def inherit_doc(cls): + """ + A decorator that makes a class inherit documentation from its parents. + """ + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 210060140fd91..21751cc68f308 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,7 +18,7 @@ import numpy as np from numpy import array -from pyspark.mllib.common import callMLlibFunc +from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', @@ -69,6 +69,7 @@ def __repr__(self): return "(weights=%s, intercept=%r)" % (self._coeff, self._intercept) +@inherit_doc class LinearRegressionModelBase(LinearModel): """A linear regression model. @@ -89,6 +90,7 @@ def predict(self, x): return self.weights.dot(x) + self.intercept +@inherit_doc class LinearRegressionModel(LinearRegressionModelBase): """A linear regression model derived from a least-squares fit. @@ -162,7 +164,7 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features - are activated or not). + are activated or not). (default: False) """ def train(rdd, i): return callMLlibFunc("trainLinearRegressionModelWithSGD", rdd, int(iterations), @@ -172,6 +174,7 @@ def train(rdd, i): return _regression_train_wrapper(train, LinearRegressionModel, data, initialWeights) +@inherit_doc class LassoModel(LinearRegressionModelBase): """A linear regression model derived from a least-squares fit with an @@ -218,6 +221,7 @@ def train(rdd, i): return _regression_train_wrapper(train, LassoModel, data, initialWeights) +@inherit_doc class RidgeRegressionModel(LinearRegressionModelBase): """A linear regression model derived from a least-squares fit with an diff --git a/python/pyspark/mllib/stat/__init__.py b/python/pyspark/mllib/stat/__init__.py index b686d955a0080..e3e128513e0d7 100644 --- a/python/pyspark/mllib/stat/__init__.py +++ b/python/pyspark/mllib/stat/__init__.py @@ -21,5 +21,7 @@ from pyspark.mllib.stat._statistics import * from pyspark.mllib.stat.distribution import MultivariateGaussian +from pyspark.mllib.stat.test import ChiSqTestResult -__all__ = ["Statistics", "MultivariateStatisticalSummary", "MultivariateGaussian"] +__all__ = ["Statistics", "MultivariateStatisticalSummary", "ChiSqTestResult", + "MultivariateGaussian"] diff --git a/python/pyspark/mllib/stat/distribution.py b/python/pyspark/mllib/stat/distribution.py index 07792e1532046..46f7a1d2f277a 100644 --- a/python/pyspark/mllib/stat/distribution.py +++ b/python/pyspark/mllib/stat/distribution.py @@ -22,7 +22,8 @@ class MultivariateGaussian(namedtuple('MultivariateGaussian', ['mu', 'sigma'])): - """ Represents a (mu, sigma) tuple + """Represents a (mu, sigma) tuple + >>> m = MultivariateGaussian(Vectors.dense([11,12]),DenseMatrix(2, 2, (1.0, 3.0, 5.0, 2.0))) >>> (m.mu, m.sigma.toArray()) (DenseVector([11.0, 12.0]), array([[ 1., 5.],[ 3., 2.]])) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index aae48f213246b..02d551b87dcc0 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -20,12 +20,12 @@ import random from pyspark import SparkContext, RDD -from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.common import callMLlibFunc, inherit_doc, JavaModelWrapper from pyspark.mllib.linalg import _convert_to_vector from pyspark.mllib.regression import LabeledPoint __all__ = ['DecisionTreeModel', 'DecisionTree', 'RandomForestModel', - 'RandomForest', 'GradientBoostedTrees'] + 'RandomForest', 'GradientBoostedTreesModel', 'GradientBoostedTrees'] class TreeEnsembleModel(JavaModelWrapper): @@ -216,6 +216,7 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) +@inherit_doc class RandomForestModel(TreeEnsembleModel): """ .. note:: Experimental @@ -381,6 +382,7 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt featureSubsetStrategy, impurity, maxDepth, maxBins, seed) +@inherit_doc class GradientBoostedTreesModel(TreeEnsembleModel): """ .. note:: Experimental From 5b0a42cb17b840c82d3f8a5ad061d99e261ceadf Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 20 Feb 2015 15:35:05 -0800 Subject: [PATCH 210/817] [SPARK-5898] [SPARK-5896] [SQL] [PySpark] create DataFrame from pandas and tuple/list Fix createDataFrame() from pandas DataFrame (not tested by jenkins, depends on SPARK-5693). It also support to create DataFrame from plain tuple/list without column names, `_1`, `_2` will be used as column names. Author: Davies Liu Closes #4679 from davies/pandas and squashes the following commits: c0cbe0b [Davies Liu] fix tests 8466d1d [Davies Liu] fix create DataFrame from pandas --- python/pyspark/sql/context.py | 12 ++++++++++-- python/pyspark/sql/tests.py | 2 +- python/pyspark/sql/types.py | 26 +++++++++----------------- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 3f168f718bd7f..313f15e6d9b6f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -351,6 +351,8 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): :return: a DataFrame >>> l = [('Alice', 1)] + >>> sqlCtx.createDataFrame(l).collect() + [Row(_1=u'Alice', _2=1)] >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect() [Row(name=u'Alice', age=1)] @@ -359,6 +361,8 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): [Row(age=1, name=u'Alice')] >>> rdd = sc.parallelize(l) + >>> sqlCtx.createDataFrame(rdd).collect() + [Row(_1=u'Alice', _2=1)] >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name=u'Alice', age=1)] @@ -377,14 +381,17 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): >>> df3 = sqlCtx.createDataFrame(rdd, schema) >>> df3.collect() [Row(name=u'Alice', age=1)] + + >>> sqlCtx.createDataFrame(df.toPandas()).collect() # doctest: +SKIP + [Row(name=u'Alice', age=1)] """ if isinstance(data, DataFrame): raise TypeError("data is already a DataFrame") if has_pandas and isinstance(data, pandas.DataFrame): - data = self._sc.parallelize(data.to_records(index=False)) if schema is None: schema = list(data.columns) + data = [r.tolist() for r in data.to_records(index=False)] if not isinstance(data, RDD): try: @@ -399,7 +406,8 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): if isinstance(schema, (list, tuple)): first = data.first() if not isinstance(first, (list, tuple)): - raise ValueError("each row in `rdd` should be list or tuple") + raise ValueError("each row in `rdd` should be list or tuple, " + "but got %r" % type(first)) row_cls = Row(*schema) schema = self._inferSchema(data.map(lambda r: row_cls(*r)), samplingRatio) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 8e1bb36598727..39071e7e35ca1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -186,7 +186,7 @@ def test_serialize_nested_array_and_map(self): self.assertEqual("2", row.d) def test_infer_schema(self): - d = [Row(l=[], d={}), + d = [Row(l=[], d={}, s=None), Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")] rdd = self.sc.parallelize(d) df = self.sqlCtx.createDataFrame(rdd) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 9409c6f9f6556..b6e41cf0b29ff 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -604,7 +604,7 @@ def _infer_type(obj): ExamplePointUDT """ if obj is None: - raise ValueError("Can not infer type for None") + return NullType() if hasattr(obj, '__UDT__'): return obj.__UDT__ @@ -637,15 +637,14 @@ def _infer_schema(row): if isinstance(row, dict): items = sorted(row.items()) - elif isinstance(row, tuple): + elif isinstance(row, (tuple, list)): if hasattr(row, "_fields"): # namedtuple items = zip(row._fields, tuple(row)) elif hasattr(row, "__FIELDS__"): # Row items = zip(row.__FIELDS__, tuple(row)) - elif all(isinstance(x, tuple) and len(x) == 2 for x in row): - items = row else: - raise ValueError("Can't infer schema from tuple") + names = ['_%d' % i for i in range(1, len(row) + 1)] + items = zip(names, row) elif hasattr(row, "__dict__"): # object items = sorted(row.__dict__.items()) @@ -812,17 +811,10 @@ def convert_struct(obj): if obj is None: return - if isinstance(obj, tuple): - if hasattr(obj, "_fields"): - d = dict(zip(obj._fields, obj)) - elif hasattr(obj, "__FIELDS__"): - d = dict(zip(obj.__FIELDS__, obj)) - elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): - d = dict(obj) - else: - raise ValueError("unexpected tuple: %s" % str(obj)) + if isinstance(obj, (tuple, list)): + return tuple(conv(v) for v, conv in zip(obj, converters)) - elif isinstance(obj, dict): + if isinstance(obj, dict): d = obj elif hasattr(obj, "__dict__"): # object d = obj.__dict__ @@ -1022,7 +1014,7 @@ def _verify_type(obj, dataType): return _type = type(dataType) - assert _type in _acceptable_types, "unkown datatype: %s" % dataType + assert _type in _acceptable_types, "unknown datatype: %s" % dataType # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: @@ -1040,7 +1032,7 @@ def _verify_type(obj, dataType): elif isinstance(dataType, StructType): if len(obj) != len(dataType.fields): - raise ValueError("Length of object (%d) does not match with" + raise ValueError("Length of object (%d) does not match with " "length of fields (%d)" % (len(obj), len(dataType.fields))) for v, f in zip(obj, dataType.fields): _verify_type(v, f.dataType) From e155324711740da97698b93526128b0eae2dc0ce Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 21 Feb 2015 13:00:16 +0000 Subject: [PATCH 211/817] [MLlib] fix typo fix typo: it should be "default:" instead of "default;" Author: Jacky Li Closes #4713 from jackylk/patch-10 and squashes the following commits: 15daf2e [Jacky Li] [MLlib] fix typo --- .../scala/org/apache/spark/examples/mllib/DenseKMeans.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala index 11e35598baf50..14cc5cbb679c5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -56,7 +56,7 @@ object DenseKMeans { .text(s"number of clusters, required") .action((x, c) => c.copy(k = x)) opt[Int]("numIterations") - .text(s"number of iterations, default; ${defaultParams.numIterations}") + .text(s"number of iterations, default: ${defaultParams.numIterations}") .action((x, c) => c.copy(numIterations = x)) opt[String]("initMode") .text(s"initialization mode (${InitializationMode.values.mkString(",")}), " + From d3cbd38c33e6a2addcf8caa18eeb10036fbfd01b Mon Sep 17 00:00:00 2001 From: Nishkam Ravi Date: Sat, 21 Feb 2015 09:59:28 -0800 Subject: [PATCH 212/817] SPARK-5841 [CORE] [HOTFIX 2] Memory leak in DiskBlockManager Continue to see IllegalStateException in YARN cluster mode. Adding a simple workaround for now. Author: Nishkam Ravi Author: nishkamravi2 Author: nravi Closes #4690 from nishkamravi2/master_nravi and squashes the following commits: d453197 [nishkamravi2] Update NewHadoopRDD.scala 6f41a1d [nishkamravi2] Update NewHadoopRDD.scala 0ce2c32 [nishkamravi2] Update HadoopRDD.scala f7e33c2 [Nishkam Ravi] Merge branch 'master_nravi' of https://github.com/nishkamravi2/spark into master_nravi ba1eb8b [Nishkam Ravi] Try-catch block around the two occurrences of removeShutDownHook. Deletion of semi-redundant occurrences of expensive operation inShutDown. 71d0e17 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 494d8c0 [nishkamravi2] Update DiskBlockManager.scala 3c5ddba [nishkamravi2] Update DiskBlockManager.scala f0d12de [Nishkam Ravi] Workaround for IllegalStateException caused by recent changes to BlockManager.stop 79ea8b4 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi b446edc [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 5c9a4cb [nishkamravi2] Update TaskSetManagerSuite.scala 535295a [nishkamravi2] Update TaskSetManager.scala 3e1b616 [Nishkam Ravi] Modify test for maxResultSize 9f6583e [Nishkam Ravi] Changes to maxResultSize code (improve error message and add condition to check if maxResultSize > 0) 5f8f9ed [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 636a9ff [nishkamravi2] Update YarnAllocator.scala 8f76c8b [Nishkam Ravi] Doc change for yarn memory overhead 35daa64 [Nishkam Ravi] Slight change in the doc for yarn memory overhead 5ac2ec1 [Nishkam Ravi] Remove out dac1047 [Nishkam Ravi] Additional documentation for yarn memory overhead issue 42c2c3d [Nishkam Ravi] Additional changes for yarn memory overhead issue 362da5e [Nishkam Ravi] Additional changes for yarn memory overhead c726bd9 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi f00fa31 [Nishkam Ravi] Improving logging for AM memoryOverhead 1cf2d1e [nishkamravi2] Update YarnAllocator.scala ebcde10 [Nishkam Ravi] Modify default YARN memory_overhead-- from an additive constant to a multiplier (redone to resolve merge conflicts) 2e69f11 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi efd688a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark 2b630f9 [nravi] Accept memory input as "30g", "512M" instead of an int value, to be consistent with rest of Spark 3bf8fad [nravi] Merge branch 'master' of https://github.com/apache/spark 5423a03 [nravi] Merge branch 'master' of https://github.com/apache/spark eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456) 6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed) 5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456) 681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles --- .../org/apache/spark/deploy/worker/ExecutorRunner.scala | 6 +++++- .../scala/org/apache/spark/storage/DiskBlockManager.scala | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 0add3064da452..bea04cd542fd1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -105,7 +105,11 @@ private[spark] class ExecutorRunner( workerThread.interrupt() workerThread = null state = ExecutorState.KILLED - Runtime.getRuntime.removeShutdownHook(shutdownHook) + try { + Runtime.getRuntime.removeShutdownHook(shutdownHook) + } catch { + case e: IllegalStateException => None + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index b297f3fd9dd1e..12cd8ea3bdf1f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -148,7 +148,11 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { // Remove the shutdown hook. It causes memory leaks if we leave it around. - Runtime.getRuntime.removeShutdownHook(shutdownHook) + try { + Runtime.getRuntime.removeShutdownHook(shutdownHook) + } catch { + case e: IllegalStateException => None + } doStop() } From 7138816abe1060a1e967c4c77c72d5752586d557 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 21 Feb 2015 10:01:01 -0800 Subject: [PATCH 213/817] [SPARK-5937][YARN] Fix ClientSuite to set YARN mode, so that the correct class is used in t... ...ests. Without this SparkHadoopUtil is used by the Client instead of YarnSparkHadoopUtil. Author: Hari Shreedharan Closes #4711 from harishreedharan/SPARK-5937 and squashes the following commits: d154de6 [Hari Shreedharan] Use System.clearProperty() instead of setting the value of SPARK_YARN_MODE to empty string. f729f70 [Hari Shreedharan] Fix ClientSuite to set YARN mode, so that the correct class is used in tests. --- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index f8f8129d220e4..92f04b4b859b3 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -28,8 +28,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } @@ -39,7 +38,15 @@ import scala.util.Try import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils -class ClientSuite extends FunSuite with Matchers { +class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + System.setProperty("SPARK_YARN_MODE", "true") + } + + override def afterAll(): Unit = { + System.clearProperty("SPARK_YARN_MODE") + } test("default Yarn application classpath") { Client.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) From 7683982faf920b8ac6cf46b79842450e7d46c5cc Mon Sep 17 00:00:00 2001 From: Evan Yu Date: Sat, 21 Feb 2015 20:40:21 +0000 Subject: [PATCH 214/817] [SPARK-5860][CORE] JdbcRDD: overflow on large range with high number of partitions Fix a overflow bug in JdbcRDD when calculating partitions for large BIGINT ids Author: Evan Yu Closes #4701 from hotou/SPARK-5860 and squashes the following commits: 9e038d1 [Evan Yu] [SPARK-5860][CORE] Prevent overflowing at the length level 7883ad9 [Evan Yu] [SPARK-5860][CORE] Prevent overflowing at the length level c88755a [Evan Yu] [SPARK-5860][CORE] switch to BigInt instead of BigDecimal 4e9ff4f [Evan Yu] [SPARK-5860][CORE] JdbcRDD overflow on large range with high number of partitions --- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 8 +-- .../org/apache/spark/rdd/JdbcRDDSuite.scala | 60 ++++++++++++++----- 2 files changed, 50 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 4fe7622bda00f..e2267861e79df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -62,11 +62,11 @@ class JdbcRDD[T: ClassTag]( override def getPartitions: Array[Partition] = { // bounds are inclusive, hence the + 1 here and - 1 on end - val length = 1 + upperBound - lowerBound + val length = BigInt(1) + upperBound - lowerBound (0 until numPartitions).map(i => { - val start = lowerBound + ((i * length) / numPartitions).toLong - val end = lowerBound + (((i + 1) * length) / numPartitions).toLong - 1 - new JdbcPartition(i, start, end) + val start = lowerBound + ((i * length) / numPartitions) + val end = lowerBound + (((i + 1) * length) / numPartitions) - 1 + new JdbcPartition(i, start.toLong, end.toLong) }).toArray } diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 6138d0bbd57f6..0dc59888f7304 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -29,22 +29,42 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { Class.forName("org.apache.derby.jdbc.EmbeddedDriver") val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") try { - val create = conn.createStatement - create.execute(""" - CREATE TABLE FOO( - ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), - DATA INTEGER - )""") - create.close() - val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") - (1 to 100).foreach { i => - insert.setInt(1, i * 2) - insert.executeUpdate + + try { + val create = conn.createStatement + create.execute(""" + CREATE TABLE FOO( + ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), + DATA INTEGER + )""") + create.close() + val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") + (1 to 100).foreach { i => + insert.setInt(1, i * 2) + insert.executeUpdate + } + insert.close() + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => + // table exists } - insert.close() - } catch { - case e: SQLException if e.getSQLState == "X0Y32" => + + try { + val create = conn.createStatement + create.execute("CREATE TABLE BIGINT_TEST(ID BIGINT NOT NULL, DATA INTEGER)") + create.close() + val insert = conn.prepareStatement("INSERT INTO BIGINT_TEST VALUES(?,?)") + (1 to 100).foreach { i => + insert.setLong(1, 100000000000000000L + 4000000000000000L * i) + insert.setInt(2, i) + insert.executeUpdate + } + insert.close() + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => // table exists + } + } finally { conn.close() } @@ -62,6 +82,18 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { assert(rdd.count === 100) assert(rdd.reduce(_+_) === 10100) } + + test("large id overflow") { + sc = new SparkContext("local", "test") + val rdd = new JdbcRDD( + sc, + () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, + "SELECT DATA FROM BIGINT_TEST WHERE ? <= ID AND ID <= ?", + 1131544775L, 567279358897692673L, 20, + (r: ResultSet) => { r.getInt(1) } ).cache() + assert(rdd.count === 100) + assert(rdd.reduce(_+_) === 5050) + } after { try { From 46462ff255b0eef8263ed798f3d5aeb8460ecaf1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 21 Feb 2015 23:07:30 -0800 Subject: [PATCH 215/817] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3490 (close requested by 'andrewor14') Closes #4646 (close requested by 'srowen') Closes #3591 (close requested by 'andrewor14') Closes #3656 (close requested by 'andrewor14') Closes #4553 (close requested by 'JoshRosen') Closes #4202 (close requested by 'srowen') Closes #4497 (close requested by 'marmbrus') Closes #4150 (close requested by 'andrewor14') Closes #2409 (close requested by 'andrewor14') Closes #4221 (close requested by 'srowen') From a7f90390251ff62a0e10edf4c2eb876538597791 Mon Sep 17 00:00:00 2001 From: Alexander Date: Sun, 22 Feb 2015 08:53:05 +0000 Subject: [PATCH 216/817] =?UTF-8?q?[DOCS]=20Fix=20typo=20in=20API=20for=20?= =?UTF-8?q?custom=20InputFormats=20based=20on=20the=20=E2=80=9Cnew?= =?UTF-8?q?=E2=80=9D=20MapReduce=20API?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This looks like a simple typo ```SparkContext.newHadoopRDD``` instead of ```SparkContext.newAPIHadoopRDD``` as in actual http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.SparkContext Author: Alexander Closes #4718 from bzz/hadoop-InputFormats-doc-fix and squashes the following commits: 680a4c4 [Alexander] Fix typo in docs on custom Hadoop InputFormats --- docs/programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 4e4af76316863..7b0701828878e 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -335,7 +335,7 @@ Apart from text files, Spark's Scala API also supports several other data format * For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts. -* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). +* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). * `RDD.saveAsObjectFile` and `SparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. @@ -367,7 +367,7 @@ Apart from text files, Spark's Java API also supports several other data formats * For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). -* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). +* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). * `JavaRDD.saveAsObjectFile` and `JavaSparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. From 275b1bef897d775f1f7743378ca3e09e36160136 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sun, 22 Feb 2015 08:56:30 +0000 Subject: [PATCH 217/817] [DataFrame] [Typo] Fix the typo Author: Cheng Hao Closes #4717 from chenghao-intel/typo1 and squashes the following commits: 858d7b0 [Cheng Hao] update the typo --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 810f7c77477bb..69e5f6a07da7f 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 @@ -138,7 +138,7 @@ class DataFrame protected[sql]( /** * An implicit conversion function internal to this class for us to avoid doing - * "new DataFrameImpl(...)" everywhere. + * "new DataFrame(...)" everywhere. */ @inline private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { new DataFrame(sqlContext, logicalPlan) From e4f9d03d728bc6fbfb6ebc7d15b4ba328f98f3dc Mon Sep 17 00:00:00 2001 From: Aaron Josephs Date: Sun, 22 Feb 2015 22:09:06 -0800 Subject: [PATCH 218/817] [SPARK-911] allow efficient queries for a range if RDD is partitioned wi... ...th RangePartitioner Author: Aaron Josephs Closes #1381 from aaronjosephs/PLAT-911 and squashes the following commits: e30ade5 [Aaron Josephs] [SPARK-911] allow efficient queries for a range if RDD is partitioned with RangePartitioner --- .../spark/rdd/OrderedRDDFunctions.scala | 23 +++++++++++++++ .../org/apache/spark/rdd/SortingSuite.scala | 28 +++++++++++++++++++ 2 files changed, 51 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 144f679a59460..6fdfdb734d1b8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -75,4 +75,27 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering) } + /** + * Returns an RDD containing only the elements in the the inclusive range `lower` to `upper`. + * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be + * performed efficiently by only scanning the partitions that might contain matching elements. + * Otherwise, a standard `filter` is applied to all partitions. + */ + def filterByRange(lower: K, upper: K): RDD[P] = { + + def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper) + + val rddToFilter: RDD[P] = self.partitioner match { + case Some(rp: RangePartitioner[K, V]) => { + val partitionIndicies = (rp.getPartition(lower), rp.getPartition(upper)) match { + case (l, u) => Math.min(l, u) to Math.max(l, u) + } + PartitionPruningRDD.create(self, partitionIndicies.contains) + } + case _ => + self + } + rddToFilter.filter { case (k, v) => inRange(k) } + } + } diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index a40f2ffeffdf9..64b1c24c47168 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -119,5 +119,33 @@ class SortingSuite extends FunSuite with SharedSparkContext with Matchers with L partitions(1).last should be > partitions(2).head partitions(2).last should be > partitions(3).head } + + test("get a range of elements in a sorted RDD that is on one partition") { + val pairArr = (1 to 1000).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey() + val range = sorted.filterByRange(20, 40).collect() + assert((20 to 40).toArray === range.map(_._1)) + } + + test("get a range of elements over multiple partitions in a descendingly sorted RDD") { + val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey(false) + val range = sorted.filterByRange(200, 800).collect() + assert((800 to 200 by -1).toArray === range.map(_._1)) + } + + test("get a range of elements in an array not partitioned by a range partitioner") { + val pairArr = util.Random.shuffle((1 to 1000).toList).map(x => (x, x)) + val pairs = sc.parallelize(pairArr,10) + val range = pairs.filterByRange(200, 800).collect() + assert((800 to 200 by -1).toArray.sorted === range.map(_._1).sorted) + } + + test("get a range of elements over multiple partitions but not taking up full partitions") { + val pairArr = (1000 to 1 by -1).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 10).sortByKey(false) + val range = sorted.filterByRange(250, 850).collect() + assert((850 to 250 by -1).toArray === range.map(_._1)) + } } From 95cd643aa954b7e4229e94fa8bdc99bf3b2bb1da Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Sun, 22 Feb 2015 22:43:04 -0800 Subject: [PATCH 219/817] [SPARK-3885] Provide mechanism to remove accumulators once they are no longer used Instead of storing a strong reference to accumulators, I've replaced this with a weak reference and updated any code that uses these accumulators to check whether the reference resolves before using the accumulator. A weak reference will be cleared when there is no longer an existing copy of the variable versus using a soft reference in which case accumulators would only be cleared when the GC explicitly ran out of memory. Author: Ilya Ganelin Closes #4021 from ilganeli/SPARK-3885 and squashes the following commits: 4ba9575 [Ilya Ganelin] Fixed error in test suite 8510943 [Ilya Ganelin] Extra code bb76ef0 [Ilya Ganelin] File deleted somehow 283a333 [Ilya Ganelin] Added cleanup method for accumulators to remove stale references within Accumulators.original to accumulators that are now out of scope 345fd4f [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885 7485a82 [Ilya Ganelin] Fixed build error c8e0f2b [Ilya Ganelin] Added working test for accumulator garbage collection 94ce754 [Ilya Ganelin] Still not being properly garbage collected 8722b63 [Ilya Ganelin] Fixing gc test 7414a9c [Ilya Ganelin] Added test for accumulator garbage collection 18d62ec [Ilya Ganelin] Updated to throw Exception when accessing a GCd accumulator 9a81928 [Ilya Ganelin] Reverting permissions changes 28f705c [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885 b820ab4b [Ilya Ganelin] reset d78f4bf [Ilya Ganelin] Removed obsolete comment 0746e61 [Ilya Ganelin] Updated DAGSchedulerSUite to fix bug 3350852 [Ilya Ganelin] Updated DAGScheduler and Suite to correctly use new implementation of WeakRef Accumulator storage c49066a [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885 cbb9023 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-3885 a77d11b [Ilya Ganelin] Updated Accumulators class to store weak references instead of strong references to allow garbage collection of old accumulators --- .../scala/org/apache/spark/Accumulators.scala | 36 ++++++++++++++----- .../org/apache/spark/ContextCleaner.scala | 20 +++++++++++ .../scala/org/apache/spark/SparkContext.scala | 28 +++++++++++---- .../apache/spark/scheduler/DAGScheduler.scala | 10 +++++- .../org/apache/spark/AccumulatorSuite.scala | 20 +++++++++++ .../apache/spark/ContextCleanerSuite.scala | 4 +++ .../spark/scheduler/DAGSchedulerSuite.scala | 6 +++- 7 files changed, 107 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5f31bfba3f8d6..30f0ccd73ccca 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -23,6 +23,7 @@ import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map +import scala.ref.WeakReference import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer @@ -280,10 +281,12 @@ object AccumulatorParam { // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right private[spark] object Accumulators { - // TODO: Use soft references? => need to make readObject work properly then - val originals = Map[Long, Accumulable[_, _]]() - val localAccums = new ThreadLocal[Map[Long, Accumulable[_, _]]]() { - override protected def initialValue() = Map[Long, Accumulable[_, _]]() + // Store a WeakReference instead of a StrongReference because this way accumulators can be + // appropriately garbage collected during long-running jobs and release memory + type WeakAcc = WeakReference[Accumulable[_, _]] + val originals = Map[Long, WeakAcc]() + val localAccums = new ThreadLocal[Map[Long, WeakAcc]]() { + override protected def initialValue() = Map[Long, WeakAcc]() } var lastId: Long = 0 @@ -294,9 +297,9 @@ private[spark] object Accumulators { def register(a: Accumulable[_, _], original: Boolean): Unit = synchronized { if (original) { - originals(a.id) = a + originals(a.id) = new WeakAcc(a) } else { - localAccums.get()(a.id) = a + localAccums.get()(a.id) = new WeakAcc(a) } } @@ -307,11 +310,22 @@ private[spark] object Accumulators { } } + def remove(accId: Long) { + synchronized { + originals.remove(accId) + } + } + // Get the values of the local accumulators for the current thread (by ID) def values: Map[Long, Any] = synchronized { val ret = Map[Long, Any]() for ((id, accum) <- localAccums.get) { - ret(id) = accum.localValue + // Since we are now storing weak references, we must check whether the underlying data + // is valid. + ret(id) = accum.get match { + case Some(values) => values.localValue + case None => None + } } return ret } @@ -320,7 +334,13 @@ private[spark] object Accumulators { def add(values: Map[Long, Any]): Unit = synchronized { for ((id, value) <- values) { if (originals.contains(id)) { - originals(id).asInstanceOf[Accumulable[Any, Any]] ++= value + // Since we are now storing weak references, we must check whether the underlying data + // is valid. + originals(id).get match { + case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] ++= value + case None => + throw new IllegalAccessError("Attempted to access garbage collected Accumulator.") + } } } } diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index ede1e23f4fcc5..434f1e47cf822 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -32,6 +32,7 @@ private sealed trait CleanupTask private case class CleanRDD(rddId: Int) extends CleanupTask private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask +private case class CleanAccum(accId: Long) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -114,6 +115,10 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { registerForCleanup(rdd, CleanRDD(rdd.id)) } + def registerAccumulatorForCleanup(a: Accumulable[_, _]): Unit = { + registerForCleanup(a, CleanAccum(a.id)) + } + /** Register a ShuffleDependency for cleanup when it is garbage collected. */ def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) @@ -145,6 +150,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) + case CleanAccum(accId) => + doCleanupAccum(accId, blocking = blockOnCleanupTasks) } } } catch { @@ -190,6 +197,18 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } + /** Perform accumulator cleanup. */ + def doCleanupAccum(accId: Long, blocking: Boolean) { + try { + logDebug("Cleaning accumulator " + accId) + Accumulators.remove(accId) + listeners.foreach(_.accumCleaned(accId)) + logInfo("Cleaned accumulator " + accId) + } catch { + case e: Exception => logError("Error cleaning accumulator " + accId, e) + } + } + private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] @@ -206,4 +225,5 @@ private[spark] trait CleanerListener { def rddCleaned(rddId: Int) def shuffleCleaned(shuffleId: Int) def broadcastCleaned(broadcastId: Long) + def accumCleaned(accId: Long) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 85ec5ea11357e..930d4bea4785b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -986,7 +986,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = - new Accumulator(initialValue, param) + { + val acc = new Accumulator(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc)) + acc + } /** * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display @@ -994,7 +998,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * driver can access the accumulator's `value`. */ def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = { - new Accumulator(initialValue, param, Some(name)) + val acc = new Accumulator(initialValue, param, Some(name)) + cleaner.foreach(_.registerAccumulatorForCleanup(acc)) + acc } /** @@ -1003,8 +1009,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = - new Accumulable(initialValue, param) + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = { + val acc = new Accumulable(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc)) + acc + } /** * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the @@ -1013,8 +1022,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = - new Accumulable(initialValue, param, Some(name)) + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = { + val acc = new Accumulable(initialValue, param, Some(name)) + cleaner.foreach(_.registerAccumulatorForCleanup(acc)) + acc + } /** * Create an accumulator from a "mutable collection" type. @@ -1025,7 +1037,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] - new Accumulable(initialValue, param) + val acc = new Accumulable(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc)) + acc } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c58721c2c82b7..bc84e2351ad74 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -890,8 +890,16 @@ class DAGScheduler( if (event.accumUpdates != null) { try { Accumulators.add(event.accumUpdates) + event.accumUpdates.foreach { case (id, partialValue) => - val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] + // In this instance, although the reference in Accumulators.originals is a WeakRef, + // it's guaranteed to exist since the event.accumUpdates Map exists + + val acc = Accumulators.originals(id).get match { + case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] + case None => throw new NullPointerException("Non-existent reference to Accumulator") + } + // To avoid UI cruft, ignore cases where value wasn't updated if (acc.name.isDefined && partialValue != acc.zero) { val name = acc.name.get diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index f087fc550dde3..bd0f8bdefa171 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import scala.collection.mutable +import scala.ref.WeakReference import org.scalatest.FunSuite import org.scalatest.Matchers @@ -136,4 +137,23 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { } } + test ("garbage collection") { + // Create an accumulator and let it go out of scope to test that it's properly garbage collected + sc = new SparkContext("local", "test") + var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val accId = acc.id + val ref = WeakReference(acc) + + // Ensure the accumulator is present + assert(ref.get.isDefined) + + // Remove the explicit reference to it and allow weak reference to get garbage collected + acc = null + System.gc() + assert(ref.get.isEmpty) + + Accumulators.remove(accId) + assert(!Accumulators.originals.get(accId).isDefined) + } + } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index ae2ae7ed0d3aa..cdfaacee7da40 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -382,6 +382,10 @@ class CleanerTester( toBeCleanedBroadcstIds -= broadcastId logInfo("Broadcast" + broadcastId + " cleaned") } + + def accumCleaned(accId: Long): Unit = { + logInfo("Cleaned accId " + accId + " cleaned") + } } val MAX_VALIDATION_ATTEMPTS = 10 diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9d0c1273695f6..4bf7f9e647d55 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -735,7 +735,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assert(Accumulators.originals(accum.id).value === 1) + + val accVal = Accumulators.originals(accum.id).get.get.value + + assert(accVal === 1) + assertDataStructuresEmpty } From 934876741683fc254fed18e7ff630614f78944be Mon Sep 17 00:00:00 2001 From: Makoto Fukuhara Date: Mon, 23 Feb 2015 09:24:33 +0000 Subject: [PATCH 220/817] [EXAMPLES] fix typo. Author: Makoto Fukuhara Closes #4724 from fukuo33/fix-typo and squashes the following commits: 8c806b9 [Makoto Fukuhara] fix typo. --- .../main/scala/org/apache/spark/examples/BroadcastTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 1b53f3edbe92e..4c129dbe2d12d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -29,7 +29,7 @@ object BroadcastTest { val blockSize = if (args.length > 3) args(3) else "4096" val sparkConf = new SparkConf().setAppName("Broadcast Test") - .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory") + .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroadcastFactory") .set("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(sparkConf) From 757b14b862a1d39c1bad7b321dae1a3ea8338fbb Mon Sep 17 00:00:00 2001 From: Saisai Shao Date: Mon, 23 Feb 2015 11:27:27 +0000 Subject: [PATCH 221/817] [SPARK-5943][Streaming] Update the test to use new API to reduce the warning Author: Saisai Shao Closes #4722 from jerryshao/SPARK-5943 and squashes the following commits: 1b01233 [Saisai Shao] Update the test to use new API to reduce the warning --- python/pyspark/streaming/context.py | 2 +- .../apache/spark/streaming/StreamingContextSuite.scala | 10 +++++----- .../org/apache/spark/streaming/TestSuiteBase.scala | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index b06ab650370bd..2c73083c9f9a8 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -189,7 +189,7 @@ def awaitTermination(self, timeout=None): if timeout is None: self._jssc.awaitTermination() else: - self._jssc.awaitTermination(int(timeout * 1000)) + self._jssc.awaitTerminationOrTimeout(int(timeout * 1000)) def awaitTerminationOrTimeout(self, timeout): """ 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 2aa5e0876b6e0..6a7cd97aa3222 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -190,7 +190,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w logInfo("Count = " + count + ", Running count = " + runningCount) } ssc.start() - ssc.awaitTermination(500) + ssc.awaitTerminationOrTimeout(500) ssc.stop(stopSparkContext = false, stopGracefully = true) logInfo("Running count = " + runningCount) logInfo("TestReceiver.counter = " + TestReceiver.counter.get()) @@ -223,7 +223,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w logInfo("Count = " + count + ", Running count = " + runningCount) } ssc.start() - ssc.awaitTermination(500) + ssc.awaitTerminationOrTimeout(500) ssc.stop(stopSparkContext = false, stopGracefully = true) logInfo("Running count = " + runningCount) assert(runningCount > 0) @@ -243,7 +243,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w // test whether awaitTermination() exits after give amount of time failAfter(1000 millis) { - ssc.awaitTermination(500) + ssc.awaitTerminationOrTimeout(500) } // test whether awaitTermination() does not exit if not time is given @@ -288,7 +288,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val exception = intercept[Exception] { ssc.start() - ssc.awaitTermination(5000) + ssc.awaitTerminationOrTimeout(5000) } assert(exception.getMessage.contains("map task"), "Expected exception not thrown") } @@ -299,7 +299,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w inputStream.transform { rdd => throw new TestException("error in transform"); rdd }.register() val exception = intercept[TestException] { ssc.start() - ssc.awaitTermination(5000) + ssc.awaitTerminationOrTimeout(5000) } assert(exception.getMessage.contains("transform"), "Expected exception not thrown") } 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 c2375ff65edac..3565d621e8a6c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -348,7 +348,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { val startTime = System.currentTimeMillis() while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput) - ssc.awaitTermination(50) + ssc.awaitTerminationOrTimeout(50) } val timeTaken = System.currentTimeMillis() - startTime logInfo("Output generated in " + timeTaken + " milliseconds") From 242d49584c6aa21d928db2552033661950f760a5 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 23 Feb 2015 11:29:25 +0000 Subject: [PATCH 222/817] [SPARK-5724] fix the misconfiguration in AkkaUtils https://issues.apache.org/jira/browse/SPARK-5724 In AkkaUtil, we set several failure detector related the parameters as following ``` al akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]) .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize |akka.log-config-on-start = $logAkkaConfig |akka.remote.log-remote-lifecycle-events = $lifecycleEvents |akka.log-dead-letters = $lifecycleEvents |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) ``` Actually, we do not have any parameter naming "akka.remote.transport-failure-detector.threshold" see: http://doc.akka.io/docs/akka/2.3.4/general/configuration.html what we have is "akka.remote.watch-failure-detector.threshold" Author: CodingCat Closes #4512 from CodingCat/SPARK-5724 and squashes the following commits: bafe56e [CodingCat] fix the grammar in configuration doc 338296e [CodingCat] remove failure-detector related info 8bfcfd4 [CodingCat] fix the misconfiguration in AkkaUtils --- .../org/apache/spark/util/AkkaUtils.scala | 3 -- docs/configuration.md | 36 +++++++------------ 2 files changed, 12 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 3d9c6192ff7f7..48a6ede05e17b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -79,8 +79,6 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaFailureDetector = - conf.getDouble("spark.akka.failure-detector.threshold", 300.0) val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) val secretKey = securityManager.getSecretKey() @@ -106,7 +104,6 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.secure-cookie = "$secureCookie" |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s - |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" diff --git a/docs/configuration.md b/docs/configuration.md index 541695c83ae36..c8db338cb6f89 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -903,36 +903,24 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses 6000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause - in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in - combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` - if you need to. - - - - spark.akka.failure-detector.threshold - 300.0 - - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). This maps to akka's - `akka.remote.transport-failure-detector.threshold`. Tune this in combination of - `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to. + This is set to a larger value to disable the transport failure detector that comes built in to Akka. + It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart + beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + this along with `spark.akka.heartbeat.interval` if you need to. spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). A larger interval value in - seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for - akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and - `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using - failure detector can be, a sensistive failure detector can help evict rogue executors really - quick. However this is usually not the case as gc pauses and network lags are expected in a - real Spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats - between nodes leading to flooding the network with those. + This is set to a larger value to disable the transport failure detector that comes built in to Akka. + It can be enabled again, if you plan to use this feature (Not recommended). A larger interval + value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative + for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need + to. A likely positive use case for using failure detector would be: a sensistive failure detector + can help evict rogue executors quickly. However this is usually not the case as GC pauses + and network lags are expected in a real Spark cluster. Apart from that enabling this leads to + a lot of exchanges of heart beats between nodes leading to flooding the network with those. From 651a1c019eb911005e234a46cc559d63da352377 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 23 Feb 2015 08:47:28 -0800 Subject: [PATCH 223/817] [SPARK-5939][MLLib] make FPGrowth example app take parameters Add parameter parsing in FPGrowth example app in Scala and Java And a sample data file is added in data/mllib folder Author: Jacky Li Closes #4714 from jackylk/parameter and squashes the following commits: 8c478b3 [Jacky Li] fix according to comments 3bb74f6 [Jacky Li] make FPGrowth exampl app take parameters f0e4d10 [Jacky Li] make FPGrowth exampl app take parameters --- data/mllib/sample_fpgrowth.txt | 6 ++ .../examples/mllib/JavaFPGrowthExample.java | 43 ++++++++++---- .../examples/mllib/FPGrowthExample.scala | 59 ++++++++++++++----- 3 files changed, 81 insertions(+), 27 deletions(-) create mode 100644 data/mllib/sample_fpgrowth.txt diff --git a/data/mllib/sample_fpgrowth.txt b/data/mllib/sample_fpgrowth.txt new file mode 100644 index 0000000000000..c451583e51317 --- /dev/null +++ b/data/mllib/sample_fpgrowth.txt @@ -0,0 +1,6 @@ +r z h k p +z y x w v u t s +s x o n r +x z y m t s q e +z +x z y r q t p diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java index f50e802cf683c..36baf5868736c 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java @@ -25,32 +25,49 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.fpm.FPGrowth; import org.apache.spark.mllib.fpm.FPGrowthModel; /** * Java example for mining frequent itemsets using FP-growth. + * Example usage: ./bin/run-example mllib.JavaFPGrowthExample ./data/mllib/sample_fpgrowth.txt */ public class JavaFPGrowthExample { public static void main(String[] args) { + String inputFile; + double minSupport = 0.3; + int numPartition = -1; + if (args.length < 1) { + System.err.println( + "Usage: JavaFPGrowth [minSupport] [numPartition]"); + System.exit(1); + } + inputFile = args[0]; + if (args.length >= 2) { + minSupport = Double.parseDouble(args[1]); + } + if (args.length >= 3) { + numPartition = Integer.parseInt(args[2]); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample"); JavaSparkContext sc = new JavaSparkContext(sparkConf); + JavaRDD> transactions = sc.textFile(inputFile).map( + new Function>() { + @Override + public ArrayList call(String s) { + return Lists.newArrayList(s.split(" ")); + } + } + ); - // TODO: Read a user-specified input file. - @SuppressWarnings("unchecked") - JavaRDD> transactions = sc.parallelize(Lists.newArrayList( - Lists.newArrayList("r z h k p".split(" ")), - Lists.newArrayList("z y x w v u t s".split(" ")), - Lists.newArrayList("s x o n r".split(" ")), - Lists.newArrayList("x z y m t s q e".split(" ")), - Lists.newArrayList("z".split(" ")), - Lists.newArrayList("x z y r q t p".split(" "))), 2); - - FPGrowth fpg = new FPGrowth() - .setMinSupport(0.3); - FPGrowthModel model = fpg.run(transactions); + FPGrowthModel model = new FPGrowth() + .setMinSupport(minSupport) + .setNumPartitions(numPartition) + .run(transactions); for (FPGrowth.FreqItemset s: model.freqItemsets().toJavaRDD().collect()) { System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq()); diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala index aaae275ec5524..13f24a1e59610 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -17,30 +17,61 @@ package org.apache.spark.examples.mllib +import scopt.OptionParser + import org.apache.spark.mllib.fpm.FPGrowth -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkConf, SparkContext} /** * Example for mining frequent itemsets using FP-growth. + * Example usage: ./bin/run-example mllib.FPGrowthExample \ + * --minSupport 0.8 --numPartition 2 ./data/mllib/sample_fpgrowth.txt */ object FPGrowthExample { + case class Params( + input: String = null, + minSupport: Double = 0.3, + numPartition: Int = -1) extends AbstractParams[Params] + def main(args: Array[String]) { - val conf = new SparkConf().setAppName("FPGrowthExample") + val defaultParams = Params() + + val parser = new OptionParser[Params]("FPGrowthExample") { + head("FPGrowth: an example FP-growth app.") + opt[Double]("minSupport") + .text(s"minimal support level, default: ${defaultParams.minSupport}") + .action((x, c) => c.copy(minSupport = x)) + opt[Int]("numPartition") + .text(s"number of partition, default: ${defaultParams.numPartition}") + .action((x, c) => c.copy(numPartition = x)) + arg[String]("") + .text("input paths to input data set, whose file format is that each line " + + "contains a transaction with each item in String and separated by a space") + .required() + .action((x, c) => c.copy(input = x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"FPGrowthExample with $params") val sc = new SparkContext(conf) + val transactions = sc.textFile(params.input).map(_.split(" ")).cache() + + println(s"Number of transactions: ${transactions.count()}") + + val model = new FPGrowth() + .setMinSupport(params.minSupport) + .setNumPartitions(params.numPartition) + .run(transactions) - // TODO: Read a user-specified input file. - val transactions = sc.parallelize(Seq( - "r z h k p", - "z y x w v u t s", - "s x o n r", - "x z y m t s q e", - "z", - "x z y r q t p").map(_.split(" ")), numSlices = 2) - - val fpg = new FPGrowth() - .setMinSupport(0.3) - val model = fpg.run(transactions) + println(s"Number of frequent itemsets: ${model.freqItemsets.count()}") model.freqItemsets.collect().foreach { itemset => println(itemset.items.mkString("[", ",", "]") + ", " + itemset.freq) From 28ccf5ee769a1df019e38985112065c01724fbd9 Mon Sep 17 00:00:00 2001 From: Alexander Ulanov Date: Mon, 23 Feb 2015 12:09:40 -0800 Subject: [PATCH 224/817] [MLLIB] SPARK-5912 Programming guide for feature selection Added description of ChiSqSelector and few words about feature selection in general. I could add a code example, however it would not look reasonable in the absence of feature discretizer or a dataset in the `data` folder that has redundant features. Author: Alexander Ulanov Closes #4709 from avulanov/SPARK-5912 and squashes the following commits: 19a8a4e [Alexander Ulanov] Addressing reviewers comments @jkbradley 58d9e4d [Alexander Ulanov] Addressing reviewers comments @jkbradley eb6b9fe [Alexander Ulanov] Typo 2921a1d [Alexander Ulanov] ChiSqSelector example of use c845350 [Alexander Ulanov] ChiSqSelector docs --- docs/mllib-feature-extraction.md | 54 ++++++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index d4a61a7fbf3d7..d588b9cb46697 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -375,3 +375,57 @@ data2 = labels.zip(normalizer2.transform(features)) {% endhighlight %}
    + +## Feature selection +[Feature selection](http://en.wikipedia.org/wiki/Feature_selection) allows selecting the most relevant features for use in model construction. The number of features to select can be determined using the validation set. Feature selection is usually applied on sparse data, for example in text classification. Feature selection reduces the size of the vector space and, in turn, the complexity of any subsequent operation with vectors. + +### ChiSqSelector +ChiSqSelector stands for Chi-Squared feature selection. It operates on the labeled data. ChiSqSelector orders categorical features based on their values of Chi-Squared test on independence from class and filters (selects) top given features. + +#### Model Fitting + +[`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) has the +following parameters in the constructor: + +* `numTopFeatures` number of top features that selector will select (filter). + +We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) method in +`ChiSqSelector` which can take an input of `RDD[LabeledPoint]` with categorical features, learn the summary statistics, and then +return a model which can transform the input dataset into the reduced feature space. + +This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the Chi-Squared feature selection on a `Vector` to produce a reduced `Vector` or on +an `RDD[Vector]` to produce a reduced `RDD[Vector]`. + +Note that the model that performs actual feature filtering can be instantiated independently with array of feature indices that has to be sorted ascending. + +#### Example + +The following example shows the basic use of ChiSqSelector. + +
    +
    +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils + +// load some data in libsvm format, each point is in the range 0..255 +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// discretize data in 16 equal bins +val discretizedData = data.map { lp => + LabeledPoint(lp.label, Vectors.dense(lp.features.toArray.map { x => x / 16 } ) ) +} +// create ChiSqSelector that will select 50 features +val selector = new ChiSqSelector(50) +// create ChiSqSelector model +val transformer = selector.fit(disctetizedData) +// filter top 50 features from each feature vector +val filteredData = disctetizedData.map { lp => + LabeledPoint(lp.label, transformer.transform(lp.features)) +} +{% endhighlight %} +
    +
    + From 59536cc87e10e5011560556729dd901280958f43 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 23 Feb 2015 16:15:57 -0800 Subject: [PATCH 225/817] [SPARK-5912] [docs] [mllib] Small fixes to ChiSqSelector docs Fixes: * typo in Scala example * Removed comment "usually applied on sparse data" since that is debatable * small edits to text for clarity CC: avulanov I noticed a typo post-hoc and ended up making a few small edits. Do the changes look OK? Author: Joseph K. Bradley Closes #4732 from jkbradley/chisqselector-docs and squashes the following commits: 9656a3b [Joseph K. Bradley] added Java example for ChiSqSelector to guide 3f3f9f4 [Joseph K. Bradley] small fixes to ChiSqSelector docs --- docs/mllib-feature-extraction.md | 72 ++++++++++++++++++++++++++------ 1 file changed, 60 insertions(+), 12 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index d588b9cb46697..80842b27effd8 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -377,27 +377,27 @@ data2 = labels.zip(normalizer2.transform(features))
    ## Feature selection -[Feature selection](http://en.wikipedia.org/wiki/Feature_selection) allows selecting the most relevant features for use in model construction. The number of features to select can be determined using the validation set. Feature selection is usually applied on sparse data, for example in text classification. Feature selection reduces the size of the vector space and, in turn, the complexity of any subsequent operation with vectors. +[Feature selection](http://en.wikipedia.org/wiki/Feature_selection) allows selecting the most relevant features for use in model construction. Feature selection reduces the size of the vector space and, in turn, the complexity of any subsequent operation with vectors. The number of features to select can be tuned using a held-out validation set. ### ChiSqSelector -ChiSqSelector stands for Chi-Squared feature selection. It operates on the labeled data. ChiSqSelector orders categorical features based on their values of Chi-Squared test on independence from class and filters (selects) top given features. +[`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) stands for Chi-Squared feature selection. It operates on labeled data with categorical features. `ChiSqSelector` orders features based on a Chi-Squared test of independence from the class, and then filters (selects) the top features which are most closely related to the label. #### Model Fitting [`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) has the following parameters in the constructor: -* `numTopFeatures` number of top features that selector will select (filter). +* `numTopFeatures` number of top features that the selector will select (filter). We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) method in `ChiSqSelector` which can take an input of `RDD[LabeledPoint]` with categorical features, learn the summary statistics, and then -return a model which can transform the input dataset into the reduced feature space. +return a `ChiSqSelectorModel` which can transform an input dataset into the reduced feature space. This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) which can apply the Chi-Squared feature selection on a `Vector` to produce a reduced `Vector` or on an `RDD[Vector]` to produce a reduced `RDD[Vector]`. -Note that the model that performs actual feature filtering can be instantiated independently with array of feature indices that has to be sorted ascending. +Note that the user can also construct a `ChiSqSelectorModel` by hand by providing an array of selected feature indices (which must be sorted in ascending order). #### Example @@ -411,21 +411,69 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils -// load some data in libsvm format, each point is in the range 0..255 +// Load some data in libsvm format val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// discretize data in 16 equal bins +// Discretize data in 16 equal bins since ChiSqSelector requires categorical features val discretizedData = data.map { lp => LabeledPoint(lp.label, Vectors.dense(lp.features.toArray.map { x => x / 16 } ) ) } -// create ChiSqSelector that will select 50 features +// Create ChiSqSelector that will select 50 features val selector = new ChiSqSelector(50) -// create ChiSqSelector model -val transformer = selector.fit(disctetizedData) -// filter top 50 features from each feature vector -val filteredData = disctetizedData.map { lp => +// Create ChiSqSelector model (selecting features) +val transformer = selector.fit(discretizedData) +// Filter the top 50 features from each feature vector +val filteredData = discretizedData.map { lp => LabeledPoint(lp.label, transformer.transform(lp.features)) } {% endhighlight %} + +
    +{% highlight java %} +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.feature.ChiSqSelector; +import org.apache.spark.mllib.feature.ChiSqSelectorModel; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaChiSqSelector"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); +JavaRDD points = MLUtils.loadLibSVMFile(sc.sc(), + "data/mllib/sample_libsvm_data.txt").toJavaRDD().cache(); + +// Discretize data in 16 equal bins since ChiSqSelector requires categorical features +JavaRDD discretizedData = points.map( + new Function() { + @Override + public LabeledPoint call(LabeledPoint lp) { + final double[] discretizedFeatures = new double[lp.features().size()]; + for (int i = 0; i < lp.features().size(); ++i) { + discretizedFeatures[i] = lp.features().apply(i) / 16; + } + return new LabeledPoint(lp.label(), Vectors.dense(discretizedFeatures)); + } + }); + +// Create ChiSqSelector that will select 50 features +ChiSqSelector selector = new ChiSqSelector(50); +// Create ChiSqSelector model (selecting features) +final ChiSqSelectorModel transformer = selector.fit(discretizedData.rdd()); +// Filter the top 50 features from each feature vector +JavaRDD filteredData = discretizedData.map( + new Function() { + @Override + public LabeledPoint call(LabeledPoint lp) { + return new LabeledPoint(lp.label(), transformer.transform(lp.features())); + } + } +); + +sc.stop(); +{% endhighlight %} +
    From 48376bfe9c97bf31279918def6c6615849c88f4d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 23 Feb 2015 17:16:34 -0800 Subject: [PATCH 226/817] [SPARK-5935][SQL] Accept MapType in the schema provided to a JSON dataset. JIRA: https://issues.apache.org/jira/browse/SPARK-5935 Author: Yin Huai Author: Yin Huai Closes #4710 from yhuai/jsonMapType and squashes the following commits: 3e40390 [Yin Huai] Remove unnecessary changes. f8e6267 [Yin Huai] Fix test. baa36e3 [Yin Huai] Accept MapType in the schema provided to jsonFile/jsonRDD. --- .../org/apache/spark/sql/json/JsonRDD.scala | 3 + .../org/apache/spark/sql/json/JsonSuite.scala | 56 +++++++++++++++++++ .../apache/spark/sql/json/TestJsonData.scala | 17 ++++++ 3 files changed, 76 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 3b8dde1823370..d83bdc2f7ff9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -416,6 +416,9 @@ private[sql] object JsonRDD extends Logging { case NullType => null case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) + case MapType(StringType, valueType, _) => + val map = value.asInstanceOf[Map[String, Any]] + map.mapValues(enforceCorrectType(_, valueType)).map(identity) case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) case DateType => toDate(value) case TimestampType => toTimestamp(value) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index c94e44bd7c397..005f20b96df79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -657,6 +657,62 @@ class JsonSuite extends QueryTest { ) } + test("Applying schemas with MapType") { + val schemaWithSimpleMap = StructType( + StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) + val jsonWithSimpleMap = jsonRDD(mapType1, schemaWithSimpleMap) + + jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap") + + checkAnswer( + sql("select map from jsonWithSimpleMap"), + Row(Map("a" -> 1)) :: + Row(Map("b" -> 2)) :: + Row(Map("c" -> 3)) :: + Row(Map("c" -> 1, "d" -> 4)) :: + Row(Map("e" -> null)) :: Nil + ) + + checkAnswer( + sql("select map['c'] from jsonWithSimpleMap"), + Row(null) :: + Row(null) :: + Row(3) :: + Row(1) :: + Row(null) :: Nil + ) + + val innerStruct = StructType( + StructField("field1", ArrayType(IntegerType, true), true) :: + StructField("field2", IntegerType, true) :: Nil) + val schemaWithComplexMap = StructType( + StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) + + val jsonWithComplexMap = jsonRDD(mapType2, schemaWithComplexMap) + + jsonWithComplexMap.registerTempTable("jsonWithComplexMap") + + checkAnswer( + sql("select map from jsonWithComplexMap"), + Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: + Row(Map("b" -> Row(null, 2))) :: + Row(Map("c" -> Row(Seq(), 4))) :: + Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) :: + Row(Map("e" -> null)) :: + Row(Map("f" -> Row(null, null))) :: Nil + ) + + checkAnswer( + sql("select map['a'].field1, map['c'].field2 from jsonWithComplexMap"), + Row(Seq(1, 2, 3, null), null) :: + Row(null, null) :: + Row(null, 4) :: + Row(null, 3) :: + Row(null, null) :: + Row(null, null) :: Nil + ) + } + test("SPARK-2096 Correctly parse dot notations") { val jsonDF = jsonRDD(complexFieldAndType2) jsonDF.registerTempTable("jsonTable") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 3370b3c98b4be..15698f61e0837 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -146,6 +146,23 @@ object TestJsonData { ]] }""" :: Nil) + val mapType1 = + TestSQLContext.sparkContext.parallelize( + """{"map": {"a": 1}}""" :: + """{"map": {"b": 2}}""" :: + """{"map": {"c": 3}}""" :: + """{"map": {"c": 1, "d": 4}}""" :: + """{"map": {"e": null}}""" :: Nil) + + val mapType2 = + TestSQLContext.sparkContext.parallelize( + """{"map": {"a": {"field1": [1, 2, 3, null]}}}""" :: + """{"map": {"b": {"field2": 2}}}""" :: + """{"map": {"c": {"field1": [], "field2": 4}}}""" :: + """{"map": {"c": {"field2": 3}, "d": {"field1": [null]}}}""" :: + """{"map": {"e": null}}""" :: + """{"map": {"f": {"field1": null}}}""" :: Nil) + val nullsInArrays = TestSQLContext.sparkContext.parallelize( """{"field1":[[null], [[["Test"]]]]}""" :: From 1ed57086d402c38d95cda6c3d9d7aea806609bf9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 23 Feb 2015 17:34:54 -0800 Subject: [PATCH 227/817] [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution Author: Michael Armbrust Closes #4684 from marmbrus/explainAnalysis and squashes the following commits: afbaa19 [Michael Armbrust] fix python d93278c [Michael Armbrust] fix hive e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis 52119f2 [Michael Armbrust] more tests 82a5431 [Michael Armbrust] fix tests 25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis aee1e6a [Michael Armbrust] fix hive b23a844 [Michael Armbrust] newline de8dc51 [Michael Armbrust] more comments acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in query execution --- python/pyspark/sql/context.py | 30 ++--- .../apache/spark/sql/catalyst/SqlParser.scala | 2 + .../sql/catalyst/analysis/Analyzer.scala | 83 -------------- .../sql/catalyst/analysis/CheckAnalysis.scala | 105 ++++++++++++++++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 35 +++--- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 5 +- .../org/apache/spark/sql/SQLContext.scala | 14 ++- .../org/apache/spark/sql/sources/rules.scala | 10 +- .../spark/sql/sources/DataSourceTest.scala | 1 - .../spark/sql/sources/InsertSuite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 1 - 12 files changed, 164 insertions(+), 126 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 313f15e6d9b6f..125933c9d3ae0 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -267,20 +267,20 @@ def applySchema(self, rdd, schema): ... StructField("byte2", ByteType(), False), ... StructField("short1", ShortType(), False), ... StructField("short2", ShortType(), False), - ... StructField("int", IntegerType(), False), - ... StructField("float", FloatType(), False), - ... StructField("date", DateType(), False), - ... StructField("time", TimestampType(), False), - ... StructField("map", + ... StructField("int1", IntegerType(), False), + ... StructField("float1", FloatType(), False), + ... StructField("date1", DateType(), False), + ... StructField("time1", TimestampType(), False), + ... StructField("map1", ... MapType(StringType(), IntegerType(), False), False), - ... StructField("struct", + ... StructField("struct1", ... StructType([StructField("b", ShortType(), False)]), False), - ... StructField("list", ArrayType(ByteType(), False), False), - ... StructField("null", DoubleType(), True)]) + ... StructField("list1", ArrayType(ByteType(), False), False), + ... StructField("null1", DoubleType(), True)]) >>> df = sqlCtx.applySchema(rdd, schema) >>> results = df.map( - ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date, - ... x.time, x.map["a"], x.struct.b, x.list, x.null)) + ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, + ... x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) @@ -288,20 +288,20 @@ def applySchema(self, rdd, schema): >>> df.registerTempTable("table2") >>> sqlCtx.sql( ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + - ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + - ... "float + 1.5 as float FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] + ... "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + + ... "float1 + 1.5 as float1 FROM table2").collect() + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int1=2147483646, float1=2.5)] >>> from pyspark.sql.types import _parse_schema_abstract, _infer_schema_type >>> rdd = sc.parallelize([(127, -32768, 1.0, ... datetime(2010, 1, 1, 1, 1, 1), ... {"a": 1}, (2,), [1, 2, 3])]) - >>> abstract = "byte short float time map{} struct(b) list[]" + >>> abstract = "byte1 short1 float1 time1 map1{} struct1(b) list1[]" >>> schema = _parse_schema_abstract(abstract) >>> typedSchema = _infer_schema_type(rdd.first(), schema) >>> df = sqlCtx.applySchema(rdd, typedSchema) >>> df.collect() - [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])] + [Row(byte1=127, short1=-32768, float1=1.0, time1=..., list1=[1, 2, 3])] """ if isinstance(rdd, DataFrame): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 124f083669358..b16aff99af1c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -78,6 +78,7 @@ class SqlParser extends AbstractSparkSQLParser { protected val IF = Keyword("IF") protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") + protected val INT = Keyword("INT") protected val INSERT = Keyword("INSERT") protected val INTERSECT = Keyword("INTERSECT") protected val INTO = Keyword("INTO") @@ -394,6 +395,7 @@ class SqlParser extends AbstractSparkSQLParser { | fixedDecimalType | DECIMAL ^^^ DecimalType.Unlimited | DATE ^^^ DateType + | INT ^^^ IntegerType ) protected lazy val fixedDecimalType: Parser[DataType] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fc37b8cde0806..e4e542562f22d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -52,12 +52,6 @@ class Analyzer(catalog: Catalog, */ val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil - /** - * Override to provide additional rules for the "Check Analysis" batch. - * These rules will be evaluated after our built-in check rules. - */ - val extendedCheckRules: Seq[Rule[LogicalPlan]] = Nil - lazy val batches: Seq[Batch] = Seq( Batch("Resolution", fixedPoint, ResolveRelations :: @@ -71,87 +65,10 @@ class Analyzer(catalog: Catalog, TrimGroupingAliases :: typeCoercionRules ++ extendedResolutionRules : _*), - Batch("Check Analysis", Once, - CheckResolution +: - extendedCheckRules: _*), Batch("Remove SubQueries", fixedPoint, EliminateSubQueries) ) - /** - * Makes sure all attributes and logical plans have been resolved. - */ - object CheckResolution extends Rule[LogicalPlan] { - def failAnalysis(msg: String) = { throw new AnalysisException(msg) } - - def apply(plan: LogicalPlan): LogicalPlan = { - // We transform up and order the rules so as to catch the first possible failure instead - // of the result of cascading resolution failures. - plan.foreachUp { - case operator: LogicalPlan => - operator transformExpressionsUp { - case a: Attribute if !a.resolved => - val from = operator.inputSet.map(_.name).mkString(", ") - a.failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") - - case c: Cast if !c.resolved => - failAnalysis( - s"invalid cast from ${c.child.dataType.simpleString} to ${c.dataType.simpleString}") - - case b: BinaryExpression if !b.resolved => - failAnalysis( - s"invalid expression ${b.prettyString} " + - s"between ${b.left.simpleString} and ${b.right.simpleString}") - } - - operator match { - case f: Filter if f.condition.dataType != BooleanType => - failAnalysis( - s"filter expression '${f.condition.prettyString}' " + - s"of type ${f.condition.dataType.simpleString} is not a boolean.") - - case aggregatePlan @ Aggregate(groupingExprs, aggregateExprs, child) => - def checkValidAggregateExpression(expr: Expression): Unit = expr match { - case _: AggregateExpression => // OK - case e: Attribute if !groupingExprs.contains(e) => - failAnalysis( - s"expression '${e.prettyString}' is neither present in the group by, " + - s"nor is it an aggregate function. " + - "Add to group by or wrap in first() if you don't care which value you get.") - case e if groupingExprs.contains(e) => // OK - case e if e.references.isEmpty => // OK - case e => e.children.foreach(checkValidAggregateExpression) - } - - val cleaned = aggregateExprs.map(_.transform { - // Should trim aliases around `GetField`s. These aliases are introduced while - // resolving struct field accesses, because `GetField` is not a `NamedExpression`. - // (Should we just turn `GetField` into a `NamedExpression`?) - case Alias(g, _) => g - }) - - cleaned.foreach(checkValidAggregateExpression) - - case o if o.children.nonEmpty && - !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => - val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") - val input = o.inputSet.map(_.prettyString).mkString(",") - - failAnalysis(s"resolved attributes $missingAttributes missing from $input") - - // Catch all - case o if !o.resolved => - failAnalysis( - s"unresolved operator ${operator.simpleString}") - - case _ => // Analysis successful! - } - } - - plan - } - } - /** * Removes no-op Alias expressions from the plan. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala new file mode 100644 index 0000000000000..4e8fc892f3eea --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types._ + +/** + * Throws user facing errors when passed invalid queries that fail to analyze. + */ +class CheckAnalysis { + + /** + * Override to provide additional checks for correct analysis. + * These rules will be evaluated after our built-in check rules. + */ + val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil + + def failAnalysis(msg: String) = { + throw new AnalysisException(msg) + } + + def apply(plan: LogicalPlan): Unit = { + // We transform up and order the rules so as to catch the first possible failure instead + // of the result of cascading resolution failures. + plan.foreachUp { + case operator: LogicalPlan => + operator transformExpressionsUp { + case a: Attribute if !a.resolved => + val from = operator.inputSet.map(_.name).mkString(", ") + a.failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") + + case c: Cast if !c.resolved => + failAnalysis( + s"invalid cast from ${c.child.dataType.simpleString} to ${c.dataType.simpleString}") + + case b: BinaryExpression if !b.resolved => + failAnalysis( + s"invalid expression ${b.prettyString} " + + s"between ${b.left.simpleString} and ${b.right.simpleString}") + } + + operator match { + case f: Filter if f.condition.dataType != BooleanType => + failAnalysis( + s"filter expression '${f.condition.prettyString}' " + + s"of type ${f.condition.dataType.simpleString} is not a boolean.") + + case aggregatePlan@Aggregate(groupingExprs, aggregateExprs, child) => + def checkValidAggregateExpression(expr: Expression): Unit = expr match { + case _: AggregateExpression => // OK + case e: Attribute if !groupingExprs.contains(e) => + failAnalysis( + s"expression '${e.prettyString}' is neither present in the group by, " + + s"nor is it an aggregate function. " + + "Add to group by or wrap in first() if you don't care which value you get.") + case e if groupingExprs.contains(e) => // OK + case e if e.references.isEmpty => // OK + case e => e.children.foreach(checkValidAggregateExpression) + } + + val cleaned = aggregateExprs.map(_.transform { + // Should trim aliases around `GetField`s. These aliases are introduced while + // resolving struct field accesses, because `GetField` is not a `NamedExpression`. + // (Should we just turn `GetField` into a `NamedExpression`?) + case Alias(g, _) => g + }) + + cleaned.foreach(checkValidAggregateExpression) + + case o if o.children.nonEmpty && + !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => + val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") + val input = o.inputSet.map(_.prettyString).mkString(",") + + failAnalysis(s"resolved attributes $missingAttributes missing from $input") + + // Catch all + case o if !o.resolved => + failAnalysis( + s"unresolved operator ${operator.simpleString}") + + case _ => // Analysis successful! + } + } + extendedCheckRules.foreach(_(plan)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index aec7847356cd4..c1dd5aa913ddc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -30,11 +30,21 @@ import org.apache.spark.sql.catalyst.dsl.plans._ class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) - val caseSensitiveAnalyze = + + val caseSensitiveAnalyzer = new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) - val caseInsensitiveAnalyze = + val caseInsensitiveAnalyzer = new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) + val checkAnalysis = new CheckAnalysis + + + def caseSensitiveAnalyze(plan: LogicalPlan) = + checkAnalysis(caseSensitiveAnalyzer(plan)) + + def caseInsensitiveAnalyze(plan: LogicalPlan) = + checkAnalysis(caseInsensitiveAnalyzer(plan)) + val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) val testRelation2 = LocalRelation( AttributeReference("a", StringType)(), @@ -55,7 +65,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { a.select(UnresolvedStar(None)).select('a).unionAll(b.select(UnresolvedStar(None))) } - assert(caseInsensitiveAnalyze(plan).resolved) + assert(caseInsensitiveAnalyzer(plan).resolved) } test("check project's resolved") { @@ -71,11 +81,11 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { test("analyze project") { assert( - caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + caseSensitiveAnalyzer(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) assert( - caseSensitiveAnalyze( + caseSensitiveAnalyzer( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -88,13 +98,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage().toLowerCase.contains("cannot resolve")) assert( - caseInsensitiveAnalyze( + caseInsensitiveAnalyzer( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) assert( - caseInsensitiveAnalyze( + caseInsensitiveAnalyzer( Project(Seq(UnresolvedAttribute("tBl.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -107,16 +117,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage == "Table Not Found: tAbLe") assert( - caseSensitiveAnalyze(UnresolvedRelation(Seq("TaBlE"), None)) === - testRelation) + caseSensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) assert( - caseInsensitiveAnalyze(UnresolvedRelation(Seq("tAbLe"), None)) === - testRelation) + caseInsensitiveAnalyzer(UnresolvedRelation(Seq("tAbLe"), None)) === testRelation) assert( - caseInsensitiveAnalyze(UnresolvedRelation(Seq("TaBlE"), None)) === - testRelation) + caseInsensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) } def errorTest( @@ -177,7 +184,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) - val plan = caseInsensitiveAnalyze( + val plan = caseInsensitiveAnalyzer( testRelation2.select( 'a / Literal(2) as 'div1, 'a / 'b as 'div2, 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 69e5f6a07da7f..27ac398063d43 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 @@ -117,7 +117,7 @@ class DataFrame protected[sql]( this(sqlContext, { val qe = sqlContext.executePlan(logicalPlan) if (sqlContext.conf.dataFrameEagerAnalysis) { - qe.analyzed // This should force analysis and throw errors if there are any + qe.assertAnalyzed() // This should force analysis and throw errors if there are any } qe }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 39f6c2f4bc8b4..a08c0f5ce3ff4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -52,8 +52,9 @@ private[spark] object SQLConf { // This is used to set the default data source val DEFAULT_DATA_SOURCE_NAME = "spark.sql.sources.default" - // Whether to perform eager analysis on a DataFrame. - val DATAFRAME_EAGER_ANALYSIS = "spark.sql.dataframe.eagerAnalysis" + // Whether to perform eager analysis when constructing a dataframe. + // Set to false when debugging requires the ability to look at invalid query plans. + val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4bdaa023914b8..ce800e0754559 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -114,7 +114,6 @@ class SQLContext(@transient val sparkContext: SparkContext) new Analyzer(catalog, functionRegistry, caseSensitive = true) { override val extendedResolutionRules = ExtractPythonUdfs :: - sources.PreWriteCheck(catalog) :: sources.PreInsertCastAndRename :: Nil } @@ -1057,6 +1056,13 @@ class SQLContext(@transient val sparkContext: SparkContext) Batch("Add exchange", Once, AddExchange(self)) :: Nil } + @transient + protected[sql] lazy val checkAnalysis = new CheckAnalysis { + override val extendedCheckRules = Seq( + sources.PreWriteCheck(catalog) + ) + } + /** * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy @@ -1064,9 +1070,13 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @DeveloperApi protected[sql] class QueryExecution(val logical: LogicalPlan) { + def assertAnalyzed(): Unit = checkAnalysis(analyzed) lazy val analyzed: LogicalPlan = analyzer(logical) - lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed) + lazy val withCachedData: LogicalPlan = { + assertAnalyzed + cacheManager.useCachedData(analyzed) + } lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData) // TODO: Don't just pick the first one... diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 36a9c0bdc41e6..8440581074877 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -78,10 +78,10 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { /** * A rule to do various checks before inserting into or writing to a data source table. */ -private[sql] case class PreWriteCheck(catalog: Catalog) extends Rule[LogicalPlan] { +private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => Unit) { def failAnalysis(msg: String) = { throw new AnalysisException(msg) } - def apply(plan: LogicalPlan): LogicalPlan = { + def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => @@ -93,7 +93,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends Rule[LogicalPlan val srcRelations = query.collect { case LogicalRelation(src: BaseRelation) => src } - if (srcRelations.exists(src => src == t)) { + if (srcRelations.contains(t)) { failAnalysis( "Cannot insert overwrite into table that is also being read from.") } else { @@ -119,7 +119,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends Rule[LogicalPlan val srcRelations = query.collect { case LogicalRelation(src: BaseRelation) => src } - if (srcRelations.exists(src => src == dest)) { + if (srcRelations.contains(dest)) { failAnalysis( s"Cannot overwrite table $tableName that is also being read from.") } else { @@ -134,7 +134,5 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends Rule[LogicalPlan case _ => // OK } - - plan } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 0ec6881d7afe6..91c6367371f15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -30,7 +30,6 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { override protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = false) { override val extendedResolutionRules = - PreWriteCheck(catalog) :: PreInsertCastAndRename :: Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5682e5a2bcea9..b5b16f9546691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -205,7 +205,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll { val message = intercept[AnalysisException] { sql( s""" - |INSERT OVERWRITE TABLE oneToTen SELECT a FROM jt + |INSERT OVERWRITE TABLE oneToTen SELECT CAST(a AS INT) FROM jt """.stripMargin) }.getMessage assert( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 2e205e67c0fdd..c439dfe0a71f8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -268,7 +268,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.PreInsertionCasts :: ExtractPythonUdfs :: ResolveUdtfsAlias :: - sources.PreWriteCheck(catalog) :: sources.PreInsertCastAndRename :: Nil } From cf2e41653de778dc8db8b03385a053aae1152e19 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 23 Feb 2015 22:08:44 -0800 Subject: [PATCH 228/817] [SPARK-5958][MLLIB][DOC] update block matrix user guide * Removed SVD code from examples. * Corrected Java API doc link. * Updated variable names: `AtransposeA` -> `ata`. * Minor changes. brkyvz Author: Xiangrui Meng Closes #4737 from mengxr/update-block-matrix-user-guide and squashes the following commits: 70f53ac [Xiangrui Meng] update block matrix user guide --- docs/mllib-data-types.md | 41 +++++++++++++++------------------------- 1 file changed, 15 insertions(+), 26 deletions(-) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 24d22b9bcdfa4..fe6c1bf7bfd99 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -298,23 +298,22 @@ In general the use of non-deterministic RDDs can lead to errors. ### BlockMatrix -A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock`s, where `MatrixBlock` is +A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock`s, where a `MatrixBlock` is a tuple of `((Int, Int), Matrix)`, where the `(Int, Int)` is the index of the block, and `Matrix` is the sub-matrix at the given index with size `rowsPerBlock` x `colsPerBlock`. -`BlockMatrix` supports methods such as `.add` and `.multiply` with another `BlockMatrix`. -`BlockMatrix` also has a helper function `.validate` which can be used to debug whether the +`BlockMatrix` supports methods such as `add` and `multiply` with another `BlockMatrix`. +`BlockMatrix` also has a helper function `validate` which can be used to check whether the `BlockMatrix` is set up properly.
    A [`BlockMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.BlockMatrix) can be -most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` using `.toBlockMatrix()`. -`.toBlockMatrix()` will create blocks of size 1024 x 1024. Users may change the sizes of their blocks -by supplying the values through `.toBlockMatrix(rowsPerBlock, colsPerBlock)`. +most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` by calling `toBlockMatrix`. +`toBlockMatrix` creates blocks of size 1024 x 1024 by default. +Users may change the block size by supplying the values through `toBlockMatrix(rowsPerBlock, colsPerBlock)`. {% highlight scala %} -import org.apache.spark.mllib.linalg.SingularValueDecomposition import org.apache.spark.mllib.linalg.distributed.{BlockMatrix, CoordinateMatrix, MatrixEntry} val entries: RDD[MatrixEntry] = ... // an RDD of (i, j, v) matrix entries @@ -323,29 +322,24 @@ val coordMat: CoordinateMatrix = new CoordinateMatrix(entries) // Transform the CoordinateMatrix to a BlockMatrix val matA: BlockMatrix = coordMat.toBlockMatrix().cache() -// validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. +// Validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. // Nothing happens if it is valid. -matA.validate +matA.validate() // Calculate A^T A. -val AtransposeA = matA.transpose.multiply(matA) - -// get SVD of 2 * A -val A2 = matA.add(matA) -val svd = A2.toIndexedRowMatrix().computeSVD(20, false, 1e-9) +val ata = matA.transpose.multiply(matA) {% endhighlight %}
    -A [`BlockMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.BlockMatrix) can be -most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` using `.toBlockMatrix()`. -`.toBlockMatrix()` will create blocks of size 1024 x 1024. Users may change the sizes of their blocks -by supplying the values through `.toBlockMatrix(rowsPerBlock, colsPerBlock)`. +A [`BlockMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/BlockMatrix.html) can be +most easily created from an `IndexedRowMatrix` or `CoordinateMatrix` by calling `toBlockMatrix`. +`toBlockMatrix` creates blocks of size 1024 x 1024 by default. +Users may change the block size by supplying the values through `toBlockMatrix(rowsPerBlock, colsPerBlock)`. {% highlight java %} import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.mllib.linalg.SingularValueDecomposition; import org.apache.spark.mllib.linalg.distributed.BlockMatrix; import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; @@ -356,17 +350,12 @@ CoordinateMatrix coordMat = new CoordinateMatrix(entries.rdd()); // Transform the CoordinateMatrix to a BlockMatrix BlockMatrix matA = coordMat.toBlockMatrix().cache(); -// validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. +// Validate whether the BlockMatrix is set up properly. Throws an Exception when it is not valid. // Nothing happens if it is valid. matA.validate(); // Calculate A^T A. -BlockMatrix AtransposeA = matA.transpose().multiply(matA); - -// get SVD of 2 * A -BlockMatrix A2 = matA.add(matA); -SingularValueDecomposition svd = - A2.toIndexedRowMatrix().computeSVD(20, false, 1e-9); +BlockMatrix ata = matA.transpose().multiply(matA); {% endhighlight %}
    From 840333133396d443e747f62fce9967f7681fb276 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 24 Feb 2015 10:45:38 -0800 Subject: [PATCH 229/817] [SPARK-5968] [SQL] Suppresses ParquetOutputCommitter WARN logs Please refer to the [JIRA ticket] [1] for the motivation. [1]: https://issues.apache.org/jira/browse/SPARK-5968 [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4744) Author: Cheng Lian Closes #4744 from liancheng/spark-5968 and squashes the following commits: caac6a8 [Cheng Lian] Suppresses ParquetOutputCommitter WARN logs --- .../apache/spark/sql/parquet/ParquetRelation.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index b0db9943a506c..a0d1005c0cae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.parquet import java.io.IOException +import java.util.logging.Level import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.permission.FsAction -import parquet.hadoop.ParquetOutputFormat +import parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat} import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType @@ -91,7 +92,7 @@ private[sql] object ParquetRelation { // checks first to see if there's any handlers already set // and if not it creates them. If this method executes prior // to that class being loaded then: - // 1) there's no handlers installed so there's none to + // 1) there's no handlers installed so there's none to // remove. But when it IS finally loaded the desired affect // of removing them is circumvented. // 2) The parquet.Log static initializer calls setUseParentHanders(false) @@ -99,7 +100,7 @@ private[sql] object ParquetRelation { // // Therefore we need to force the class to be loaded. // This should really be resolved by Parquet. - Class.forName(classOf[parquet.Log].getName()) + Class.forName(classOf[parquet.Log].getName) // Note: Logger.getLogger("parquet") has a default logger // that appends to Console which needs to be cleared. @@ -108,6 +109,11 @@ private[sql] object ParquetRelation { // TODO(witgo): Need to set the log level ? // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null) if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true) + + // Disables WARN log message in ParquetOutputCommitter. + // See https://issues.apache.org/jira/browse/SPARK-5968 for details + Class.forName(classOf[ParquetOutputCommitter].getName) + java.util.logging.Logger.getLogger(classOf[ParquetOutputCommitter].getName).setLevel(Level.OFF) } // The element type for the RDDs that this relation maps to. From 0a59e45e2f2e6f00ccd5f10c79f629fb796fd8d0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Feb 2015 10:49:51 -0800 Subject: [PATCH 230/817] [SPARK-5910][SQL] Support for as in selectExpr Author: Michael Armbrust Closes #4736 from marmbrus/asExprs and squashes the following commits: 5ba97e4 [Michael Armbrust] [SPARK-5910][SQL] Support for as in selectExpr --- .../scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b16aff99af1c5..c363a5efacde8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -40,7 +40,7 @@ class SqlParser extends AbstractSparkSQLParser { def parseExpression(input: String): Expression = { // Initialize the Keywords. lexical.initialize(reservedWords) - phrase(expression)(new lexical.Scanner(input)) match { + phrase(projection)(new lexical.Scanner(input)) match { case Success(plan, _) => plan case failureOrError => sys.error(failureOrError.toString) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 6b9b3a8425964..e71e9bee3a6d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -130,6 +130,12 @@ class DataFrameSuite extends QueryTest { testData.collect().map(row => Row(math.abs(row.getInt(0)), row.getString(1))).toSeq) } + test("selectExpr with alias") { + checkAnswer( + testData.selectExpr("key as k").select("k"), + testData.select("key").collect().toSeq) + } + test("filterExpr") { checkAnswer( testData.filter("key > 90"), From 201236628a344194f7c20ba8e9afeeaefbe9318c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Feb 2015 10:52:18 -0800 Subject: [PATCH 231/817] [SPARK-5532][SQL] Repartition should not use external rdd representation Author: Michael Armbrust Closes #4738 from marmbrus/udtRepart and squashes the following commits: c06d7b5 [Michael Armbrust] fix compilation 91c8829 [Michael Armbrust] [SQL][SPARK-5532] Repartition should not use external rdd representation --- .../scala/org/apache/spark/sql/DataFrame.scala | 5 +++-- .../spark/sql/execution/debug/package.scala | 1 + .../apache/spark/sql/UserDefinedTypeSuite.scala | 16 +++++++++++++++- 3 files changed, 19 insertions(+), 3 deletions(-) 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 27ac398063d43..04bf5d9b0f931 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 @@ -799,14 +799,15 @@ class DataFrame protected[sql]( * Returns the number of rows in the [[DataFrame]]. * @group action */ - override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) + override def count(): Long = groupBy().count().collect().head.getLong(0) /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. * @group rdd */ override def repartition(numPartitions: Int): DataFrame = { - sqlContext.createDataFrame(rdd.repartition(numPartitions), schema) + sqlContext.createDataFrame( + queryExecution.toRdd.map(_.copy()).repartition(numPartitions), schema) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 73162b22fa9cd..ffe388cfa9532 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -168,6 +168,7 @@ package object debug { case (_: Short, ShortType) => case (_: Boolean, BooleanType) => case (_: Double, DoubleType) => + case (v, udt: UserDefinedType[_]) => typeCheck(v, udt.sqlType) case (d, t) => sys.error(s"Invalid data found: got $d (${d.getClass}) expected $t") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 5f21d990e2e5b..9c098df24c65f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql +import java.io.File + import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -91,4 +92,17 @@ class UserDefinedTypeSuite extends QueryTest { sql("SELECT testType(features) from points"), Seq(Row(true), Row(true))) } + + + test("UDTs with Parquet") { + val tempDir = File.createTempFile("parquet", "test") + tempDir.delete() + pointsRDD.saveAsParquetFile(tempDir.getCanonicalPath) + } + + test("Repartition UDTs with Parquet") { + val tempDir = File.createTempFile("parquet", "test") + tempDir.delete() + pointsRDD.repartition(1).saveAsParquetFile(tempDir.getCanonicalPath) + } } From 64d2c01ff1048de83b9b8efce987b55e457298f9 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 24 Feb 2015 11:02:47 -0800 Subject: [PATCH 232/817] [Spark-5967] [UI] Correctly clean JobProgressListener.stageIdToActiveJobIds Patch should be self-explanatory pwendell JoshRosen Author: Tathagata Das Closes #4741 from tdas/SPARK-5967 and squashes the following commits: 653b5bb [Tathagata Das] Fixed the fix and added test e2de972 [Tathagata Das] Clear stages which have no corresponding active jobs. --- .../spark/ui/jobs/JobProgressListener.scala | 3 +++ .../ui/jobs/JobProgressListenerSuite.scala | 22 +++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 0b6fe70bd2062..937d95a934b59 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -203,6 +203,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { for (stageId <- jobData.stageIds) { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => jobsUsingStage.remove(jobEnd.jobId) + if (jobsUsingStage.isEmpty) { + stageIdToActiveJobIds.remove(stageId) + } stageIdToInfo.get(stageId).foreach { stageInfo => if (stageInfo.submissionTime.isEmpty) { // if this stage is pending, it won't complete, so mark it as "skipped": diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 6019282d2fb70..730a4b54f5aa1 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -88,6 +88,28 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc listener.completedStages.map(_.stageId).toSet should be (Set(50, 49, 48, 47, 46)) } + test("test clearing of stageIdToActiveJobs") { + val conf = new SparkConf() + conf.set("spark.ui.retainedStages", 5.toString) + val listener = new JobProgressListener(conf) + val jobId = 0 + val stageIds = 1 to 50 + // Start a job with 50 stages + listener.onJobStart(createJobStartEvent(jobId, stageIds)) + for (stageId <- stageIds) { + listener.onStageSubmitted(createStageStartEvent(stageId)) + } + listener.stageIdToActiveJobIds.size should be > 0 + + // Complete the stages and job + for (stageId <- stageIds) { + listener.onStageCompleted(createStageEndEvent(stageId, failed = false)) + } + listener.onJobEnd(createJobEndEvent(jobId, false)) + assertActiveJobsStateIsEmpty(listener) + listener.stageIdToActiveJobIds.size should be (0) + } + test("test LRU eviction of jobs") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) From 6d2caa576fcdc5c848d1472b09c685b3871e220e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 24 Feb 2015 11:08:07 -0800 Subject: [PATCH 233/817] [SPARK-5965] Standalone Worker UI displays {{USER_JAR}} For screenshot see: https://issues.apache.org/jira/browse/SPARK-5965 This was caused by 20a6013106b56a1a1cc3e8cda092330ffbe77cc3. Author: Andrew Or Closes #4739 from andrewor14/user-jar-blocker and squashes the following commits: 23c4a9e [Andrew Or] Use right argument --- .../scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 327b905032800..720f13bfa829b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -134,7 +134,7 @@ private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { def driverRow(driver: DriverRunner): Seq[Node] = { {driver.driverId} - {driver.driverDesc.command.arguments(1)} + {driver.driverDesc.command.arguments(2)} {driver.finalState.getOrElse(DriverState.RUNNING)} {driver.driverDesc.cores.toString} From 105791e35cee694f3b2ac1e06758650fe44e2c71 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Feb 2015 11:38:59 -0800 Subject: [PATCH 234/817] [MLLIB] Change x_i to y_i in Variance's user guide Variance is calculated on labels/responses. Author: Xiangrui Meng Closes #4740 from mengxr/patch-1 and squashes the following commits: 673317b [Xiangrui Meng] [MLLIB] Change x_i to y_i in Variance's user guide --- docs/mllib-decision-tree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index d1537def851e7..6675133a810db 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -54,8 +54,8 @@ impurity measure for regression (variance). Variance Regression - $\frac{1}{N} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, - $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N x_i$. + $\frac{1}{N} \sum_{i=1}^{N} (y_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N y_i$. From c5ba975ee85521f708ebeec81144347cf1b40fba Mon Sep 17 00:00:00 2001 From: Judy Date: Tue, 24 Feb 2015 20:50:16 +0000 Subject: [PATCH 235/817] [Spark-5708] Add Slf4jSink to Spark Metrics Add Slf4jSink to Spark Metrics using Coda Hale's SlfjReporter. This sends metrics to log4j, allowing spark users to reuse log4j pipeline for metrics collection. Reviewed existing unit tests and didn't see any sink-related tests. Please advise on if tests should be added. Author: Judy Author: judynash Closes #4644 from judynash/master and squashes the following commits: 57ef214 [judynash] doc clarification and indent fixes a751a66 [Judy] Spark-5708: Add Slf4jSink to Spark Metrics --- conf/metrics.properties.template | 9 +++ .../apache/spark/metrics/sink/Slf4jSink.scala | 68 +++++++++++++++++++ docs/monitoring.md | 1 + 3 files changed, 78 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 464c14457e53f..2e0cb5db170ac 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -122,6 +122,15 @@ #worker.sink.csv.unit=minutes +# Enable Slf4jSink for all instances by class name +#*.sink.slf4j.class=org.apache.spark.metrics.sink.Slf4jSink + +# Polling period for Slf4JSink +#*.sink.sl4j.period=1 + +#*.sink.sl4j.unit=minutes + + # Enable jvm source for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala new file mode 100644 index 0000000000000..e8b3074e8f1a6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{Slf4jReporter, MetricRegistry} + +import org.apache.spark.SecurityManager +import org.apache.spark.metrics.MetricsSystem + +private[spark] class Slf4jSink( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink { + val SLF4J_DEFAULT_PERIOD = 10 + val SLF4J_DEFAULT_UNIT = "SECONDS" + + val SLF4J_KEY_PERIOD = "period" + val SLF4J_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(SLF4J_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => SLF4J_DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = Option(property.getProperty(SLF4J_KEY_UNIT)) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(SLF4J_DEFAULT_UNIT) + } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter: Slf4jReporter = Slf4jReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + override def start() { + reporter.start(pollPeriod, pollUnit) + } + + override def stop() { + reporter.stop() + } + + override def report() { + reporter.report() + } +} + diff --git a/docs/monitoring.md b/docs/monitoring.md index 7a5cadc171d6d..009a344dff4bb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -176,6 +176,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the * `JmxSink`: Registers metrics for viewing in a JMX console. * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data. * `GraphiteSink`: Sends metrics to a Graphite node. +* `Slf4jSink`: Sends metrics to slf4j as log entries. Spark also supports a Ganglia sink which is not included in the default build due to licensing restrictions: From a2b9137923e0ba328da8fff2fbbfcf2abf50b033 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Feb 2015 13:39:29 -0800 Subject: [PATCH 236/817] [SPARK-5952][SQL] Lock when using hive metastore client Author: Michael Armbrust Closes #4746 from marmbrus/hiveLock and squashes the following commits: 8b871cf [Michael Armbrust] [SPARK-5952][SQL] Lock when using hive metastore client --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f7ad2efc9544e..2cc8d65d3cb79 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -52,6 +52,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with /** Connection to hive metastore. Usages should lock on `this`. */ protected[hive] val client = Hive.get(hive.hiveconf) + /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) // TODO: Use this everywhere instead of tuples or databaseName, tableName,. @@ -65,7 +66,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = client.getTable(in.database, in.name) + val table = synchronized { + client.getTable(in.database, in.name) + } val schemaString = table.getProperty("spark.sql.sources.schema") val userSpecifiedSchema = if (schemaString == null) { @@ -134,15 +137,18 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } - def hiveDefaultTableFilePath(tableName: String): String = { + def hiveDefaultTableFilePath(tableName: String): String = synchronized { val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase) + hiveWarehouse.getTablePath(currentDatabase, tableName).toString } - def tableExists(tableIdentifier: Seq[String]): Boolean = { + def tableExists(tableIdentifier: Seq[String]): Boolean = synchronized { val tableIdent = processTableIdentifier(tableIdentifier) - val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( - hive.sessionState.getCurrentDatabase) + val databaseName = + tableIdent + .lift(tableIdent.size - 2) + .getOrElse(hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last client.getTable(databaseName, tblName, false) != null } @@ -219,7 +225,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } - override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { + override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { val dbName = if (!caseSensitive) { if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None } else { From da505e59274d1c838653c1109db65ad374e65304 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 24 Feb 2015 14:50:00 -0800 Subject: [PATCH 237/817] [SPARK-5973] [PySpark] fix zip with two RDDs with AutoBatchedSerializer Author: Davies Liu Closes #4745 from davies/fix_zip and squashes the following commits: 2124b2c [Davies Liu] Update tests.py b5c828f [Davies Liu] increase the number of records c1e40fd [Davies Liu] fix zip with two RDDs with AutoBatchedSerializer --- python/pyspark/rdd.py | 2 +- python/pyspark/tests.py | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index ba2347ae76844..d3148de6f41a3 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1950,7 +1950,7 @@ def batch_as(rdd, batchSize): my_batch = get_batch_size(self._jrdd_deserializer) other_batch = get_batch_size(other._jrdd_deserializer) - if my_batch != other_batch: + if my_batch != other_batch or not my_batch: # use the smallest batchSize for both of them batchSize = min(my_batch, other_batch) if batchSize <= 0: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 52e82091c9f81..06ba2b461d53e 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -543,6 +543,12 @@ def test_zip_with_different_serializers(self): # regression test for bug in _reserializer() self.assertEqual(cnt, t.zip(rdd).count()) + def test_zip_with_different_object_sizes(self): + # regress test for SPARK-5973 + a = self.sc.parallelize(range(10000)).map(lambda i: '*' * i) + b = self.sc.parallelize(range(10000, 20000)).map(lambda i: '*' * i) + self.assertEqual(10000, a.zip(b).count()) + def test_zip_with_different_number_of_items(self): a = self.sc.parallelize(range(5), 2) # different number of partitions From 2a0fe34891882e0fde1b5722d8227aa99acc0f1f Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 24 Feb 2015 15:13:22 -0800 Subject: [PATCH 238/817] [SPARK-5436] [MLlib] Validate GradientBoostedTrees using runWithValidation One can early stop if the decrease in error rate is lesser than a certain tol or if the error increases if the training data is overfit. This introduces a new method runWithValidation which takes in a pair of RDD's , one for the training data and the other for the validation. Author: MechCoder Closes #4677 from MechCoder/spark-5436 and squashes the following commits: 1bb21d4 [MechCoder] Combine regression and classification tests into a single one e4d799b [MechCoder] Addresses indentation and doc comments b48a70f [MechCoder] COSMIT b928a19 [MechCoder] Move validation while training section under usage tips fad9b6e [MechCoder] Made the following changes 1. Add section to documentation 2. Return corresponding to bestValidationError 3. Allow negative tolerance. 55e5c3b [MechCoder] One liner for prevValidateError 3e74372 [MechCoder] TST: Add test for classification 77549a9 [MechCoder] [SPARK-5436] Validate GradientBoostedTrees using runWithValidation --- docs/mllib-ensembles.md | 11 +++ .../mllib/tree/GradientBoostedTrees.scala | 75 +++++++++++++++++-- .../tree/configuration/BoostingStrategy.scala | 6 +- .../tree/GradientBoostedTreesSuite.scala | 36 +++++++++ 4 files changed, 122 insertions(+), 6 deletions(-) diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index fb90b7039971c..00040e6073d0d 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -427,6 +427,17 @@ We omit some decision tree parameters since those are covered in the [decision t * **`algo`**: The algorithm or task (classification vs. regression) is set using the tree [Strategy] parameter. +#### Validation while training + +Gradient boosting can overfit when trained with more trees. In order to prevent overfitting, it is useful to validate while +training. The method runWithValidation has been provided to make use of this option. It takes a pair of RDD's as arguments, the +first one being the training dataset and the second being the validation dataset. + +The training is stopped when the improvement in the validation error is not more than a certain tolerance +(supplied by the `validationTol` argument in `BoostingStrategy`). In practice, the validation error +decreases initially and later increases. There might be cases in which the validation error does not change monotonically, +and the user is advised to set a large enough negative tolerance and examine the validation curve to to tune the number of +iterations. ### Examples diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 61f6b1313f82e..b4466ff40937f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -60,11 +60,12 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo algo match { - case Regression => GradientBoostedTrees.boost(input, boostingStrategy) + case Regression => GradientBoostedTrees.boost(input, input, boostingStrategy, validate=false) case Classification => // Map labels to -1, +1 so binary classification can be treated as regression. val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) - GradientBoostedTrees.boost(remappedInput, boostingStrategy) + GradientBoostedTrees.boost(remappedInput, + remappedInput, boostingStrategy, validate=false) case _ => throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") } @@ -76,8 +77,46 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) def run(input: JavaRDD[LabeledPoint]): GradientBoostedTreesModel = { run(input.rdd) } -} + /** + * Method to validate a gradient boosting model + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @param validationInput Validation dataset: + RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + Should be different from and follow the same distribution as input. + e.g., these two datasets could be created from an original dataset + by using [[org.apache.spark.rdd.RDD.randomSplit()]] + * @return a gradient boosted trees model that can be used for prediction + */ + def runWithValidation( + input: RDD[LabeledPoint], + validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { + val algo = boostingStrategy.treeStrategy.algo + algo match { + case Regression => GradientBoostedTrees.boost( + input, validationInput, boostingStrategy, validate=true) + case Classification => + // Map labels to -1, +1 so binary classification can be treated as regression. + val remappedInput = input.map( + x => new LabeledPoint((x.label * 2) - 1, x.features)) + val remappedValidationInput = validationInput.map( + x => new LabeledPoint((x.label * 2) - 1, x.features)) + GradientBoostedTrees.boost(remappedInput, remappedValidationInput, boostingStrategy, + validate=true) + case _ => + throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") + } + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees!#runWithValidation]]. + */ + def runWithValidation( + input: JavaRDD[LabeledPoint], + validationInput: JavaRDD[LabeledPoint]): GradientBoostedTreesModel = { + runWithValidation(input.rdd, validationInput.rdd) + } +} object GradientBoostedTrees extends Logging { @@ -108,12 +147,16 @@ object GradientBoostedTrees extends Logging { /** * Internal method for performing regression using trees as base learners. * @param input training dataset + * @param validationInput validation dataset, ignored if validate is set to false. * @param boostingStrategy boosting parameters + * @param validate whether or not to use the validation dataset. * @return a gradient boosted trees model that can be used for prediction */ private def boost( input: RDD[LabeledPoint], - boostingStrategy: BoostingStrategy): GradientBoostedTreesModel = { + validationInput: RDD[LabeledPoint], + boostingStrategy: BoostingStrategy, + validate: Boolean): GradientBoostedTreesModel = { val timer = new TimeTracker() timer.start("total") @@ -129,6 +172,7 @@ object GradientBoostedTrees extends Logging { val learningRate = boostingStrategy.learningRate // Prepare strategy for individual trees, which use regression with variance impurity. val treeStrategy = boostingStrategy.treeStrategy.copy + val validationTol = boostingStrategy.validationTol treeStrategy.algo = Regression treeStrategy.impurity = Variance treeStrategy.assertValid() @@ -152,13 +196,16 @@ object GradientBoostedTrees extends Logging { baseLearnerWeights(0) = 1.0 val startingModel = new GradientBoostedTreesModel(Regression, Array(firstTreeModel), Array(1.0)) logDebug("error of gbt = " + loss.computeError(startingModel, input)) + // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") + var bestValidateError = if (validate) loss.computeError(startingModel, validationInput) else 0.0 + var bestM = 1 + // psuedo-residual for second iteration data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), point.features)) - var m = 1 while (m < numIterations) { timer.start(s"building tree $m") @@ -177,6 +224,23 @@ object GradientBoostedTrees extends Logging { val partialModel = new GradientBoostedTreesModel( Regression, baseLearners.slice(0, m + 1), baseLearnerWeights.slice(0, m + 1)) logDebug("error of gbt = " + loss.computeError(partialModel, input)) + + if (validate) { + // Stop training early if + // 1. Reduction in error is less than the validationTol or + // 2. If the error increases, that is if the model is overfit. + // We want the model returned corresponding to the best validation error. + val currentValidateError = loss.computeError(partialModel, validationInput) + if (bestValidateError - currentValidateError < validationTol) { + return new GradientBoostedTreesModel( + boostingStrategy.treeStrategy.algo, + baseLearners.slice(0, bestM), + baseLearnerWeights.slice(0, bestM)) + } else if (currentValidateError < bestValidateError) { + bestValidateError = currentValidateError + bestM = m + 1 + } + } // Update data with pseudo-residuals data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), point.features)) @@ -191,4 +255,5 @@ object GradientBoostedTrees extends Logging { new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, baseLearners, baseLearnerWeights) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index ed8e6a796f8c4..664c8df019233 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -34,6 +34,9 @@ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} * weak hypotheses used in the final model. * @param learningRate Learning rate for shrinking the contribution of each estimator. The * learning rate should be between in the interval (0, 1] + * @param validationTol Useful when runWithValidation is used. If the error rate on the + * validation input between two iterations is less than the validationTol + * then stop. Ignored when [[run]] is used. */ @Experimental case class BoostingStrategy( @@ -42,7 +45,8 @@ case class BoostingStrategy( @BeanProperty var loss: Loss, // Optional boosting parameters @BeanProperty var numIterations: Int = 100, - @BeanProperty var learningRate: Double = 0.1) extends Serializable { + @BeanProperty var learningRate: Double = 0.1, + @BeanProperty var validationTol: Double = 1e-5) extends Serializable { /** * Check validity of parameters. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index bde47606eb001..b437aeaaf0547 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -158,6 +158,40 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { } } } + + test("runWithValidation stops early and performs better on a validation dataset") { + // Set numIterations large enough so that it stops early. + val numIterations = 20 + val trainRdd = sc.parallelize(GradientBoostedTreesSuite.trainData, 2) + val validateRdd = sc.parallelize(GradientBoostedTreesSuite.validateData, 2) + + val algos = Array(Regression, Regression, Classification) + val losses = Array(SquaredError, AbsoluteError, LogLoss) + (algos zip losses) map { + case (algo, loss) => { + val treeStrategy = new Strategy(algo = algo, impurity = Variance, maxDepth = 2, + categoricalFeaturesInfo = Map.empty) + val boostingStrategy = + new BoostingStrategy(treeStrategy, loss, numIterations, validationTol = 0.0) + val gbtValidate = new GradientBoostedTrees(boostingStrategy) + .runWithValidation(trainRdd, validateRdd) + assert(gbtValidate.numTrees !== numIterations) + + // Test that it performs better on the validation dataset. + val gbt = GradientBoostedTrees.train(trainRdd, boostingStrategy) + val (errorWithoutValidation, errorWithValidation) = { + if (algo == Classification) { + val remappedRdd = validateRdd.map(x => new LabeledPoint(2 * x.label - 1, x.features)) + (loss.computeError(gbt, remappedRdd), loss.computeError(gbtValidate, remappedRdd)) + } else { + (loss.computeError(gbt, validateRdd), loss.computeError(gbtValidate, validateRdd)) + } + } + assert(errorWithValidation <= errorWithoutValidation) + } + } + } + } private object GradientBoostedTreesSuite { @@ -166,4 +200,6 @@ private object GradientBoostedTreesSuite { val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) val data = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) + val trainData = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120) + val validateData = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80) } From f816e73902b8ca28e24bf1f79a70533f75f239db Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Feb 2015 08:34:55 +0800 Subject: [PATCH 239/817] [SPARK-5751] [SQL] [WIP] Revamped HiveThriftServer2Suite for robustness **NOTICE** Do NOT merge this, as we're waiting for #3881 to be merged. `HiveThriftServer2Suite` has been notorious for its flakiness for a while. This was mostly due to spawning and communicate with external server processes. This PR revamps this test suite for better robustness: 1. Fixes a racing condition occurred while using `tail -f` to check log file It's possible that the line we are looking for has already been printed into the log file before we start the `tail -f` process. This PR uses `tail -n +0 -f` to ensure all lines are checked. 2. Retries up to 3 times if the server fails to start In most of the cases, the server fails to start because of port conflict. This PR no longer asks the system to choose an available TCP port, but uses a random port first, and retries up to 3 times if the server fails to start. 3. A server instance is reused among all test cases within a single suite The original `HiveThriftServer2Suite` is splitted into two test suites, `HiveThriftBinaryServerSuite` and `HiveThriftHttpServerSuite`. Each suite starts a `HiveThriftServer2` instance and reuses it for all of its test cases. **TODO** - [ ] Starts the Thrift server in foreground once #3881 is merged (adding `--foreground` flag to `spark-daemon.sh`) [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4720) Author: Cheng Lian Closes #4720 from liancheng/revamp-thrift-server-tests and squashes the following commits: d6c80eb [Cheng Lian] Relaxes server startup timeout 6f14eb1 [Cheng Lian] Revamped HiveThriftServer2Suite for robustness --- .../thriftserver/HiveThriftServer2Suite.scala | 387 ----------------- .../HiveThriftServer2Suites.scala | 403 ++++++++++++++++++ 2 files changed, 403 insertions(+), 387 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala deleted file mode 100644 index b52a51d11e4ad..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ /dev/null @@ -1,387 +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.thriftserver - -import java.io.File -import java.net.ServerSocket -import java.sql.{Date, DriverManager, Statement} -import java.util.concurrent.TimeoutException - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration._ -import scala.concurrent.{Await, Promise} -import scala.sys.process.{Process, ProcessLogger} -import scala.util.Try - -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.jdbc.HiveDriver -import org.apache.hive.service.auth.PlainSaslHelper -import org.apache.hive.service.cli.GetInfoType -import org.apache.hive.service.cli.thrift.TCLIService.Client -import org.apache.hive.service.cli.thrift._ -import org.apache.thrift.protocol.TBinaryProtocol -import org.apache.thrift.transport.TSocket -import org.scalatest.FunSuite - -import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.hive.HiveShim - -/** - * Tests for the HiveThriftServer2 using JDBC. - * - * NOTE: SPARK_PREPEND_CLASSES is explicitly disabled in this test suite. Assembly jar must be - * rebuilt after changing HiveThriftServer2 related code. - */ -class HiveThriftServer2Suite extends FunSuite with Logging { - Class.forName(classOf[HiveDriver].getCanonicalName) - - object TestData { - def getTestDataFilePath(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") - } - - val smallKv = getTestDataFilePath("small_kv.txt") - val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") - } - - def randomListeningPort = { - // Let the system to choose a random available port to avoid collision with other parallel - // builds. - val socket = new ServerSocket(0) - val port = socket.getLocalPort - socket.close() - port - } - - def withJdbcStatement( - serverStartTimeout: FiniteDuration = 1.minute, - httpMode: Boolean = false)( - f: Statement => Unit) { - val port = randomListeningPort - - startThriftServer(port, serverStartTimeout, httpMode) { - val jdbcUri = if (httpMode) { - s"jdbc:hive2://${"localhost"}:$port/" + - "default?hive.server2.transport.mode=http;hive.server2.thrift.http.path=cliservice" - } else { - s"jdbc:hive2://${"localhost"}:$port/" - } - - val user = System.getProperty("user.name") - val connection = DriverManager.getConnection(jdbcUri, user, "") - val statement = connection.createStatement() - - try { - f(statement) - } finally { - statement.close() - connection.close() - } - } - } - - def withCLIServiceClient( - serverStartTimeout: FiniteDuration = 1.minute)( - f: ThriftCLIServiceClient => Unit) { - val port = randomListeningPort - - startThriftServer(port) { - // Transport creation logics below mimics HiveConnection.createBinaryTransport - val rawTransport = new TSocket("localhost", port) - val user = System.getProperty("user.name") - val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) - val protocol = new TBinaryProtocol(transport) - val client = new ThriftCLIServiceClient(new Client(protocol)) - - transport.open() - - try { - f(client) - } finally { - transport.close() - } - } - } - - def startThriftServer( - port: Int, - serverStartTimeout: FiniteDuration = 1.minute, - httpMode: Boolean = false)( - f: => Unit) { - val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) - val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) - - val warehousePath = getTempFilePath("warehouse") - val metastorePath = getTempFilePath("metastore") - val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - - val command = - if (httpMode) { - s"""$startScript - | --master local - | --hiveconf hive.root.logger=INFO,console - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost - | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=http - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT}=$port - | --driver-class-path ${sys.props("java.class.path")} - | --conf spark.ui.enabled=false - """.stripMargin.split("\\s+").toSeq - } else { - s"""$startScript - | --master local - | --hiveconf hive.root.logger=INFO,console - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port - | --driver-class-path ${sys.props("java.class.path")} - | --conf spark.ui.enabled=false - """.stripMargin.split("\\s+").toSeq - } - - val serverRunning = Promise[Unit]() - val buffer = new ArrayBuffer[String]() - val LOGGING_MARK = - s"starting ${HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$")}, logging to " - var logTailingProcess: Process = null - var logFilePath: String = null - - def captureLogOutput(line: String): Unit = { - buffer += line - if (line.contains("ThriftBinaryCLIService listening on") || - line.contains("Started ThriftHttpCLIService in http")) { - serverRunning.success(()) - } - } - - def captureThriftServerOutput(source: String)(line: String): Unit = { - if (line.startsWith(LOGGING_MARK)) { - logFilePath = line.drop(LOGGING_MARK.length).trim - // Ensure that the log file is created so that the `tail' command won't fail - Try(new File(logFilePath).createNewFile()) - logTailingProcess = Process(s"/usr/bin/env tail -f $logFilePath") - .run(ProcessLogger(captureLogOutput, _ => ())) - } - } - - val env = Seq( - // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths - "SPARK_TESTING" -> "0") - - Process(command, None, env: _*).run(ProcessLogger( - captureThriftServerOutput("stdout"), - captureThriftServerOutput("stderr"))) - - try { - Await.result(serverRunning.future, serverStartTimeout) - f - } catch { - case cause: Exception => - cause match { - case _: TimeoutException => - logError(s"Failed to start Hive Thrift server within $serverStartTimeout", cause) - case _ => - } - logError( - s""" - |===================================== - |HiveThriftServer2Suite failure output - |===================================== - |HiveThriftServer2 command line: ${command.mkString(" ")} - |Binding port: $port - |System user: ${System.getProperty("user.name")} - | - |${buffer.mkString("\n")} - |========================================= - |End HiveThriftServer2Suite failure output - |========================================= - """.stripMargin, cause) - throw cause - } finally { - warehousePath.delete() - metastorePath.delete() - Process(stopScript, None, env: _*).run().exitValue() - // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while. - Thread.sleep(3.seconds.toMillis) - Option(logTailingProcess).map(_.destroy()) - Option(logFilePath).map(new File(_).delete()) - } - } - - test("Test JDBC query execution") { - withJdbcStatement() { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") - - queries.foreach(statement.execute) - - assertResult(5, "Row count mismatch") { - val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") - resultSet.next() - resultSet.getInt(1) - } - } - } - - test("Test JDBC query execution in Http Mode") { - withJdbcStatement(httpMode = true) { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") - - queries.foreach(statement.execute) - - assertResult(5, "Row count mismatch") { - val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") - resultSet.next() - resultSet.getInt(1) - } - } - } - - test("SPARK-3004 regression: result set containing NULL") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_null", - "CREATE TABLE test_null(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") - - queries.foreach(statement.execute) - - val resultSet = statement.executeQuery("SELECT * FROM test_null WHERE key IS NULL") - - (0 until 5).foreach { _ => - resultSet.next() - assert(resultSet.getInt(1) === 0) - assert(resultSet.wasNull()) - } - - assert(!resultSet.next()) - } - } - - test("GetInfo Thrift API") { - withCLIServiceClient() { client => - val user = System.getProperty("user.name") - val sessionHandle = client.openSession(user, "") - - assertResult("Spark SQL", "Wrong GetInfo(CLI_DBMS_NAME) result") { - client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_NAME).getStringValue - } - - assertResult("Spark SQL", "Wrong GetInfo(CLI_SERVER_NAME) result") { - client.getInfo(sessionHandle, GetInfoType.CLI_SERVER_NAME).getStringValue - } - - assertResult(true, "Spark version shouldn't be \"Unknown\"") { - val version = client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_VER).getStringValue - logInfo(s"Spark version: $version") - version != "Unknown" - } - } - } - - test("Checks Hive version") { - withJdbcStatement() { statement => - val resultSet = statement.executeQuery("SET spark.sql.hive.version") - resultSet.next() - assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") - } - } - - test("Checks Hive version in Http Mode") { - withJdbcStatement(httpMode = true) { statement => - val resultSet = statement.executeQuery("SET spark.sql.hive.version") - resultSet.next() - assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") - } - } - - test("SPARK-4292 regression: result set iterator issue") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_4292", - "CREATE TABLE test_4292(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") - - queries.foreach(statement.execute) - - val resultSet = statement.executeQuery("SELECT key FROM test_4292") - - Seq(238, 86, 311, 27, 165).foreach { key => - resultSet.next() - assert(resultSet.getInt(1) === key) - } - - statement.executeQuery("DROP TABLE IF EXISTS test_4292") - } - } - - test("SPARK-4309 regression: Date type support") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_date", - "CREATE TABLE test_date(key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") - - queries.foreach(statement.execute) - - assertResult(Date.valueOf("2011-01-01")) { - val resultSet = statement.executeQuery( - "SELECT CAST('2011-01-01' as date) FROM test_date LIMIT 1") - resultSet.next() - resultSet.getDate(1) - } - } - } - - test("SPARK-4407 regression: Complex type support") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_map", - "CREATE TABLE test_map(key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") - - queries.foreach(statement.execute) - - assertResult("""{238:"val_238"}""") { - val resultSet = statement.executeQuery("SELECT MAP(key, value) FROM test_map LIMIT 1") - resultSet.next() - resultSet.getString(1) - } - - assertResult("""["238","val_238"]""") { - val resultSet = statement.executeQuery( - "SELECT ARRAY(CAST(key AS STRING), value) FROM test_map LIMIT 1") - resultSet.next() - resultSet.getString(1) - } - } - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala new file mode 100644 index 0000000000000..77ef37253e38f --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -0,0 +1,403 @@ +/* + * 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.thriftserver + +import java.io.File +import java.sql.{Date, DriverManager, Statement} + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.concurrent.{Await, Promise} +import scala.sys.process.{Process, ProcessLogger} +import scala.util.{Random, Try} + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.jdbc.HiveDriver +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.GetInfoType +import org.apache.hive.service.cli.thrift.TCLIService.Client +import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.TSocket +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.hive.HiveShim + +object TestData { + def getTestDataFilePath(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") + } + + val smallKv = getTestDataFilePath("small_kv.txt") + val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") +} + +class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { + override def mode = ServerMode.binary + + private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { + // Transport creation logics below mimics HiveConnection.createBinaryTransport + val rawTransport = new TSocket("localhost", serverPort) + val user = System.getProperty("user.name") + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val protocol = new TBinaryProtocol(transport) + val client = new ThriftCLIServiceClient(new Client(protocol)) + + transport.open() + try f(client) finally transport.close() + } + + test("GetInfo Thrift API") { + withCLIServiceClient { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + assertResult("Spark SQL", "Wrong GetInfo(CLI_DBMS_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_NAME).getStringValue + } + + assertResult("Spark SQL", "Wrong GetInfo(CLI_SERVER_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_SERVER_NAME).getStringValue + } + + assertResult(true, "Spark version shouldn't be \"Unknown\"") { + val version = client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_VER).getStringValue + logInfo(s"Spark version: $version") + version != "Unknown" + } + } + } + + test("JDBC query execution") { + withJdbcStatement { statement => + val queries = Seq( + "SET spark.sql.shuffle.partitions=3", + "DROP TABLE IF EXISTS test", + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } + } + + test("Checks Hive version") { + withJdbcStatement { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") + } + } + + test("SPARK-3004 regression: result set containing NULL") { + withJdbcStatement { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_null", + "CREATE TABLE test_null(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") + + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + + (0 until 5).foreach { _ => + resultSet.next() + assert(resultSet.getInt(1) === 0) + assert(resultSet.wasNull()) + } + + assert(!resultSet.next()) + } + } + + test("SPARK-4292 regression: result set iterator issue") { + withJdbcStatement { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_4292", + "CREATE TABLE test_4292(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") + + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT key FROM test_4292") + + Seq(238, 86, 311, 27, 165).foreach { key => + resultSet.next() + assert(resultSet.getInt(1) === key) + } + + statement.executeQuery("DROP TABLE IF EXISTS test_4292") + } + } + + test("SPARK-4309 regression: Date type support") { + withJdbcStatement { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_date", + "CREATE TABLE test_date(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") + + queries.foreach(statement.execute) + + assertResult(Date.valueOf("2011-01-01")) { + val resultSet = statement.executeQuery( + "SELECT CAST('2011-01-01' as date) FROM test_date LIMIT 1") + resultSet.next() + resultSet.getDate(1) + } + } + } + + test("SPARK-4407 regression: Complex type support") { + withJdbcStatement { statement => + val queries = Seq( + "DROP TABLE IF EXISTS test_map", + "CREATE TABLE test_map(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") + + queries.foreach(statement.execute) + + assertResult("""{238:"val_238"}""") { + val resultSet = statement.executeQuery("SELECT MAP(key, value) FROM test_map LIMIT 1") + resultSet.next() + resultSet.getString(1) + } + + assertResult("""["238","val_238"]""") { + val resultSet = statement.executeQuery( + "SELECT ARRAY(CAST(key AS STRING), value) FROM test_map LIMIT 1") + resultSet.next() + resultSet.getString(1) + } + } + } +} + +class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { + override def mode = ServerMode.http + + test("JDBC query execution") { + withJdbcStatement { statement => + val queries = Seq( + "SET spark.sql.shuffle.partitions=3", + "DROP TABLE IF EXISTS test", + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } + } + + test("Checks Hive version") { + withJdbcStatement { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") + } + } +} + +object ServerMode extends Enumeration { + val binary, http = Value +} + +abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { + Class.forName(classOf[HiveDriver].getCanonicalName) + + private def jdbcUri = if (mode == ServerMode.http) { + s"""jdbc:hive2://localhost:$serverPort/ + |default? + |hive.server2.transport.mode=http; + |hive.server2.thrift.http.path=cliservice + """.stripMargin.split("\n").mkString.trim + } else { + s"jdbc:hive2://localhost:$serverPort/" + } + + protected def withJdbcStatement(f: Statement => Unit): Unit = { + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() + + try f(statement) finally { + statement.close() + connection.close() + } + } +} + +abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll with Logging { + def mode: ServerMode.Value + + private val CLASS_NAME = HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$") + private val LOG_FILE_MARK = s"starting $CLASS_NAME, logging to " + + private val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + private val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) + + private var listeningPort: Int = _ + protected def serverPort: Int = listeningPort + + protected def user = System.getProperty("user.name") + + private var warehousePath: File = _ + private var metastorePath: File = _ + private def metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + + private var logPath: File = _ + private var logTailingProcess: Process = _ + private var diagnosisBuffer: ArrayBuffer[String] = ArrayBuffer.empty[String] + + private def serverStartCommand(port: Int) = { + val portConf = if (mode == ServerMode.binary) { + ConfVars.HIVE_SERVER2_THRIFT_PORT + } else { + ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT + } + + s"""$startScript + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost + | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=$mode + | --hiveconf $portConf=$port + | --driver-class-path ${sys.props("java.class.path")} + | --conf spark.ui.enabled=false + """.stripMargin.split("\\s+").toSeq + } + + private def startThriftServer(port: Int, attempt: Int) = { + warehousePath = util.getTempFilePath("warehouse") + metastorePath = util.getTempFilePath("metastore") + logPath = null + logTailingProcess = null + + val command = serverStartCommand(port) + + diagnosisBuffer ++= + s""" + |### Attempt $attempt ### + |HiveThriftServer2 command line: $command + |Listening port: $port + |System user: $user + """.stripMargin.split("\n") + + logInfo(s"Trying to start HiveThriftServer2: port=$port, mode=$mode, attempt=$attempt") + + logPath = Process(command, None, "SPARK_TESTING" -> "0").lines.collectFirst { + case line if line.contains(LOG_FILE_MARK) => new File(line.drop(LOG_FILE_MARK.length)) + }.getOrElse { + throw new RuntimeException("Failed to find HiveThriftServer2 log file.") + } + + val serverStarted = Promise[Unit]() + + // Ensures that the following "tail" command won't fail. + logPath.createNewFile() + logTailingProcess = + // Using "-n +0" to make sure all lines in the log file are checked. + Process(s"/usr/bin/env tail -n +0 -f ${logPath.getCanonicalPath}").run(ProcessLogger( + (line: String) => { + diagnosisBuffer += line + + if (line.contains("ThriftBinaryCLIService listening on") || + line.contains("Started ThriftHttpCLIService in http")) { + serverStarted.trySuccess(()) + } else if (line.contains("HiveServer2 is stopped")) { + // This log line appears when the server fails to start and terminates gracefully (e.g. + // because of port contention). + serverStarted.tryFailure(new RuntimeException("Failed to start HiveThriftServer2")) + } + })) + + Await.result(serverStarted.future, 2.minute) + } + + private def stopThriftServer(): Unit = { + // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while. + Process(stopScript, None).run().exitValue() + Thread.sleep(3.seconds.toMillis) + + warehousePath.delete() + warehousePath = null + + metastorePath.delete() + metastorePath = null + + Option(logPath).foreach(_.delete()) + logPath = null + + Option(logTailingProcess).foreach(_.destroy()) + logTailingProcess = null + } + + private def dumpLogs(): Unit = { + logError( + s""" + |===================================== + |HiveThriftServer2Suite failure output + |===================================== + |${diagnosisBuffer.mkString("\n")} + |========================================= + |End HiveThriftServer2Suite failure output + |========================================= + """.stripMargin) + } + + override protected def beforeAll(): Unit = { + // Chooses a random port between 10000 and 19999 + listeningPort = 10000 + Random.nextInt(10000) + diagnosisBuffer.clear() + + // Retries up to 3 times with different port numbers if the server fails to start + (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) => + started.orElse { + listeningPort += 1 + stopThriftServer() + Try(startThriftServer(listeningPort, attempt)) + } + }.recover { + case cause: Throwable => + dumpLogs() + throw cause + }.get + + logInfo(s"HiveThriftServer2 started successfully") + } + + override protected def afterAll(): Unit = { + stopThriftServer() + logInfo("HiveThriftServer2 stopped") + } +} From 53a1ebf33b5c349ae3a40d7eebf357b839b363af Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 Feb 2015 18:51:41 -0800 Subject: [PATCH 240/817] [SPARK-5904][SQL] DataFrame Java API test suites. Added a new test suite to make sure Java DF programs can use varargs properly. Also moved all suites into test.org.apache.spark package to make sure the suites also test for method visibility. Author: Reynold Xin Closes #4751 from rxin/df-tests and squashes the following commits: 1e8b8e4 [Reynold Xin] Fixed imports and renamed JavaAPISuite. a6ca53b [Reynold Xin] [SPARK-5904][SQL] DataFrame Java API test suites. --- .../apache/spark/sql/api/java/JavaDsl.java | 120 ------------------ .../spark/sql}/JavaApplySchemaSuite.java | 28 ++-- .../apache/spark/sql/JavaDataFrameSuite.java | 84 ++++++++++++ .../org/apache/spark/sql}/JavaRowSuite.java | 2 +- .../org/apache/spark/sql/JavaUDFSuite.java} | 10 +- .../spark/sql/sources/JavaSaveLoadSuite.java | 3 +- .../org/apache/spark/sql/DataFrameSuite.scala | 4 +- 7 files changed, 108 insertions(+), 143 deletions(-) delete mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java rename sql/core/src/test/java/{org/apache/spark/sql/api/java => test/org/apache/spark/sql}/JavaApplySchemaSuite.java (90%) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java rename sql/core/src/test/java/{org/apache/spark/sql/api/java => test/org/apache/spark/sql}/JavaRowSuite.java (99%) rename sql/core/src/test/java/{org/apache/spark/sql/api/java/JavaAPISuite.java => test/org/apache/spark/sql/JavaUDFSuite.java} (94%) rename sql/core/src/test/java/{ => test}/org/apache/spark/sql/sources/JavaSaveLoadSuite.java (98%) diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java deleted file mode 100644 index 05233dc5ffc58..0000000000000 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaDsl.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import com.google.common.collect.ImmutableMap; - -import org.apache.spark.sql.Column; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.types.DataTypes; - -import static org.apache.spark.sql.functions.*; - -/** - * This test doesn't actually run anything. It is here to check the API compatibility for Java. - */ -public class JavaDsl { - - public static void testDataFrame(final DataFrame df) { - DataFrame df1 = df.select("colA"); - df1 = df.select("colA", "colB"); - - df1 = df.select(col("colA"), col("colB"), lit("literal value").$plus(1)); - - df1 = df.filter(col("colA")); - - java.util.Map aggExprs = ImmutableMap.builder() - .put("colA", "sum") - .put("colB", "avg") - .build(); - - df1 = df.agg(aggExprs); - - df1 = df.groupBy("groupCol").agg(aggExprs); - - df1 = df.join(df1, col("key1").$eq$eq$eq(col("key2")), "outer"); - - df.orderBy("colA"); - df.orderBy("colA", "colB", "colC"); - df.orderBy(col("colA").desc()); - df.orderBy(col("colA").desc(), col("colB").asc()); - - df.sort("colA"); - df.sort("colA", "colB", "colC"); - df.sort(col("colA").desc()); - df.sort(col("colA").desc(), col("colB").asc()); - - df.as("b"); - - df.limit(5); - - df.unionAll(df1); - df.intersect(df1); - df.except(df1); - - df.sample(true, 0.1, 234); - - df.head(); - df.head(5); - df.first(); - df.count(); - } - - public static void testColumn(final Column c) { - c.asc(); - c.desc(); - - c.endsWith("abcd"); - c.startsWith("afgasdf"); - - c.like("asdf%"); - c.rlike("wef%asdf"); - - c.as("newcol"); - - c.cast("int"); - c.cast(DataTypes.IntegerType); - } - - public static void testDsl() { - // Creating a column. - Column c = col("abcd"); - Column c1 = column("abcd"); - - // Literals - Column l1 = lit(1); - Column l2 = lit(1.0); - Column l3 = lit("abcd"); - - // Functions - Column a = upper(c); - a = lower(c); - a = sqrt(c); - a = abs(c); - - // Aggregates - a = min(c); - a = max(c); - a = sum(c); - a = sumDistinct(c); - a = countDistinct(c, a); - a = avg(c); - a = first(c); - a = last(c); - } -} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java similarity index 90% rename from sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index 643b891ab1b63..c344a9b095c52 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package test.org.apache.spark.sql; import java.io.Serializable; import java.util.ArrayList; @@ -39,18 +39,18 @@ // see http://stackoverflow.com/questions/758570/. public class JavaApplySchemaSuite implements Serializable { private transient JavaSparkContext javaCtx; - private transient SQLContext javaSqlCtx; + private transient SQLContext sqlContext; @Before public void setUp() { - javaSqlCtx = TestSQLContext$.MODULE$; - javaCtx = new JavaSparkContext(javaSqlCtx.sparkContext()); + sqlContext = TestSQLContext$.MODULE$; + javaCtx = new JavaSparkContext(sqlContext.sparkContext()); } @After public void tearDown() { javaCtx = null; - javaSqlCtx = null; + sqlContext = null; } public static class Person implements Serializable { @@ -98,9 +98,9 @@ public Row call(Person person) throws Exception { fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); StructType schema = DataTypes.createStructType(fields); - DataFrame df = javaSqlCtx.applySchema(rowRDD, schema); + DataFrame df = sqlContext.applySchema(rowRDD, schema); df.registerTempTable("people"); - Row[] actual = javaSqlCtx.sql("SELECT * FROM people").collect(); + Row[] actual = sqlContext.sql("SELECT * FROM people").collect(); List expected = new ArrayList(2); expected.add(RowFactory.create("Michael", 29)); @@ -109,8 +109,6 @@ public Row call(Person person) throws Exception { Assert.assertEquals(expected, Arrays.asList(actual)); } - - @Test public void dataFrameRDDOperations() { List personList = new ArrayList(2); @@ -135,9 +133,9 @@ public Row call(Person person) throws Exception { fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); StructType schema = DataTypes.createStructType(fields); - DataFrame df = javaSqlCtx.applySchema(rowRDD, schema); + DataFrame df = sqlContext.applySchema(rowRDD, schema); df.registerTempTable("people"); - List actual = javaSqlCtx.sql("SELECT * FROM people").toJavaRDD().map(new Function() { + List actual = sqlContext.sql("SELECT * FROM people").toJavaRDD().map(new Function() { public String call(Row row) { return row.getString(0) + "_" + row.get(1).toString(); @@ -189,18 +187,18 @@ public void applySchemaToJSON() { null, "this is another simple string.")); - DataFrame df1 = javaSqlCtx.jsonRDD(jsonRDD); + DataFrame df1 = sqlContext.jsonRDD(jsonRDD); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); df1.registerTempTable("jsonTable1"); - List actual1 = javaSqlCtx.sql("select * from jsonTable1").collectAsList(); + List actual1 = sqlContext.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); - DataFrame df2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); + DataFrame df2 = sqlContext.jsonRDD(jsonRDD, expectedSchema); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); df2.registerTempTable("jsonTable2"); - List actual2 = javaSqlCtx.sql("select * from jsonTable2").collectAsList(); + List actual2 = sqlContext.sql("select * from jsonTable2").collectAsList(); Assert.assertEquals(expectedResult, actual2); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java new file mode 100644 index 0000000000000..c1c51f80d6586 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.*; +import org.apache.spark.sql.test.TestSQLContext$; +import static org.apache.spark.sql.functions.*; + + +public class JavaDataFrameSuite { + private transient SQLContext context; + + @Before + public void setUp() { + // Trigger static initializer of TestData + TestData$.MODULE$.testData(); + context = TestSQLContext$.MODULE$; + } + + @After + public void tearDown() { + context = null; + } + + @Test + public void testExecution() { + DataFrame df = context.table("testData").filter("key = 1"); + Assert.assertEquals(df.select("key").collect()[0].get(0), 1); + } + + /** + * See SPARK-5904. Abstract vararg methods defined in Scala do not work in Java. + */ + @Test + public void testVarargMethods() { + DataFrame df = context.table("testData"); + + df.toDF("key1", "value1"); + + df.select("key", "value"); + df.select(col("key"), col("value")); + df.selectExpr("key", "value + 1"); + + df.sort("key", "value"); + df.sort(col("key"), col("value")); + df.orderBy("key", "value"); + df.orderBy(col("key"), col("value")); + + df.groupBy("key", "value").agg(col("key"), col("value"), sum("value")); + df.groupBy(col("key"), col("value")).agg(col("key"), col("value"), sum("value")); + df.agg(first("key"), sum("value")); + + df.groupBy().avg("key"); + df.groupBy().mean("key"); + df.groupBy().max("key"); + df.groupBy().min("key"); + df.groupBy().sum("key"); + + // Varargs in column expressions + df.groupBy().agg(countDistinct("key", "value")); + df.groupBy().agg(countDistinct(col("key"), col("value"))); + df.select(coalesce(col("key"))); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java similarity index 99% rename from sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java index fbfcd3f59d910..4ce1d1dddb26a 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package test.org.apache.spark.sql; import java.math.BigDecimal; import java.sql.Date; diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java similarity index 94% rename from sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index a21a15409080c..79d92734ff375 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -15,24 +15,26 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package test.org.apache.spark.sql; import java.io.Serializable; -import org.apache.spark.sql.test.TestSQLContext$; import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.api.java.UDF2; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.test.TestSQLContext$; import org.apache.spark.sql.types.DataTypes; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. -public class JavaAPISuite implements Serializable { +public class JavaUDFSuite implements Serializable { private transient JavaSparkContext sc; private transient SQLContext sqlContext; diff --git a/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java similarity index 98% rename from sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java rename to sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index 311f1bdd07510..b76f7d421f643 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -14,7 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.sources; + +package test.org.apache.spark.sql.sources; import java.io.File; import java.io.IOException; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index e71e9bee3a6d8..30e77e4ef30f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -411,7 +411,7 @@ class DataFrameSuite extends QueryTest { ) } - test("addColumn") { + test("withColumn") { val df = testData.toDF().withColumn("newCol", col("key") + 1) checkAnswer( df, @@ -421,7 +421,7 @@ class DataFrameSuite extends QueryTest { assert(df.schema.map(_.name).toSeq === Seq("key", "value", "newCol")) } - test("renameColumn") { + test("withColumnRenamed") { val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") checkAnswer( From fba11c2f55dd81e4f6230e7edca3c7b2e01ccd9d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 Feb 2015 18:59:23 -0800 Subject: [PATCH 241/817] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python. Also added desc/asc function for constructing sorting expressions more conveniently. And added a small fix to lift alias out of cast expression. Author: Reynold Xin Closes #4752 from rxin/SPARK-5985 and squashes the following commits: aeda5ae [Reynold Xin] Added Experimental flag to ColumnName. 047ad03 [Reynold Xin] Lift alias out of cast. c9cf17c [Reynold Xin] [SPARK-5985][SQL] DataFrame sortBy -> orderBy in Python. --- python/pyspark/sql/dataframe.py | 11 +++++-- python/pyspark/sql/functions.py | 3 ++ .../scala/org/apache/spark/sql/Column.scala | 13 +++++++-- .../org/apache/spark/sql/functions.scala | 29 +++++++++++++++++++ .../spark/sql/ColumnExpressionSuite.scala | 4 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 4 +++ 6 files changed, 59 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 010c38f93b9cf..6f746d136b22d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -504,13 +504,18 @@ def join(self, other, joinExprs=None, joinType=None): return DataFrame(jdf, self.sql_ctx) def sort(self, *cols): - """ Return a new :class:`DataFrame` sorted by the specified column. + """ Return a new :class:`DataFrame` sorted by the specified column(s). :param cols: The columns or expressions used for sorting >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] - >>> df.sortBy(df.age.desc()).collect() + >>> df.orderBy(df.age.desc()).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> from pyspark.sql.functions import * + >>> df.sort(asc("age")).collect() + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] + >>> df.orderBy(desc("age"), "name").collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] """ if not cols: @@ -520,7 +525,7 @@ def sort(self, *cols): jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) return DataFrame(jdf, self.sql_ctx) - sortBy = sort + orderBy = sort def head(self, n=None): """ Return the first `n` rows or the first row if n is None. diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index fc61162f0b827..8aa44765205c1 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -48,6 +48,9 @@ def _(col): 'lit': 'Creates a :class:`Column` of literal value.', 'col': 'Returns a :class:`Column` based on the given column name.', 'column': 'Returns a :class:`Column` based on the given column name.', + 'asc': 'Returns a sort expression based on the ascending order of the given column name.', + 'desc': 'Returns a sort expression based on the descending order of the given column name.', + 'upper': 'Converts a string expression to upper case.', 'lower': 'Converts a string expression to upper case.', 'sqrt': 'Computes the square root of the specified float value.', diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 980754322e6c8..a2cc9a9b93eb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -600,7 +600,11 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def cast(to: DataType): Column = Cast(expr, to) + def cast(to: DataType): Column = expr match { + // Lift alias out of cast so we can support col.as("name").cast(IntegerType) + case Alias(childExpr, name) => Alias(Cast(childExpr, to), name)() + case _ => Cast(expr, to) + } /** * Casts the column to a different data type, using the canonical string representation @@ -613,7 +617,7 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def cast(to: String): Column = Cast(expr, to.toLowerCase match { + def cast(to: String): Column = cast(to.toLowerCase match { case "string" | "str" => StringType case "boolean" => BooleanType case "byte" => ByteType @@ -671,6 +675,11 @@ class Column(protected[sql] val expr: Expression) { } +/** + * :: Experimental :: + * A convenient class used for constructing schema. + */ +@Experimental class ColumnName(name: String) extends Column(name) { /** Creates a new AttributeReference of type boolean */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 2a1e086891423..4fdbfc6d22c9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.types._ * * @groupname udf_funcs UDF functions * @groupname agg_funcs Aggregate functions + * @groupname sort_funcs Sorting functions * @groupname normal_funcs Non-aggregate functions * @groupname Ungrouped Support functions for DataFrames. */ @@ -96,6 +97,33 @@ object functions { } ////////////////////////////////////////////////////////////////////////////////////////////// + // Sort functions + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** + * Returns a sort expression based on ascending order of the column. + * {{ + * // Sort by dept in ascending order, and then age in descending order. + * df.sort(asc("dept"), desc("age")) + * }} + * + * @group sort_funcs + */ + def asc(columnName: String): Column = Column(columnName).asc + + /** + * Returns a sort expression based on the descending order of the column. + * {{ + * // Sort by dept in ascending order, and then age in descending order. + * df.sort(asc("dept"), desc("age")) + * }} + * + * @group sort_funcs + */ + def desc(columnName: String): Column = Column(columnName).desc + + ////////////////////////////////////////////////////////////////////////////////////////////// + // Aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// /** @@ -263,6 +291,7 @@ object functions { def max(columnName: String): Column = max(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// + // Non-aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 928b0deb61921..37c02aaa5460b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -309,4 +309,8 @@ class ColumnExpressionSuite extends QueryTest { (1 to 100).map(n => Row(null)) ) } + + test("lift alias out of cast") { + assert(col("1234").as("name").cast("int").expr === col("1234").cast("int").as("name").expr) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 30e77e4ef30f2..c392a553c03f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -239,6 +239,10 @@ class DataFrameSuite extends QueryTest { testData2.orderBy('a.asc, 'b.asc), Seq(Row(1,1), Row(1,2), Row(2,1), Row(2,2), Row(3,1), Row(3,2))) + checkAnswer( + testData2.orderBy(asc("a"), desc("b")), + Seq(Row(1,2), Row(1,1), Row(2,2), Row(2,1), Row(3,2), Row(3,1))) + checkAnswer( testData2.orderBy('a.asc, 'b.desc), Seq(Row(1,2), Row(1,1), Row(2,2), Row(2,1), Row(3,2), Row(3,1))) From 922b43b3cc1cca04e0313bf9e31c5f944ac06d1f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 24 Feb 2015 19:10:37 -0800 Subject: [PATCH 242/817] [SPARK-5993][Streaming][Build] Fix assembly jar location of kafka-assembly Published Kafka-assembly JAR was empty in 1.3.0-RC1 This is because the maven build generated two Jars- 1. an empty JAR file (since kafka-assembly has no code of its own) 2. a assembly JAR file containing everything in a different location as 1 The maven publishing plugin uploaded 1 and not 2. Instead if 2 is not configure to generate in a different location, there is only 1 jar containing everything, which gets published. Author: Tathagata Das Closes #4753 from tdas/SPARK-5993 and squashes the following commits: c390db8 [Tathagata Das] Fix assembly jar location of kafka-assembly --- external/kafka-assembly/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 503fc129dc4f2..8daa7ed608f6a 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -33,9 +33,6 @@ streaming-kafka-assembly - scala-${scala.binary.version} - spark-streaming-kafka-assembly-${project.version}.jar - ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} @@ -61,7 +58,6 @@ maven-shade-plugin false - ${spark.jar} *:* From 769e092bdc51582372093f76dbaece27149cc4ea Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 24 Feb 2015 19:51:36 -0800 Subject: [PATCH 243/817] [SPARK-5286][SQL] SPARK-5286 followup https://issues.apache.org/jira/browse/SPARK-5286 Author: Yin Huai Closes #4755 from yhuai/SPARK-5286-throwable and squashes the following commits: 4c0c450 [Yin Huai] Catch Throwable instead of Exception. --- .../org/apache/spark/sql/hive/execution/commands.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index c88d0e6b79491..9934a5d3c30a2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -63,10 +63,10 @@ case class DropTable( } catch { // This table's metadata is not in case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException => - // Other exceptions can be caused by users providing wrong parameters in OPTIONS + // Other Throwables can be caused by users providing wrong parameters in OPTIONS // (e.g. invalid paths). We catch it and log a warning message. - // Users should be able to drop such kinds of tables regardless if there is an exception. - case e: Exception => log.warn(s"${e.getMessage}") + // Users should be able to drop such kinds of tables regardless if there is an error. + case e: Throwable => log.warn(s"${e.getMessage}") } hiveContext.invalidateTable(tableName) hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") From d641fbb39c90b1d734cc55396ca43d7e98788975 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 24 Feb 2015 20:51:55 -0800 Subject: [PATCH 244/817] [SPARK-5994] [SQL] Python DataFrame documentation fixes select empty should NOT be the same as select. make sure selectExpr is behaving the same. join param documentation link to source doesn't work in jekyll generated file cross reference of columns (i.e. enabling linking) show(): move df example before df.show() move tests in SQLContext out of docstring otherwise doc is too long Column.desc and .asc doesn't have any documentation in documentation, sort functions.*) Author: Davies Liu Closes #4756 from davies/df_docs and squashes the following commits: f30502c [Davies Liu] fix doc 32f0d46 [Davies Liu] fix DataFrame docs --- docs/_config.yml | 1 + python/docs/pyspark.sql.rst | 3 - python/pyspark/sql/context.py | 182 ++++++-------------------------- python/pyspark/sql/dataframe.py | 56 +++++----- python/pyspark/sql/functions.py | 1 + python/pyspark/sql/tests.py | 68 +++++++++++- python/pyspark/sql/types.py | 2 +- 7 files changed, 130 insertions(+), 183 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index e2db274e1f619..0652927a8ce9b 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -10,6 +10,7 @@ kramdown: include: - _static + - _modules # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index e03379e521a07..2e3f69b9a562a 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -7,7 +7,6 @@ Module Context .. automodule:: pyspark.sql :members: :undoc-members: - :show-inheritance: pyspark.sql.types module @@ -15,7 +14,6 @@ pyspark.sql.types module .. automodule:: pyspark.sql.types :members: :undoc-members: - :show-inheritance: pyspark.sql.functions module @@ -23,4 +21,3 @@ pyspark.sql.functions module .. automodule:: pyspark.sql.functions :members: :undoc-members: - :show-inheritance: diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 125933c9d3ae0..5d7aeb664cadf 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -129,6 +129,7 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() [Row(c0=u'4')] + >>> from pyspark.sql.types import IntegerType >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() @@ -197,31 +198,6 @@ def inferSchema(self, rdd, samplingRatio=None): >>> df = sqlCtx.inferSchema(rdd) >>> df.collect()[0] Row(field1=1, field2=u'row1') - - >>> NestedRow = Row("f1", "f2") - >>> nestedRdd1 = sc.parallelize([ - ... NestedRow(array('i', [1, 2]), {"row1": 1.0}), - ... NestedRow(array('i', [2, 3]), {"row2": 2.0})]) - >>> df = sqlCtx.inferSchema(nestedRdd1) - >>> df.collect() - [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})] - - >>> nestedRdd2 = sc.parallelize([ - ... NestedRow([[1, 2], [2, 3]], [1, 2]), - ... NestedRow([[2, 3], [3, 4]], [2, 3])]) - >>> df = sqlCtx.inferSchema(nestedRdd2) - >>> df.collect() - [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] - - >>> from collections import namedtuple - >>> CustomRow = namedtuple('CustomRow', 'field1 field2') - >>> rdd = sc.parallelize( - ... [CustomRow(field1=1, field2="row1"), - ... CustomRow(field1=2, field2="row2"), - ... CustomRow(field1=3, field2="row3")]) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.collect()[0] - Row(field1=1, field2=u'row1') """ if isinstance(rdd, DataFrame): @@ -252,56 +228,8 @@ def applySchema(self, rdd, schema): >>> schema = StructType([StructField("field1", IntegerType(), False), ... StructField("field2", StringType(), False)]) >>> df = sqlCtx.applySchema(rdd2, schema) - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT * from table1") - >>> df2.collect() - [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] - - >>> from datetime import date, datetime - >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, - ... date(2010, 1, 1), - ... datetime(2010, 1, 1, 1, 1, 1), - ... {"a": 1}, (2,), [1, 2, 3], None)]) - >>> schema = StructType([ - ... StructField("byte1", ByteType(), False), - ... StructField("byte2", ByteType(), False), - ... StructField("short1", ShortType(), False), - ... StructField("short2", ShortType(), False), - ... StructField("int1", IntegerType(), False), - ... StructField("float1", FloatType(), False), - ... StructField("date1", DateType(), False), - ... StructField("time1", TimestampType(), False), - ... StructField("map1", - ... MapType(StringType(), IntegerType(), False), False), - ... StructField("struct1", - ... StructType([StructField("b", ShortType(), False)]), False), - ... StructField("list1", ArrayType(ByteType(), False), False), - ... StructField("null1", DoubleType(), True)]) - >>> df = sqlCtx.applySchema(rdd, schema) - >>> results = df.map( - ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, - ... x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) - >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE - (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), - datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) - - >>> df.registerTempTable("table2") - >>> sqlCtx.sql( - ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + - ... "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + - ... "float1 + 1.5 as float1 FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int1=2147483646, float1=2.5)] - - >>> from pyspark.sql.types import _parse_schema_abstract, _infer_schema_type - >>> rdd = sc.parallelize([(127, -32768, 1.0, - ... datetime(2010, 1, 1, 1, 1, 1), - ... {"a": 1}, (2,), [1, 2, 3])]) - >>> abstract = "byte1 short1 float1 time1 map1{} struct1(b) list1[]" - >>> schema = _parse_schema_abstract(abstract) - >>> typedSchema = _infer_schema_type(rdd.first(), schema) - >>> df = sqlCtx.applySchema(rdd, typedSchema) >>> df.collect() - [Row(byte1=127, short1=-32768, float1=1.0, time1=..., list1=[1, 2, 3])] + [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] """ if isinstance(rdd, DataFrame): @@ -459,46 +387,28 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() >>> shutil.rmtree(jsonFile) - >>> ofn = open(jsonFile, 'w') - >>> for json in jsonStrings: - ... print>>ofn, json - >>> ofn.close() + >>> with open(jsonFile, 'w') as f: + ... f.writelines(jsonStrings) >>> df1 = sqlCtx.jsonFile(jsonFile) - >>> sqlCtx.registerDataFrameAsTable(df1, "table1") - >>> df2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table1") - >>> for r in df2.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> df3 = sqlCtx.jsonFile(jsonFile, df1.schema) - >>> sqlCtx.registerDataFrameAsTable(df3, "table2") - >>> df4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table2") - >>> for r in df4.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> df1.printSchema() + root + |-- field1: long (nullable = true) + |-- field2: string (nullable = true) + |-- field3: struct (nullable = true) + | |-- field4: long (nullable = true) >>> from pyspark.sql.types import * >>> schema = StructType([ - ... StructField("field2", StringType(), True), + ... StructField("field2", StringType()), ... StructField("field3", - ... StructType([ - ... StructField("field5", - ... ArrayType(IntegerType(), False), True)]), False)]) - >>> df5 = sqlCtx.jsonFile(jsonFile, schema) - >>> sqlCtx.registerDataFrameAsTable(df5, "table3") - >>> df6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, " - ... "field3.field5[0] as f3 from table3") - >>> df6.collect() - [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] + ... StructType([StructField("field5", ArrayType(IntegerType()))]))]) + >>> df2 = sqlCtx.jsonFile(jsonFile, schema) + >>> df2.printSchema() + root + |-- field2: string (nullable = true) + |-- field3: struct (nullable = true) + | |-- field5: array (nullable = true) + | | |-- element: integer (containsNull = true) """ if schema is None: df = self._ssql_ctx.jsonFile(path, samplingRatio) @@ -517,48 +427,23 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): determine the schema. >>> df1 = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerDataFrameAsTable(df1, "table1") - >>> df2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table1") - >>> for r in df2.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) - - >>> df3 = sqlCtx.jsonRDD(json, df1.schema) - >>> sqlCtx.registerDataFrameAsTable(df3, "table2") - >>> df4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " - ... "field6 as f4 from table2") - >>> for r in df4.collect(): - ... print r - Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) - Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) - Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> df1.first() + Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) + + >>> df2 = sqlCtx.jsonRDD(json, df1.schema) + >>> df2.first() + Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) >>> from pyspark.sql.types import * >>> schema = StructType([ - ... StructField("field2", StringType(), True), + ... StructField("field2", StringType()), ... StructField("field3", - ... StructType([ - ... StructField("field5", - ... ArrayType(IntegerType(), False), True)]), False)]) - >>> df5 = sqlCtx.jsonRDD(json, schema) - >>> sqlCtx.registerDataFrameAsTable(df5, "table3") - >>> df6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, " - ... "field3.field5[0] as f3 from table3") - >>> df6.collect() - [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] - - >>> sqlCtx.jsonRDD(sc.parallelize(['{}', - ... '{"key0": {"key1": "value1"}}'])).collect() - [Row(key0=None), Row(key0=Row(key1=u'value1'))] - >>> sqlCtx.jsonRDD(sc.parallelize(['{"key0": null}', - ... '{"key0": {"key1": "value1"}}'])).collect() - [Row(key0=None), Row(key0=Row(key1=u'value1'))] + ... StructType([StructField("field5", ArrayType(IntegerType()))])) + ... ]) + >>> df3 = sqlCtx.jsonRDD(json, schema) + >>> df3.first() + Row(field2=u'row1', field3=Row(field5=None)) + """ def func(iterator): @@ -848,7 +733,8 @@ def _test(): globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) (failure_count, test_count) = doctest.testmod( - pyspark.sql.context, globs=globs, optionflags=doctest.ELLIPSIS) + pyspark.sql.context, globs=globs, + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 6f746d136b22d..6d42410020b64 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -96,7 +96,7 @@ def applySchema(it): return self._lazy_rdd def toJSON(self, use_unicode=False): - """Convert a DataFrame into a MappedRDD of JSON documents; one document per row. + """Convert a :class:`DataFrame` into a MappedRDD of JSON documents; one document per row. >>> df.toJSON().first() '{"age":2,"name":"Alice"}' @@ -108,7 +108,7 @@ def saveAsParquetFile(self, path): """Save the contents as a Parquet file, preserving the schema. Files that are written out using this method can be read back in as - a DataFrame using the L{SQLContext.parquetFile} method. + a :class:`DataFrame` using the L{SQLContext.parquetFile} method. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() @@ -139,7 +139,7 @@ def registerAsTable(self, name): self.registerTempTable(name) def insertInto(self, tableName, overwrite=False): - """Inserts the contents of this DataFrame into the specified table. + """Inserts the contents of this :class:`DataFrame` into the specified table. Optionally overwriting any existing data. """ @@ -165,7 +165,7 @@ def _java_save_mode(self, mode): return jmode def saveAsTable(self, tableName, source=None, mode="append", **options): - """Saves the contents of the DataFrame to a data source as a table. + """Saves the contents of the :class:`DataFrame` to a data source as a table. The data source is specified by the `source` and a set of `options`. If `source` is not specified, the default data source configured by @@ -174,12 +174,13 @@ def saveAsTable(self, tableName, source=None, mode="append", **options): Additionally, mode is used to specify the behavior of the saveAsTable operation when table already exists in the data source. There are four modes: - * append: Contents of this DataFrame are expected to be appended to existing table. - * overwrite: Data in the existing table is expected to be overwritten by the contents of \ - this DataFrame. + * append: Contents of this :class:`DataFrame` are expected to be appended \ + to existing table. + * overwrite: Data in the existing table is expected to be overwritten by \ + the contents of this DataFrame. * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of the DataFrame and \ - to not change the existing table. + * ignore: The save operation is expected to not save the contents of the \ + :class:`DataFrame` and to not change the existing table. """ if source is None: source = self.sql_ctx.getConf("spark.sql.sources.default", @@ -190,7 +191,7 @@ def saveAsTable(self, tableName, source=None, mode="append", **options): self._jdf.saveAsTable(tableName, source, jmode, joptions) def save(self, path=None, source=None, mode="append", **options): - """Saves the contents of the DataFrame to a data source. + """Saves the contents of the :class:`DataFrame` to a data source. The data source is specified by the `source` and a set of `options`. If `source` is not specified, the default data source configured by @@ -199,11 +200,11 @@ def save(self, path=None, source=None, mode="append", **options): Additionally, mode is used to specify the behavior of the save operation when data already exists in the data source. There are four modes: - * append: Contents of this DataFrame are expected to be appended to existing data. + * append: Contents of this :class:`DataFrame` are expected to be appended to existing data. * overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of the DataFrame and \ - to not change the existing data. + * ignore: The save operation is expected to not save the contents of \ + the :class:`DataFrame` and to not change the existing data. """ if path is not None: options["path"] = path @@ -217,7 +218,7 @@ def save(self, path=None, source=None, mode="append", **options): @property def schema(self): - """Returns the schema of this DataFrame (represented by + """Returns the schema of this :class:`DataFrame` (represented by a L{StructType}). >>> df.schema @@ -275,12 +276,12 @@ def show(self): """ Print the first 20 rows. + >>> df + DataFrame[age: int, name: string] >>> df.show() age name 2 Alice 5 Bob - >>> df - DataFrame[age: int, name: string] """ print self._jdf.showString().encode('utf8', 'ignore') @@ -481,8 +482,8 @@ def columns(self): def join(self, other, joinExprs=None, joinType=None): """ - Join with another DataFrame, using the given join expression. - The following performs a full outer join between `df1` and `df2`:: + Join with another :class:`DataFrame`, using the given join expression. + The following performs a full outer join between `df1` and `df2`. :param other: Right side of the join :param joinExprs: Join expression @@ -582,8 +583,6 @@ def __getattr__(self, name): def select(self, *cols): """ Selecting a set of expressions. - >>> df.select().collect() - [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] >>> df.select('*').collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] >>> df.select('name', 'age').collect() @@ -591,8 +590,6 @@ def select(self, *cols): >>> df.select(df.name, (df.age + 10).alias('age')).collect() [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] """ - if not cols: - cols = ["*"] jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) @@ -612,7 +609,7 @@ def selectExpr(self, *expr): def filter(self, condition): """ Filtering rows using the given condition, which could be - Column expression or string of SQL expression. + :class:`Column` expression or string of SQL expression. where() is an alias for filter(). @@ -666,7 +663,7 @@ def agg(self, *exprs): return self.groupBy().agg(*exprs) def unionAll(self, other): - """ Return a new DataFrame containing union of rows in this + """ Return a new :class:`DataFrame` containing union of rows in this frame and another frame. This is equivalent to `UNION ALL` in SQL. @@ -919,9 +916,10 @@ class Column(object): """ A column in a DataFrame. - `Column` instances can be created by:: + :class:`Column` instances can be created by:: # 1. Select a column out of a DataFrame + df.colName df["colName"] @@ -975,7 +973,7 @@ def __init__(self, jc): def substr(self, startPos, length): """ - Return a Column which is a substring of the column + Return a :class:`Column` which is a substring of the column :param startPos: start position (int or Column) :param length: length of the substring (int or Column) @@ -996,8 +994,10 @@ def substr(self, startPos, length): __getslice__ = substr # order - asc = _unary_op("asc") - desc = _unary_op("desc") + asc = _unary_op("asc", "Returns a sort expression based on the" + " ascending order of the given column name.") + desc = _unary_op("desc", "Returns a sort expression based on the" + " descending order of the given column name.") isNull = _unary_op("isNull", "True if the current expression is null.") isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8aa44765205c1..5873f09ae3275 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -72,6 +72,7 @@ def _(col): globals()[_name] = _create_function(_name, _doc) del _name, _doc __all__ += _functions.keys() +__all__.sort() def countDistinct(col, *cols): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 39071e7e35ca1..83899ad4b1b12 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -36,9 +36,9 @@ else: import unittest -from pyspark.sql import SQLContext, HiveContext, Column -from pyspark.sql.types import IntegerType, Row, ArrayType, StructType, StructField, \ - UserDefinedType, DoubleType, LongType, StringType, _infer_type +from pyspark.sql import SQLContext, HiveContext, Column, Row +from pyspark.sql.types import * +from pyspark.sql.types import UserDefinedType, _infer_type from pyspark.tests import ReusedPySparkTestCase @@ -204,6 +204,68 @@ def test_infer_schema(self): result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) + def test_infer_nested_schema(self): + NestedRow = Row("f1", "f2") + nestedRdd1 = self.sc.parallelize([NestedRow([1, 2], {"row1": 1.0}), + NestedRow([2, 3], {"row2": 2.0})]) + df = self.sqlCtx.inferSchema(nestedRdd1) + self.assertEqual(Row(f1=[1, 2], f2={u'row1': 1.0}), df.collect()[0]) + + nestedRdd2 = self.sc.parallelize([NestedRow([[1, 2], [2, 3]], [1, 2]), + NestedRow([[2, 3], [3, 4]], [2, 3])]) + df = self.sqlCtx.inferSchema(nestedRdd2) + self.assertEqual(Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), df.collect()[0]) + + from collections import namedtuple + CustomRow = namedtuple('CustomRow', 'field1 field2') + rdd = self.sc.parallelize([CustomRow(field1=1, field2="row1"), + CustomRow(field1=2, field2="row2"), + CustomRow(field1=3, field2="row3")]) + df = self.sqlCtx.inferSchema(rdd) + self.assertEquals(Row(field1=1, field2=u'row1'), df.first()) + + def test_apply_schema(self): + from datetime import date, datetime + rdd = self.sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + date(2010, 1, 1), datetime(2010, 1, 1, 1, 1, 1), + {"a": 1}, (2,), [1, 2, 3], None)]) + schema = StructType([ + StructField("byte1", ByteType(), False), + StructField("byte2", ByteType(), False), + StructField("short1", ShortType(), False), + StructField("short2", ShortType(), False), + StructField("int1", IntegerType(), False), + StructField("float1", FloatType(), False), + StructField("date1", DateType(), False), + StructField("time1", TimestampType(), False), + StructField("map1", MapType(StringType(), IntegerType(), False), False), + StructField("struct1", StructType([StructField("b", ShortType(), False)]), False), + StructField("list1", ArrayType(ByteType(), False), False), + StructField("null1", DoubleType(), True)]) + df = self.sqlCtx.applySchema(rdd, schema) + results = df.map(lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, + x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) + r = (127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), + datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + self.assertEqual(r, results.first()) + + df.registerTempTable("table2") + r = self.sqlCtx.sql("SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + + "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + + "float1 + 1.5 as float1 FROM table2").first() + + self.assertEqual((126, -127, -32767, 32766, 2147483646, 2.5), tuple(r)) + + from pyspark.sql.types import _parse_schema_abstract, _infer_schema_type + rdd = self.sc.parallelize([(127, -32768, 1.0, datetime(2010, 1, 1, 1, 1, 1), + {"a": 1}, (2,), [1, 2, 3])]) + abstract = "byte1 short1 float1 time1 map1{} struct1(b) list1[]" + schema = _parse_schema_abstract(abstract) + typedSchema = _infer_schema_type(rdd.first(), schema) + df = self.sqlCtx.applySchema(rdd, typedSchema) + r = (127, -32768, 1.0, datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, Row(b=2), [1, 2, 3]) + self.assertEqual(r, tuple(df.first())) + def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] df = self.sc.parallelize(d).toDF() diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index b6e41cf0b29ff..0f5dc2be6dab8 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -28,7 +28,7 @@ __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", - "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", ] + "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType"] class DataType(object): From d51ed263ee791967380de6b9c892985ce87f6fcb Mon Sep 17 00:00:00 2001 From: prabs Date: Wed, 25 Feb 2015 14:37:35 +0000 Subject: [PATCH 245/817] [SPARK-5666][streaming][MQTT streaming] some trivial fixes modified to adhere to accepted coding standards as pointed by tdas in PR #3844 Author: prabs Author: Prabeesh K Closes #4178 from prabeesh/master and squashes the following commits: bd2cb49 [Prabeesh K] adress the comment ccc0765 [prabs] adress the comment 46f9619 [prabs] adress the comment c035bdc [prabs] adress the comment 22dd7f7 [prabs] address the comments 0cc67bd [prabs] adress the comment 838c38e [prabs] adress the comment cd57029 [prabs] address the comments 66919a3 [Prabeesh K] changed MqttDefaultFilePersistence to MemoryPersistence 5857989 [prabs] modified to adhere to accepted coding standards --- .../examples/streaming/MQTTWordCount.scala | 49 +++++++++++-------- .../streaming/mqtt/MQTTInputDStream.scala | 26 +++++----- .../spark/streaming/mqtt/MQTTUtils.scala | 3 +- .../streaming/mqtt/MQTTStreamSuite.scala | 12 ++--- 4 files changed, 50 insertions(+), 40 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 6ff0c47793a25..f40caad322f59 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -17,8 +17,8 @@ package org.apache.spark.examples.streaming -import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic} -import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence +import org.eclipse.paho.client.mqttv3._ +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -31,8 +31,6 @@ import org.apache.spark.SparkConf */ object MQTTPublisher { - var client: MqttClient = _ - def main(args: Array[String]) { if (args.length < 2) { System.err.println("Usage: MQTTPublisher ") @@ -42,25 +40,36 @@ object MQTTPublisher { StreamingExamples.setStreamingLogLevels() val Seq(brokerUrl, topic) = args.toSeq + + var client: MqttClient = null try { - var peristance:MqttClientPersistence =new MqttDefaultFilePersistence("/tmp") - client = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) + val persistence = new MemoryPersistence() + client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) + + client.connect() + + val msgtopic = client.getTopic(topic) + val msgContent = "hello mqtt demo for spark streaming" + val message = new MqttMessage(msgContent.getBytes("utf-8")) + + while (true) { + try { + msgtopic.publish(message) + println(s"Published data. topic: {msgtopic.getName()}; Message: {message}") + } catch { + case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => + Thread.sleep(10) + println("Queue is full, wait for to consume data from the message queue") + } + } } catch { case e: MqttException => println("Exception Caught: " + e) + } finally { + if (client != null) { + client.disconnect() + } } - - client.connect() - - val msgtopic: MqttTopic = client.getTopic(topic) - val msg: String = "hello mqtt demo for spark streaming" - - while (true) { - val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8")) - msgtopic.publish(message) - println("Published data. topic: " + msgtopic.getName() + " Message: " + message) - } - client.disconnect() } } @@ -96,9 +105,9 @@ object MQTTWordCount { val sparkConf = new SparkConf().setAppName("MQTTWordCount") val ssc = new StreamingContext(sparkConf, Seconds(2)) val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) - - val words = lines.flatMap(x => x.toString.split(" ")) + val words = lines.flatMap(x => x.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) + wordCounts.print() ssc.start() ssc.awaitTermination() diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 1ef91dd49284f..3c0ef94cb0fab 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -17,23 +17,23 @@ package org.apache.spark.streaming.mqtt +import java.io.IOException +import java.util.concurrent.Executors +import java.util.Properties + +import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ import scala.reflect.ClassTag -import java.util.Properties -import java.util.concurrent.Executors -import java.io.IOException - +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken import org.eclipse.paho.client.mqttv3.MqttCallback import org.eclipse.paho.client.mqttv3.MqttClient import org.eclipse.paho.client.mqttv3.MqttClientPersistence -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken import org.eclipse.paho.client.mqttv3.MqttException import org.eclipse.paho.client.mqttv3.MqttMessage import org.eclipse.paho.client.mqttv3.MqttTopic +import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel @@ -82,18 +82,18 @@ class MQTTReceiver( val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) // Callback automatically triggers as and when new message arrives on specified topic - val callback: MqttCallback = new MqttCallback() { + val callback = new MqttCallback() { // Handles Mqtt message - override def messageArrived(arg0: String, arg1: MqttMessage) { - store(new String(arg1.getPayload(),"utf-8")) + override def messageArrived(topic: String, message: MqttMessage) { + store(new String(message.getPayload(),"utf-8")) } - override def deliveryComplete(arg0: IMqttDeliveryToken) { + override def deliveryComplete(token: IMqttDeliveryToken) { } - override def connectionLost(arg0: Throwable) { - restart("Connection lost ", arg0) + override def connectionLost(cause: Throwable) { + restart("Connection lost ", cause) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index c5ffe51f9986c..1142d0f56ba34 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.mqtt +import scala.reflect.ClassTag + import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream} -import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object MQTTUtils { diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 19c9271af77be..0f3298af6234a 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -42,8 +42,8 @@ import org.apache.spark.util.Utils class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { private val batchDuration = Milliseconds(500) - private val master: String = "local[2]" - private val framework: String = this.getClass.getSimpleName + private val master = "local[2]" + private val framework = this.getClass.getSimpleName private val freePort = findFreePort() private val brokerUri = "//localhost:" + freePort private val topic = "def" @@ -69,7 +69,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { test("mqtt input stream") { val sendMessage = "MQTT demo for spark streaming" - val receiveStream: ReceiverInputDStream[String] = + val receiveStream = MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) @volatile var receiveMessage: List[String] = List() receiveStream.foreachRDD { rdd => @@ -123,12 +123,12 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { def publishData(data: String): Unit = { var client: MqttClient = null try { - val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) + val persistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence) client.connect() if (client.isConnected) { - val msgTopic: MqttTopic = client.getTopic(topic) - val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) + val msgTopic = client.getTopic(topic) + val message = new MqttMessage(data.getBytes("utf-8")) message.setQos(1) message.setRetained(true) From 5b8480e0359d5af8bdf570f115acb0b1b8997735 Mon Sep 17 00:00:00 2001 From: Benedikt Linse Date: Wed, 25 Feb 2015 14:46:17 +0000 Subject: [PATCH 246/817] [GraphX] fixing 3 typos in the graphx programming guide Corrected 3 Typos in the GraphX programming guide. I hope this is the correct way to contribute. Author: Benedikt Linse Closes #4766 from 1123/master and squashes the following commits: 8a63812 [Benedikt Linse] fixing 3 typos in the graphx programming guide --- docs/graphx-programming-guide.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 826f6d8f371c7..28bdf81ca0ca5 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -538,7 +538,7 @@ val joinedGraph = graph.joinVertices(uniqueCosts, ## Neighborhood Aggregation -A key step in may graph analytics tasks is aggregating information about the neighborhood of each +A key step in many graph analytics tasks is aggregating information about the neighborhood of each vertex. For example, we might want to know the number of followers each user has or the average age of the the followers of each user. Many iterative graph algorithms (e.g., PageRank, Shortest Path, and @@ -634,7 +634,7 @@ avgAgeOfOlderFollowers.collect.foreach(println(_)) ### Map Reduce Triplets Transition Guide (Legacy) -In earlier versions of GraphX we neighborhood aggregation was accomplished using the +In earlier versions of GraphX neighborhood aggregation was accomplished using the [`mapReduceTriplets`][Graph.mapReduceTriplets] operator: {% highlight scala %} @@ -682,8 +682,8 @@ val result = graph.aggregateMessages[String](msgFun, reduceFun) ### Computing Degree Information A common aggregation task is computing the degree of each vertex: the number of edges adjacent to -each vertex. In the context of directed graphs it often necessary to know the in-degree, out- -degree, and the total degree of each vertex. The [`GraphOps`][GraphOps] class contains a +each vertex. In the context of directed graphs it is often necessary to know the in-degree, +out-degree, and the total degree of each vertex. The [`GraphOps`][GraphOps] class contains a collection of operators to compute the degrees of each vertex. For example in the following we compute the max in, out, and total degrees: From dd077abf2e2949fdfec31074b760b587f00efcf2 Mon Sep 17 00:00:00 2001 From: guliangliang Date: Wed, 25 Feb 2015 14:48:02 +0000 Subject: [PATCH 247/817] [SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called In Standalone mode, the number of cores in Completed Applications of the Master Web Page will always be zero, if sc.stop() is called. But the number will always be right, if sc.stop() is not called. The reason maybe: after sc.stop() is called, the function removeExecutor of class ApplicationInfo will be called, thus reduce the variable coresGranted to zero. The variable coresGranted is used to display the number of Cores on the Web Page. Author: guliangliang Closes #4567 from marsishandsome/Spark5771 and squashes the following commits: 694796e [guliangliang] remove duplicate code a20e390 [guliangliang] change to Cores Using & Requested 0c19c95 [guliangliang] change Cores to Cores (max) cfbd97d [guliangliang] [SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called --- .../spark/deploy/master/ApplicationInfo.scala | 4 +-- .../spark/deploy/master/ui/MasterPage.scala | 31 +++++++++++++++---- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index ede0a9dbefb8d..a962dc4af2f6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationInfo( } } - private val myMaxCores = desc.maxCores.getOrElse(defaultCores) + val requestedCores = desc.maxCores.getOrElse(defaultCores) - def coresLeft: Int = myMaxCores - coresGranted + def coresLeft: Int = requestedCores - coresGranted private var _retryCount = 0 diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index fd514f07664a9..9dd96493ee48d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -50,12 +50,16 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per Node", "Submitted Time", - "User", "State", "Duration") + val activeAppHeaders = Seq("Application ID", "Name", "Cores in Use", + "Cores Requested", "Memory per Node", "Submitted Time", "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse - val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) + val activeAppsTable = UIUtils.listingTable(activeAppHeaders, activeAppRow, activeApps) + + val completedAppHeaders = Seq("Application ID", "Name", "Cores Requested", "Memory per Node", + "Submitted Time", "User", "State", "Duration") val completedApps = state.completedApps.sortBy(_.endTime).reverse - val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) + val completedAppsTable = UIUtils.listingTable(completedAppHeaders, completeAppRow, + completedApps) val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class") @@ -162,7 +166,7 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = { {app.id} @@ -170,8 +174,15 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.desc.name} + { + if (active) { + + {app.coresGranted} + + } + } - {app.coresGranted} + {app.requestedCores} {Utils.megabytesToString(app.desc.memoryPerSlave)} @@ -183,6 +194,14 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } + private def activeAppRow(app: ApplicationInfo): Seq[Node] = { + appRow(app, active = true) + } + + private def completeAppRow(app: ApplicationInfo): Seq[Node] = { + appRow(app, active = false) + } + private def driverRow(driver: DriverInfo): Seq[Node] = { {driver.id} From f84c799ea0b82abca6a4fad39532c2515743b632 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Feb 2015 10:13:40 -0800 Subject: [PATCH 248/817] [SPARK-5996][SQL] Fix specialized outbound conversions Author: Michael Armbrust Closes #4757 from marmbrus/udtConversions and squashes the following commits: 3714aad [Michael Armbrust] [SPARK-5996][SQL] Fix specialized outbound conversions --- .../apache/spark/sql/execution/LocalTableScan.scala | 7 +++++-- .../apache/spark/sql/execution/basicOperators.scala | 8 +++++--- .../org/apache/spark/sql/UserDefinedTypeSuite.scala | 10 ++++++++++ 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala index d6d8258f46a9a..d3a18b37d52b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Attribute @@ -30,7 +31,9 @@ case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNo override def execute() = rdd - override def executeCollect() = rows.toArray + override def executeCollect() = + rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray - override def executeTake(limit: Int) = rows.take(limit).toArray + override def executeTake(limit: Int) = + rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 4dc506c21ab9e..710268584cff1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -134,13 +134,15 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) val ord = new RowOrdering(sortOrder, child.output) + private def collectData() = child.execute().map(_.copy()).takeOrdered(limit)(ord) + // TODO: Is this copying for no reason? - override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ord) - .map(ScalaReflection.convertRowToScala(_, this.schema)) + override def executeCollect() = + collectData().map(ScalaReflection.convertRowToScala(_, this.schema)) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - override def execute() = sparkContext.makeRDD(executeCollect(), 1) + override def execute() = sparkContext.makeRDD(collectData(), 1) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 9c098df24c65f..47fdb5543235c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -22,6 +22,7 @@ import java.io.File import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -105,4 +106,13 @@ class UserDefinedTypeSuite extends QueryTest { tempDir.delete() pointsRDD.repartition(1).saveAsParquetFile(tempDir.getCanonicalPath) } + + // Tests to make sure that all operators correctly convert types on the way out. + test("Local UDTs") { + val df = Seq((1, new MyDenseVector(Array(0.1, 1.0)))).toDF("int", "vec") + df.collect()(0).getAs[MyDenseVector](1) + df.take(1)(0).getAs[MyDenseVector](1) + df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) + df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) + } } From 7d8e6a2e44e13a6d6cdcd98a0d0c33b243ef0dc2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 25 Feb 2015 12:20:44 -0800 Subject: [PATCH 249/817] SPARK-5930 [DOCS] Documented default of spark.shuffle.io.retryWait is confusing Clarify default max wait in spark.shuffle.io.retryWait docs CC andrewor14 Author: Sean Owen Closes #4769 from srowen/SPARK-5930 and squashes the following commits: ae2792b [Sean Owen] Clarify default max wait in spark.shuffle.io.retryWait docs --- docs/configuration.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index c8db338cb6f89..81298514a7cb2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -955,7 +955,9 @@ Apart from these, the following properties are also available, and may be useful 5 (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + is simply maxRetries * retryWait. The default maximum delay is therefore + 15 seconds, because the default value of maxRetries is 3, and the default + retryWait here is 5 seconds. From a777c65da9bc636e5cf5426e15a2e76d6b21b744 Mon Sep 17 00:00:00 2001 From: Milan Straka Date: Wed, 25 Feb 2015 21:33:34 +0000 Subject: [PATCH 250/817] [SPARK-5970][core] Register directory created in getOrCreateLocalRootDirs for automatic deletion. As documented in createDirectory, the result of createDirectory is not registered for automatic removal. Currently there are 4 directories left in `/tmp` after just running `pyspark`. Author: Milan Straka Closes #4759 from foxik/remove-tmp-dirs and squashes the following commits: 280450d [Milan Straka] Use createTempDir in getOrCreateLocalRootDirs... --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index df21ed37e76b1..4803ff9403b1d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -696,7 +696,7 @@ private[spark] object Utils extends Logging { try { val rootDir = new File(root) if (rootDir.exists || rootDir.mkdirs()) { - val dir = createDirectory(root) + val dir = createTempDir(root) chmod700(dir) Some(dir.getAbsolutePath) } else { From 9f603fce78fcc997926e9a72dec44d48cbc396fc Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 25 Feb 2015 14:11:12 -0800 Subject: [PATCH 251/817] [SPARK-1955][GraphX]: VertexRDD can incorrectly assume index sharing Fixes the issue whereby when VertexRDD's are `diff`ed, `innerJoin`ed, or `leftJoin`ed and have different partition sizes they fail under the `zipPartitions` method. This fix tests whether the partitions are equal or not and, if not, will repartition the other to match the partition size of the calling VertexRDD. Author: Brennon York Closes #4705 from brennonyork/SPARK-1955 and squashes the following commits: 0882590 [Brennon York] updated to properly handle differently-partitioned vertexRDDs --- .../org/apache/spark/graphx/impl/VertexRDDImpl.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 6dad167fa7411..904be213147dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -104,8 +104,14 @@ class VertexRDDImpl[VD] private[graphx] ( this.mapVertexPartitions(_.map(f)) override def diff(other: VertexRDD[VD]): VertexRDD[VD] = { + val otherPartition = other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + other.partitionsRDD + case _ => + VertexRDD(other.partitionBy(this.partitioner.get)).partitionsRDD + } val newPartitionsRDD = partitionsRDD.zipPartitions( - other.partitionsRDD, preservesPartitioning = true + otherPartition, preservesPartitioning = true ) { (thisIter, otherIter) => val thisPart = thisIter.next() val otherPart = otherIter.next() @@ -133,7 +139,7 @@ class VertexRDDImpl[VD] private[graphx] ( // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient leftZipJoin other match { - case other: VertexRDD[_] => + case other: VertexRDD[_] if this.partitioner == other.partitioner => leftZipJoin(other)(f) case _ => this.withPartitionsRDD[VD3]( @@ -162,7 +168,7 @@ class VertexRDDImpl[VD] private[graphx] ( // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient innerZipJoin other match { - case other: VertexRDD[_] => + case other: VertexRDD[_] if this.partitioner == other.partitioner => innerZipJoin(other)(f) case _ => this.withPartitionsRDD( From 838a48036c050cef03b8c3620e16b5495cd7beab Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 25 Feb 2015 14:55:24 -0800 Subject: [PATCH 252/817] [SPARK-5982] Remove incorrect Local Read Time Metric This metric is incomplete, because the files are memory mapped, so much of the read from disk occurs later as tasks actually read the file's data. This should be merged into 1.3, so that we never expose this incorrect metric to users. CC pwendell ksakellis sryza Author: Kay Ousterhout Closes #4749 from kayousterhout/SPARK-5982 and squashes the following commits: 9737b5e [Kay Ousterhout] More fixes a1eb300 [Kay Ousterhout] Removed one more use of local read time cf13497 [Kay Ousterhout] [SPARK-5982] Remove incorrectwq Local Read Time Metric --- .../scala/org/apache/spark/executor/TaskMetrics.scala | 8 -------- .../main/scala/org/apache/spark/scheduler/JobLogger.scala | 1 - .../spark/storage/ShuffleBlockFetcherIterator.scala | 2 -- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 2 -- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 3 --- 5 files changed, 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index df36566bec4b1..07b152651dedf 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -203,7 +203,6 @@ class TaskMetrics extends Serializable { merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) merged.incRemoteBytesRead(depMetrics.remoteBytesRead) merged.incLocalBytesRead(depMetrics.localBytesRead) - merged.incLocalReadTime(depMetrics.localReadTime) merged.incRecordsRead(depMetrics.recordsRead) } _shuffleReadMetrics = Some(merged) @@ -345,13 +344,6 @@ class ShuffleReadMetrics extends Serializable { private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value - /** - * Time the task spent (in milliseconds) reading local shuffle blocks (from the local disk). - */ - private var _localReadTime: Long = _ - def localReadTime = _localReadTime - private[spark] def incLocalReadTime(value: Long) = _localReadTime += value - /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index f9fc8aa30454e..8aa528ac573d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -170,7 +170,6 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + - " LOCAL_READ_TIME=" + metrics.localReadTime + " LOCAL_BYTES_READ=" + metrics.localBytesRead case None => "" } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 2ebb79989da43..8f28ef49a8a6f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -228,7 +228,6 @@ final class ShuffleBlockFetcherIterator( * track in-memory are the ManagedBuffer references themselves. */ private[this] def fetchLocalBlocks() { - val startTime = System.currentTimeMillis val iter = localBlocks.iterator while (iter.hasNext) { val blockId = iter.next() @@ -246,7 +245,6 @@ final class ShuffleBlockFetcherIterator( return } } - shuffleMetrics.incLocalReadTime(System.currentTimeMillis - startTime) } private[this] def initialize(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 58d37e2d667f7..8e20864db5673 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -294,7 +294,6 @@ private[spark] object JsonProtocol { ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ - ("Local Read Time" -> shuffleReadMetrics.localReadTime) ~ ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> shuffleReadMetrics.recordsRead) } @@ -676,7 +675,6 @@ private[spark] object JsonProtocol { metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int]) metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) - metrics.incLocalReadTime((json \ "Local Read Time").extractOpt[Long].getOrElse(0)) metrics.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) metrics diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c181baf6844b0..a2be724254d7c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -270,7 +270,6 @@ class JsonProtocolSuite extends FunSuite { .removeField { case (field, _) => field == "Local Read Time" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.shuffleReadMetrics.get.localBytesRead == 0) - assert(newMetrics.shuffleReadMetrics.get.localReadTime == 0) } test("SparkListenerApplicationStart backwards compatibility") { @@ -708,7 +707,6 @@ class JsonProtocolSuite extends FunSuite { sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1) - sr.incLocalReadTime(a + e) sr.incLocalBytesRead(a + f) t.setShuffleReadMetrics(Some(sr)) } @@ -956,7 +954,6 @@ class JsonProtocolSuite extends FunSuite { | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, - | "Local Read Time": 1000, | "Local Bytes Read": 1100, | "Total Records Read" : 10 | }, From f3f4c87b3d944c10d1200dfe49091ebb2a149be6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 25 Feb 2015 15:13:34 -0800 Subject: [PATCH 253/817] [SPARK-5944] [PySpark] fix version in Python API docs use RELEASE_VERSION when building the Python API docs Author: Davies Liu Closes #4731 from davies/api_version and squashes the following commits: c9744c9 [Davies Liu] Update create-release.sh 08cbc3f [Davies Liu] fix python docs --- dev/create-release/create-release.sh | 2 +- python/docs/conf.py | 6 +++--- python/docs/pyspark.sql.rst | 2 +- python/pyspark/rdd.py | 4 ++++ 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 607ce1c803507..da15ce3e0e2f7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -237,7 +237,7 @@ if [[ ! "$@" =~ --skip-package ]]; then sbt/sbt clean cd docs # Compile docs with Java 7 to use nicer format - JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build + JAVA_HOME="$JAVA_7_HOME" PRODUCTION=1 RELEASE_VERSION="$RELEASE_VERSION" jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs ssh $ASF_USERNAME@people.apache.org \ diff --git a/python/docs/conf.py b/python/docs/conf.py index cbbf7ffb08992..163987dd8e5fa 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -48,16 +48,16 @@ # General information about the project. project = u'PySpark' -copyright = u'2014, Author' +copyright = u'' # The version info for the project you're documenting, acts as replacement for # |version| and |release|, also used in various other places throughout the # built documents. # # The short X.Y version. -version = '1.3-SNAPSHOT' +version = 'master' # The full version, including alpha/beta/rc tags. -release = '1.3-SNAPSHOT' +release = os.environ.get('RELEASE_VERSION', version) # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst index 2e3f69b9a562a..6259379ed05b7 100644 --- a/python/docs/pyspark.sql.rst +++ b/python/docs/pyspark.sql.rst @@ -17,7 +17,7 @@ pyspark.sql.types module pyspark.sql.functions module ------------------------- +---------------------------- .. automodule:: pyspark.sql.functions :members: :undoc-members: diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d3148de6f41a3..cb12fed98c53d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2111,6 +2111,7 @@ def _to_java_object_rdd(self): def countApprox(self, timeout, confidence=0.95): """ .. note:: Experimental + Approximate version of count() that returns a potentially incomplete result within a timeout, even if not all tasks have finished. @@ -2124,6 +2125,7 @@ def countApprox(self, timeout, confidence=0.95): def sumApprox(self, timeout, confidence=0.95): """ .. note:: Experimental + Approximate operation to return the sum within a timeout or meet the confidence. @@ -2140,6 +2142,7 @@ def sumApprox(self, timeout, confidence=0.95): def meanApprox(self, timeout, confidence=0.95): """ .. note:: Experimental + Approximate operation to return the mean within a timeout or meet the confidence. @@ -2156,6 +2159,7 @@ def meanApprox(self, timeout, confidence=0.95): def countApproxDistinct(self, relativeSD=0.05): """ .. note:: Experimental + Return approximate number of distinct elements in the RDD. The algorithm used is based on streamlib's implementation of From e0fdd467e277867d6bec5c6605cc1cabce70ac89 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Feb 2015 15:15:22 -0800 Subject: [PATCH 254/817] [SPARK-6010] [SQL] Merging compatible Parquet schemas before computing splits `ReadContext.init` calls `InitContext.getMergedKeyValueMetadata`, which doesn't know how to merge conflicting user defined key-value metadata and throws exception. In our case, when dealing with different but compatible schemas, we have different Spark SQL schema JSON strings in different Parquet part-files, thus causes this problem. Reading similar Parquet files generated by Hive doesn't suffer from this issue. In this PR, we manually merge the schemas before passing it to `ReadContext` to avoid the exception. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4768) Author: Cheng Lian Closes #4768 from liancheng/spark-6010 and squashes the following commits: 9002f0a [Cheng Lian] Fixes SPARK-6010 --- .../sql/parquet/ParquetTableOperations.scala | 20 +++++++++++++++++- .../spark/sql/parquet/ParquetTest.scala | 5 +++++ .../ParquetPartitionDiscoverySuite.scala | 21 +++++++++++++++++++ 3 files changed, 45 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 65966458eb670..4dc13b036cd4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -48,6 +48,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.{Logging, SerializableWritable, TaskContext} /** @@ -459,13 +460,30 @@ private[parquet] class FilteringParquetRowInputFormat val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) getGlobalMetaData.setAccessible(true) - val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] + var globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] if (globalMetaData == null) { val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] return splits } + Option(globalMetaData.getKeyValueMetaData.get(RowReadSupport.SPARK_METADATA_KEY)).foreach { + schemas => + val mergedSchema = schemas + .map(DataType.fromJson(_).asInstanceOf[StructType]) + .reduce(_ merge _) + .json + + val mergedMetadata = globalMetaData + .getKeyValueMetaData + .updated(RowReadSupport.SPARK_METADATA_KEY, setAsJavaSet(Set(mergedSchema))) + + globalMetaData = new GlobalMetaData( + globalMetaData.getSchema, + mergedMetadata, + globalMetaData.getCreatedBy) + } + val readContext = getReadSupport(configuration).init( new InitContext(configuration, globalMetaData.getKeyValueMetaData, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index 0fa2fe90f9674..d6ea6679c5966 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -131,6 +131,11 @@ private[sql] trait ParquetTest { data.toDF().save(path.getCanonicalPath, "org.apache.spark.sql.parquet", SaveMode.Overwrite) } + protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( + df: DataFrame, path: File): Unit = { + df.save(path.getCanonicalPath, "org.apache.spark.sql.parquet", SaveMode.Overwrite) + } + protected def makePartitionDir( basePath: File, defaultPartitionName: String, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index 3bf0116c8f7e9..adb3c9391f6c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -36,6 +36,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { override val sqlContext: SQLContext = TestSQLContext import sqlContext._ + import sqlContext.implicits._ val defaultPartitionName = "__NULL__" @@ -319,4 +320,24 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { } } } + + test("read partitioned table - merging compatible schemas") { + withTempDir { base => + makeParquetFile( + (1 to 10).map(i => Tuple1(i)).toDF("intField"), + makePartitionDir(base, defaultPartitionName, "pi" -> 1)) + + makeParquetFile( + (1 to 10).map(i => (i, i.toString)).toDF("intField", "stringField"), + makePartitionDir(base, defaultPartitionName, "pi" -> 2)) + + load(base.getCanonicalPath, "org.apache.spark.sql.parquet").registerTempTable("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + (1 to 10).map(i => Row(i, null, 1)) ++ (1 to 10).map(i => Row(i, i.toString, 2))) + } + } + } } From 12dbf98c5d270e3846e946592666160b1541d9dc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 25 Feb 2015 15:22:33 -0800 Subject: [PATCH 255/817] [SPARK-5999][SQL] Remove duplicate Literal matching block Author: Liang-Chi Hsieh Closes #4760 from viirya/dup_literal and squashes the following commits: 06e7516 [Liang-Chi Hsieh] Remove duplicate Literal matching block. --- .../sql/catalyst/expressions/literals.scala | 2 ++ .../org/apache/spark/sql/functions.scala | 20 +------------------ 2 files changed, 3 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 97bb96f48e2c7..9ff66563c8164 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -38,6 +38,8 @@ object Literal { case d: Date => Literal(DateUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) + case _ => + throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 4fdbfc6d22c9b..111e751588a8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -74,25 +74,7 @@ object functions { case _ => // continue } - val literalExpr = literal match { - case v: Boolean => Literal(v, BooleanType) - case v: Byte => Literal(v, ByteType) - case v: Short => Literal(v, ShortType) - case v: Int => Literal(v, IntegerType) - case v: Long => Literal(v, LongType) - case v: Float => Literal(v, FloatType) - case v: Double => Literal(v, DoubleType) - case v: String => Literal(v, StringType) - case v: BigDecimal => Literal(Decimal(v), DecimalType.Unlimited) - case v: java.math.BigDecimal => Literal(Decimal(v), DecimalType.Unlimited) - case v: Decimal => Literal(v, DecimalType.Unlimited) - case v: java.sql.Timestamp => Literal(v, TimestampType) - case v: java.sql.Date => Literal(v, DateType) - case v: Array[Byte] => Literal(v, BinaryType) - case null => Literal(null, NullType) - case _ => - throw new RuntimeException("Unsupported literal type " + literal.getClass + " " + literal) - } + val literalExpr = Literal(literal) Column(literalExpr) } From 41e2e5acb749c25641f1f8dea5a2e1d8af319486 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 25 Feb 2015 15:37:13 -0800 Subject: [PATCH 256/817] [SPARK-5926] [SQL] make DataFrame.explain leverage queryExecution.logical DataFrame.explain return wrong result when the query is DDL command. For example, the following two queries should print out the same execution plan, but it not. sql("create table tb as select * from src where key > 490").explain(true) sql("explain extended create table tb as select * from src where key > 490") This is because DataFrame.explain leverage logicalPlan which had been forced executed, we should use the unexecuted plan queryExecution.logical. Author: Yanbo Liang Closes #4707 from yanboliang/spark-5926 and squashes the following commits: fa6db63 [Yanbo Liang] logicalPlan is not lazy 0e40a1b [Yanbo Liang] make DataFrame.explain leverage queryExecution.logical --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 04bf5d9b0f931..f045da305ca5d 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 @@ -264,7 +264,7 @@ class DataFrame protected[sql]( */ def explain(extended: Boolean): Unit = { ExplainCommand( - logicalPlan, + queryExecution.logical, extended = extended).queryExecution.executedPlan.executeCollect().map { r => println(r.getString(0)) } From 46a044a36a2aff1306f7f677e952ce253ddbefac Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 25 Feb 2015 16:12:56 -0800 Subject: [PATCH 257/817] [SPARK-1182][Docs] Sort the configuration parameters in configuration.md Sorts all configuration options present on the `configuration.md` page to ease readability. Author: Brennon York Closes #3863 from brennonyork/SPARK-1182 and squashes the following commits: 5696f21 [Brennon York] fixed merge conflict with port comments 81a7b10 [Brennon York] capitalized A in Allocation e240486 [Brennon York] moved all spark.mesos properties into the running-on-mesos doc 7de5f75 [Brennon York] moved serialization from application to compression and serialization section a16fec0 [Brennon York] moved shuffle settings from network to shuffle f8fa286 [Brennon York] sorted encryption category 1023f15 [Brennon York] moved initialExecutors e9d62aa [Brennon York] fixed akka.heartbeat.interval 25e6f6f [Brennon York] moved spark.executer.user* 4625ade [Brennon York] added spark.executor.extra* items 4ee5648 [Brennon York] fixed merge conflicts 1b49234 [Brennon York] sorting mishap 2b5758b [Brennon York] sorting mishap 6fbdf42 [Brennon York] sorting mishap 55dc6f8 [Brennon York] sorted security ec34294 [Brennon York] sorted dynamic allocation 2a7c4a3 [Brennon York] sorted scheduling aa9acdc [Brennon York] sorted networking a4380b8 [Brennon York] sorted execution behavior 27f3919 [Brennon York] sorted compression and serialization 80a5bbb [Brennon York] sorted spark ui 3f32e5b [Brennon York] sorted shuffle behavior 6c51b38 [Brennon York] sorted runtime environment efe9d6f [Brennon York] sorted application properties --- docs/configuration.md | 1008 +++++++++++++++++++------------------- docs/running-on-mesos.md | 24 +- 2 files changed, 508 insertions(+), 524 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 81298514a7cb2..8dd2bad61344f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -94,14 +94,6 @@ of the most common options to set are: The name of your application. This will appear in the UI and in log data. - - spark.master - (none) - - The cluster manager to connect to. See the list of - allowed master URL's. - - spark.driver.cores 1 @@ -109,27 +101,6 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode. - - spark.driver.memory - 512m - - Amount of memory to use for the driver process, i.e. where SparkContext is initialized. - (e.g. 512m, 2g). - -
    Note: In client mode, this config must not be set through the SparkConf - directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-memory command line option - or in your default properties file. - - - spark.executor.memory - 512m - - Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. 512m, 2g). - - - spark.driver.maxResultSize 1g @@ -142,38 +113,35 @@ of the most common options to set are: - spark.serializer - org.apache.spark.serializer.
    JavaSerializer + spark.driver.memory + 512m - Class to use for serializing objects that will be sent over the network or need to be cached - in serialized form. The default of Java serialization works with any Serializable Java object - but is quite slow, so we recommend using - org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization - when speed is necessary. Can be any subclass of - - org.apache.spark.Serializer. + Amount of memory to use for the driver process, i.e. where SparkContext is initialized. + (e.g. 512m, 2g). + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-memory command line option + or in your default properties file. - spark.kryo.classesToRegister - (none) + spark.executor.memory + 512m - If you use Kryo serialization, give a comma-separated list of custom class names to register - with Kryo. - See the tuning guide for more details. + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m, 2g). - spark.kryo.registrator + spark.extraListeners (none) - If you use Kryo serialization, set this class to register your custom classes with Kryo. This - property is useful if you need to register your classes in a custom way, e.g. to specify a custom - field serializer. Otherwise spark.kryo.classesToRegister is simpler. It should be - set to a class that extends - - KryoRegistrator. - See the tuning guide for more details. + A comma-separated list of classes that implement SparkListener; when initializing + SparkContext, instances of these classes will be created and registered with Spark's listener + bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor + will be called; otherwise, a zero-argument constructor will be called. If no valid constructor + can be found, the SparkContext creation will fail with an exception. @@ -196,14 +164,11 @@ of the most common options to set are: - spark.extraListeners + spark.master (none) - A comma-separated list of classes that implement SparkListener; when initializing - SparkContext, instances of these classes will be created and registered with Spark's listener - bus. If a class has a single-argument constructor that accepts a SparkConf, that constructor - will be called; otherwise, a zero-argument constructor will be called. If no valid constructor - can be found, the SparkContext creation will fail with an exception. + The cluster manager to connect to. See the list of + allowed master URL's. @@ -214,26 +179,26 @@ Apart from these, the following properties are also available, and may be useful - + - + @@ -261,23 +226,22 @@ Apart from these, the following properties are also available, and may be useful - + - + @@ -287,6 +251,24 @@ Apart from these, the following properties are also available, and may be useful Set a special library path to use when launching executor JVM's. + + + + + + + + + + @@ -308,24 +290,6 @@ Apart from these, the following properties are also available, and may be useful for automatic cleaning of old logs. - - - - - - - - - - @@ -335,12 +299,11 @@ Apart from these, the following properties are also available, and may be useful - - + + @@ -366,6 +329,15 @@ Apart from these, the following properties are also available, and may be useful automatically. + + + + + @@ -376,40 +348,38 @@ Apart from these, the following properties are also available, and may be useful from JVM to Python worker for every task. +
    Property NameDefaultMeaning
    spark.driver.extraJavaOptionsspark.driver.extraClassPath (none) - A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. - + Extra classpath entries to append to the classpath of the driver. +
    Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-java-options command line option or in + Instead, please set this through the --driver-class-path command line option or in your default properties file.
    spark.driver.extraClassPathspark.driver.extraJavaOptions (none) - Extra classpath entries to append to the classpath of the driver. - + A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. +
    Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. - Instead, please set this through the --driver-class-path command line option or in + Instead, please set this through the --driver-java-options command line option or in your default properties file.
    spark.executor.extraJavaOptionsspark.executor.extraClassPath (none) - A string of extra JVM options to pass to executors. For instance, GC settings or other - logging. Note that it is illegal to set Spark properties or heap size settings with this - option. Spark properties should be set using a SparkConf object or the - spark-defaults.conf file used with the spark-submit script. Heap size settings can be set - with spark.executor.memory. + Extra classpath entries to append to the classpath of executors. This exists primarily for + backwards-compatibility with older versions of Spark. Users typically should not need to set + this option.
    spark.executor.extraClassPathspark.executor.extraJavaOptions (none) - Extra classpath entries to append to the classpath of executors. This exists primarily - for backwards-compatibility with older versions of Spark. Users typically should not need - to set this option. + A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or heap size settings with this option. Spark + properties should be set using a SparkConf object or the spark-defaults.conf file used with the + spark-submit script. Heap size settings can be set with spark.executor.memory.
    spark.executor.logs.rolling.maxRetainedFiles(none) + Sets the number of latest rolling log files that are going to be retained by the system. + Older log files will be deleted. Disabled by default. +
    spark.executor.logs.rolling.size.maxBytes(none) + Set the max size of the file by which the executor logs will be rolled over. + Rolling is disabled by default. Value is set in terms of bytes. + See spark.executor.logs.rolling.maxRetainedFiles + for automatic cleaning of old logs. +
    spark.executor.logs.rolling.strategy (none)
    spark.executor.logs.rolling.size.maxBytes(none) - Set the max size of the file by which the executor logs will be rolled over. - Rolling is disabled by default. Value is set in terms of bytes. - See spark.executor.logs.rolling.maxRetainedFiles - for automatic cleaning of old logs. -
    spark.executor.logs.rolling.maxRetainedFiles(none) - Sets the number of latest rolling log files that are going to be retained by the system. - Older log files will be deleted. Disabled by default. -
    spark.executor.userClassPathFirst false
    spark.python.worker.memory512mspark.executorEnv.[EnvironmentVariableName](none) - Amount of memory to use per python worker process during aggregation, in the same - format as JVM memory strings (e.g. 512m, 2g). If the memory - used during aggregation goes above this amount, it will spill the data into disks. + Add the environment variable specified by EnvironmentVariableName to the Executor + process. The user can specify multiple of these to set multiple environment variables.
    spark.python.worker.memory512m + Amount of memory to use per python worker process during aggregation, in the same + format as JVM memory strings (e.g. 512m, 2g). If the memory + used during aggregation goes above this amount, it will spill the data into disks. +
    spark.python.worker.reuse true
    + +#### Shuffle Behavior + + - - + + - - + + - - + + -
    Property NameDefaultMeaning
    spark.executorEnv.[EnvironmentVariableName](none)spark.reducer.maxMbInFlight48 - Add the environment variable specified by EnvironmentVariableName to the Executor - process. The user can specify multiple of these to set multiple environment variables. + Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since + each output requires us to create a buffer to receive it, this represents a fixed memory + overhead per reduce task, so keep it small unless you have a large amount of memory.
    spark.mesos.executor.homedriver side SPARK_HOMEspark.shuffle.blockTransferServicenetty - Set the directory in which Spark is installed on the executors in Mesos. By default, the - executors will simply use the driver's Spark home directory, which may not be visible to - them. Note that this is only relevant if a Spark binary package is not specified through - spark.executor.uri. + Implementation to use for transferring shuffle and cached blocks between executors. There + are two implementations available: netty and nio. Netty-based + block transfer is intended to be simpler but equally efficient and is the default option + starting in 1.2.
    spark.mesos.executor.memoryOverheadexecutor memory * 0.07, with minimum of 384spark.shuffle.compresstrue - This value is an additive for spark.executor.memory, specified in MiB, - which is used to calculate the total Mesos task memory. A value of 384 - implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum - overhead. The final overhead will be the larger of either - `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. + Whether to compress map output files. Generally a good idea. Compression will use + spark.io.compression.codec.
    - -#### Shuffle Behavior - - @@ -421,55 +391,46 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + - - + + - + - - - - - - - + + @@ -481,6 +442,17 @@ Apart from these, the following properties are also available, and may be useful the default option starting in 1.2. + + + + + @@ -490,41 +462,49 @@ Apart from these, the following properties are also available, and may be useful - - + + -
    Property NameDefaultMeaning
    spark.shuffle.consolidateFiles false
    spark.shuffle.spilltruespark.shuffle.file.buffer.kb32 - If set to "true", limits the amount of memory used during reduces by spilling data out to disk. - This spilling threshold is specified by spark.shuffle.memoryFraction. + Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers + reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
    spark.shuffle.spill.compresstruespark.shuffle.io.maxRetries3 - Whether to compress data spilled during shuffles. Compression will use - spark.io.compression.codec. + (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is + set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC + pauses or transient network connectivity issues.
    spark.shuffle.memoryFraction0.2spark.shuffle.io.numConnectionsPerPeer1 - Fraction of Java heap to use for aggregation and cogroups during shuffles, if - spark.shuffle.spill is true. At any given time, the collective size of - all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will - begin to spill to disk. If spills are often, consider increasing this value at the expense of - spark.storage.memoryFraction. + (Netty only) Connections between hosts are reused in order to reduce connection buildup for + large clusters. For clusters with many hard disks and few hosts, this may result in insufficient + concurrency to saturate all disks, and so users may consider increasing this value.
    spark.shuffle.compressspark.shuffle.io.preferDirectBufs true - Whether to compress map output files. Generally a good idea. Compression will use - spark.io.compression.codec. -
    spark.shuffle.file.buffer.kb32 - Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers - reduce the number of disk seeks and system calls made in creating intermediate shuffle files. + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to + turn this off to force all allocations from Netty to be on-heap.
    spark.reducer.maxMbInFlight48spark.shuffle.io.retryWait5 - Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since - each output requires us to create a buffer to receive it, this represents a fixed memory - overhead per reduce task, so keep it small unless you have a large amount of memory. + (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 15 seconds.
    spark.shuffle.memoryFraction0.2 + Fraction of Java heap to use for aggregation and cogroups during shuffles, if + spark.shuffle.spill is true. At any given time, the collective size of + all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will + begin to spill to disk. If spills are often, consider increasing this value at the expense of + spark.storage.memoryFraction. +
    spark.shuffle.sort.bypassMergeThreshold 200
    spark.shuffle.blockTransferServicenettyspark.shuffle.spilltrue - Implementation to use for transferring shuffle and cached blocks between executors. There - are two implementations available: netty and nio. Netty-based - block transfer is intended to be simpler but equally efficient and is the default option - starting in 1.2. + If set to "true", limits the amount of memory used during reduces by spilling data out to disk. + This spilling threshold is specified by spark.shuffle.memoryFraction.
    - -#### Spark UI + + spark.shuffle.spill.compress + true + + Whether to compress data spilled during shuffles. Compression will use + spark.io.compression.codec. + + + + +#### Spark UI - - + + - - + + - - + + @@ -535,28 +515,26 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + - - + +
    Property NameDefaultMeaning
    spark.ui.port4040spark.eventLog.compressfalse - Port for your application's dashboard, which shows memory and workload data. + Whether to compress logged events, if spark.eventLog.enabled is true.
    spark.ui.retainedStages1000spark.eventLog.dirfile:///tmp/spark-events - How many stages the Spark UI and status APIs remember before garbage - collecting. + Base directory in which Spark events are logged, if spark.eventLog.enabled is true. + Within this base directory, Spark creates a sub-directory for each application, and logs the + events specific to the application in this directory. Users may want to set this to + a unified location like an HDFS directory so history files can be read by the history server.
    spark.ui.retainedJobs1000spark.eventLog.enabledfalse - How many jobs the Spark UI and status APIs remember before garbage - collecting. + Whether to log Spark events, useful for reconstructing the Web UI after the application has + finished.
    spark.eventLog.enabledfalsespark.ui.port4040 - Whether to log Spark events, useful for reconstructing the Web UI after the application has - finished. + Port for your application's dashboard, which shows memory and workload data.
    spark.eventLog.compressfalsespark.ui.retainedJobs1000 - Whether to compress logged events, if spark.eventLog.enabled is true. + How many jobs the Spark UI and status APIs remember before garbage + collecting.
    spark.eventLog.dirfile:///tmp/spark-eventsspark.ui.retainedStages1000 - Base directory in which Spark events are logged, if spark.eventLog.enabled is true. - Within this base directory, Spark creates a sub-directory for each application, and logs the - events specific to the application in this directory. Users may want to set this to - a unified location like an HDFS directory so history files can be read by the history server. + How many stages the Spark UI and status APIs remember before garbage + collecting.
    @@ -572,12 +550,10 @@ Apart from these, the following properties are also available, and may be useful - spark.rdd.compress - false + spark.closure.serializer + org.apache.spark.serializer.
    JavaSerializer - Whether to compress serialized RDD partitions (e.g. for - StorageLevel.MEMORY_ONLY_SER). Can save substantial space at the cost of some - extra CPU time. + Serializer class to use for closures. Currently only the Java serializer is supported. @@ -593,14 +569,6 @@ Apart from these, the following properties are also available, and may be useful and org.apache.spark.io.SnappyCompressionCodec. - - spark.io.compression.snappy.block.size - 32768 - - Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec - is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. - - spark.io.compression.lz4.block.size 32768 @@ -610,21 +578,20 @@ Apart from these, the following properties are also available, and may be useful - spark.closure.serializer - org.apache.spark.serializer.
    JavaSerializer + spark.io.compression.snappy.block.size + 32768 - Serializer class to use for closures. Currently only the Java serializer is supported. + Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec + is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. - spark.serializer.objectStreamReset - 100 + spark.kryo.classesToRegister + (none) - When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches - objects to prevent writing redundant data, however that stops garbage collection of those - objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to -1. - By default it will reset the serializer every 100 objects. + If you use Kryo serialization, give a comma-separated list of custom class names to register + with Kryo. + See the tuning guide for more details. @@ -649,12 +616,16 @@ Apart from these, the following properties are also available, and may be useful - spark.kryoserializer.buffer.mb - 0.064 + spark.kryo.registrator + (none) - Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer - per core on each worker. This buffer will grow up to - spark.kryoserializer.buffer.max.mb if needed. + If you use Kryo serialization, set this class to register your custom classes with Kryo. This + property is useful if you need to register your classes in a custom way, e.g. to specify a custom + field serializer. Otherwise spark.kryo.classesToRegister is simpler. It should be + set to a class that extends + + KryoRegistrator. + See the tuning guide for more details. @@ -666,11 +637,80 @@ Apart from these, the following properties are also available, and may be useful inside Kryo. + + spark.kryoserializer.buffer.mb + 0.064 + + Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer + per core on each worker. This buffer will grow up to + spark.kryoserializer.buffer.max.mb if needed. + + + + spark.rdd.compress + false + + Whether to compress serialized RDD partitions (e.g. for + StorageLevel.MEMORY_ONLY_SER). Can save substantial space at the cost of some + extra CPU time. + + + + spark.serializer + org.apache.spark.serializer.
    JavaSerializer + + Class to use for serializing objects that will be sent over the network or need to be cached + in serialized form. The default of Java serialization works with any Serializable Java object + but is quite slow, so we recommend using + org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization + when speed is necessary. Can be any subclass of + + org.apache.spark.Serializer. + + + + spark.serializer.objectStreamReset + 100 + + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to -1. + By default it will reset the serializer every 100 objects. + + #### Execution Behavior + + + + + + + + + + + + + + + - - - + + + - - + + @@ -713,12 +752,23 @@ Apart from these, the following properties are also available, and may be useful - - - + + + + + + + + @@ -729,6 +779,15 @@ Apart from these, the following properties are also available, and may be useful increase it if you configure your own old generation size. + + + + + @@ -747,15 +806,6 @@ Apart from these, the following properties are also available, and may be useful directories on Tachyon file system. - - - - - @@ -763,106 +813,19 @@ Apart from these, the following properties are also available, and may be useful The URL of the underlying Tachyon file system in the TachyonStore. - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.broadcast.blockSize4096 + Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Too large a value decreases parallelism during broadcast (makes it slower); however, if it is + too small, BlockManager might take a performance hit. +
    spark.broadcast.factoryorg.apache.spark.broadcast.
    TorrentBroadcastFactory
    + Which broadcast implementation to use. +
    spark.cleaner.ttl(infinite) + Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks + generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be + forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in + case of Spark Streaming applications). Note that any RDD that persists in memory for more than + this duration will be cleared as well. +
    spark.default.parallelism @@ -689,19 +729,18 @@ Apart from these, the following properties are also available, and may be useful
    spark.broadcast.factoryorg.apache.spark.broadcast.
    TorrentBroadcastFactory
    - Which broadcast implementation to use. - spark.executor.heartbeatInterval10000Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + the driver know that the executor is still alive and update it with metrics for in-progress + tasks.
    spark.broadcast.blockSize4096spark.files.fetchTimeout60 - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. - Too large a value decreases parallelism during broadcast (makes it slower); however, if it is - too small, BlockManager might take a performance hit. + Communication timeout to use when fetching files added through SparkContext.addFile() from + the driver, in seconds.
    spark.files.fetchTimeout60 - Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. - spark.hadoop.cloneConffalseIf set to true, clones a new Hadoop Configuration object for each task. This + option should be enabled to work around Configuration thread-safety issues (see + SPARK-2546 for more details). + This is disabled by default in order to avoid unexpected performance regressions for jobs that + are not affected by these issues.
    spark.hadoop.validateOutputSpecstrueIf set to true, validates the output specification (e.g. checking if the output directory already exists) + used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing + output directories. We recommend that users do not disable this except if trying to achieve compatibility with + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since + data may need to be rewritten to pre-existing output directories during checkpoint recovery.
    spark.storage.memoryFraction
    spark.storage.memoryMapThreshold2097152 + Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + This prevents Spark from memory mapping very small blocks. In general, memory + mapping has high overhead for blocks close to or below the page size of the operating system. +
    spark.storage.unrollFraction 0.2
    spark.storage.memoryMapThreshold2097152 - Size of a block, in bytes, above which Spark memory maps when reading a block from disk. - This prevents Spark from memory mapping very small blocks. In general, memory - mapping has high overhead for blocks close to or below the page size of the operating system. -
    spark.tachyonStore.url tachyon://localhost:19998
    spark.cleaner.ttl(infinite) - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks - generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be - forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in - case of Spark Streaming applications). Note that any RDD that persists in memory for more than - this duration will be cleared as well. -
    spark.hadoop.validateOutputSpecstrueIf set to true, validates the output specification (e.g. checking if the output directory already exists) - used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing - output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. - This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since - data may need to be rewritten to pre-existing output directories during checkpoint recovery.
    spark.hadoop.cloneConffalseIf set to true, clones a new Hadoop Configuration object for each task. This - option should be enabled to work around Configuration thread-safety issues (see - SPARK-2546 for more details). - This is disabled by default in order to avoid unexpected performance regressions for jobs that - are not affected by these issues.
    spark.executor.heartbeatInterval10000Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let - the driver know that the executor is still alive and update it with metrics for in-progress - tasks.
    #### Networking - - - - - - - - - - - - - - - - - - - - - - + + - - - - - - - - - - - - - - - @@ -875,28 +838,18 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - - - - - - + + @@ -910,136 +863,114 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - + + - - + + - - + + - - + + -
    Property NameDefaultMeaning
    spark.driver.host(local hostname) - Hostname or IP address for the driver to listen on. - This is used for communicating with the executors and the standalone Master. -
    spark.driver.port(random) - Port for the driver to listen on. - This is used for communicating with the executors and the standalone Master. -
    spark.fileserver.port(random) - Port for the driver's HTTP file server to listen on. -
    spark.broadcast.port(random) - Port for the driver's HTTP broadcast server to listen on. - This is not relevant for torrent broadcast. -
    spark.replClassServer.port(random)spark.akka.failure-detector.threshold300.0 - Port for the driver's HTTP class server to listen on. - This is only relevant for the Spark shell. -
    spark.blockManager.port(random) - Port for all block managers to listen on. These exist on both the driver and the executors. -
    spark.executor.port(random) - Port for the executor to listen on. This is used for communicating with the driver. -
    spark.port.maxRetries16 - Default maximum number of retries when binding to a port before giving up. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be + enabled again, if you plan to use this feature (Not recommended). This maps to akka's + `akka.remote.transport-failure-detector.threshold`. Tune this in combination of + `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
    spark.akka.threads4 - Number of actor threads to use for communication. Can be useful to increase on large clusters - when the driver has a lot of CPU cores. -
    spark.akka.timeout100 - Communication timeout between Spark nodes, in seconds. -
    spark.network.timeout120spark.akka.heartbeat.interval1000 - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, - spark.storage.blockManagerSlaveTimeoutMs or - spark.shuffle.io.connectionTimeout, if they are not configured. + This is set to a larger value to disable the transport failure detector that comes built in to + Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger + interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` + if you need to. A likely positive use case for using failure detector would be: a sensistive + failure detector can help evict rogue executors quickly. However this is usually not the case + as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling + this leads to a lot of exchanges of heart beats between nodes leading to flooding the network + with those.
    spark.akka.heartbeat.interval1000 - This is set to a larger value to disable the transport failure detector that comes built in to Akka. - It can be enabled again, if you plan to use this feature (Not recommended). A larger interval - value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative - for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need - to. A likely positive use case for using failure detector would be: a sensistive failure detector - can help evict rogue executors quickly. However this is usually not the case as GC pauses - and network lags are expected in a real Spark cluster. Apart from that enabling this leads to - a lot of exchanges of heart beats between nodes leading to flooding the network with those. -
    spark.shuffle.io.preferDirectBufstruespark.akka.threads4 - (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache - block transfer. For environments where off-heap memory is tightly limited, users may wish to - turn this off to force all allocations from Netty to be on-heap. + Number of actor threads to use for communication. Can be useful to increase on large clusters + when the driver has a lot of CPU cores.
    spark.shuffle.io.numConnectionsPerPeer1spark.akka.timeout100 - (Netty only) Connections between hosts are reused in order to reduce connection buildup for - large clusters. For clusters with many hard disks and few hosts, this may result in insufficient - concurrency to saturate all disks, and so users may consider increasing this value. + Communication timeout between Spark nodes, in seconds.
    spark.shuffle.io.maxRetries3spark.blockManager.port(random) - (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is - set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC - pauses or transient network connectivity issues. + Port for all block managers to listen on. These exist on both the driver and the executors.
    spark.shuffle.io.retryWait5spark.broadcast.port(random) - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait. The default maximum delay is therefore - 15 seconds, because the default value of maxRetries is 3, and the default - retryWait here is 5 seconds. + Port for the driver's HTTP broadcast server to listen on. + This is not relevant for torrent broadcast.
    - -#### Scheduling - - - - + + - - + + - - + + - - + + - - + + - - + + - - + + +
    Property NameDefaultMeaning
    spark.task.cpus1spark.driver.host(local hostname) - Number of cores to allocate for each task. + Hostname or IP address for the driver to listen on. + This is used for communicating with the executors and the standalone Master.
    spark.task.maxFailures4spark.driver.port(random) - Number of individual task failures before giving up on the job. - Should be greater than or equal to 1. Number of allowed retries = this value - 1. + Port for the driver to listen on. + This is used for communicating with the executors and the standalone Master.
    spark.scheduler.modeFIFOspark.executor.port(random) - The scheduling mode between - jobs submitted to the same SparkContext. Can be set to FAIR - to use fair sharing instead of queueing jobs one after another. Useful for - multi-user services. + Port for the executor to listen on. This is used for communicating with the driver.
    spark.cores.max(not set)spark.fileserver.port(random) - When running on a standalone deploy cluster or a - Mesos cluster in "coarse-grained" - sharing mode, the maximum amount of CPU cores to request for the application from - across the cluster (not from each machine). If not set, the default will be - spark.deploy.defaultCores on Spark's standalone cluster manager, or - infinite (all available cores) on Mesos. + Port for the driver's HTTP file server to listen on.
    spark.mesos.coarsefalsespark.network.timeout120 - If set to "true", runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per - Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use - for the whole duration of the Spark job. + Default timeout for all network interactions, in seconds. This config will be used in + place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.storage.blockManagerSlaveTimeoutMs or + spark.shuffle.io.connectionTimeout, if they are not configured.
    spark.speculationfalsespark.port.maxRetries16 - If set to "true", performs speculative execution of tasks. This means if one or more tasks are - running slowly in a stage, they will be re-launched. + Default maximum number of retries when binding to a port before giving up.
    spark.speculation.interval100spark.replClassServer.port(random) - How often Spark will check for tasks to speculate, in milliseconds. + Port for the driver's HTTP class server to listen on. + This is only relevant for the Spark shell.
    + +#### Scheduling + + - - + + - - + + @@ -1055,19 +986,19 @@ Apart from these, the following properties are also available, and may be useful - + - + @@ -1078,14 +1009,14 @@ Apart from these, the following properties are also available, and may be useful - - + + - + - - + + + + + + + - + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.speculation.quantile0.75spark.cores.max(not set) - Percentage of tasks which must be complete before speculation is enabled for a particular stage. + When running on a standalone deploy cluster or a + Mesos cluster in "coarse-grained" + sharing mode, the maximum amount of CPU cores to request for the application from + across the cluster (not from each machine). If not set, the default will be + spark.deploy.defaultCores on Spark's standalone cluster manager, or + infinite (all available cores) on Mesos.
    spark.speculation.multiplier1.5spark.localExecution.enabledfalse - How many times slower a task is than the median to be considered for speculation. + Enables Spark to run certain jobs, such as first() or take() on the driver, without sending + tasks to the cluster. This can make certain jobs execute very quickly, but may require + shipping a whole partition of data to the driver.
    spark.locality.wait.processspark.locality.wait.node spark.locality.wait - Customize the locality wait for process locality. This affects tasks that attempt to access - cached data in a particular executor process. + Customize the locality wait for node locality. For example, you can set this to 0 to skip + node locality and search immediately for rack locality (if your cluster has rack information).
    spark.locality.wait.nodespark.locality.wait.process spark.locality.wait - Customize the locality wait for node locality. For example, you can set this to 0 to skip - node locality and search immediately for rack locality (if your cluster has rack information). + Customize the locality wait for process locality. This affects tasks that attempt to access + cached data in a particular executor process.
    spark.scheduler.revive.interval1000spark.scheduler.maxRegisteredResourcesWaitingTime30000 - The interval length for the scheduler to revive the worker resource offers to run tasks + Maximum amount of time to wait for resources to register before scheduling begins (in milliseconds).
    spark.scheduler.minRegisteredResourcesRatio 0.0 for Mesos and Standalone mode, 0.8 for YARN @@ -1098,25 +1029,70 @@ Apart from these, the following properties are also available, and may be useful
    spark.scheduler.maxRegisteredResourcesWaitingTime30000spark.scheduler.modeFIFO - Maximum amount of time to wait for resources to register before scheduling begins + The scheduling mode between + jobs submitted to the same SparkContext. Can be set to FAIR + to use fair sharing instead of queueing jobs one after another. Useful for + multi-user services. +
    spark.scheduler.revive.interval1000 + The interval length for the scheduler to revive the worker resource offers to run tasks (in milliseconds).
    spark.localExecution.enabledspark.speculation false - Enables Spark to run certain jobs, such as first() or take() on the driver, without sending - tasks to the cluster. This can make certain jobs execute very quickly, but may require - shipping a whole partition of data to the driver. + If set to "true", performs speculative execution of tasks. This means if one or more tasks are + running slowly in a stage, they will be re-launched. +
    spark.speculation.interval100 + How often Spark will check for tasks to speculate, in milliseconds. +
    spark.speculation.multiplier1.5 + How many times slower a task is than the median to be considered for speculation. +
    spark.speculation.quantile0.75 + Percentage of tasks which must be complete before speculation is enabled for a particular stage. +
    spark.task.cpus1 + Number of cores to allocate for each task. +
    spark.task.maxFailures4 + Number of individual task failures before giving up on the job. + Should be greater than or equal to 1. Number of allowed retries = this value - 1.
    -#### Dynamic allocation +#### Dynamic Allocation @@ -1136,10 +1112,19 @@ Apart from these, the following properties are also available, and may be useful + + + + + + - @@ -1150,10 +1135,10 @@ Apart from these, the following properties are also available, and may be useful - + @@ -1174,20 +1159,30 @@ Apart from these, the following properties are also available, and may be useful description. - - - - -
    Property NameDefaultMeaning
    spark.dynamicAllocation.executorIdleTimeout600 + If dynamic allocation is enabled and an executor has been idle for more than this duration + (in seconds), the executor will be removed. For more detail, see this + description. +
    spark.dynamicAllocation.initialExecutors spark.dynamicAllocation.minExecutors0 - Lower bound for the number of executors if dynamic allocation is enabled. + Initial number of executors to run if dynamic allocation is enabled.
    spark.dynamicAllocation.maxExecutors spark.dynamicAllocation.minExecutors0 - Initial number of executors to run if dynamic allocation is enabled. + Lower bound for the number of executors if dynamic allocation is enabled.
    spark.dynamicAllocation.executorIdleTimeout600 - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this - description. -
    #### Security + + + + + + + + + + @@ -1204,6 +1199,15 @@ Apart from these, the following properties are also available, and may be useful not running on YARN and authentication is enabled. + + + + + @@ -1213,12 +1217,11 @@ Apart from these, the following properties are also available, and may be useful - - + + @@ -1235,16 +1238,6 @@ Apart from these, the following properties are also available, and may be useful -Dspark.com.test.filter1.params='param1=foo,param2=testing' - - - - - @@ -1253,23 +1246,6 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. - - - - - - - - - -
    Property NameDefaultMeaning
    spark.acls.enablefalse + Whether Spark acls should are enabled. If enabled, this checks to see if the user has + access permissions to view or modify the job. Note this requires the user to be known, + so if the user comes across as null no checks are done. Filters can be used with the UI + to authenticate and set the user. +
    spark.admin.aclsEmpty + Comma separated list of users/administrators that have view and modify access to all Spark jobs. + This can be used if you run on a shared cluster and have a set of administrators or devs who + help debug when things work. +
    spark.authenticate false
    spark.core.connection.ack.wait.timeout60 + Number of seconds for the connection to wait for ack to occur before timing + out and giving up. To avoid unwilling timeout caused by long pause like GC, + you can set larger value. +
    spark.core.connection.auth.wait.timeout 30
    spark.core.connection.ack.wait.timeout60spark.modify.aclsEmpty - Number of seconds for the connection to wait for ack to occur before timing - out and giving up. To avoid unwilling timeout caused by long pause like GC, - you can set larger value. + Comma separated list of users that have modify access to the Spark job. By default only the + user that started the Spark job has access to modify it (kill it for example).
    spark.acls.enablefalse - Whether Spark acls should are enabled. If enabled, this checks to see if the user has - access permissions to view or modify the job. Note this requires the user to be known, - so if the user comes across as null no checks are done. Filters can be used with the UI - to authenticate and set the user. -
    spark.ui.view.acls Empty
    spark.modify.aclsEmpty - Comma separated list of users that have modify access to the Spark job. By default only the - user that started the Spark job has access to modify it (kill it for example). -
    spark.admin.aclsEmpty - Comma separated list of users/administrators that have view and modify access to all Spark jobs. - This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. -
    #### Encryption @@ -1293,6 +1269,23 @@ Apart from these, the following properties are also available, and may be useful file server.

    + + spark.ssl.enabledAlgorithms + Empty + + A comma separated list of ciphers. The specified ciphers must be supported by JVM. + The reference list of protocols one can find on + this + page. + + + + spark.ssl.keyPassword + None + + A password to the private key in key-store. + + spark.ssl.keyStore None @@ -1309,10 +1302,12 @@ Apart from these, the following properties are also available, and may be useful - spark.ssl.keyPassword + spark.ssl.protocol None - A password to the private key in key-store. + A protocol name. The protocol must be supported by JVM. The reference list of protocols + one can find on this + page. @@ -1330,25 +1325,6 @@ Apart from these, the following properties are also available, and may be useful A password to the trust-store. - - spark.ssl.protocol - None - - A protocol name. The protocol must be supported by JVM. The reference list of protocols - one can find on this - page. - - - - spark.ssl.enabledAlgorithms - Empty - - A comma separated list of ciphers. The specified ciphers must be supported by JVM. - The reference list of protocols one can find on - this - page. - - diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 78358499fd01f..db1173a06b0b1 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -197,7 +197,11 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.coarse false - Set the run mode for Spark on Mesos. For more information about the run mode, refer to #Mesos Run Mode section above. + If set to "true", runs over Mesos clusters in + "coarse-grained" sharing mode, + where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per + Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use + for the whole duration of the Spark job. @@ -211,19 +215,23 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.executor.home - SPARK_HOME + driver side SPARK_HOME - The location where the mesos executor will look for Spark binaries to execute, and uses the SPARK_HOME setting on default. - This variable is only used when no spark.executor.uri is provided, and assumes Spark is installed on the specified location - on each slave. + Set the directory in which Spark is installed on the executors in Mesos. By default, the + executors will simply use the driver's Spark home directory, which may not be visible to + them. Note that this is only relevant if a Spark binary package is not specified through + spark.executor.uri. spark.mesos.executor.memoryOverhead - 384 + executor memory * 0.07, with minimum of 384 - The amount of memory that Mesos executor will request for the task to account for the overhead of running the executor itself. - The final total amount of memory allocated is the maximum value between executor memory plus memoryOverhead, and overhead fraction (1.07) plus the executor memory. + This value is an additive for spark.executor.memory, specified in MiB, + which is used to calculate the total Mesos task memory. A value of 384 + implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + overhead. The final overhead will be the larger of either + `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. From d20559b157743981b9c09e286f2aaff8cbefab59 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 25 Feb 2015 16:13:17 -0800 Subject: [PATCH 258/817] [SPARK-5974] [SPARK-5980] [mllib] [python] [docs] Update ML guide with save/load, Python GBT * Add GradientBoostedTrees Python examples to ML guide * I ran these in the pyspark shell, and they worked. * Add save/load to examples in ML guide * Added note to python docs about predict,transform not working within RDD actions,transformations in some cases (See SPARK-5981) CC: mengxr Author: Joseph K. Bradley Closes #4750 from jkbradley/SPARK-5974 and squashes the following commits: c410e38 [Joseph K. Bradley] Added note to LabeledPoint about attributes bcae18b [Joseph K. Bradley] Added import of models for save/load examples in ml guide. Fixed line length for tree.py, feature.py (but not other ML Pyspark files yet). 6d81c3e [Joseph K. Bradley] completed python GBT examples 9903309 [Joseph K. Bradley] Added note to python docs about predict,transform not working within RDD actions,transformations in some cases c7dfad8 [Joseph K. Bradley] Added model save/load to ML guide. Added GBT examples to ML guide --- docs/mllib-classification-regression.md | 9 +- docs/mllib-collaborative-filtering.md | 9 ++ docs/mllib-decision-tree.md | 20 +++ docs/mllib-ensembles.md | 94 +++++++++++++- docs/mllib-linear-methods.md | 21 +++- docs/mllib-naive-bayes.md | 10 +- python/pyspark/mllib/feature.py | 67 ++++++---- python/pyspark/mllib/regression.py | 7 +- python/pyspark/mllib/tree.py | 156 ++++++++++++++---------- 9 files changed, 296 insertions(+), 97 deletions(-) diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 5b9b4dd83b774..8e91d62f4a907 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -17,13 +17,13 @@ the supported algorithms for each type of problem. - Binary Classificationlinear SVMs, logistic regression, decision trees, naive Bayes + Binary Classificationlinear SVMs, logistic regression, decision trees, random forests, gradient-boosted trees, naive Bayes - Multiclass Classificationdecision trees, naive Bayes + Multiclass Classificationdecision trees, random forests, naive Bayes - Regressionlinear least squares, Lasso, ridge regression, decision trees, isotonic regression + Regressionlinear least squares, Lasso, ridge regression, decision trees, random forests, gradient-boosted trees, isotonic regression @@ -34,5 +34,8 @@ More details for these methods can be found here: * [binary classification (SVMs, logistic regression)](mllib-linear-methods.html#binary-classification) * [linear regression (least squares, Lasso, ridge)](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) * [Decision trees](mllib-decision-tree.html) +* [Ensembles of decision trees](mllib-ensembles.html) + * [random forests](mllib-ensembles.html#random-forests) + * [gradient-boosted trees](mllib-ensembles.html#gradient-boosted-trees-gbts) * [Naive Bayes](mllib-naive-bayes.html) * [Isotonic regression](mllib-isotonic-regression.html) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index ef18cec9371d6..935cd8dad3b25 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -66,6 +66,7 @@ recommendation model by measuring the Mean Squared Error of rating prediction. {% highlight scala %} import org.apache.spark.mllib.recommendation.ALS +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel import org.apache.spark.mllib.recommendation.Rating // Load and parse the data @@ -95,6 +96,9 @@ val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => err * err }.mean() println("Mean Squared Error = " + MSE) + +model.save("myModelPath") +val sameModel = MatrixFactorizationModel.load("myModelPath") {% endhighlight %} If the rating matrix is derived from another source of information (e.g., it is inferred from @@ -181,6 +185,9 @@ public class CollaborativeFiltering { } ).rdd()).mean(); System.out.println("Mean Squared Error = " + MSE); + + model.save("myModelPath"); + MatrixFactorizationModel sameModel = MatrixFactorizationModel.load("myModelPath"); } } {% endhighlight %} @@ -191,6 +198,8 @@ In the following example we load rating data. Each row consists of a user, a pro We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation by measuring the Mean Squared Error of rating prediction. +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.recommendation import ALS, Rating diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 6675133a810db..4695d1cde4901 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -194,6 +194,7 @@ maximum tree depth of 5. The test error is calculated to measure the algorithm a
    {% highlight scala %} import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.model.DecisionTreeModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -221,6 +222,9 @@ val labelAndPreds = testData.map { point => val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() println("Test Error = " + testErr) println("Learned classification tree model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = DecisionTreeModel.load("myModelPath") {% endhighlight %}
    @@ -279,10 +283,16 @@ Double testErr = }).count() / testData.count(); System.out.println("Test Error: " + testErr); System.out.println("Learned classification tree model:\n" + model.toDebugString()); + +model.save("myModelPath"); +DecisionTreeModel sameModel = DecisionTreeModel.load("myModelPath"); {% endhighlight %}
    + +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.tree import DecisionTree @@ -324,6 +334,7 @@ depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate
    {% highlight scala %} import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.model.DecisionTreeModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -350,6 +361,9 @@ val labelsAndPredictions = testData.map { point => val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() println("Test Mean Squared Error = " + testMSE) println("Learned regression tree model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = DecisionTreeModel.load("myModelPath") {% endhighlight %}
    @@ -414,10 +428,16 @@ Double testMSE = }) / data.count(); System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression tree model:\n" + model.toDebugString()); + +model.save("myModelPath"); +DecisionTreeModel sameModel = DecisionTreeModel.load("myModelPath"); {% endhighlight %}
    + +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.tree import DecisionTree diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index 00040e6073d0d..ddae84165f8a9 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -98,6 +98,7 @@ The test error is calculated to measure the algorithm accuracy.
    {% highlight scala %} import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.tree.model.RandomForestModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -127,6 +128,9 @@ val labelAndPreds = testData.map { point => val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() println("Test Error = " + testErr) println("Learned classification forest model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = RandomForestModel.load("myModelPath") {% endhighlight %}
    @@ -188,10 +192,16 @@ Double testErr = }).count() / testData.count(); System.out.println("Test Error: " + testErr); System.out.println("Learned classification forest model:\n" + model.toDebugString()); + +model.save("myModelPath"); +RandomForestModel sameModel = RandomForestModel.load("myModelPath"); {% endhighlight %}
    + +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.tree import RandomForest from pyspark.mllib.util import MLUtils @@ -235,6 +245,7 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
    {% highlight scala %} import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.tree.model.RandomForestModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -264,6 +275,9 @@ val labelsAndPredictions = testData.map { point => val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() println("Test Mean Squared Error = " + testMSE) println("Learned regression forest model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = RandomForestModel.load("myModelPath") {% endhighlight %}
    @@ -328,10 +342,16 @@ Double testMSE = }) / testData.count(); System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression forest model:\n" + model.toDebugString()); + +model.save("myModelPath"); +RandomForestModel sameModel = RandomForestModel.load("myModelPath"); {% endhighlight %}
    + +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.tree import RandomForest from pyspark.mllib.util import MLUtils @@ -441,8 +461,6 @@ iterations. ### Examples -GBTs currently have APIs in Scala and Java. Examples in both languages are shown below. - #### Classification The example below demonstrates how to load a @@ -457,6 +475,7 @@ The test error is calculated to measure the algorithm accuracy. {% highlight scala %} import org.apache.spark.mllib.tree.GradientBoostedTrees import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -484,6 +503,9 @@ val labelAndPreds = testData.map { point => val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() println("Test Error = " + testErr) println("Learned classification GBT model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = GradientBoostedTreesModel.load("myModelPath") {% endhighlight %}
    @@ -545,6 +567,38 @@ Double testErr = }).count() / testData.count(); System.out.println("Test Error: " + testErr); System.out.println("Learned classification GBT model:\n" + model.toDebugString()); + +model.save("myModelPath"); +GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load("myModelPath"); +{% endhighlight %} + + +
    + +Note that the Python API does not yet support model save/load but will in the future. + +{% highlight python %} +from pyspark.mllib.tree import GradientBoostedTrees +from pyspark.mllib.util import MLUtils + +# Load and parse the data file. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a GradientBoostedTrees model. +# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. +# (b) Use more iterations in practice. +model = GradientBoostedTrees.trainClassifier(trainingData, + categoricalFeaturesInfo={}, numIterations=3) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) +print('Test Error = ' + str(testErr)) +print('Learned classification GBT model:') +print(model.toDebugString()) {% endhighlight %}
    @@ -565,6 +619,7 @@ The Mean Squared Error (MSE) is computed at the end to evaluate {% highlight scala %} import org.apache.spark.mllib.tree.GradientBoostedTrees import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. @@ -591,6 +646,9 @@ val labelsAndPredictions = testData.map { point => val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() println("Test Mean Squared Error = " + testMSE) println("Learned regression GBT model:\n" + model.toDebugString) + +model.save("myModelPath") +val sameModel = GradientBoostedTreesModel.load("myModelPath") {% endhighlight %} @@ -658,6 +716,38 @@ Double testMSE = }) / data.count(); System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression GBT model:\n" + model.toDebugString()); + +model.save("myModelPath"); +GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load("myModelPath"); +{% endhighlight %} + + +
    + +Note that the Python API does not yet support model save/load but will in the future. + +{% highlight python %} +from pyspark.mllib.tree import GradientBoostedTrees +from pyspark.mllib.util import MLUtils + +# Load and parse the data file. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a GradientBoostedTrees model. +# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. +# (b) Use more iterations in practice. +model = GradientBoostedTrees.trainRegressor(trainingData, + categoricalFeaturesInfo={}, numIterations=3) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) +print('Test Mean Squared Error = ' + str(testMSE)) +print('Learned regression GBT model:') +print(model.toDebugString()) {% endhighlight %}
    diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 44b7f67c57734..d9fc63b37d116 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -190,7 +190,7 @@ error. {% highlight scala %} import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.SVMWithSGD +import org.apache.spark.mllib.classification.{SVMModel, SVMWithSGD} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors @@ -222,6 +222,9 @@ val metrics = new BinaryClassificationMetrics(scoreAndLabels) val auROC = metrics.areaUnderROC() println("Area under ROC = " + auROC) + +model.save("myModelPath") +val sameModel = SVMModel.load("myModelPath") {% endhighlight %} The `SVMWithSGD.train()` method by default performs L2 regularization with the @@ -304,6 +307,9 @@ public class SVMClassifier { double auROC = metrics.areaUnderROC(); System.out.println("Area under ROC = " + auROC); + + model.save("myModelPath"); + SVMModel sameModel = SVMModel.load("myModelPath"); } } {% endhighlight %} @@ -338,6 +344,8 @@ a dependency. The following example shows how to load a sample dataset, build Logistic Regression model, and make predictions with the resulting model to compute the training error. +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.classification import LogisticRegressionWithSGD from pyspark.mllib.regression import LabeledPoint @@ -391,8 +399,9 @@ values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). {% highlight scala %} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.mllib.linalg.Vectors // Load and parse the data @@ -413,6 +422,9 @@ val valuesAndPreds = parsedData.map { point => } val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) + +model.save("myModelPath") +val sameModel = LinearRegressionModel.load("myModelPath") {% endhighlight %} [`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) @@ -483,6 +495,9 @@ public class LinearRegression { } ).rdd()).mean(); System.out.println("training Mean Squared Error = " + MSE); + + model.save("myModelPath"); + LinearRegressionModel sameModel = LinearRegressionModel.load("myModelPath"); } } {% endhighlight %} @@ -494,6 +509,8 @@ The example then uses LinearRegressionWithSGD to build a simple linear model to values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD from numpy import array diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index d5b044d94fdd7..81173255b590d 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -37,7 +37,7 @@ smoothing parameter `lambda` as input, and output a can be used for evaluation and prediction. {% highlight scala %} -import org.apache.spark.mllib.classification.NaiveBayes +import org.apache.spark.mllib.classification.{NaiveBayes, NaiveBayesModel} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint @@ -55,6 +55,9 @@ val model = NaiveBayes.train(training, lambda = 1.0) val predictionAndLabel = test.map(p => (model.predict(p.features), p.label)) val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() + +model.save("myModelPath") +val sameModel = NaiveBayesModel.load("myModelPath") {% endhighlight %} @@ -93,6 +96,9 @@ double accuracy = predictionAndLabel.filter(new Function, return pl._1().equals(pl._2()); } }).count() / (double) test.count(); + +model.save("myModelPath"); +NaiveBayesModel sameModel = NaiveBayesModel.load("myModelPath"); {% endhighlight %} @@ -105,6 +111,8 @@ smoothing parameter `lambda` as input, and output a [NaiveBayesModel](api/python/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be used for evaluation and prediction. +Note that the Python API does not yet support model save/load but will in the future. + {% highlight python %} from pyspark.mllib.regression import LabeledPoint diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 10df6288065b8..0ffe092a07365 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -58,7 +58,8 @@ class Normalizer(VectorTransformer): For any 1 <= `p` < float('inf'), normalizes samples using sum(abs(vector) :sup:`p`) :sup:`(1/p)` as norm. - For `p` = float('inf'), max(abs(vector)) will be used as norm for normalization. + For `p` = float('inf'), max(abs(vector)) will be used as norm for + normalization. >>> v = Vectors.dense(range(3)) >>> nor = Normalizer(1) @@ -120,9 +121,14 @@ def transform(self, vector): """ Applies standardization transformation on a vector. + Note: In Python, transform cannot currently be used within + an RDD transformation or action. + Call transform directly on the RDD instead. + :param vector: Vector or RDD of Vector to be standardized. - :return: Standardized vector. If the variance of a column is zero, - it will return default `0.0` for the column with zero variance. + :return: Standardized vector. If the variance of a column is + zero, it will return default `0.0` for the column with + zero variance. """ return JavaVectorTransformer.transform(self, vector) @@ -148,9 +154,10 @@ def __init__(self, withMean=False, withStd=True): """ :param withMean: False by default. Centers the data with mean before scaling. It will build a dense output, so this - does not work on sparse input and will raise an exception. - :param withStd: True by default. Scales the data to unit standard - deviation. + does not work on sparse input and will raise an + exception. + :param withStd: True by default. Scales the data to unit + standard deviation. """ if not (withMean or withStd): warnings.warn("Both withMean and withStd are false. The model does nothing.") @@ -159,10 +166,11 @@ def __init__(self, withMean=False, withStd=True): def fit(self, dataset): """ - Computes the mean and variance and stores as a model to be used for later scaling. + Computes the mean and variance and stores as a model to be used + for later scaling. - :param data: The data used to compute the mean and variance to build - the transformation model. + :param data: The data used to compute the mean and variance + to build the transformation model. :return: a StandardScalarModel """ dataset = dataset.map(_convert_to_vector) @@ -174,7 +182,8 @@ class HashingTF(object): """ .. note:: Experimental - Maps a sequence of terms to their term frequencies using the hashing trick. + Maps a sequence of terms to their term frequencies using the hashing + trick. Note: the terms must be hashable (can not be dict/set/list...). @@ -195,8 +204,9 @@ def indexOf(self, term): def transform(self, document): """ - Transforms the input document (list of terms) to term frequency vectors, - or transform the RDD of document to RDD of term frequency vectors. + Transforms the input document (list of terms) to term frequency + vectors, or transform the RDD of document to RDD of term + frequency vectors. """ if isinstance(document, RDD): return document.map(self.transform) @@ -220,7 +230,12 @@ def transform(self, x): the terms which occur in fewer than `minDocFreq` documents will have an entry of 0. - :param x: an RDD of term frequency vectors or a term frequency vector + Note: In Python, transform cannot currently be used within + an RDD transformation or action. + Call transform directly on the RDD instead. + + :param x: an RDD of term frequency vectors or a term frequency + vector :return: an RDD of TF-IDF vectors or a TF-IDF vector """ if isinstance(x, RDD): @@ -241,9 +256,9 @@ class IDF(object): of documents that contain term `t`. This implementation supports filtering out terms which do not appear - in a minimum number of documents (controlled by the variable `minDocFreq`). - For terms that are not in at least `minDocFreq` documents, the IDF is - found as 0, resulting in TF-IDFs of 0. + in a minimum number of documents (controlled by the variable + `minDocFreq`). For terms that are not in at least `minDocFreq` + documents, the IDF is found as 0, resulting in TF-IDFs of 0. >>> n = 4 >>> freqs = [Vectors.sparse(n, (1, 3), (1.0, 2.0)), @@ -325,15 +340,16 @@ class Word2Vec(object): The vector representation can be used as features in natural language processing and machine learning algorithms. - We used skip-gram model in our implementation and hierarchical softmax - method to train the model. The variable names in the implementation - matches the original C implementation. + We used skip-gram model in our implementation and hierarchical + softmax method to train the model. The variable names in the + implementation matches the original C implementation. - For original C implementation, see https://code.google.com/p/word2vec/ + For original C implementation, + see https://code.google.com/p/word2vec/ For research papers, see Efficient Estimation of Word Representations in Vector Space - and - Distributed Representations of Words and Phrases and their Compositionality. + and Distributed Representations of Words and Phrases and their + Compositionality. >>> sentence = "a b " * 100 + "a c " * 10 >>> localDoc = [sentence, sentence] @@ -374,15 +390,16 @@ def setLearningRate(self, learningRate): def setNumPartitions(self, numPartitions): """ - Sets number of partitions (default: 1). Use a small number for accuracy. + Sets number of partitions (default: 1). Use a small number for + accuracy. """ self.numPartitions = numPartitions return self def setNumIterations(self, numIterations): """ - Sets number of iterations (default: 1), which should be smaller than or equal to number of - partitions. + Sets number of iterations (default: 1), which should be smaller + than or equal to number of partitions. """ self.numIterations = numIterations return self diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 21751cc68f308..66617abb85670 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -31,8 +31,11 @@ class LabeledPoint(object): The features and labels of a data point. :param label: Label for this data point. - :param features: Vector of features for this point (NumPy array, list, - pyspark.mllib.linalg.SparseVector, or scipy.sparse column matrix) + :param features: Vector of features for this point (NumPy array, + list, pyspark.mllib.linalg.SparseVector, or scipy.sparse + column matrix) + + Note: 'label' and 'features' are accessible as class attributes. """ def __init__(self, label, features): diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 02d551b87dcc0..73618f0449ad4 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -33,6 +33,10 @@ def predict(self, x): """ Predict values for a single data point or an RDD of points using the model trained. + + Note: In Python, predict cannot currently be used within an RDD + transformation or action. + Call predict directly on the RDD instead. """ if isinstance(x, RDD): return self.call("predict", x.map(_convert_to_vector)) @@ -48,7 +52,8 @@ def numTrees(self): def totalNumNodes(self): """ - Get total number of nodes, summed over all trees in the ensemble. + Get total number of nodes, summed over all trees in the + ensemble. """ return self.call("totalNumNodes") @@ -71,6 +76,10 @@ def predict(self, x): """ Predict the label of one or more examples. + Note: In Python, predict cannot currently be used within an RDD + transformation or action. + Call predict directly on the RDD instead. + :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ @@ -99,7 +108,8 @@ class DecisionTree(object): """ .. note:: Experimental - Learning algorithm for a decision tree model for classification or regression. + Learning algorithm for a decision tree model for classification or + regression. """ @classmethod @@ -176,17 +186,17 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, :param data: Training data: RDD of LabeledPoint. Labels are real numbers. - :param categoricalFeaturesInfo: Map from categorical feature index - to number of categories. - Any feature not in this map - is treated as continuous. + :param categoricalFeaturesInfo: Map from categorical feature + index to number of categories. + Any feature not in this map is treated as continuous. :param impurity: Supported values: "variance" :param maxDepth: Max depth of tree. - E.g., depth 0 means 1 leaf node. - Depth 1 means 1 internal node + 2 leaf nodes. - :param maxBins: Number of bins used for finding splits at each node. - :param minInstancesPerNode: Min number of instances required at child - nodes to create the parent split + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each + node. + :param minInstancesPerNode: Min number of instances required at + child nodes to create the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel @@ -229,7 +239,8 @@ class RandomForest(object): """ .. note:: Experimental - Learning algorithm for a random forest model for classification or regression. + Learning algorithm for a random forest model for classification or + regression. """ supportedFeatureSubsetStrategies = ("auto", "all", "sqrt", "log2", "onethird") @@ -256,26 +267,33 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, Method to train a decision tree model for binary or multiclass classification. - :param data: Training dataset: RDD of LabeledPoint. Labels should take - values {0, 1, ..., numClasses-1}. + :param data: Training dataset: RDD of LabeledPoint. Labels + should take values {0, 1, ..., numClasses-1}. :param numClasses: number of classes for classification. - :param categoricalFeaturesInfo: Map storing arity of categorical features. - E.g., an entry (n -> k) indicates that feature n is categorical - with k categories indexed from 0: {0, 1, ..., k-1}. + :param categoricalFeaturesInfo: Map storing arity of categorical + features. E.g., an entry (n -> k) indicates that + feature n is categorical with k categories indexed + from 0: {0, 1, ..., k-1}. :param numTrees: Number of trees in the random forest. - :param featureSubsetStrategy: Number of features to consider for splits at - each node. - Supported: "auto" (default), "all", "sqrt", "log2", "onethird". - If "auto" is set, this parameter is set based on numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "sqrt". - :param impurity: Criterion used for information gain calculation. + :param featureSubsetStrategy: Number of features to consider for + splits at each node. + Supported: "auto" (default), "all", "sqrt", "log2", + "onethird". + If "auto" is set, this parameter is set based on + numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "sqrt". + :param impurity: Criterion used for information gain + calculation. Supported values: "gini" (recommended) or "entropy". - :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 leaf node; - depth 1 means 1 internal node + 2 leaf nodes. (default: 4) - :param maxBins: maximum number of bins used for splitting features + :param maxDepth: Maximum depth of the tree. + E.g., depth 0 means 1 leaf node; depth 1 means + 1 internal node + 2 leaf nodes. (default: 4) + :param maxBins: maximum number of bins used for splitting + features (default: 100) - :param seed: Random seed for bootstrapping and choosing feature subsets. + :param seed: Random seed for bootstrapping and choosing feature + subsets. :return: RandomForestModel that can be used for prediction Example usage: @@ -337,19 +355,24 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt {0, 1, ..., k-1}. :param numTrees: Number of trees in the random forest. :param featureSubsetStrategy: Number of features to consider for - splits at each node. - Supported: "auto" (default), "all", "sqrt", "log2", "onethird". - If "auto" is set, this parameter is set based on numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "onethird" for regression. - :param impurity: Criterion used for information gain calculation. - Supported values: "variance". - :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 - leaf node; depth 1 means 1 internal node + 2 leaf nodes. - (default: 4) - :param maxBins: maximum number of bins used for splitting features - (default: 100) - :param seed: Random seed for bootstrapping and choosing feature subsets. + splits at each node. + Supported: "auto" (default), "all", "sqrt", "log2", + "onethird". + If "auto" is set, this parameter is set based on + numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "onethird" for + regression. + :param impurity: Criterion used for information gain + calculation. + Supported values: "variance". + :param maxDepth: Maximum depth of the tree. E.g., depth 0 means + 1 leaf node; depth 1 means 1 internal node + 2 leaf + nodes. (default: 4) + :param maxBins: maximum number of bins used for splitting + features (default: 100) + :param seed: Random seed for bootstrapping and choosing feature + subsets. :return: RandomForestModel that can be used for prediction Example usage: @@ -395,7 +418,8 @@ class GradientBoostedTrees(object): """ .. note:: Experimental - Learning algorithm for a gradient boosted trees model for classification or regression. + Learning algorithm for a gradient boosted trees model for + classification or regression. """ @classmethod @@ -411,24 +435,29 @@ def _train(cls, data, algo, categoricalFeaturesInfo, def trainClassifier(cls, data, categoricalFeaturesInfo, loss="logLoss", numIterations=100, learningRate=0.1, maxDepth=3): """ - Method to train a gradient-boosted trees model for classification. + Method to train a gradient-boosted trees model for + classification. - :param data: Training dataset: RDD of LabeledPoint. Labels should take values {0, 1}. + :param data: Training dataset: RDD of LabeledPoint. + Labels should take values {0, 1}. :param categoricalFeaturesInfo: Map storing arity of categorical features. E.g., an entry (n -> k) indicates that feature n is categorical with k categories indexed from 0: {0, 1, ..., k-1}. - :param loss: Loss function used for minimization during gradient boosting. - Supported: {"logLoss" (default), "leastSquaresError", "leastAbsoluteError"}. + :param loss: Loss function used for minimization during gradient + boosting. Supported: {"logLoss" (default), + "leastSquaresError", "leastAbsoluteError"}. :param numIterations: Number of iterations of boosting. (default: 100) - :param learningRate: Learning rate for shrinking the contribution of each estimator. - The learning rate should be between in the interval (0, 1] - (default: 0.1) - :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 - leaf node; depth 1 means 1 internal node + 2 leaf nodes. - (default: 3) - :return: GradientBoostedTreesModel that can be used for prediction + :param learningRate: Learning rate for shrinking the + contribution of each estimator. The learning rate + should be between in the interval (0, 1]. + (default: 0.1) + :param maxDepth: Maximum depth of the tree. E.g., depth 0 means + 1 leaf node; depth 1 means 1 internal node + 2 leaf + nodes. (default: 3) + :return: GradientBoostedTreesModel that can be used for + prediction Example usage: @@ -472,17 +501,20 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, features. E.g., an entry (n -> k) indicates that feature n is categorical with k categories indexed from 0: {0, 1, ..., k-1}. - :param loss: Loss function used for minimization during gradient boosting. - Supported: {"logLoss" (default), "leastSquaresError", "leastAbsoluteError"}. + :param loss: Loss function used for minimization during gradient + boosting. Supported: {"logLoss" (default), + "leastSquaresError", "leastAbsoluteError"}. :param numIterations: Number of iterations of boosting. (default: 100) - :param learningRate: Learning rate for shrinking the contribution of each estimator. - The learning rate should be between in the interval (0, 1] - (default: 0.1) - :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 - leaf node; depth 1 means 1 internal node + 2 leaf nodes. - (default: 3) - :return: GradientBoostedTreesModel that can be used for prediction + :param learningRate: Learning rate for shrinking the + contribution of each estimator. The learning rate + should be between in the interval (0, 1]. + (default: 0.1) + :param maxDepth: Maximum depth of the tree. E.g., depth 0 means + 1 leaf node; depth 1 means 1 internal node + 2 leaf + nodes. (default: 3) + :return: GradientBoostedTreesModel that can be used for + prediction Example usage: From e43139f40309995b1133c7ef2936ab858b7b44fc Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 25 Feb 2015 23:43:29 -0800 Subject: [PATCH 259/817] [SPARK-5976][MLLIB] Add partitioner to factors returned by ALS The model trained by ALS requires partitioning information to do quick lookup of a user/item factor for making recommendation on individual requests. In the new implementation, we didn't set partitioners in the factors returned by ALS, which would cause performance regression. srowen coderxiang Author: Xiangrui Meng Closes #4748 from mengxr/SPARK-5976 and squashes the following commits: 9373a09 [Xiangrui Meng] add partitioner to factors returned by ALS 260f183 [Xiangrui Meng] add a test for partitioner --- .../apache/spark/ml/recommendation/ALS.scala | 55 +++++++++++-------- .../spark/ml/recommendation/ALSSuite.scala | 32 ++++++++++- 2 files changed, 64 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index c2ec716f08b7c..7bb69df65362b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -29,7 +29,7 @@ import com.github.fommil.netlib.LAPACK.{getInstance => lapack} import org.jblas.DoubleMatrix import org.netlib.util.intW -import org.apache.spark.{HashPartitioner, Logging, Partitioner} +import org.apache.spark.{Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ @@ -501,8 +501,8 @@ object ALS extends Logging { require(intermediateRDDStorageLevel != StorageLevel.NONE, "ALS is not designed to run without persisting intermediate RDDs.") val sc = ratings.sparkContext - val userPart = new HashPartitioner(numUserBlocks) - val itemPart = new HashPartitioner(numItemBlocks) + val userPart = new ALSPartitioner(numUserBlocks) + val itemPart = new ALSPartitioner(numItemBlocks) val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) val solver = if (nonnegative) new NNLSSolver else new CholeskySolver @@ -550,13 +550,23 @@ object ALS extends Logging { val userIdAndFactors = userInBlocks .mapValues(_.srcIds) .join(userFactors) - .values + .mapPartitions({ items => + items.flatMap { case (_, (ids, factors)) => + ids.view.zip(factors) + } + // Preserve the partitioning because IDs are consistent with the partitioners in userInBlocks + // and userFactors. + }, preservesPartitioning = true) .setName("userFactors") .persist(finalRDDStorageLevel) val itemIdAndFactors = itemInBlocks .mapValues(_.srcIds) .join(itemFactors) - .values + .mapPartitions({ items => + items.flatMap { case (_, (ids, factors)) => + ids.view.zip(factors) + } + }, preservesPartitioning = true) .setName("itemFactors") .persist(finalRDDStorageLevel) if (finalRDDStorageLevel != StorageLevel.NONE) { @@ -569,13 +579,7 @@ object ALS extends Logging { itemOutBlocks.unpersist() blockRatings.unpersist() } - val userOutput = userIdAndFactors.flatMap { case (ids, factors) => - ids.view.zip(factors) - } - val itemOutput = itemIdAndFactors.flatMap { case (ids, factors) => - ids.view.zip(factors) - } - (userOutput, itemOutput) + (userIdAndFactors, itemIdAndFactors) } /** @@ -995,15 +999,15 @@ object ALS extends Logging { "Converting to local indices took " + (System.nanoTime() - start) / 1e9 + " seconds.") val dstLocalIndices = dstIds.map(dstIdToLocalIndex.apply) (srcBlockId, (dstBlockId, srcIds, dstLocalIndices, ratings)) - }.groupByKey(new HashPartitioner(srcPart.numPartitions)) - .mapValues { iter => - val builder = - new UncompressedInBlockBuilder[ID](new LocalIndexEncoder(dstPart.numPartitions)) - iter.foreach { case (dstBlockId, srcIds, dstLocalIndices, ratings) => - builder.add(dstBlockId, srcIds, dstLocalIndices, ratings) - } - builder.build().compress() - }.setName(prefix + "InBlocks") + }.groupByKey(new ALSPartitioner(srcPart.numPartitions)) + .mapValues { iter => + val builder = + new UncompressedInBlockBuilder[ID](new LocalIndexEncoder(dstPart.numPartitions)) + iter.foreach { case (dstBlockId, srcIds, dstLocalIndices, ratings) => + builder.add(dstBlockId, srcIds, dstLocalIndices, ratings) + } + builder.build().compress() + }.setName(prefix + "InBlocks") .persist(storageLevel) val outBlocks = inBlocks.mapValues { case InBlock(srcIds, dstPtrs, dstEncodedIndices, _) => val encoder = new LocalIndexEncoder(dstPart.numPartitions) @@ -1064,7 +1068,7 @@ object ALS extends Logging { (dstBlockId, (srcBlockId, activeIndices.map(idx => srcFactors(idx)))) } } - val merged = srcOut.groupByKey(new HashPartitioner(dstInBlocks.partitions.length)) + val merged = srcOut.groupByKey(new ALSPartitioner(dstInBlocks.partitions.length)) dstInBlocks.join(merged).mapValues { case (InBlock(dstIds, srcPtrs, srcEncodedIndices, ratings), srcFactors) => val sortedSrcFactors = new Array[FactorBlock](numSrcBlocks) @@ -1149,4 +1153,11 @@ object ALS extends Logging { encoded & localIndexMask } } + + /** + * Partitioner used by ALS. We requires that getPartition is a projection. That is, for any key k, + * we have getPartition(getPartition(k)) = getPartition(k). Since the the default HashPartitioner + * satisfies this requirement, we simply use a type alias here. + */ + private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 376c3626f9bbb..bb86bafc0eb0a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} import org.apache.spark.ml.recommendation.ALS._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -455,4 +455,34 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { assert(isNonnegative(itemFactors)) // TODO: Validate the solution. } + + test("als partitioner is a projection") { + for (p <- Seq(1, 10, 100, 1000)) { + val part = new ALSPartitioner(p) + var k = 0 + while (k < p) { + assert(k === part.getPartition(k)) + assert(k === part.getPartition(k.toLong)) + k += 1 + } + } + } + + test("partitioner in returned factors") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + val (userFactors, itemFactors) = ALS.train( + ratings, rank = 2, maxIter = 4, numUserBlocks = 3, numItemBlocks = 4) + for ((tpe, factors) <- Seq(("User", userFactors), ("Item", itemFactors))) { + assert(userFactors.partitioner.isDefined, s"$tpe factors should have partitioner.") + val part = userFactors.partitioner.get + userFactors.mapPartitionsWithIndex { (idx, items) => + items.foreach { case (id, _) => + if (part.getPartition(id) != idx) { + throw new SparkException(s"$tpe with ID $id should not be in partition $idx.") + } + } + Iterator.empty + }.count() + } + } } From 51a6f9097bb475cb518ca766a46c7226640cf58e Mon Sep 17 00:00:00 2001 From: Judy Nash Date: Thu, 26 Feb 2015 11:14:37 +0000 Subject: [PATCH 260/817] [SPARK-5914] to run spark-submit requiring only user perm on windows Because windows on-default does not grant read permission to jars except to admin, spark-submit would fail with "ClassNotFound" exception if user runs slave service with only user permission. This fix is to add read permission to owner of the jar (which would be the slave service account in windows ) Author: Judy Nash Closes #4742 from judynash/SPARK-5914 and squashes the following commits: e288e56 [Judy Nash] Fix spacing and refactor code 1de3c0e [Judy Nash] [SPARK-5914] Enable spark-submit to run requiring only user permission on windows --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4803ff9403b1d..1396f167eb8be 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -440,6 +440,12 @@ private[spark] object Utils extends Logging { } // Make the file executable - That's necessary for scripts FileUtil.chmod(targetFile.getAbsolutePath, "a+x") + + // Windows does not grant read permission by default to non-admin users + // Add read permission to owner explicitly + if (isWindows) { + FileUtil.chmod(targetFile.getAbsolutePath, "u+r") + } } /** From f02394d06473889d0d7897c4583239e6e136ff46 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 26 Feb 2015 22:39:49 +0800 Subject: [PATCH 261/817] [SPARK-6023][SQL] ParquetConversions fails to replace the destination MetastoreRelation of an InsertIntoTable node to ParquetRelation2 JIRA: https://issues.apache.org/jira/browse/SPARK-6023 Author: Yin Huai Closes #4782 from yhuai/parquetInsertInto and squashes the following commits: ae7e806 [Yin Huai] Convert MetastoreRelation in InsertIntoTable and InsertIntoHiveTable. ba543cd [Yin Huai] More tests. 50b6d0f [Yin Huai] Update error messages. 346780c [Yin Huai] Failed test. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 21 +++ .../spark/sql/parquet/parquetSuites.scala | 138 +++++++++++++++++- 2 files changed, 152 insertions(+), 7 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 2cc8d65d3cb79..8af5a4848fd44 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -440,6 +440,17 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val attributedRewrites = relation.output.zip(parquetRelation.output) (relation, parquetRelation, attributedRewrites) + // Write path + case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) + // Inserting into partitioned table is not supported in Parquet data source (yet). + if !relation.hiveQlTable.isPartitioned && + hive.convertMetastoreParquet && + hive.conf.parquetUseDataSourceApi && + relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => + val parquetRelation = convertToParquetRelation(relation) + val attributedRewrites = relation.output.zip(parquetRelation.output) + (relation, parquetRelation, attributedRewrites) + // Read path case p @ PhysicalOperation(_, _, relation: MetastoreRelation) if hive.convertMetastoreParquet && @@ -464,6 +475,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with withAlias } + case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) + if relationMap.contains(r) => { + val parquetRelation = relationMap(r) + InsertIntoTable(parquetRelation, partition, child, overwrite) + } + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) + if relationMap.contains(r) => { + val parquetRelation = relationMap(r) + InsertIntoTable(parquetRelation, partition, child, overwrite) + } case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 653f4b47367c4..80fd5cda20e20 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -24,11 +24,11 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.PhysicalRDD -import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} +import org.apache.spark.sql.hive.execution.{InsertIntoHiveTable, HiveTableScan} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ -import org.apache.spark.sql.sources.LogicalRelation +import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -93,6 +93,11 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { sql(s"ALTER TABLE partitioned_parquet_with_key ADD PARTITION (p=$p)") } + val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + jsonRDD(rdd1).registerTempTable("jt") + val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":[$i, null]}""")) + jsonRDD(rdd2).registerTempTable("jt_array") + setConf("spark.sql.hive.convertMetastoreParquet", "true") } @@ -100,6 +105,8 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { sql("DROP TABLE partitioned_parquet") sql("DROP TABLE partitioned_parquet_with_key") sql("DROP TABLE normal_parquet") + sql("DROP TABLE IF EXISTS jt") + sql("DROP TABLE IF EXISTS jt_array") setConf("spark.sql.hive.convertMetastoreParquet", "false") } @@ -122,9 +129,6 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { override def beforeAll(): Unit = { super.beforeAll() - val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - jsonRDD(rdd).registerTempTable("jt") - sql( """ |create table test_parquet @@ -143,7 +147,6 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { override def afterAll(): Unit = { super.afterAll() - sql("DROP TABLE IF EXISTS jt") sql("DROP TABLE IF EXISTS test_parquet") setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) @@ -238,6 +241,70 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { sql("DROP TABLE IF EXISTS test_parquet_ctas") } + + test("MetastoreRelation in InsertIntoTable will be converted") { + sql( + """ + |create table test_insert_parquet + |( + | intField INT + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") + df.queryExecution.executedPlan match { + case ExecutedCommand( + InsertIntoDataSource( + LogicalRelation(r: ParquetRelation2), query, overwrite)) => // OK + case o => fail("test_insert_parquet should be converted to a " + + s"${classOf[ParquetRelation2].getCanonicalName} and " + + s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan." + + s"However, found a ${o.toString} ") + } + + checkAnswer( + sql("SELECT intField FROM test_insert_parquet WHERE test_insert_parquet.intField > 5"), + sql("SELECT a FROM jt WHERE jt.a > 5").collect() + ) + + sql("DROP TABLE IF EXISTS test_insert_parquet") + } + + test("MetastoreRelation in InsertIntoHiveTable will be converted") { + sql( + """ + |create table test_insert_parquet + |( + | int_array array + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") + df.queryExecution.executedPlan match { + case ExecutedCommand( + InsertIntoDataSource( + LogicalRelation(r: ParquetRelation2), query, overwrite)) => // OK + case o => fail("test_insert_parquet should be converted to a " + + s"${classOf[ParquetRelation2].getCanonicalName} and " + + s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan." + + s"However, found a ${o.toString} ") + } + + checkAnswer( + sql("SELECT int_array FROM test_insert_parquet"), + sql("SELECT a FROM jt_array").collect() + ) + + sql("DROP TABLE IF EXISTS test_insert_parquet") + } } class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { @@ -252,6 +319,63 @@ class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { super.afterAll() setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("MetastoreRelation in InsertIntoTable will not be converted") { + sql( + """ + |create table test_insert_parquet + |( + | intField INT + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") + df.queryExecution.executedPlan match { + case insert: InsertIntoHiveTable => // OK + case o => fail(s"The SparkPlan should be ${classOf[InsertIntoHiveTable].getCanonicalName}. " + + s"However, found ${o.toString}.") + } + + checkAnswer( + sql("SELECT intField FROM test_insert_parquet WHERE test_insert_parquet.intField > 5"), + sql("SELECT a FROM jt WHERE jt.a > 5").collect() + ) + + sql("DROP TABLE IF EXISTS test_insert_parquet") + } + + // TODO: enable it after the fix of SPARK-5950. + ignore("MetastoreRelation in InsertIntoHiveTable will not be converted") { + sql( + """ + |create table test_insert_parquet + |( + | int_array array + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") + df.queryExecution.executedPlan match { + case insert: InsertIntoHiveTable => // OK + case o => fail(s"The SparkPlan should be ${classOf[InsertIntoHiveTable].getCanonicalName}. " + + s"However, found ${o.toString}.") + } + + checkAnswer( + sql("SELECT int_array FROM test_insert_parquet"), + sql("SELECT a FROM jt_array").collect() + ) + + sql("DROP TABLE IF EXISTS test_insert_parquet") + } } /** From 192e42a2933eb283e12bfdfb46e2ef895228af4a Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 27 Feb 2015 01:01:32 +0800 Subject: [PATCH 262/817] [SPARK-6016][SQL] Cannot read the parquet table after overwriting the existing table when spark.sql.parquet.cacheMetadata=true Please see JIRA (https://issues.apache.org/jira/browse/SPARK-6016) for details of the bug. Author: Yin Huai Closes #4775 from yhuai/parquetFooterCache and squashes the following commits: 78787b1 [Yin Huai] Remove footerCache in FilteringParquetRowInputFormat. dff6fba [Yin Huai] Failed unit test. --- .../sql/parquet/ParquetTableOperations.scala | 49 +++---------------- .../apache/spark/sql/parquet/newParquet.scala | 8 ++- .../spark/sql/parquet/parquetSuites.scala | 27 ++++++++++ 3 files changed, 42 insertions(+), 42 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 4dc13b036cd4e..9061d3f5fee4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -374,8 +374,6 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private[parquet] class FilteringParquetRowInputFormat extends parquet.hadoop.ParquetInputFormat[Row] with Logging { - private var footers: JList[Footer] = _ - private var fileStatuses = Map.empty[Path, FileStatus] override def createRecordReader( @@ -396,46 +394,15 @@ private[parquet] class FilteringParquetRowInputFormat } } - override def getFooters(jobContext: JobContext): JList[Footer] = { - import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.footerCache - - if (footers eq null) { - val conf = ContextUtil.getConfiguration(jobContext) - val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) - val statuses = listStatus(jobContext) - fileStatuses = statuses.map(file => file.getPath -> file).toMap - if (statuses.isEmpty) { - footers = Collections.emptyList[Footer] - } else if (!cacheMetadata) { - // Read the footers from HDFS - footers = getFooters(conf, statuses) - } else { - // Read only the footers that are not in the footerCache - val foundFooters = footerCache.getAllPresent(statuses) - val toFetch = new ArrayList[FileStatus] - for (s <- statuses) { - if (!foundFooters.containsKey(s)) { - toFetch.add(s) - } - } - val newFooters = new mutable.HashMap[FileStatus, Footer] - if (toFetch.size > 0) { - val startFetch = System.currentTimeMillis - val fetched = getFooters(conf, toFetch) - logInfo(s"Fetched $toFetch footers in ${System.currentTimeMillis - startFetch} ms") - for ((status, i) <- toFetch.zipWithIndex) { - newFooters(status) = fetched.get(i) - } - footerCache.putAll(newFooters) - } - footers = new ArrayList[Footer](statuses.size) - for (status <- statuses) { - footers.add(newFooters.getOrElse(status, foundFooters.get(status))) - } - } - } + // This is only a temporary solution sicne we need to use fileStatuses in + // both getClientSideSplits and getTaskSideSplits. It can be removed once we get rid of these + // two methods. + override def getSplits(jobContext: JobContext): JList[InputSplit] = { + // First set fileStatuses. + val statuses = listStatus(jobContext) + fileStatuses = statuses.map(file => file.getPath -> file).toMap - footers + super.getSplits(jobContext) } // TODO Remove this method and related code once PARQUET-16 is fixed 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 16b771344bfcd..e648618468d5d 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 @@ -200,7 +200,7 @@ private[sql] case class ParquetRelation2( private var commonMetadataStatuses: Array[FileStatus] = _ // Parquet footer cache. - private var footers: Map[FileStatus, Footer] = _ + var footers: Map[FileStatus, Footer] = _ // `FileStatus` objects of all data files (Parquet part-files). var dataStatuses: Array[FileStatus] = _ @@ -400,6 +400,7 @@ private[sql] case class ParquetRelation2( } else { metadataCache.dataStatuses.toSeq } + val selectedFooters = selectedFiles.map(metadataCache.footers) // FileInputFormat cannot handle empty lists. if (selectedFiles.nonEmpty) { @@ -447,11 +448,16 @@ private[sql] case class ParquetRelation2( @transient val cachedStatus = selectedFiles + @transient + val cachedFooters = selectedFooters + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { val inputFormat = if (cacheMetadata) { new FilteringParquetRowInputFormat { override def listStatus(jobContext: JobContext): JList[FileStatus] = cachedStatus + + override def getFooters(jobContext: JobContext): JList[Footer] = cachedFooters } } else { new FilteringParquetRowInputFormat diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 80fd5cda20e20..6a9d9daf6750c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.hive.execution.{InsertIntoHiveTable, HiveTableScan} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} +import org.apache.spark.sql.SaveMode // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -409,6 +410,32 @@ class ParquetSourceSuiteBase extends ParquetPartitioningTest { ) """) } + + test("SPARK-6016 make sure to use the latest footers") { + sql("drop table if exists spark_6016_fix") + + // Create a DataFrame with two partitions. So, the created table will have two parquet files. + val df1 = jsonRDD(sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i}"""), 2)) + df1.saveAsTable("spark_6016_fix", "parquet", SaveMode.Overwrite) + checkAnswer( + sql("select * from spark_6016_fix"), + (1 to 10).map(i => Row(i)) + ) + + // Create a DataFrame with four partitions. So, the created table will have four parquet files. + val df2 = jsonRDD(sparkContext.parallelize((1 to 10).map(i => s"""{"b":$i}"""), 4)) + df2.saveAsTable("spark_6016_fix", "parquet", SaveMode.Overwrite) + // For the bug of SPARK-6016, we are caching two outdated footers for df1. Then, + // since the new table has four parquet files, we are trying to read new footers from two files + // and then merge metadata in footers of these four (two outdated ones and two latest one), + // which will cause an error. + checkAnswer( + sql("select * from spark_6016_fix"), + (1 to 10).map(i => Row(i)) + ) + + sql("drop table spark_6016_fix") + } } class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { From df3d559b32f1ceb8ca3491e2a1169c56a6faab58 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 26 Feb 2015 17:35:03 +0000 Subject: [PATCH 263/817] [SPARK-5801] [core] Avoid creating nested directories. Cache the value of the local root dirs to use for storing local data, so that the same directories are reused. Also, to avoid an extra level of nesting, use a different env variable to propagate the local dirs from the Worker to the executors. And make the executor directory use a different name. Author: Marcelo Vanzin Closes #4747 from vanzin/SPARK-5801 and squashes the following commits: e0114e1 [Marcelo Vanzin] Update unit test. 18ee0a7 [Marcelo Vanzin] [SPARK-5801] [core] Avoid creating nested directories. --- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 4 ++-- .../scala/org/apache/spark/util/Utils.scala | 23 +++++++++++++++++++ .../apache/spark/storage/LocalDirsSuite.scala | 8 +++++-- 4 files changed, 32 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index bea04cd542fd1..6653aca0a0f06 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -135,7 +135,7 @@ private[spark] class ExecutorRunner( logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) builder.directory(executorDir) - builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(",")) + builder.environment.put("SPARK_EXECUTOR_DIRS", appLocalDirs.mkString(File.pathSeparator)) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 10929eb516041..2473a90aa9309 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -345,11 +345,11 @@ private[spark] class Worker( } // Create local dirs for the executor. These are passed to the executor via the - // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the + // SPARK_EXECUTOR_DIRS environment variable, and deleted by the Worker when the // application finishes. val appLocalDirs = appDirectories.get(appId).getOrElse { Utils.getOrCreateLocalRootDirs(conf).map { dir => - Utils.createDirectory(dir).getAbsolutePath() + Utils.createDirectory(dir, namePrefix = "executor").getAbsolutePath() }.toSeq } appDirectories(appId) = appLocalDirs diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 1396f167eb8be..4644088f19f4b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -63,6 +63,7 @@ private[spark] object Utils extends Logging { val random = new Random() private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + @volatile private var localRootDirs: Array[String] = null /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { @@ -683,14 +684,31 @@ private[spark] object Utils extends Logging { * and returns only the directories that exist / could be created. * * If no directories could be created, this will return an empty list. + * + * This method will cache the local directories for the application when it's first invoked. + * So calling it multiple times with a different configuration will always return the same + * set of directories. */ private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { + if (localRootDirs == null) { + this.synchronized { + if (localRootDirs == null) { + localRootDirs = getOrCreateLocalRootDirsImpl(conf) + } + } + } + localRootDirs + } + + private def getOrCreateLocalRootDirsImpl(conf: SparkConf): Array[String] = { if (isRunningInYarnContainer(conf)) { // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. Note this assumes that Yarn has // created the directories already, and that they are secured so that only the // user has access to them. getYarnLocalDirs(conf).split(",") + } else if (conf.getenv("SPARK_EXECUTOR_DIRS") != null) { + conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator) } else { // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user // configuration to point to a secure directory. So create a subdirectory with restricted @@ -734,6 +752,11 @@ private[spark] object Utils extends Logging { localDirs } + /** Used by unit tests. Do not call from other places. */ + private[spark] def clearLocalRootDirs(): Unit = { + localRootDirs = null + } + /** * Shuffle the elements of a collection into a random order, returning the * result in a new collection. Unlike scala.util.Random.shuffle, this method diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index 8cf951adb354b..82a82e23eecf2 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.io.File import org.apache.spark.util.Utils -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SparkConf @@ -28,7 +28,11 @@ import org.apache.spark.SparkConf /** * Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options. */ -class LocalDirsSuite extends FunSuite { +class LocalDirsSuite extends FunSuite with BeforeAndAfter { + + before { + Utils.clearLocalRootDirs() + } test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") { // Regression test for SPARK-2974 From 2358657547016d647cdd2e2d363426fcd8d3e9ff Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 26 Feb 2015 10:40:58 -0800 Subject: [PATCH 264/817] [SPARK-6007][SQL] Add numRows param in DataFrame.show() It is useful to let the user decide the number of rows to show in DataFrame.show Author: Jacky Li Closes #4767 from jackylk/show and squashes the following commits: a0e0f4b [Jacky Li] fix testcase 7cdbe91 [Jacky Li] modify according to comment bb54537 [Jacky Li] for Java compatibility d7acc18 [Jacky Li] modify according to comments 981be52 [Jacky Li] add numRows param in DataFrame.show() --- python/pyspark/sql/dataframe.py | 6 +++--- .../main/scala/org/apache/spark/sql/DataFrame.scala | 13 ++++++++++--- .../org/apache/spark/sql/JavaDataFrameSuite.java | 9 +++++++++ .../scala/org/apache/spark/sql/DataFrameSuite.scala | 5 +++++ 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 6d42410020b64..aec99017fbdc1 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -272,9 +272,9 @@ def isLocal(self): """ return self._jdf.isLocal() - def show(self): + def show(self, n=20): """ - Print the first 20 rows. + Print the first n rows. >>> df DataFrame[age: int, name: string] @@ -283,7 +283,7 @@ def show(self): 2 Alice 5 Bob """ - print self._jdf.showString().encode('utf8', 'ignore') + print self._jdf.showString(n).encode('utf8', 'ignore') def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) 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 f045da305ca5d..060ab5e9a0cfa 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 @@ -159,9 +159,10 @@ class DataFrame protected[sql]( /** * Internal API for Python + * @param numRows Number of rows to show */ - private[sql] def showString(): String = { - val data = take(20) + private[sql] def showString(numRows: Int): String = { + val data = take(numRows) val numCols = schema.fieldNames.length // For cells that are beyond 20 characters, replace it with the first 17 and "..." @@ -293,9 +294,15 @@ class DataFrame protected[sql]( * 1983 03 0.410516 0.442194 * 1984 04 0.450090 0.483521 * }}} + * @param numRows Number of rows to show * @group basic */ - def show(): Unit = println(showString()) + def show(numRows: Int): Unit = println(showString(numRows)) + + /** + * Displays the top 20 rows of [[DataFrame]] in a tabular form. + */ + def show(): Unit = show(20) /** * Cartesian join with another [[DataFrame]]. diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index c1c51f80d6586..2d586f784ac5a 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -20,6 +20,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.apache.spark.sql.*; @@ -81,4 +82,12 @@ public void testVarargMethods() { df.groupBy().agg(countDistinct(col("key"), col("value"))); df.select(coalesce(col("key"))); } + + @Ignore + public void testShow() { + // This test case is intended ignored, but to make sure it compiles correctly + DataFrame df = context.table("testData"); + df.show(); + df.show(1000); + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index c392a553c03f3..ff441ef26f9c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -441,4 +441,9 @@ class DataFrameSuite extends QueryTest { checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) } + ignore("show") { + // This test case is intended ignored, but to make sure it compiles correctly + testData.select($"*").show() + testData.select($"*").show(1000) + } } From cfff397f0adb27ca102cca43a7696e9fb1819ee0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 26 Feb 2015 10:51:47 -0800 Subject: [PATCH 265/817] [SPARK-6004][MLlib] Pick the best model when training GradientBoostedTrees with validation Since the validation error does not change monotonically, in practice, it should be proper to pick the best model when training GradientBoostedTrees with validation instead of stopping it early. Author: Liang-Chi Hsieh Closes #4763 from viirya/gbt_record_model and squashes the following commits: 452e049 [Liang-Chi Hsieh] Address comment. ea2fae2 [Liang-Chi Hsieh] Pick the best model when training GradientBoostedTrees with validation. --- .../spark/mllib/tree/GradientBoostedTrees.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index b4466ff40937f..a9c93e181e3ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -251,9 +251,15 @@ object GradientBoostedTrees extends Logging { logInfo("Internal timing for DecisionTree:") logInfo(s"$timer") - - new GradientBoostedTreesModel( - boostingStrategy.treeStrategy.algo, baseLearners, baseLearnerWeights) + if (validate) { + new GradientBoostedTreesModel( + boostingStrategy.treeStrategy.algo, + baseLearners.slice(0, bestM), + baseLearnerWeights.slice(0, bestM)) + } else { + new GradientBoostedTreesModel( + boostingStrategy.treeStrategy.algo, baseLearners, baseLearnerWeights) + } } } From 7fa960e653a905fc48d4097b49ce560cff919fa2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 26 Feb 2015 11:54:17 -0800 Subject: [PATCH 266/817] [SPARK-5363] Fix bug in PythonRDD: remove() inside iterator is not safe Removing elements from a mutable HashSet while iterating over it can cause the iteration to incorrectly skip over entries that were not removed. If this happened, PythonRDD would write fewer broadcast variables than the Python worker was expecting to read, which would cause the Python worker to hang indefinitely. Author: Davies Liu Closes #4776 from davies/fix_hang and squashes the following commits: a4384a5 [Davies Liu] fix bug: remvoe() inside iterator is not safe --- .../org/apache/spark/api/python/PythonRDD.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index dcb6e6313a1d2..b1cec0f6472b0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -219,14 +219,13 @@ private[spark] class PythonRDD( val oldBids = PythonRDD.getWorkerBroadcasts(worker) val newBids = broadcastVars.map(_.id).toSet // number of different broadcasts - val cnt = oldBids.diff(newBids).size + newBids.diff(oldBids).size + val toRemove = oldBids.diff(newBids) + val cnt = toRemove.size + newBids.diff(oldBids).size dataOut.writeInt(cnt) - for (bid <- oldBids) { - if (!newBids.contains(bid)) { - // remove the broadcast from worker - dataOut.writeLong(- bid - 1) // bid >= 0 - oldBids.remove(bid) - } + for (bid <- toRemove) { + // remove the broadcast from worker + dataOut.writeLong(- bid - 1) // bid >= 0 + oldBids.remove(bid) } for (broadcast <- broadcastVars) { if (!oldBids.contains(broadcast.id)) { From cd5c8d7bbd3ea410df08af6cdd3833e0ed4b91a0 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 Feb 2015 12:56:54 -0800 Subject: [PATCH 267/817] SPARK-4704 [CORE] SparkSubmitDriverBootstrap doesn't flush output Join on output threads to make sure any lingering output from process reaches stdout, stderr before exiting CC andrewor14 since I believe he created this section of code Author: Sean Owen Closes #4788 from srowen/SPARK-4704 and squashes the following commits: ad7114e [Sean Owen] Join on output threads to make sure any lingering output from process reaches stdout, stderr before exiting --- .../apache/spark/deploy/SparkSubmitDriverBootstrapper.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 2eab9981845e8..311048cdaa324 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import java.io.File - import scala.collection.JavaConversions._ import org.apache.spark.util.{RedirectThread, Utils} @@ -164,6 +162,8 @@ private[spark] object SparkSubmitDriverBootstrapper { } } val returnCode = process.waitFor() + stdoutThread.join() + stderrThread.join() sys.exit(returnCode) } From 10094a523e3993b775111ae9b22ca31cc0d76e03 Mon Sep 17 00:00:00 2001 From: Li Zhihui Date: Thu, 26 Feb 2015 13:07:07 -0800 Subject: [PATCH 268/817] Modify default value description for spark.scheduler.minRegisteredResourcesRatio on docs. The configuration is not supported in mesos mode now. See https://github.com/apache/spark/pull/1462 Author: Li Zhihui Closes #4781 from li-zhihui/fixdocconf and squashes the following commits: 63e7a44 [Li Zhihui] Modify default value description for spark.scheduler.minRegisteredResourcesRatio on docs. --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8dd2bad61344f..c11787b17eb8c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1018,7 +1018,7 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0.0 for Mesos and Standalone mode, 0.8 for YARN + 0.8 for YARN mode; 0.0 otherwise The minimum ratio of registered resources (registered resources / total expected resources) (resources are executors in yarn mode, CPU cores in standalone mode) From 8942b522d8a3269a2a357e3a274ed4b3e66ebdde Mon Sep 17 00:00:00 2001 From: xukun 00228947 Date: Thu, 26 Feb 2015 13:24:00 -0800 Subject: [PATCH 269/817] [SPARK-3562]Periodic cleanup event logs Author: xukun 00228947 Closes #4214 from viper-kun/cleaneventlog and squashes the following commits: 7a5b9c5 [xukun 00228947] fix issue 31674ee [xukun 00228947] fix issue 6e3d06b [xukun 00228947] fix issue 373f3b9 [xukun 00228947] fix issue 71782b5 [xukun 00228947] fix issue 5b45035 [xukun 00228947] fix issue 70c28d6 [xukun 00228947] fix issues adcfe86 [xukun 00228947] Periodic cleanup event logs --- .../scala/org/apache/spark/SparkConf.scala | 8 +- .../deploy/history/FsHistoryProvider.scala | 112 ++++++++++++------ docs/monitoring.md | 25 +++- 3 files changed, 110 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0dbd26146cb13..0f4922ab4e310 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -362,7 +362,13 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.files.userClassPathFirst", "spark.executor.userClassPathFirst", "1.3"), DeprecatedConfig("spark.yarn.user.classpath.first", null, "1.3", - "Use spark.{driver,executor}.userClassPathFirst instead.")) + "Use spark.{driver,executor}.userClassPathFirst instead."), + DeprecatedConfig("spark.history.fs.updateInterval", + "spark.history.fs.update.interval.seconds", + "1.3", "Use spark.history.fs.update.interval.seconds instead"), + DeprecatedConfig("spark.history.updateInterval", + "spark.history.fs.update.interval.seconds", + "1.3", "Use spark.history.fs.update.interval.seconds instead")) configs.map { x => (x.oldName, x) }.toMap } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 885fa0fdbf85b..1aaa7b72735ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -17,9 +17,13 @@ package org.apache.spark.deploy.history -import java.io.{BufferedInputStream, FileNotFoundException, InputStream} +import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream} +import java.util.concurrent.{Executors, TimeUnit} import scala.collection.mutable +import scala.concurrent.duration.Duration + +import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.fs.permission.AccessControlException @@ -44,8 +48,15 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val NOT_STARTED = "" // Interval between each check for event log updates - private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", - conf.getInt("spark.history.updateInterval", 10)) * 1000 + private val UPDATE_INTERVAL_MS = conf.getOption("spark.history.fs.update.interval.seconds") + .orElse(conf.getOption(SparkConf.translateConfKey("spark.history.fs.updateInterval", true))) + .orElse(conf.getOption(SparkConf.translateConfKey("spark.history.updateInterval", true))) + .map(_.toInt) + .getOrElse(10) * 1000 + + // Interval between each cleaner checks for event logs to delete + private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", + DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S) * 1000 private val logDir = conf.getOption("spark.history.fs.logDirectory") .map { d => Utils.resolveURI(d).toString } @@ -53,8 +64,11 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val fs = Utils.getHadoopFileSystem(logDir, SparkHadoopUtil.get.newConfiguration(conf)) - // A timestamp of when the disk was last accessed to check for log updates - private var lastLogCheckTimeMs = -1L + // Used by check event thread and clean log thread. + // Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs + // and applications between check task and clean task. + private val pool = Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder() + .setNameFormat("spark-history-task-%d").setDaemon(true).build()) // The modification time of the newest log detected during the last scan. This is used // to ignore logs that are older during subsequent scans, to avoid processing data that @@ -73,25 +87,13 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private[history] val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" /** - * A background thread that periodically checks for event log updates on disk. - * - * If a log check is invoked manually in the middle of a period, this thread re-adjusts the - * time at which it performs the next log check to maintain the same period as before. - * - * TODO: Add a mechanism to update manually. + * Return a runnable that performs the given operation on the event logs. + * This operation is expected to be executed periodically. */ - private val logCheckingThread = new Thread("LogCheckingThread") { - override def run() = Utils.logUncaughtExceptions { - while (true) { - val now = getMonotonicTimeMs() - if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) { - Thread.sleep(UPDATE_INTERVAL_MS) - } else { - // If the user has manually checked for logs recently, wait until - // UPDATE_INTERVAL_MS after the last check time - Thread.sleep(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now) - } - checkForLogs() + private def getRunner(operateFun: () => Unit): Runnable = { + new Runnable() { + override def run() = Utils.logUncaughtExceptions { + operateFun() } } } @@ -113,12 +115,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis "Logging directory specified is not a directory: %s".format(logDir)) } - checkForLogs() - // Disable the background thread during tests. if (!conf.contains("spark.testing")) { - logCheckingThread.setDaemon(true) - logCheckingThread.start() + // A task that periodically checks for event log updates on disk. + pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_MS, + TimeUnit.MILLISECONDS) + + if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { + // A task that periodically cleans event logs on disk. + pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_MS, + TimeUnit.MILLISECONDS) + } } } @@ -163,9 +170,6 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * applications that haven't been updated since last time the logs were checked. */ private[history] def checkForLogs(): Unit = { - lastLogCheckTimeMs = getMonotonicTimeMs() - logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) - try { var newLastModifiedTime = lastModifiedTime val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) @@ -230,6 +234,45 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } + /** + * Delete event logs from the log directory according to the clean policy defined by the user. + */ + private def cleanLogs(): Unit = { + try { + val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) + .getOrElse(Seq[FileStatus]()) + val maxAge = conf.getLong("spark.history.fs.cleaner.maxAge.seconds", + DEFAULT_SPARK_HISTORY_FS_MAXAGE_S) * 1000 + + val now = System.currentTimeMillis() + val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + + applications.values.foreach { info => + if (now - info.lastUpdated <= maxAge) { + appsToRetain += (info.id -> info) + } + } + + applications = appsToRetain + + // Scan all logs from the log directory. + // Only directories older than the specified max age will be deleted + statusList.foreach { dir => + try { + if (now - dir.getModificationTime() > maxAge) { + // if path is a directory and set to true, + // the directory is deleted else throws an exception + fs.delete(dir.getPath, true) + } + } catch { + case t: IOException => logError(s"IOException in cleaning logs of $dir", t) + } + } + } catch { + case t: Exception => logError("Exception in cleaning logs", t) + } + } + /** * Comparison function that defines the sort order for the application listing. * @@ -336,9 +379,6 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } - /** Returns the system's mononotically increasing time. */ - private def getMonotonicTimeMs(): Long = System.nanoTime() / (1000 * 1000) - /** * Return true when the application has completed. */ @@ -354,6 +394,12 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private object FsHistoryProvider { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" + + // One day + val DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S = Duration(1, TimeUnit.DAYS).toSeconds + + // One week + val DEFAULT_SPARK_HISTORY_FS_MAXAGE_S = Duration(7, TimeUnit.DAYS).toSeconds } private class FsApplicationHistoryInfo( diff --git a/docs/monitoring.md b/docs/monitoring.md index 009a344dff4bb..37ede476c187d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -86,7 +86,7 @@ follows: - spark.history.fs.updateInterval + spark.history.fs.update.interval.seconds 10 The period, in seconds, at which information displayed by this history server is updated. @@ -145,6 +145,29 @@ follows: If disabled, no access control checks are made. + + spark.history.fs.cleaner.enabled + false + + Specifies whether the History Server should periodically clean up event logs from storage. + + + + spark.history.fs.cleaner.interval.seconds + 86400 + + How often the job history cleaner checks for files to delete, in seconds. Defaults to 86400 (one day). + Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.seconds. + + + + spark.history.fs.cleaner.maxAge.seconds + 3600 * 24 * 7 + + Job history files older than this many seconds will be deleted when the history cleaner runs. + Defaults to 3600 * 24 * 7 (1 week). + + Note that in all of these UIs, the tables are sortable by clicking their headers, From aa63f633d39efa8c29095295f161eaad5495071d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 26 Feb 2015 13:46:07 -0800 Subject: [PATCH 270/817] [SPARK-6027][SPARK-5546] Fixed --jar and --packages not working for KafkaUtils and improved error message The problem with SPARK-6027 in short is that JARs like the kafka-assembly.jar does not work in python as the added JAR is not visible in the classloader used by Py4J. Py4J uses Class.forName(), which does not uses the systemclassloader, but the JARs are only visible in the Thread's contextclassloader. So this back uses the context class loader to create the KafkaUtils dstream object. This works for both cases where the Kafka libraries are added with --jars spark-streaming-kafka-assembly.jar or with --packages spark-streaming-kafka Also improves the error message. davies Author: Tathagata Das Closes #4779 from tdas/kafka-python-fix and squashes the following commits: fb16b04 [Tathagata Das] Removed import c1fdf35 [Tathagata Das] Fixed long line and improved documentation 7b88be8 [Tathagata Das] Fixed --jar not working for KafkaUtils and improved error message --- .../spark/streaming/kafka/KafkaUtils.scala | 29 ++++++++++++- python/pyspark/streaming/kafka.py | 42 ++++++++++++------- 2 files changed, 55 insertions(+), 16 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index af04bc6576148..62a659518943d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConversions._ import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import kafka.serializer.{Decoder, StringDecoder} +import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.{SparkContext, SparkException} @@ -532,3 +532,30 @@ object KafkaUtils { ) } } + +/** + * This is a helper class that wraps the KafkaUtils.createStream() into more + * Python-friendly class and function so that it can be easily + * instantiated and called from Python's KafkaUtils (see SPARK-6027). + * + * The zero-arg constructor helps instantiate this class from the Class object + * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() + * takes care of known parameters instead of passing them from Python + */ +private class KafkaUtilsPythonHelper { + def createStream( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = { + KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( + jssc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + kafkaParams, + topics, + storageLevel) + } +} diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 19ad71f99d4d5..0002dc10e8a17 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -16,7 +16,7 @@ # from py4j.java_collections import MapConverter -from py4j.java_gateway import java_import, Py4JError +from py4j.java_gateway import java_import, Py4JError, Py4JJavaError from pyspark.storagelevel import StorageLevel from pyspark.serializers import PairDeserializer, NoOpSerializer @@ -50,8 +50,6 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, :param valueDecoder: A function used to decode value (default is utf8_decoder) :return: A DStream object """ - java_import(ssc._jvm, "org.apache.spark.streaming.kafka.KafkaUtils") - kafkaParams.update({ "zookeeper.connect": zkQuorum, "group.id": groupId, @@ -63,20 +61,34 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client) jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - def getClassByName(name): - return ssc._jvm.org.apache.spark.util.Utils.classForName(name) - try: - array = getClassByName("[B") - decoder = getClassByName("kafka.serializer.DefaultDecoder") - jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, array, decoder, decoder, - jparam, jtopics, jlevel) - except Py4JError, e: + # Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see SPARK-6027) + helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ + .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") + helper = helperClass.newInstance() + jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) + except Py4JJavaError, e: # TODO: use --jar once it also work on driver - if not e.message or 'call a package' in e.message: - print "No kafka package, please put the assembly jar into classpath:" - print " $ bin/spark-submit --driver-class-path external/kafka-assembly/target/" + \ - "scala-*/spark-streaming-kafka-assembly-*.jar" + if 'ClassNotFoundException' in str(e.java_exception): + print """ +________________________________________________________________________________________________ + + Spark Streaming's Kafka libraries not found in class path. Try one of the following. + + 1. Include the Kafka library and its dependencies with in the + spark-submit command as + + $ bin/spark-submit --packages org.apache.spark:spark-streaming-kafka:%s ... + + 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, + Group Id = org.apache.spark, Artifact Id = spark-streaming-kafka-assembly, Version = %s. + Then, innclude the jar in the spark-submit command as + + $ bin/spark-submit --jars ... + +________________________________________________________________________________________________ + +""" % (ssc.sparkContext.version, ssc.sparkContext.version) raise e ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) stream = DStream(jstream, ssc, ser) From 5f3238b3b0157091d28803aa3b1d248dfa6cdc59 Mon Sep 17 00:00:00 2001 From: Cheolsoo Park Date: Thu, 26 Feb 2015 13:53:49 -0800 Subject: [PATCH 271/817] [SPARK-6018] [YARN] NoSuchMethodError in Spark app is swallowed by YARN AM Author: Cheolsoo Park Closes #4773 from piaozhexiu/SPARK-6018 and squashes the following commits: 2a919d5 [Cheolsoo Park] Rename e with cause to avoid duplicate names 1e71d2d [Cheolsoo Park] Replace placeholder with throwable eb5750d [Cheolsoo Park] NoSuchMethodError in Spark app is swallowed by YARN AM --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 a9bf861d160c1..20fc19166ac4e 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 @@ -485,12 +485,12 @@ private[spark] class ApplicationMaster( e.getCause match { case _: InterruptedException => // Reporter thread can interrupt to stop user class - case e: Exception => + case cause: Throwable => finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, - "User class threw exception: " + e.getMessage) + "User class threw exception: " + cause.getMessage) // re-throw to get it logged - throw e + throw cause } } } From 3fb53c0298761ba227890525ae79ce4ec6300deb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 Feb 2015 14:08:56 -0800 Subject: [PATCH 272/817] SPARK-4300 [CORE] Race condition during SparkWorker shutdown Close appender saving stdout/stderr before destroying process to avoid exception on reading closed input stream. (This also removes a redundant `waitFor()` although it was harmless) CC tdas since I think you wrote this method. Author: Sean Owen Closes #4787 from srowen/SPARK-4300 and squashes the following commits: e0cdabf [Sean Owen] Close appender saving stdout/stderr before destroying process to avoid exception on reading closed input stream --- .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 6653aca0a0f06..066d46c4473eb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -85,14 +85,13 @@ private[spark] class ExecutorRunner( var exitCode: Option[Int] = None if (process != null) { logInfo("Killing process!") - process.destroy() - process.waitFor() if (stdoutAppender != null) { stdoutAppender.stop() } if (stderrAppender != null) { stderrAppender.stop() } + process.destroy() exitCode = Some(process.waitFor()) } worker ! ExecutorStateChanged(appId, execId, state, message, exitCode) From c871e2dae0182e914135560d14304242e1f97f7e Mon Sep 17 00:00:00 2001 From: moussa taifi Date: Thu, 26 Feb 2015 14:19:43 -0800 Subject: [PATCH 273/817] Add a note for context termination for History server on Yarn The history server on Yarn only shows completed jobs. This adds a note concerning the needed explicit context termination at the end of a spark job which is a best practice anyway. Related to SPARK-2972 and SPARK-3458 Author: moussa taifi Closes #4721 from moutai/add-history-server-note-for-closing-the-spark-context and squashes the following commits: 9f5b6c3 [moussa taifi] Fix upper case typo for YARN 3ad3db4 [moussa taifi] Add context termination for History server on Yarn --- docs/monitoring.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/monitoring.md b/docs/monitoring.md index 37ede476c187d..6816671ffbf46 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -173,6 +173,8 @@ follows: Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. +Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI. + # Metrics Spark has a configurable metrics system based on the From b38dec2ffdf724ff4e181cc8c7427d074b442670 Mon Sep 17 00:00:00 2001 From: "mohit.goyal" Date: Thu, 26 Feb 2015 14:27:47 -0800 Subject: [PATCH 274/817] [SPARK-5951][YARN] Remove unreachable driver memory properties in yarn client mode Remove unreachable driver memory properties in yarn client mode Author: mohit.goyal Closes #4730 from zuxqoj/master and squashes the following commits: 977dc96 [mohit.goyal] remove not rechable deprecated variables in yarn client mode --- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index f1b5aafac4066..8abdc26b43806 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -79,18 +79,12 @@ private[spark] class YarnClientSchedulerBackend( ) // Warn against the following deprecated environment variables: env var -> suggestion val deprecatedEnvVars = Map( - "SPARK_MASTER_MEMORY" -> "SPARK_DRIVER_MEMORY or --driver-memory through spark-submit", "SPARK_WORKER_INSTANCES" -> "SPARK_WORKER_INSTANCES or --num-executors through spark-submit", "SPARK_WORKER_MEMORY" -> "SPARK_EXECUTOR_MEMORY or --executor-memory through spark-submit", "SPARK_WORKER_CORES" -> "SPARK_EXECUTOR_CORES or --executor-cores through spark-submit") - // Do the same for deprecated properties: property -> suggestion - val deprecatedProps = Map("spark.master.memory" -> "--driver-memory through spark-submit") optionTuples.foreach { case (optionName, envVar, sparkProp) => if (sc.getConf.contains(sparkProp)) { extraArgs += (optionName, sc.getConf.get(sparkProp)) - if (deprecatedProps.contains(sparkProp)) { - logWarning(s"NOTE: $sparkProp is deprecated. Use ${deprecatedProps(sparkProp)} instead.") - } } else if (System.getenv(envVar) != null) { extraArgs += (optionName, System.getenv(envVar)) if (deprecatedEnvVars.contains(envVar)) { From e60ad2f4c47b011be7a3198689ac2b82ee317d96 Mon Sep 17 00:00:00 2001 From: tedyu Date: Thu, 26 Feb 2015 23:26:07 +0000 Subject: [PATCH 275/817] SPARK-6045 RecordWriter should be checked against null in PairRDDFunctio... ...ns#saveAsNewAPIHadoopDataset Author: tedyu Closes #4794 from tedyu/master and squashes the following commits: 2632a57 [tedyu] SPARK-6045 RecordWriter should be checked against null in PairRDDFunctions#saveAsNewAPIHadoopDataset 2d8d4b1 [tedyu] SPARK-6045 RecordWriter should be checked against null in PairRDDFunctions#saveAsNewAPIHadoopDataset --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 1 + 1 file changed, 1 insertion(+) 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 955b42c3baaa1..6b4f097ea9ae5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -993,6 +993,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] + require(writer != null, "Unable to obtain RecordWriter") var recordsWritten = 0L try { while (iter.hasNext) { From fbc469473dd529eb72046186b85dd8fc2b7c5bb5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 Feb 2015 17:35:09 -0800 Subject: [PATCH 276/817] SPARK-4579 [WEBUI] Scheduling Delay appears negative Ensure scheduler delay handles unfinished task case, and ensure delay is never negative even due to rounding Author: Sean Owen Closes #4796 from srowen/SPARK-4579 and squashes the following commits: ad6713c [Sean Owen] Ensure scheduler delay handles unfinished task case, and ensure delay is never negative even due to rounding --- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d752434ad58ae..110f8780a9a12 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -626,15 +626,16 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { - val totalExecutionTime = { - if (info.gettingResultTime > 0) { - (info.gettingResultTime - info.launchTime) + val totalExecutionTime = + if (info.gettingResult) { + info.gettingResultTime - info.launchTime + } else if (info.finished) { + info.finishTime - info.launchTime } else { - (info.finishTime - info.launchTime) + 0 } - } val executorOverhead = (metrics.executorDeserializeTime + metrics.resultSerializationTime) - totalExecutionTime - metrics.executorRunTime - executorOverhead + math.max(0, totalExecutionTime - metrics.executorRunTime - executorOverhead) } } From 18f2098433e0bfef9497bacd601fdf098ed03eab Mon Sep 17 00:00:00 2001 From: Hong Shen Date: Thu, 26 Feb 2015 18:43:23 -0800 Subject: [PATCH 277/817] [SPARK-5529][CORE]Add expireDeadHosts in HeartbeatReceiver If a blockManager has not send heartBeat more than 120s, BlockManagerMasterActor will remove it. But coarseGrainedSchedulerBackend can only remove executor after an DisassociatedEvent. We should expireDeadHosts at HeartbeatReceiver. Author: Hong Shen Closes #4363 from shenh062326/my_change3 and squashes the following commits: 2c9a46a [Hong Shen] Change some code style. 1a042ff [Hong Shen] Change some code style. 2dc456e [Hong Shen] Change some code style. d221493 [Hong Shen] Fix test failed 7448ac6 [Hong Shen] A minor change in sparkContext and heartbeatReceiver b904aed [Hong Shen] Fix failed test 52725af [Hong Shen] Remove assert in SparkContext.killExecutors 5bedcb8 [Hong Shen] Remove assert in SparkContext.killExecutors a858fb5 [Hong Shen] A minor change in HeartbeatReceiver 3e221d9 [Hong Shen] A minor change in HeartbeatReceiver 6bab7aa [Hong Shen] Change a code style. 07952f3 [Hong Shen] Change configs name and code style. ce9257e [Hong Shen] Fix test failed bccd515 [Hong Shen] Fix test failed 8e77408 [Hong Shen] Fix test failed c1dfda1 [Hong Shen] Fix test failed e197e20 [Hong Shen] Fix test failed fb5df97 [Hong Shen] Remove ExpireDeadHosts in BlockManagerMessages b5c0441 [Hong Shen] Remove expireDeadHosts in BlockManagerMasterActor c922cb0 [Hong Shen] Add expireDeadHosts in HeartbeatReceiver --- .../org/apache/spark/HeartbeatReceiver.scala | 65 +++++++++++++++++-- .../scala/org/apache/spark/SparkContext.scala | 15 +++-- .../spark/scheduler/TaskScheduler.scala | 6 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../storage/BlockManagerMasterActor.scala | 36 +--------- .../spark/storage/BlockManagerMessages.scala | 2 - .../spark/scheduler/DAGSchedulerSuite.scala | 2 + 7 files changed, 79 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 83ae57b7f1516..69178da1a7773 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -17,33 +17,86 @@ package org.apache.spark -import akka.actor.Actor +import scala.concurrent.duration._ +import scala.collection.mutable + +import akka.actor.{Actor, Cancellable} + import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId -import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} import org.apache.spark.util.ActorLogReceive /** * A heartbeat from executors to the driver. This is a shared message used by several internal - * components to convey liveness or execution information for in-progress tasks. + * components to convey liveness or execution information for in-progress tasks. It will also + * expire the hosts that have not heartbeated for more than spark.network.timeout. */ private[spark] case class Heartbeat( executorId: String, taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) +private[spark] case object ExpireDeadHosts + private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** * Lives in the driver to receive heartbeats from executors.. */ -private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) +private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskScheduler) extends Actor with ActorLogReceive with Logging { + // executor ID -> timestamp of when the last heartbeat from this executor was received + private val executorLastSeen = new mutable.HashMap[String, Long] + + private val executorTimeoutMs = sc.conf.getLong("spark.network.timeout", + sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120)) * 1000 + + private val checkTimeoutIntervalMs = sc.conf.getLong("spark.network.timeoutInterval", + sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60)) * 1000 + + private var timeoutCheckingTask: Cancellable = null + + override def preStart(): Unit = { + import context.dispatcher + timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, + checkTimeoutIntervalMs.milliseconds, self, ExpireDeadHosts) + super.preStart() + } + override def receiveWithLogging = { case Heartbeat(executorId, taskMetrics, blockManagerId) => - val response = HeartbeatResponse( - !scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId)) + val unknownExecutor = !scheduler.executorHeartbeatReceived( + executorId, taskMetrics, blockManagerId) + val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) + executorLastSeen(executorId) = System.currentTimeMillis() sender ! response + case ExpireDeadHosts => + expireDeadHosts() + } + + private def expireDeadHosts(): Unit = { + logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.") + val now = System.currentTimeMillis() + for ((executorId, lastSeenMs) <- executorLastSeen) { + if (now - lastSeenMs > executorTimeoutMs) { + logWarning(s"Removing executor $executorId with no recent heartbeats: " + + s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms") + scheduler.executorLost(executorId, SlaveLost("Executor heartbeat " + + "timed out after ${now - lastSeenMs} ms")) + if (sc.supportDynamicAllocation) { + sc.killExecutor(executorId) + } + executorLastSeen.remove(executorId) + } + } + } + + override def postStop(): Unit = { + if (timeoutCheckingTask != null) { + timeoutCheckingTask.cancel() + } + super.postStop() } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 930d4bea4785b..d3948d4e6d91b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -351,7 +351,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] var (schedulerBackend, taskScheduler) = SparkContext.createTaskScheduler(this, master) private val heartbeatReceiver = env.actorSystem.actorOf( - Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") + Props(new HeartbeatReceiver(this, taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ try { dagScheduler = new DAGScheduler(this) @@ -398,7 +398,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = if (dynamicAllocationEnabled) { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Dynamic allocation of executors is currently only supported in YARN mode") Some(new ExecutorAllocationManager(this, listenerBus, conf)) } else { @@ -1122,6 +1122,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli postEnvironmentUpdate() } + /** + * Return whether dynamically adjusting the amount of resources allocated to + * this application is supported. This is currently only available for YARN. + */ + private[spark] def supportDynamicAllocation = + master.contains("yarn") || dynamicAllocationTesting + /** * :: DeveloperApi :: * Register a listener to receive up-calls from events that happen during execution. @@ -1155,7 +1162,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ @DeveloperApi override def requestExecutors(numAdditionalExecutors: Int): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => @@ -1173,7 +1180,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ @DeveloperApi override def killExecutors(executorIds: Seq[String]): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Killing executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index f095915352b17..ed3418676e077 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -73,5 +73,9 @@ private[spark] trait TaskScheduler { * @return An application ID */ def applicationId(): String = appId - + + /** + * Process a lost executor + */ + def executorLost(executorId: String, reason: ExecutorLossReason): Unit } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 54f8fcfc416d1..7a9cf1c2e7f30 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -436,7 +436,7 @@ private[spark] class TaskSchedulerImpl( } } - def executorLost(executorId: String, reason: ExecutorLossReason) { + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = { var failedExecutor: Option[String] = None synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 64133464d8daa..787b0f96bec32 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConversions._ import scala.concurrent.Future import scala.concurrent.duration._ -import akka.actor.{Actor, ActorRef, Cancellable} +import akka.actor.{Actor, ActorRef} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} @@ -52,19 +52,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 1000) - - val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) - - var timeoutCheckingTask: Cancellable = null - - override def preStart() { - import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, - checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) - super.preStart() - } - override def receiveWithLogging = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) @@ -118,14 +105,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case StopBlockManagerMaster => sender ! true - if (timeoutCheckingTask != null) { - timeoutCheckingTask.cancel() - } context.stop(self) - case ExpireDeadHosts => - expireDeadHosts() - case BlockManagerHeartbeat(blockManagerId) => sender ! heartbeatReceived(blockManagerId) @@ -207,21 +188,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus logInfo(s"Removing block manager $blockManagerId") } - private def expireDeadHosts() { - logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.") - val now = System.currentTimeMillis() - val minSeenTime = now - slaveTimeout - val toRemove = new mutable.HashSet[BlockManagerId] - for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime && !info.blockManagerId.isDriver) { - logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " - + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") - toRemove += info.blockManagerId - } - } - toRemove.foreach(removeBlockManager) - } - private def removeExecutor(execId: String) { logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 3f32099d08cc9..48247453edef0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -109,6 +109,4 @@ private[spark] object BlockManagerMessages { extends ToBlockManagerMaster case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - - case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 4bf7f9e647d55..30119ce5d4eec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -96,6 +96,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } /** Length of time to wait while draining listener events. */ @@ -386,6 +387,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def defaultParallelism() = 2 override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean = true + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } val noKillScheduler = new DAGScheduler( sc, From 4ad5153f5449319a7e82c9013ccff4494ab58ef1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 27 Feb 2015 11:06:47 +0800 Subject: [PATCH 278/817] [SPARK-6037][SQL] Avoiding duplicate Parquet schema merging `FilteringParquetRowInputFormat` manually merges Parquet schemas before computing splits. However, it is duplicate because the schemas are already merged in `ParquetRelation2`. We don't need to re-merge them at `InputFormat`. Author: Liang-Chi Hsieh Closes #4786 from viirya/dup_parquet_schemas_merge and squashes the following commits: ef78a5a [Liang-Chi Hsieh] Avoiding duplicate Parquet schema merging. --- .../sql/parquet/ParquetTableOperations.scala | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 9061d3f5fee4d..4e4f647767dc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -434,22 +434,13 @@ private[parquet] class FilteringParquetRowInputFormat return splits } - Option(globalMetaData.getKeyValueMetaData.get(RowReadSupport.SPARK_METADATA_KEY)).foreach { - schemas => - val mergedSchema = schemas - .map(DataType.fromJson(_).asInstanceOf[StructType]) - .reduce(_ merge _) - .json - - val mergedMetadata = globalMetaData - .getKeyValueMetaData - .updated(RowReadSupport.SPARK_METADATA_KEY, setAsJavaSet(Set(mergedSchema))) - - globalMetaData = new GlobalMetaData( - globalMetaData.getSchema, - mergedMetadata, - globalMetaData.getCreatedBy) - } + val metadata = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) + val mergedMetadata = globalMetaData + .getKeyValueMetaData + .updated(RowReadSupport.SPARK_METADATA_KEY, setAsJavaSet(Set(metadata))) + + globalMetaData = new GlobalMetaData(globalMetaData.getSchema, + mergedMetadata, globalMetaData.getCreatedBy) val readContext = getReadSupport(configuration).init( new InitContext(configuration, From 5e5ad6558d60cfbf360708584e883e80d363e33e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 26 Feb 2015 20:46:05 -0800 Subject: [PATCH 279/817] [SPARK-6024][SQL] When a data source table has too many columns, it's schema cannot be stored in metastore. JIRA: https://issues.apache.org/jira/browse/SPARK-6024 Author: Yin Huai Closes #4795 from yhuai/wideSchema and squashes the following commits: 4882e6f [Yin Huai] Address comments. 73e71b4 [Yin Huai] Address comments. 143927a [Yin Huai] Simplify code. cc1d472 [Yin Huai] Make the schema wider. 12bacae [Yin Huai] If the JSON string of a schema is too large, split it before storing it in metastore. e9b4f70 [Yin Huai] Failed test. --- .../scala/org/apache/spark/sql/SQLConf.scala | 10 +++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 29 +++++++++++++++---- .../sql/hive/MetastoreDataSourcesSuite.scala | 21 ++++++++++++++ 3 files changed, 54 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index a08c0f5ce3ff4..4815620c6fe57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -51,6 +51,11 @@ private[spark] object SQLConf { // This is used to set the default data source val DEFAULT_DATA_SOURCE_NAME = "spark.sql.sources.default" + // This is used to control the when we will split a schema's JSON string to multiple pieces + // in order to fit the JSON string in metastore's table property (by default, the value has + // a length restriction of 4000 characters). We will split the JSON string of a schema + // to its length exceeds the threshold. + val SCHEMA_STRING_LENGTH_THRESHOLD = "spark.sql.sources.schemaStringLengthThreshold" // Whether to perform eager analysis when constructing a dataframe. // Set to false when debugging requires the ability to look at invalid query plans. @@ -177,6 +182,11 @@ private[sql] class SQLConf extends Serializable { private[spark] def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.parquet") + // Do not use a value larger than 4000 as the default value of this property. + // See the comments of SCHEMA_STRING_LENGTH_THRESHOLD above for more information. + private[spark] def schemaStringLengthThreshold: Int = + getConf(SCHEMA_STRING_LENGTH_THRESHOLD, "4000").toInt + private[spark] def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS, "true").toBoolean diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 8af5a4848fd44..d3ad364328265 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -69,13 +69,23 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val table = synchronized { client.getTable(in.database, in.name) } - val schemaString = table.getProperty("spark.sql.sources.schema") val userSpecifiedSchema = - if (schemaString == null) { - None - } else { - Some(DataType.fromJson(schemaString).asInstanceOf[StructType]) + Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => + val parts = (0 until numParts.toInt).map { index => + val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") + if (part == null) { + throw new AnalysisException( + s"Could not read schema from the metastore because it is corrupted " + + s"(missing part ${index} of the schema).") + } + + part + } + // Stick all parts back to a single schema string in the JSON representation + // and convert it back to a StructType. + DataType.fromJson(parts.mkString).asInstanceOf[StructType] } + // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... val options = table.getTTable.getSd.getSerdeInfo.getParameters.toMap @@ -119,7 +129,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with tbl.setProperty("spark.sql.sources.provider", provider) if (userSpecifiedSchema.isDefined) { - tbl.setProperty("spark.sql.sources.schema", userSpecifiedSchema.get.json) + val threshold = hive.conf.schemaStringLengthThreshold + val schemaJsonString = userSpecifiedSchema.get.json + // Split the JSON string. + val parts = schemaJsonString.grouped(threshold).toSeq + tbl.setProperty("spark.sql.sources.schema.numParts", parts.size.toString) + parts.zipWithIndex.foreach { case (part, index) => + tbl.setProperty(s"spark.sql.sources.schema.part.${index}", part) + } } options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0bd82773f3a55..00306f1cd7f86 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -591,4 +591,25 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalUseDataSource) } } + + test("SPARK-6024 wide schema support") { + // We will need 80 splits for this schema if the threshold is 4000. + val schema = StructType((1 to 5000).map(i => StructField(s"c_${i}", StringType, true))) + assert( + schema.json.size > conf.schemaStringLengthThreshold, + "To correctly test the fix of SPARK-6024, the value of " + + s"spark.sql.sources.schemaStringLengthThreshold needs to be less than ${schema.json.size}") + // Manually create a metastore data source table. + catalog.createDataSourceTable( + tableName = "wide_schema", + userSpecifiedSchema = Some(schema), + provider = "json", + options = Map("path" -> "just a dummy path"), + isExternal = false) + + invalidateTable("wide_schema") + + val actualSchema = table("wide_schema").schema + assert(schema === actualSchema) + } } From 12135e90549f957962899487cd5eb95badd8976d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 26 Feb 2015 22:35:43 -0800 Subject: [PATCH 280/817] [SPARK-5771][UI][hotfix] Change Requested Cores into * if default cores is not set cc andrewor14, srowen. Author: jerryshao Closes #4800 from jerryshao/SPARK-5771 and squashes the following commits: a2483c2 [jerryshao] Change the UI of Requested Cores into * if default cores is not set --- .../scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 9dd96493ee48d..c7a71ea72a77f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -182,7 +182,7 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } } - {app.requestedCores} + {if (app.requestedCores == Int.MaxValue) "*" else app.requestedCores} {Utils.megabytesToString(app.desc.memoryPerSlave)} From 67595eb8fb563eb26654f056033a01f0199bdf68 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 26 Feb 2015 22:36:48 -0800 Subject: [PATCH 281/817] [SPARK-5495][UI] Add app and driver kill function in master web UI Add application kill function in master web UI for standalone mode. Details can be seen in [SPARK-5495](https://issues.apache.org/jira/browse/SPARK-5495). The snapshot of UI shows as below: ![snapshot](https://dl.dropboxusercontent.com/u/19230832/master_ui.png) Please help to review, thanks a lot. Author: jerryshao Closes #4288 from jerryshao/SPARK-5495 and squashes the following commits: fa3e486 [jerryshao] Add some conditions 9a7be93 [jerryshao] Add kill Driver function a239776 [jerryshao] Change the code format ff5195d [jerryshao] Add app kill function in master web UI --- .../deploy/master/ApplicationState.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 53 +++++++++++++++++-- .../spark/deploy/master/ui/MasterWebUI.scala | 8 ++- 3 files changed, 58 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 67e6c5d66af0e..f5b946329ae9b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -21,7 +21,7 @@ private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value - val WAITING, RUNNING, FINISHED, FAILED, UNKNOWN = Value + val WAITING, RUNNING, FINISHED, FAILED, KILLED, UNKNOWN = Value val MAX_NUM_RETRY = 10 } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index c7a71ea72a77f..c086cadca2c7d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -26,8 +26,8 @@ import akka.pattern.ask import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} +import org.apache.spark.deploy.DeployMessages.{RequestKillDriver, MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.master._ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils @@ -41,6 +41,31 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { JsonProtocol.writeMasterState(state) } + def handleAppKillRequest(request: HttpServletRequest): Unit = { + handleKillRequest(request, id => { + parent.master.idToApp.get(id).foreach { app => + parent.master.removeApplication(app, ApplicationState.KILLED) + } + }) + } + + def handleDriverKillRequest(request: HttpServletRequest): Unit = { + handleKillRequest(request, id => { master ! RequestKillDriver(id) }) + } + + private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = { + if (parent.killEnabled && + parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val id = Option(request.getParameter("id")) + if (id.isDefined && killFlag) { + action(id.get) + } + + Thread.sleep(100) + } + } + /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] @@ -167,9 +192,20 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = { + val killLink = if (parent.killEnabled && + (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { + val killLinkUri = s"app/kill?id=${app.id}&terminate=true" + val confirm = "return window.confirm(" + + s"'Are you sure you want to kill application ${app.id} ?');" + + (kill) + + } + {app.id} + {killLink} {app.desc.name} @@ -203,8 +239,19 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } private def driverRow(driver: DriverInfo): Seq[Node] = { + val killLink = if (parent.killEnabled && + (driver.state == DriverState.RUNNING || + driver.state == DriverState.SUBMITTED || + driver.state == DriverState.RELAUNCHING)) { + val killLinkUri = s"driver/kill?id=${driver.id}&terminate=true" + val confirm = "return window.confirm(" + + s"'Are you sure you want to kill driver ${driver.id} ?');" + + (kill) + + } - {driver.id} + {driver.id} {killLink} {driver.submitDate} {driver.worker.map(w => {w.id.toString}).getOrElse("None")} 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 73400c5affb5d..170f90a00ad2a 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 @@ -32,15 +32,21 @@ class MasterWebUI(val master: Master, requestedPort: Int) val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) + val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) initialize() /** Initialize all components of the server. */ def initialize() { + val masterPage = new MasterPage(this) attachPage(new ApplicationPage(this)) attachPage(new HistoryNotFoundPage(this)) - attachPage(new MasterPage(this)) + attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler( + createRedirectHandler("/app/kill", "/", masterPage.handleAppKillRequest)) + attachHandler( + createRedirectHandler("/driver/kill", "/", masterPage.handleDriverKillRequest)) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ From 4a8a0a8ecd836bf7fe0f2e692cf20a62dda313c0 Mon Sep 17 00:00:00 2001 From: Lukasz Jastrzebski Date: Thu, 26 Feb 2015 22:38:06 -0800 Subject: [PATCH 282/817] SPARK-2168 [Spark core] Use relative URIs for the app links in the History Server. As agreed in PR #1160 adding test to verify if history server generates relative links to applications. Author: Lukasz Jastrzebski Closes #4778 from elyast/master and squashes the following commits: 0c07fab [Lukasz Jastrzebski] Incorporating comments for SPARK-2168 6d7866d [Lukasz Jastrzebski] Adjusting test for SPARK-2168 for master branch d6f4fbe [Lukasz Jastrzebski] Added test for SPARK-2168 --- .../deploy/history/HistoryServerSuite.scala | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala 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 new file mode 100644 index 0000000000000..3a9963a5ce7b7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.mockito.Mockito.{when} +import org.scalatest.FunSuite +import org.scalatest.Matchers +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.ui.SparkUI + +class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { + + test("generate history page with relative links") { + val historyServer = mock[HistoryServer] + val request = mock[HttpServletRequest] + val ui = mock[SparkUI] + val link = "/history/app1" + val info = new ApplicationHistoryInfo("app1", "app1", 0, 2, 1, "xxx", true) + when(historyServer.getApplicationList()).thenReturn(Seq(info)) + when(ui.basePath).thenReturn(link) + when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) + val page = new HistoryPage(historyServer) + + //when + val response = page.render(request) + + //then + val links = response \\ "a" + val justHrefs = for { + l <- links + attrs <- l.attribute("href") + } yield (attrs.toString) + justHrefs should contain(link) + } +} From 7c99a014fb1e370e038e0f537c2301ca8138e620 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 26 Feb 2015 22:39:46 -0800 Subject: [PATCH 283/817] [SPARK-6046] Privatize SparkConf.translateConfKey The warning of deprecated configs is actually done when the configs are set, not when they are get. As a result we don't need to explicitly call `translateConfKey` outside of `SparkConf` just to print the warning again in vain. Author: Andrew Or Closes #4797 from andrewor14/warn-deprecated-config and squashes the following commits: 8fb43e6 [Andrew Or] Privatize SparkConf.translateConfKey --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0f4922ab4e310..61b34d524a421 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -407,7 +407,7 @@ private[spark] object SparkConf extends Logging { * @param warn Whether to print a warning if the key is deprecated. Warnings will be printed * only once for each key. */ - def translateConfKey(userKey: String, warn: Boolean = false): String = { + private def translateConfKey(userKey: String, warn: Boolean = false): String = { deprecatedConfigs.get(userKey) .map { deprecatedKey => if (warn) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 1aaa7b72735ab..3e3d6ff29faf0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -49,8 +49,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getOption("spark.history.fs.update.interval.seconds") - .orElse(conf.getOption(SparkConf.translateConfKey("spark.history.fs.updateInterval", true))) - .orElse(conf.getOption(SparkConf.translateConfKey("spark.history.updateInterval", true))) + .orElse(conf.getOption("spark.history.fs.updateInterval")) + .orElse(conf.getOption("spark.history.updateInterval")) .map(_.toInt) .getOrElse(10) * 1000 From 0375a413b8a009f5820897691570a1273ee25b97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=AE=B8=E9=B9=8F?= Date: Thu, 26 Feb 2015 23:05:56 -0800 Subject: [PATCH 284/817] fix spark-6033, clarify the spark.worker.cleanup behavior in standalone mode MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit jira case spark-6033 https://issues.apache.org/jira/browse/SPARK-6033 In standalone deploy mode, the cleanup will only remove the stopped application's directories. The original description about the cleanup behavior is incorrect. Author: 许鹏 Closes #4803 from hseagle/spark-6033 and squashes the following commits: 927a6a0 [许鹏] fix the incorrect description about the spark.worker.cleanup in standalone mode --- docs/spark-standalone.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 5c6084fb46255..74d8653a8b845 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -222,8 +222,7 @@ SPARK_WORKER_OPTS supports the following system properties: false Enable periodic cleanup of worker / application directories. Note that this only affects standalone - mode, as YARN works differently. Applications directories are cleaned up regardless of whether - the application is still running. + mode, as YARN works differently. Only the directories of stopped applications are cleaned up. From 8cd1692c9092150107bed27777951633cbf945f6 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 26 Feb 2015 23:11:43 -0800 Subject: [PATCH 285/817] [SPARK-6036][CORE] avoid race condition between eventlogListener and akka actor system For detail description, pls refer to [SPARK-6036](https://issues.apache.org/jira/browse/SPARK-6036). Author: Zhang, Liye Closes #4785 from liyezhang556520/EventLogInProcess and squashes the following commits: 8b0b0a6 [Zhang, Liye] stop listener after DAGScheduler 79b15b3 [Zhang, Liye] SPARK-6036 avoid race condition between eventlogListener and akka actor system --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d3948d4e6d91b..3cd0c218a36fd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1389,17 +1389,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli stopped = true env.metricsSystem.report() metadataCleaner.cancel() - env.actorSystem.stop(heartbeatReceiver) cleaner.foreach(_.stop()) dagScheduler.stop() dagScheduler = null + listenerBus.stop() + eventLogger.foreach(_.stop()) + env.actorSystem.stop(heartbeatReceiver) progressBar.foreach(_.stop()) taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) - listenerBus.stop() - eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") SparkContext.clearActiveContext() } else { From e747e98490f8ede23b0a9e0795e7445d0b597624 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 27 Feb 2015 13:31:46 +0000 Subject: [PATCH 286/817] [SPARK-6058][Yarn] Log the user class exception in ApplicationMaster Because ApplicationMaster doesn't set SparkUncaughtExceptionHandler, the exception in the user class won't be logged. This PR added a `logError` for it. Author: zsxwing Closes #4813 from zsxwing/SPARK-6058 and squashes the following commits: 806c932 [zsxwing] Log the user class exception --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 20fc19166ac4e..796422bc25b59 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 @@ -486,11 +486,10 @@ private[spark] class ApplicationMaster( case _: InterruptedException => // Reporter thread can interrupt to stop user class case cause: Throwable => + logError("User class threw exception: " + cause.getMessage, cause) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, "User class threw exception: " + cause.getMessage) - // re-throw to get it logged - throw cause } } } From 57566d0af3008982a1e24a763ed2f6a700b40f8f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 27 Feb 2015 13:33:39 +0000 Subject: [PATCH 287/817] [SPARK-6059][Yarn] Add volatile to ApplicationMaster's reporterThread and allocator `ApplicationMaster.reporterThread` and `ApplicationMaster.allocator` are accessed in multiple threads, so they should be marked as `volatile`. Author: zsxwing Closes #4814 from zsxwing/SPARK-6059 and squashes the following commits: 17d9386 [zsxwing] Add volatile to ApplicationMaster's reporterThread and allocator --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 796422bc25b59..e966bfba7bb7d 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 @@ -68,8 +68,8 @@ private[spark] class ApplicationMaster( @volatile private var finalMsg: String = "" @volatile private var userClassThread: Thread = _ - private var reporterThread: Thread = _ - private var allocator: YarnAllocator = _ + @volatile private var reporterThread: Thread = _ + @volatile private var allocator: YarnAllocator = _ // Fields used in client mode. private var actorSystem: ActorSystem = null From d17cb2ba33b363dd346ac5a5681e1757decd0f4d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 27 Feb 2015 13:00:36 -0800 Subject: [PATCH 288/817] [SPARK-4587] [mllib] [docs] Fixed save,load calls in ML guide examples Should pass spark context to save/load CC: mengxr Author: Joseph K. Bradley Closes #4816 from jkbradley/ml-io-doc-fix and squashes the following commits: 83d369d [Joseph K. Bradley] added comment to save,load parts of ML guide examples 2841170 [Joseph K. Bradley] Fixed save,load calls in ML guide examples --- docs/mllib-collaborative-filtering.md | 10 ++++--- docs/mllib-decision-tree.md | 20 ++++++++------ docs/mllib-ensembles.md | 40 ++++++++++++++++----------- docs/mllib-linear-methods.md | 20 ++++++++------ docs/mllib-naive-bayes.md | 10 ++++--- 5 files changed, 60 insertions(+), 40 deletions(-) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 935cd8dad3b25..27aa4d38b7617 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -97,8 +97,9 @@ val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => }.mean() println("Mean Squared Error = " + MSE) -model.save("myModelPath") -val sameModel = MatrixFactorizationModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = MatrixFactorizationModel.load(sc, "myModelPath") {% endhighlight %} If the rating matrix is derived from another source of information (e.g., it is inferred from @@ -186,8 +187,9 @@ public class CollaborativeFiltering { ).rdd()).mean(); System.out.println("Mean Squared Error = " + MSE); - model.save("myModelPath"); - MatrixFactorizationModel sameModel = MatrixFactorizationModel.load("myModelPath"); + // Save and load model + model.save(sc.sc(), "myModelPath"); + MatrixFactorizationModel sameModel = MatrixFactorizationModel.load(sc.sc(), "myModelPath"); } } {% endhighlight %} diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 4695d1cde4901..8e478ab035582 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -223,8 +223,9 @@ val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData. println("Test Error = " + testErr) println("Learned classification tree model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = DecisionTreeModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = DecisionTreeModel.load(sc, "myModelPath") {% endhighlight %} @@ -284,8 +285,9 @@ Double testErr = System.out.println("Test Error: " + testErr); System.out.println("Learned classification tree model:\n" + model.toDebugString()); -model.save("myModelPath"); -DecisionTreeModel sameModel = DecisionTreeModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath"); {% endhighlight %} @@ -362,8 +364,9 @@ val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean println("Test Mean Squared Error = " + testMSE) println("Learned regression tree model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = DecisionTreeModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = DecisionTreeModel.load(sc, "myModelPath") {% endhighlight %} @@ -429,8 +432,9 @@ Double testMSE = System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression tree model:\n" + model.toDebugString()); -model.save("myModelPath"); -DecisionTreeModel sameModel = DecisionTreeModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath"); {% endhighlight %} diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index ddae84165f8a9..ec1ef38b453d3 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -129,8 +129,9 @@ val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData. println("Test Error = " + testErr) println("Learned classification forest model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = RandomForestModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = RandomForestModel.load(sc, "myModelPath") {% endhighlight %} @@ -193,8 +194,9 @@ Double testErr = System.out.println("Test Error: " + testErr); System.out.println("Learned classification forest model:\n" + model.toDebugString()); -model.save("myModelPath"); -RandomForestModel sameModel = RandomForestModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath"); {% endhighlight %} @@ -276,8 +278,9 @@ val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean println("Test Mean Squared Error = " + testMSE) println("Learned regression forest model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = RandomForestModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = RandomForestModel.load(sc, "myModelPath") {% endhighlight %} @@ -343,8 +346,9 @@ Double testMSE = System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression forest model:\n" + model.toDebugString()); -model.save("myModelPath"); -RandomForestModel sameModel = RandomForestModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath"); {% endhighlight %} @@ -504,8 +508,9 @@ val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData. println("Test Error = " + testErr) println("Learned classification GBT model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = GradientBoostedTreesModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") {% endhighlight %} @@ -568,8 +573,9 @@ Double testErr = System.out.println("Test Error: " + testErr); System.out.println("Learned classification GBT model:\n" + model.toDebugString()); -model.save("myModelPath"); -GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath"); {% endhighlight %} @@ -647,8 +653,9 @@ val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean println("Test Mean Squared Error = " + testMSE) println("Learned regression GBT model:\n" + model.toDebugString) -model.save("myModelPath") -val sameModel = GradientBoostedTreesModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") {% endhighlight %} @@ -717,8 +724,9 @@ Double testMSE = System.out.println("Test Mean Squared Error: " + testMSE); System.out.println("Learned regression GBT model:\n" + model.toDebugString()); -model.save("myModelPath"); -GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath"); {% endhighlight %} diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index d9fc63b37d116..ffbd7ef1bff51 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -223,8 +223,9 @@ val auROC = metrics.areaUnderROC() println("Area under ROC = " + auROC) -model.save("myModelPath") -val sameModel = SVMModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = SVMModel.load(sc, "myModelPath") {% endhighlight %} The `SVMWithSGD.train()` method by default performs L2 regularization with the @@ -308,8 +309,9 @@ public class SVMClassifier { System.out.println("Area under ROC = " + auROC); - model.save("myModelPath"); - SVMModel sameModel = SVMModel.load("myModelPath"); + // Save and load model + model.save(sc.sc(), "myModelPath"); + SVMModel sameModel = SVMModel.load(sc.sc(), "myModelPath"); } } {% endhighlight %} @@ -423,8 +425,9 @@ val valuesAndPreds = parsedData.map { point => val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) -model.save("myModelPath") -val sameModel = LinearRegressionModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = LinearRegressionModel.load(sc, "myModelPath") {% endhighlight %} [`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) @@ -496,8 +499,9 @@ public class LinearRegression { ).rdd()).mean(); System.out.println("training Mean Squared Error = " + MSE); - model.save("myModelPath"); - LinearRegressionModel sameModel = LinearRegressionModel.load("myModelPath"); + // Save and load model + model.save(sc.sc(), "myModelPath"); + LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), "myModelPath"); } } {% endhighlight %} diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 81173255b590d..5224a0b49a991 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -56,8 +56,9 @@ val model = NaiveBayes.train(training, lambda = 1.0) val predictionAndLabel = test.map(p => (model.predict(p.features), p.label)) val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() -model.save("myModelPath") -val sameModel = NaiveBayesModel.load("myModelPath") +// Save and load model +model.save(sc, "myModelPath") +val sameModel = NaiveBayesModel.load(sc, "myModelPath") {% endhighlight %} @@ -97,8 +98,9 @@ double accuracy = predictionAndLabel.filter(new Function, } }).count() / (double) test.count(); -model.save("myModelPath"); -NaiveBayesModel sameModel = NaiveBayesModel.load("myModelPath"); +// Save and load model +model.save(sc.sc(), "myModelPath"); +NaiveBayesModel sameModel = NaiveBayesModel.load(sc.sc(), "myModelPath"); {% endhighlight %} From 5f7f3b938e1776168be866fc9ee87dc7494696cc Mon Sep 17 00:00:00 2001 From: Saisai Shao Date: Fri, 27 Feb 2015 13:01:42 -0800 Subject: [PATCH 289/817] [Streaming][Minor] Remove useless type signature of Java Kafka direct stream API cc tdas . Author: Saisai Shao Closes #4817 from jerryshao/signature-minor-fix and squashes the following commits: eebfaac [Saisai Shao] Remove useless type parameter --- .../scala/org/apache/spark/streaming/kafka/KafkaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 62a659518943d..5a9bd4214cf51 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -512,7 +512,7 @@ object KafkaUtils { * @param topics Names of the topics to consume */ @Experimental - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( jssc: JavaStreamingContext, keyClass: Class[K], valueClass: Class[V], From 8c468a6600e0deb5464990df60148212e64fdecd Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 28 Feb 2015 08:41:49 +0800 Subject: [PATCH 290/817] [SPARK-5751] [SQL] Sets SPARK_HOME as SPARK_PID_DIR when running Thrift server test suites This is a follow-up of #4720. By default, `spark-daemon.sh` writes PID files under `/tmp`, which makes it impossible to start multiple server instances simultaneously. This PR sets `SPARK_PID_DIR` to Spark home directory to workaround this problem. Many thanks to chenghao-intel for pointing out this issue! [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4758) Author: Cheng Lian Closes #4758 from liancheng/thriftserver-pid-dir and squashes the following commits: 252fa0f [Cheng Lian] Uses temporary directory as Thrift server PID directory 1b3d1e3 [Cheng Lian] Sets SPARK_HOME as SPARK_PID_DIR when running Thrift server test suites --- .../hive/thriftserver/HiveThriftServer2Suites.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 77ef37253e38f..d783d487b5c60 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -39,6 +39,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.util.Utils object TestData { def getTestDataFilePath(name: String) = { @@ -273,6 +274,7 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit private var metastorePath: File = _ private def metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + private val pidDir: File = Utils.createTempDir("thriftserver-pid") private var logPath: File = _ private var logTailingProcess: Process = _ private var diagnosisBuffer: ArrayBuffer[String] = ArrayBuffer.empty[String] @@ -315,7 +317,14 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit logInfo(s"Trying to start HiveThriftServer2: port=$port, mode=$mode, attempt=$attempt") - logPath = Process(command, None, "SPARK_TESTING" -> "0").lines.collectFirst { + val env = Seq( + // Disables SPARK_TESTING to exclude log4j.properties in test directories. + "SPARK_TESTING" -> "0", + // Points SPARK_PID_DIR to SPARK_HOME, otherwise only 1 Thrift server instance can be started + // at a time, which is not Jenkins friendly. + "SPARK_PID_DIR" -> pidDir.getCanonicalPath) + + logPath = Process(command, None, env: _*).lines.collectFirst { case line if line.contains(LOG_FILE_MARK) => new File(line.drop(LOG_FILE_MARK.length)) }.getOrElse { throw new RuntimeException("Failed to find HiveThriftServer2 log file.") @@ -346,7 +355,7 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit private def stopThriftServer(): Unit = { // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while. - Process(stopScript, None).run().exitValue() + Process(stopScript, None, "SPARK_PID_DIR" -> pidDir.getCanonicalPath).run().exitValue() Thread.sleep(3.seconds.toMillis) warehousePath.delete() From e0e64ba4b1b8eb72e856286f756c65fa22ab0a36 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 27 Feb 2015 20:07:17 -0800 Subject: [PATCH 291/817] [SPARK-6055] [PySpark] fix incorrect __eq__ of DataType The _eq_ of DataType is not correct, class cache is not use correctly (created class can not be find by dataType), then it will create lots of classes (saved in _cached_cls), never released. Also, all same DataType have same hash code, there will be many object in a dict with the same hash code, end with hash attach, it's very slow to access this dict (depends on the implementation of CPython). This PR also improve the performance of inferSchema (avoid the unnecessary converter of object). cc pwendell JoshRosen Author: Davies Liu Closes #4808 from davies/leak and squashes the following commits: 6a322a4 [Davies Liu] tests refactor 3da44fc [Davies Liu] fix __eq__ of Singleton 534ac90 [Davies Liu] add more checks 46999dc [Davies Liu] fix tests d9ae973 [Davies Liu] fix memory leak in sql --- python/pyspark/sql/context.py | 90 +----------------------- python/pyspark/sql/dataframe.py | 4 +- python/pyspark/sql/tests.py | 9 +++ python/pyspark/sql/types.py | 120 +++++++++++++++++++------------- 4 files changed, 86 insertions(+), 137 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 5d7aeb664cadf..795ef0dbc4c47 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -17,7 +17,6 @@ import warnings import json -from array import array from itertools import imap from py4j.protocol import Py4JError @@ -25,7 +24,7 @@ from pyspark.rdd import RDD, _prepare_for_python_RDD from pyspark.serializers import AutoBatchedSerializer, PickleSerializer -from pyspark.sql.types import StringType, StructType, _verify_type, \ +from pyspark.sql.types import Row, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter from pyspark.sql.dataframe import DataFrame @@ -620,93 +619,6 @@ def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) -def _create_row(fields, values): - row = Row(*values) - row.__FIELDS__ = fields - return row - - -class Row(tuple): - - """ - A row in L{DataFrame}. The fields in it can be accessed like attributes. - - Row can be used to create a row object by using named arguments, - the fields will be sorted by names. - - >>> row = Row(name="Alice", age=11) - >>> row - Row(age=11, name='Alice') - >>> row.name, row.age - ('Alice', 11) - - Row also can be used to create another Row like class, then it - could be used to create Row objects, such as - - >>> Person = Row("name", "age") - >>> Person - - >>> Person("Alice", 11) - Row(name='Alice', age=11) - """ - - def __new__(self, *args, **kwargs): - if args and kwargs: - raise ValueError("Can not use both args " - "and kwargs to create Row") - if args: - # create row class or objects - return tuple.__new__(self, args) - - elif kwargs: - # create row objects - names = sorted(kwargs.keys()) - values = tuple(kwargs[n] for n in names) - row = tuple.__new__(self, values) - row.__FIELDS__ = names - return row - - else: - raise ValueError("No args or kwargs") - - def asDict(self): - """ - Return as an dict - """ - if not hasattr(self, "__FIELDS__"): - raise TypeError("Cannot convert a Row class into dict") - return dict(zip(self.__FIELDS__, self)) - - # let obect acs like class - def __call__(self, *args): - """create new Row object""" - return _create_row(self, args) - - def __getattr__(self, item): - if item.startswith("__"): - raise AttributeError(item) - try: - # it will be slow when it has many fields, - # but this will not be used in normal cases - idx = self.__FIELDS__.index(item) - return self[idx] - except IndexError: - raise AttributeError(item) - - def __reduce__(self): - if hasattr(self, "__FIELDS__"): - return (_create_row, (self.__FIELDS__, tuple(self))) - else: - return tuple.__reduce__(self) - - def __repr__(self): - if hasattr(self, "__FIELDS__"): - return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) - else: - return "" % ", ".join(self) - - def _test(): import doctest from pyspark.context import SparkContext diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index aec99017fbdc1..5c3b7377c33b5 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1025,10 +1025,12 @@ def cast(self, dataType): ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) jdt = ssql_ctx.parseDataType(dataType.json()) jc = self._jc.cast(jdt) + else: + raise TypeError("unexpected type: %s" % type(dataType)) return Column(jc) def __repr__(self): - return 'Column<%s>' % self._jdf.toString().encode('utf8') + return 'Column<%s>' % self._jc.toString().encode('utf8') def _test(): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 83899ad4b1b12..2720439416682 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -24,6 +24,7 @@ import pydoc import shutil import tempfile +import pickle import py4j @@ -88,6 +89,14 @@ def __eq__(self, other): other.x == self.x and other.y == self.y +class DataTypeTests(unittest.TestCase): + # regression test for SPARK-6055 + def test_data_type_eq(self): + lt = LongType() + lt2 = pickle.loads(pickle.dumps(LongType())) + self.assertEquals(lt, lt2) + + class SQLTests(ReusedPySparkTestCase): @classmethod diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 0f5dc2be6dab8..31a861e1feb46 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -21,6 +21,7 @@ import warnings import json import re +import weakref from array import array from operator import itemgetter @@ -42,8 +43,7 @@ def __hash__(self): return hash(str(self)) def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.__dict__ == other.__dict__) + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not self.__eq__(other) @@ -64,6 +64,8 @@ def json(self): sort_keys=True) +# This singleton pattern does not work with pickle, you will get +# another object after pickle and unpickle class PrimitiveTypeSingleton(type): """Metaclass for PrimitiveType""" @@ -82,10 +84,6 @@ class PrimitiveType(DataType): __metaclass__ = PrimitiveTypeSingleton - def __eq__(self, other): - # because they should be the same object - return self is other - class NullType(PrimitiveType): @@ -242,11 +240,12 @@ def __init__(self, elementType, containsNull=True): :param elementType: the data type of elements. :param containsNull: indicates whether the list contains None values. - >>> ArrayType(StringType) == ArrayType(StringType, True) + >>> ArrayType(StringType()) == ArrayType(StringType(), True) True - >>> ArrayType(StringType, False) == ArrayType(StringType) + >>> ArrayType(StringType(), False) == ArrayType(StringType()) False """ + assert isinstance(elementType, DataType), "elementType should be DataType" self.elementType = elementType self.containsNull = containsNull @@ -292,13 +291,15 @@ def __init__(self, keyType, valueType, valueContainsNull=True): :param valueContainsNull: indicates whether values contains null values. - >>> (MapType(StringType, IntegerType) - ... == MapType(StringType, IntegerType, True)) + >>> (MapType(StringType(), IntegerType()) + ... == MapType(StringType(), IntegerType(), True)) True - >>> (MapType(StringType, IntegerType, False) - ... == MapType(StringType, FloatType)) + >>> (MapType(StringType(), IntegerType(), False) + ... == MapType(StringType(), FloatType())) False """ + assert isinstance(keyType, DataType), "keyType should be DataType" + assert isinstance(valueType, DataType), "valueType should be DataType" self.keyType = keyType self.valueType = valueType self.valueContainsNull = valueContainsNull @@ -348,13 +349,14 @@ def __init__(self, name, dataType, nullable=True, metadata=None): to simple type that can be serialized to JSON automatically - >>> (StructField("f1", StringType, True) - ... == StructField("f1", StringType, True)) + >>> (StructField("f1", StringType(), True) + ... == StructField("f1", StringType(), True)) True - >>> (StructField("f1", StringType, True) - ... == StructField("f2", StringType, True)) + >>> (StructField("f1", StringType(), True) + ... == StructField("f2", StringType(), True)) False """ + assert isinstance(dataType, DataType), "dataType should be DataType" self.name = name self.dataType = dataType self.nullable = nullable @@ -393,16 +395,17 @@ class StructType(DataType): def __init__(self, fields): """Creates a StructType - >>> struct1 = StructType([StructField("f1", StringType, True)]) - >>> struct2 = StructType([StructField("f1", StringType, True)]) + >>> struct1 = StructType([StructField("f1", StringType(), True)]) + >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 True - >>> struct1 = StructType([StructField("f1", StringType, True)]) - >>> struct2 = StructType([StructField("f1", StringType, True), - ... [StructField("f2", IntegerType, False)]]) + >>> struct1 = StructType([StructField("f1", StringType(), True)]) + >>> struct2 = StructType([StructField("f1", StringType(), True), + ... StructField("f2", IntegerType(), False)]) >>> struct1 == struct2 False """ + assert all(isinstance(f, DataType) for f in fields), "fields should be a list of DataType" self.fields = fields def simpleString(self): @@ -505,20 +508,24 @@ def __eq__(self, other): def _parse_datatype_json_string(json_string): """Parses the given data type JSON string. + >>> import pickle >>> def check_datatype(datatype): + ... pickled = pickle.loads(pickle.dumps(datatype)) + ... assert datatype == pickled ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) - ... return datatype == python_datatype - >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) - True + ... assert datatype == python_datatype + >>> for cls in _all_primitive_types.values(): + ... check_datatype(cls()) + >>> # Simple ArrayType. >>> simple_arraytype = ArrayType(StringType(), True) >>> check_datatype(simple_arraytype) - True + >>> # Simple MapType. >>> simple_maptype = MapType(StringType(), LongType()) >>> check_datatype(simple_maptype) - True + >>> # Simple StructType. >>> simple_structtype = StructType([ ... StructField("a", DecimalType(), False), @@ -526,7 +533,7 @@ def _parse_datatype_json_string(json_string): ... StructField("c", LongType(), True), ... StructField("d", BinaryType(), False)]) >>> check_datatype(simple_structtype) - True + >>> # Complex StructType. >>> complex_structtype = StructType([ ... StructField("simpleArray", simple_arraytype, True), @@ -535,22 +542,20 @@ def _parse_datatype_json_string(json_string): ... StructField("boolean", BooleanType(), False), ... StructField("withMeta", DoubleType(), False, {"name": "age"})]) >>> check_datatype(complex_structtype) - True + >>> # Complex ArrayType. >>> complex_arraytype = ArrayType(complex_structtype, True) >>> check_datatype(complex_arraytype) - True + >>> # Complex MapType. >>> complex_maptype = MapType(complex_structtype, ... complex_arraytype, False) >>> check_datatype(complex_maptype) - True + >>> check_datatype(ExamplePointUDT()) - True >>> structtype_with_udt = StructType([StructField("label", DoubleType(), False), ... StructField("point", ExamplePointUDT(), False)]) >>> check_datatype(structtype_with_udt) - True """ return _parse_datatype_json_value(json.loads(json_string)) @@ -786,8 +791,24 @@ def _merge_type(a, b): return a +def _need_converter(dataType): + if isinstance(dataType, StructType): + return True + elif isinstance(dataType, ArrayType): + return _need_converter(dataType.elementType) + elif isinstance(dataType, MapType): + return _need_converter(dataType.keyType) or _need_converter(dataType.valueType) + elif isinstance(dataType, NullType): + return True + else: + return False + + def _create_converter(dataType): """Create an converter to drop the names of fields in obj """ + if not _need_converter(dataType): + return lambda x: x + if isinstance(dataType, ArrayType): conv = _create_converter(dataType.elementType) return lambda row: map(conv, row) @@ -806,13 +827,17 @@ def _create_converter(dataType): # dataType must be StructType names = [f.name for f in dataType.fields] converters = [_create_converter(f.dataType) for f in dataType.fields] + convert_fields = any(_need_converter(f.dataType) for f in dataType.fields) def convert_struct(obj): if obj is None: return if isinstance(obj, (tuple, list)): - return tuple(conv(v) for v, conv in zip(obj, converters)) + if convert_fields: + return tuple(conv(v) for v, conv in zip(obj, converters)) + else: + return tuple(obj) if isinstance(obj, dict): d = obj @@ -821,7 +846,10 @@ def convert_struct(obj): else: raise ValueError("Unexpected obj: %s" % obj) - return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) + if convert_fields: + return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) + else: + return tuple([d.get(name) for name in names]) return convert_struct @@ -871,20 +899,20 @@ def _parse_field_abstract(s): Parse a field in schema abstract >>> _parse_field_abstract("a") - StructField(a,None,true) + StructField(a,NullType,true) >>> _parse_field_abstract("b(c d)") - StructField(b,StructType(...c,None,true),StructField(d... + StructField(b,StructType(...c,NullType,true),StructField(d... >>> _parse_field_abstract("a[]") - StructField(a,ArrayType(None,true),true) + StructField(a,ArrayType(NullType,true),true) >>> _parse_field_abstract("a{[]}") - StructField(a,MapType(None,ArrayType(None,true),true),true) + StructField(a,MapType(NullType,ArrayType(NullType,true),true),true) """ if set(_BRACKETS.keys()) & set(s): idx = min((s.index(c) for c in _BRACKETS if c in s)) name = s[:idx] return StructField(name, _parse_schema_abstract(s[idx:]), True) else: - return StructField(s, None, True) + return StructField(s, NullType(), True) def _parse_schema_abstract(s): @@ -898,11 +926,11 @@ def _parse_schema_abstract(s): >>> _parse_schema_abstract("c{} d{a b}") StructType...c,MapType...d,MapType...a...b... >>> _parse_schema_abstract("a b(t)").fields[1] - StructField(b,StructType(List(StructField(t,None,true))),true) + StructField(b,StructType(List(StructField(t,NullType,true))),true) """ s = s.strip() if not s: - return + return NullType() elif s.startswith('('): return _parse_schema_abstract(s[1:-1]) @@ -911,7 +939,7 @@ def _parse_schema_abstract(s): return ArrayType(_parse_schema_abstract(s[1:-1]), True) elif s.startswith('{'): - return MapType(None, _parse_schema_abstract(s[1:-1])) + return MapType(NullType(), _parse_schema_abstract(s[1:-1])) parts = _split_schema_abstract(s) fields = [_parse_field_abstract(p) for p in parts] @@ -931,7 +959,7 @@ def _infer_schema_type(obj, dataType): >>> _infer_schema_type(row, schema) StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ - if dataType is None: + if dataType is NullType(): return _infer_type(obj) if not obj: @@ -1037,8 +1065,7 @@ def _verify_type(obj, dataType): for v, f in zip(obj, dataType.fields): _verify_type(v, f.dataType) - -_cached_cls = {} +_cached_cls = weakref.WeakValueDictionary() def _restore_object(dataType, obj): @@ -1233,8 +1260,7 @@ def __new__(self, *args, **kwargs): elif kwargs: # create row objects names = sorted(kwargs.keys()) - values = tuple(kwargs[n] for n in names) - row = tuple.__new__(self, values) + row = tuple.__new__(self, [kwargs[n] for n in names]) row.__FIELDS__ = names return row From dba08d1fc3bdb9245aefe695970354df088a93b6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 27 Feb 2015 22:44:11 -0800 Subject: [PATCH 292/817] [SPARK-6070] [yarn] Remove unneeded classes from shuffle service jar. These may conflict with the classes already in the NM. We shouldn't be repackaging them. Author: Marcelo Vanzin Closes #4820 from vanzin/SPARK-6070 and squashes the following commits: 871b566 [Marcelo Vanzin] The "d'oh how didn't I think of it before" solution. 3cba946 [Marcelo Vanzin] Use profile instead, so that dependencies don't need to be explicitly listed. 7a18a1b [Marcelo Vanzin] [SPARK-6070] [yarn] Remove unneeded classes from shuffle service jar. --- network/yarn/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index acec8f18f2b5c..39b99f54f6dbc 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -33,6 +33,8 @@ http://spark.apache.org/ network-yarn + + provided @@ -47,7 +49,6 @@ org.apache.hadoop hadoop-client - provided From 6d8e5fbc0d83411174ffa59ff6a761a862eca32c Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 27 Feb 2015 22:59:35 -0800 Subject: [PATCH 293/817] [SPARK-5979][SPARK-6032] Smaller safer --packages fix pwendell tdas This is the safer parts of PR #4754: - SPARK-5979: All dependencies with the groupId `org.apache.spark` passed through `--packages`, were being excluded from the dependency tree on the assumption that they would be in the assembly jar. This is not the case, therefore the exclusion rules had to be defined more explicitly. - SPARK-6032: Ivy prints a whole lot of logs while retrieving dependencies. These were printed to `System.out`. Moved the logging to `System.err`. Author: Burak Yavuz Closes #4802 from brkyvz/simple-streaming-fix and squashes the following commits: e0f38cb [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into simple-streaming-fix bad921c [Burak Yavuz] [SPARK-5979][SPARK-6032] Smaller safer fix --- .../org/apache/spark/deploy/SparkSubmit.scala | 53 +++++++++++++------ .../spark/deploy/SparkSubmitUtilsSuite.scala | 16 +++++- 2 files changed, 51 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4c4110812e0a1..4a74641f4e1fa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -655,8 +655,7 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided - * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. The latter provides - * simplicity for Spark Package users. + * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ @@ -747,6 +746,35 @@ private[spark] object SparkSubmitUtils { md.addDependency(dd) } } + + /** Add exclusion rules for dependencies already included in the spark-assembly */ + private[spark] def addExclusionRules( + ivySettings: IvySettings, + ivyConfName: String, + md: DefaultModuleDescriptor): Unit = { + // Add scala exclusion rule + val scalaArtifacts = new ArtifactId(new ModuleId("*", "scala-library"), "*", "*", "*") + val scalaDependencyExcludeRule = + new DefaultExcludeRule(scalaArtifacts, ivySettings.getMatcher("glob"), null) + scalaDependencyExcludeRule.addConfiguration(ivyConfName) + md.addExcludeRule(scalaDependencyExcludeRule) + + // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka and + // other spark-streaming utility components. Underscore is there to differentiate between + // spark-streaming_2.1x and spark-streaming-kafka-assembly_2.1x + val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", + "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") + + components.foreach { comp => + val sparkArtifacts = + new ArtifactId(new ModuleId("org.apache.spark", s"spark-$comp*"), "*", "*", "*") + val sparkDependencyExcludeRule = + new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null) + sparkDependencyExcludeRule.addConfiguration(ivyConfName) + + md.addExcludeRule(sparkDependencyExcludeRule) + } + } /** A nice function to use in tests as well. Values are dummy strings. */ private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( @@ -768,6 +796,9 @@ private[spark] object SparkSubmitUtils { if (coordinates == null || coordinates.trim.isEmpty) { "" } else { + val sysOut = System.out + // To prevent ivy from logging to system out + System.setOut(printStream) val artifacts = extractMavenCoordinates(coordinates) // Default configuration name for ivy val ivyConfName = "default" @@ -811,19 +842,9 @@ private[spark] object SparkSubmitUtils { val md = getModuleDescriptor md.setDefaultConf(ivyConfName) - // Add an exclusion rule for Spark and Scala Library - val sparkArtifacts = new ArtifactId(new ModuleId("org.apache.spark", "*"), "*", "*", "*") - val sparkDependencyExcludeRule = - new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null) - sparkDependencyExcludeRule.addConfiguration(ivyConfName) - val scalaArtifacts = new ArtifactId(new ModuleId("*", "scala-library"), "*", "*", "*") - val scalaDependencyExcludeRule = - new DefaultExcludeRule(scalaArtifacts, ivySettings.getMatcher("glob"), null) - scalaDependencyExcludeRule.addConfiguration(ivyConfName) - - // Exclude any Spark dependencies, and add all supplied maven artifacts as dependencies - md.addExcludeRule(sparkDependencyExcludeRule) - md.addExcludeRule(scalaDependencyExcludeRule) + // Add exclusion rules for Spark and Scala Library + addExclusionRules(ivySettings, ivyConfName, md) + // add all supplied maven artifacts as dependencies addDependenciesToIvy(md, artifacts, ivyConfName) // resolve dependencies @@ -835,7 +856,7 @@ private[spark] object SparkSubmitUtils { ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId, packagesDirectory.getAbsolutePath + File.separator + "[artifact](-[classifier]).[ext]", retrieveOptions.setConfs(Array(ivyConfName))) - + System.setOut(sysOut) resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index ad62b35f624f6..8bcca926097a1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -117,8 +117,20 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { } test("neglects Spark and Spark's dependencies") { - val path = SparkSubmitUtils.resolveMavenCoordinates( - "org.apache.spark:spark-core_2.10:1.2.0", None, None, true) + val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", + "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") + + val coordinates = + components.map(comp => s"org.apache.spark:spark-${comp}2.10:1.2.0").mkString(",") + + ",org.apache.spark:spark-core_fake:1.2.0" + + val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, true) assert(path === "", "should return empty path") + // Should not exclude the following dependency. Will throw an error, because it doesn't exist, + // but the fact that it is checking means that it wasn't excluded. + intercept[RuntimeException] { + SparkSubmitUtils.resolveMavenCoordinates(coordinates + + ",org.apache.spark:spark-streaming-kafka-assembly_2.10:1.2.0", None, None, true) + } } } From 9168259813713a12251fb0d457ffbbed8ba857f8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 27 Feb 2015 23:10:09 -0800 Subject: [PATCH 294/817] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #1128 (close requested by 'srowen') Closes #3425 (close requested by 'srowen') Closes #4770 (close requested by 'srowen') Closes #2813 (close requested by 'srowen') From e6003f0a571ba44fcd011e695c8622e11cfee7dd Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 28 Feb 2015 21:15:43 +0800 Subject: [PATCH 295/817] [SPARK-5775] [SQL] BugFix: GenericRow cannot be cast to SpecificMutableRow when nested data and partitioned table This PR adapts anselmevignon's #4697 to master and branch-1.3. Please refer to PR description of #4697 for details. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4792) Author: Cheng Lian Author: Cheng Lian Author: Yin Huai Closes #4792 from liancheng/spark-5775 and squashes the following commits: 538f506 [Cheng Lian] Addresses comments cee55cf [Cheng Lian] Merge pull request #4 from yhuai/spark-5775-yin b0b74fb [Yin Huai] Remove runtime pattern matching. ca6e038 [Cheng Lian] Fixes SPARK-5775 --- .../sql/parquet/ParquetTableOperations.scala | 59 ++++++-- .../apache/spark/sql/parquet/newParquet.scala | 48 +++++-- .../spark/sql/parquet/parquetSuites.scala | 134 +++++++++++++++++- 3 files changed, 217 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 4e4f647767dc9..225ec6db7d553 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -126,6 +126,13 @@ private[sql] case class ParquetTableScan( conf) if (requestedPartitionOrdinals.nonEmpty) { + // This check is based on CatalystConverter.createRootConverter. + val primitiveRow = output.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType)) + + // Uses temporary variable to avoid the whole `ParquetTableScan` object being captured into + // the `mapPartitionsWithInputSplit` closure below. + val outputSize = output.size + baseRDD.mapPartitionsWithInputSplit { case (split, iter) => val partValue = "([^=]+)=([^=]+)".r val partValues = @@ -143,19 +150,47 @@ private[sql] case class ParquetTableScan( relation.partitioningAttributes .map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) - new Iterator[Row] { - def hasNext = iter.hasNext - def next() = { - val row = iter.next()._2.asInstanceOf[SpecificMutableRow] - - // Parquet will leave partitioning columns empty, so we fill them in here. - var i = 0 - while (i < requestedPartitionOrdinals.size) { - row(requestedPartitionOrdinals(i)._2) = - partitionRowValues(requestedPartitionOrdinals(i)._1) - i += 1 + if (primitiveRow) { + new Iterator[Row] { + def hasNext = iter.hasNext + def next() = { + // We are using CatalystPrimitiveRowConverter and it returns a SpecificMutableRow. + val row = iter.next()._2.asInstanceOf[SpecificMutableRow] + + // Parquet will leave partitioning columns empty, so we fill them in here. + var i = 0 + while (i < requestedPartitionOrdinals.size) { + row(requestedPartitionOrdinals(i)._2) = + partitionRowValues(requestedPartitionOrdinals(i)._1) + i += 1 + } + row + } + } + } else { + // Create a mutable row since we need to fill in values from partition columns. + val mutableRow = new GenericMutableRow(outputSize) + new Iterator[Row] { + def hasNext = iter.hasNext + def next() = { + // We are using CatalystGroupConverter and it returns a GenericRow. + // Since GenericRow is not mutable, we just cast it to a Row. + val row = iter.next()._2.asInstanceOf[Row] + + var i = 0 + while (i < row.size) { + mutableRow(i) = row(i) + i += 1 + } + // Parquet will leave partitioning columns empty, so we fill them in here. + i = 0 + while (i < requestedPartitionOrdinals.size) { + mutableRow(requestedPartitionOrdinals(i)._2) = + partitionRowValues(requestedPartitionOrdinals(i)._1) + i += 1 + } + mutableRow } - row } } } 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 e648618468d5d..6d56be3ab8dd4 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 @@ -482,6 +482,10 @@ private[sql] case class ParquetRelation2( // When the data does not include the key and the key is requested then we must fill it in // based on information from the input split. if (!partitionKeysIncludedInDataSchema && partitionKeyLocations.nonEmpty) { + // This check is based on CatalystConverter.createRootConverter. + val primitiveRow = + requestedSchema.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType)) + baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { case p if split.getPath.getParent.toString == p.path => p.values @@ -489,16 +493,42 @@ private[sql] case class ParquetRelation2( val requiredPartOrdinal = partitionKeyLocations.keys.toSeq - iterator.map { pair => - val row = pair._2.asInstanceOf[SpecificMutableRow] - var i = 0 - while (i < requiredPartOrdinal.size) { - // TODO Avoids boxing cost here! - val partOrdinal = requiredPartOrdinal(i) - row.update(partitionKeyLocations(partOrdinal), partValues(partOrdinal)) - i += 1 + if (primitiveRow) { + iterator.map { pair => + // We are using CatalystPrimitiveRowConverter and it returns a SpecificMutableRow. + val row = pair._2.asInstanceOf[SpecificMutableRow] + var i = 0 + while (i < requiredPartOrdinal.size) { + // TODO Avoids boxing cost here! + val partOrdinal = requiredPartOrdinal(i) + row.update(partitionKeyLocations(partOrdinal), partValues(partOrdinal)) + i += 1 + } + row + } + } else { + // Create a mutable row since we need to fill in values from partition columns. + val mutableRow = new GenericMutableRow(requestedSchema.size) + iterator.map { pair => + // We are using CatalystGroupConverter and it returns a GenericRow. + // Since GenericRow is not mutable, we just cast it to a Row. + val row = pair._2.asInstanceOf[Row] + var i = 0 + while (i < row.size) { + // TODO Avoids boxing cost here! + mutableRow(i) = row(i) + i += 1 + } + + i = 0 + while (i < requiredPartOrdinal.size) { + // TODO Avoids boxing cost here! + val partOrdinal = requiredPartOrdinal(i) + mutableRow.update(partitionKeyLocations(partOrdinal), partValues(partOrdinal)) + i += 1 + } + mutableRow } - row } } } else { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 6a9d9daf6750c..c8da8eea4e646 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -36,6 +36,20 @@ case class ParquetData(intField: Int, stringField: String) // The data that also includes the partitioning key case class ParquetDataWithKey(p: Int, intField: Int, stringField: String) +case class StructContainer(intStructField :Int, stringStructField: String) + +case class ParquetDataWithComplexTypes( + intField: Int, + stringField: String, + structField: StructContainer, + arrayField: Seq[Int]) + +case class ParquetDataWithKeyAndComplexTypes( + p: Int, + intField: Int, + stringField: String, + structField: StructContainer, + arrayField: Seq[Int]) /** * A suite to test the automatic conversion of metastore tables with parquet data to use the @@ -86,6 +100,38 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { location '${new File(normalTableDir, "normal").getCanonicalPath}' """) + sql(s""" + CREATE EXTERNAL TABLE partitioned_parquet_with_complextypes + ( + intField INT, + stringField STRING, + structField STRUCT, + arrayField ARRAY + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + LOCATION '${partitionedTableDirWithComplexTypes.getCanonicalPath}' + """) + + sql(s""" + CREATE EXTERNAL TABLE partitioned_parquet_with_key_and_complextypes + ( + intField INT, + stringField STRING, + structField STRUCT, + arrayField ARRAY + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + LOCATION '${partitionedTableDirWithKeyAndComplexTypes.getCanonicalPath}' + """) + (1 to 10).foreach { p => sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") } @@ -94,7 +140,15 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { sql(s"ALTER TABLE partitioned_parquet_with_key ADD PARTITION (p=$p)") } - val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet_with_key_and_complextypes ADD PARTITION (p=$p)") + } + + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet_with_complextypes ADD PARTITION (p=$p)") + } + + val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) jsonRDD(rdd1).registerTempTable("jt") val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":[$i, null]}""")) jsonRDD(rdd2).registerTempTable("jt_array") @@ -105,6 +159,8 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest { override def afterAll(): Unit = { sql("DROP TABLE partitioned_parquet") sql("DROP TABLE partitioned_parquet_with_key") + sql("DROP TABLE partitioned_parquet_with_complextypes") + sql("DROP TABLE partitioned_parquet_with_key_and_complextypes") sql("DROP TABLE normal_parquet") sql("DROP TABLE IF EXISTS jt") sql("DROP TABLE IF EXISTS jt_array") @@ -409,6 +465,22 @@ class ParquetSourceSuiteBase extends ParquetPartitioningTest { path '${new File(partitionedTableDir, "p=1").getCanonicalPath}' ) """) + + sql( s""" + CREATE TEMPORARY TABLE partitioned_parquet_with_key_and_complextypes + USING org.apache.spark.sql.parquet + OPTIONS ( + path '${partitionedTableDirWithKeyAndComplexTypes.getCanonicalPath}' + ) + """) + + sql( s""" + CREATE TEMPORARY TABLE partitioned_parquet_with_complextypes + USING org.apache.spark.sql.parquet + OPTIONS ( + path '${partitionedTableDirWithComplexTypes.getCanonicalPath}' + ) + """) } test("SPARK-6016 make sure to use the latest footers") { @@ -473,7 +545,8 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll var partitionedTableDir: File = null var normalTableDir: File = null var partitionedTableDirWithKey: File = null - + var partitionedTableDirWithComplexTypes: File = null + var partitionedTableDirWithKeyAndComplexTypes: File = null override def beforeAll(): Unit = { partitionedTableDir = File.createTempFile("parquettests", "sparksql") @@ -509,9 +582,45 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll .toDF() .saveAsParquetFile(partDir.getCanonicalPath) } + + partitionedTableDirWithKeyAndComplexTypes = File.createTempFile("parquettests", "sparksql") + partitionedTableDirWithKeyAndComplexTypes.delete() + partitionedTableDirWithKeyAndComplexTypes.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDirWithKeyAndComplexTypes, s"p=$p") + sparkContext.makeRDD(1 to 10).map { i => + ParquetDataWithKeyAndComplexTypes( + p, i, s"part-$p", StructContainer(i, f"${i}_string"), 1 to i) + }.toDF().saveAsParquetFile(partDir.getCanonicalPath) + } + + partitionedTableDirWithComplexTypes = File.createTempFile("parquettests", "sparksql") + partitionedTableDirWithComplexTypes.delete() + partitionedTableDirWithComplexTypes.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDirWithComplexTypes, s"p=$p") + sparkContext.makeRDD(1 to 10).map { i => + ParquetDataWithComplexTypes(i, s"part-$p", StructContainer(i, f"${i}_string"), 1 to i) + }.toDF().saveAsParquetFile(partDir.getCanonicalPath) + } + } + + override protected def afterAll(): Unit = { + partitionedTableDir.delete() + normalTableDir.delete() + partitionedTableDirWithKey.delete() + partitionedTableDirWithComplexTypes.delete() + partitionedTableDirWithKeyAndComplexTypes.delete() } - Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => + Seq( + "partitioned_parquet", + "partitioned_parquet_with_key", + "partitioned_parquet_with_complextypes", + "partitioned_parquet_with_key_and_complextypes").foreach { table => + test(s"ordering of the partitioning columns $table") { checkAnswer( sql(s"SELECT p, stringField FROM $table WHERE p = 1"), @@ -601,6 +710,25 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll } } + Seq( + "partitioned_parquet_with_key_and_complextypes", + "partitioned_parquet_with_complextypes").foreach { table => + + test(s"SPARK-5775 read struct from $table") { + checkAnswer( + sql(s"SELECT p, structField.intStructField, structField.stringStructField FROM $table WHERE p = 1"), + (1 to 10).map(i => Row(1, i, f"${i}_string"))) + } + + // Re-enable this after SPARK-5508 is fixed + ignore(s"SPARK-5775 read array from $table") { + checkAnswer( + sql(s"SELECT arrayField, p FROM $table WHERE p = 1"), + (1 to 10).map(i => Row(1 to i, 1))) + } + } + + test("non-part select(*)") { checkAnswer( sql("SELECT COUNT(*) FROM normal_parquet"), From b36b1bc22ea73669b0f69ed21e77d47fb0a7cd5d Mon Sep 17 00:00:00 2001 From: Michael Griffiths Date: Sat, 28 Feb 2015 14:47:39 +0000 Subject: [PATCH 296/817] SPARK-6063 MLlib doesn't pass mvn scalastyle check due to UTF chars in LDAModel.scala Remove unicode characters from MLlib file. Author: Michael Griffiths Author: Griffiths, Michael (NYC-RPM) Closes #4815 from msjgriffiths/SPARK-6063 and squashes the following commits: bcd7de1 [Griffiths, Michael (NYC-RPM)] Change \u201D quote marks around 'theta' to standard single apostrophe (\x27) 38eb535 [Michael Griffiths] Merge pull request #2 from apache/master b08e865 [Michael Griffiths] Merge pull request #1 from apache/master --- .../main/scala/org/apache/spark/mllib/clustering/LDAModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index b0e991d2f2344..0a3f21ecee0dc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -130,7 +130,7 @@ abstract class LDAModel private[clustering] { /* TODO * Compute the estimated topic distribution for each document. - * This is often called “theta” in the literature. + * This is often called 'theta' in the literature. * * @param documents RDD of documents, which are term (word) count vectors paired with IDs. * The term count vectors are "bags of words" with a fixed-size vocabulary From f91298e2c597e45af461931919372da5d33ae3da Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 28 Feb 2015 15:23:59 +0000 Subject: [PATCH 297/817] SPARK-5983 [WEBUI] Don't respond to HTTP TRACE in HTTP-based UIs Disallow TRACE HTTP method in servlets Author: Sean Owen Closes #4765 from srowen/SPARK-5983 and squashes the following commits: 421b25b [Sean Owen] Disallow TRACE HTTP method in servlets --- .../org/apache/spark/deploy/history/HistoryServer.scala | 4 ++++ core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 8 ++++++++ 2 files changed, 12 insertions(+) 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 fa9bfe5426b6c..af483d560b33e 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 @@ -96,6 +96,10 @@ class HistoryServer( } } } + // SPARK-5983 ensure TRACE is not supported + protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } } initialize() diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index bf4b24e98b134..95f254a9ef22a 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -80,6 +80,10 @@ private[spark] object JettyUtils extends Logging { response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage) } } + // SPARK-5983 ensure TRACE is not supported + protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } } } @@ -119,6 +123,10 @@ private[spark] object JettyUtils extends Logging { val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString response.sendRedirect(newUrl) } + // SPARK-5983 ensure TRACE is not supported + protected override def doTrace(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } } createServletHandler(srcPath, servlet, basePath) } From 86fcdaef62dbe624233e364ffe43fe3a1da893f0 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 28 Feb 2015 15:34:08 +0000 Subject: [PATCH 298/817] SPARK-1965 [WEBUI] Spark UI throws NPE on trying to load the app page for non-existent app Don't throw NPE if appId is unknown. kayousterhout is this a decent enough band-aid for avoiding a full-blown NPE? it should just render empty content instead Author: Sean Owen Closes #4777 from srowen/SPARK-1965 and squashes the following commits: 7e16590 [Sean Owen] Update app not found message cb878d6 [Sean Owen] Return basic "not found" page for unknown appId d8270da [Sean Owen] Don't throw NPE if appId is unknown --- .../spark/deploy/master/ui/ApplicationPage.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 3aae2b95d7396..76fc40e17d9a8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -24,6 +24,7 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue +import org.json4s.JsonAST.JNothing import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} @@ -44,7 +45,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) - JsonProtocol.writeApplicationInfo(app) + if (app == null) { + JNothing + } else { + JsonProtocol.writeApplicationInfo(app) + } } /** Executor details for a particular application */ @@ -55,6 +60,10 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) + if (app == null) { + val msg =
    No running application with ID {appId}
    + return UIUtils.basicSparkPage(msg, "Not Found") + } val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") val allExecutors = (app.executors.values ++ app.removedExecutors).toSet.toSeq From 643300a6e27dac3822f9a3ced0ad5fb3b4f2ad75 Mon Sep 17 00:00:00 2001 From: Evan Yu Date: Sat, 28 Feb 2015 18:55:34 -0800 Subject: [PATCH 299/817] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException Fix TimSort bug which causes a ArrayOutOfBoundsException. Using the proposed fix here http://envisage-project.eu/proving-android-java-and-python-sorting-algorithm-is-broken-and-how-to-fix-it/ Author: Evan Yu Closes #4804 from hotou/SPARK-5984 and squashes the following commits: 3421b6c [Evan Yu] SPARK-5984: Add info to LICENSE e61c6b8 [Evan Yu] SPARK-5984: Fix license and document 6ccc280 [Evan Yu] SPARK-5984: Add License header to file e06c0d2 [Evan Yu] SPARK-5984: Add License header to file 4d95f75 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException 479a106 [Evan Yu] SPARK-5984: Fix TimSort bug causes ArrayOutOfBoundsException --- LICENSE | 16 +++ .../apache/spark/util/collection/TimSort.java | 9 +- .../spark/util/collection/TestTimSort.java | 134 ++++++++++++++++++ .../spark/util/collection/SorterSuite.scala | 7 + 4 files changed, 161 insertions(+), 5 deletions(-) create mode 100644 core/src/test/java/org/apache/spark/util/collection/TestTimSort.java diff --git a/LICENSE b/LICENSE index 0a42d389e4c3c..9b364a4d00079 100644 --- a/LICENSE +++ b/LICENSE @@ -771,6 +771,22 @@ 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. +======================================================================== +For TestTimSort (core/src/test/java/org/apache/spark/util/collection/TestTimSort.java): +======================================================================== +Copyright (C) 2015 Stijn de Gouw + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. ======================================================================== For LimitedInputStream diff --git a/core/src/main/java/org/apache/spark/util/collection/TimSort.java b/core/src/main/java/org/apache/spark/util/collection/TimSort.java index 409e1a41c5d49..a90cc0e761f62 100644 --- a/core/src/main/java/org/apache/spark/util/collection/TimSort.java +++ b/core/src/main/java/org/apache/spark/util/collection/TimSort.java @@ -425,15 +425,14 @@ private void pushRun(int runBase, int runLen) { private void mergeCollapse() { while (stackSize > 1) { int n = stackSize - 2; - if (n > 0 && runLen[n-1] <= runLen[n] + runLen[n+1]) { + if ( (n >= 1 && runLen[n-1] <= runLen[n] + runLen[n+1]) + || (n >= 2 && runLen[n-2] <= runLen[n] + runLen[n-1])) { if (runLen[n - 1] < runLen[n + 1]) n--; - mergeAt(n); - } else if (runLen[n] <= runLen[n + 1]) { - mergeAt(n); - } else { + } else if (runLen[n] > runLen[n + 1]) { break; // Invariant is established } + mergeAt(n); } } diff --git a/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java new file mode 100644 index 0000000000000..45772b6d3c20d --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java @@ -0,0 +1,134 @@ +/** + * Copyright 2015 Stijn de Gouw + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.spark.util.collection; + +import java.util.*; + +/** + * This codes generates a int array which fails the standard TimSort. + * + * The blog that reported the bug + * http://www.envisage-project.eu/timsort-specification-and-verification/ + * + * This codes was originally wrote by Stijn de Gouw, modified by Evan Yu to adapt to + * our test suite. + * + * https://github.com/abstools/java-timsort-bug + * https://github.com/abstools/java-timsort-bug/blob/master/LICENSE + */ +public class TestTimSort { + + private static final int MIN_MERGE = 32; + + /** + * Returns an array of integers that demonstrate the bug in TimSort + */ + public static int[] getTimSortBugTestSet(int length) { + int minRun = minRunLength(length); + List runs = runsJDKWorstCase(minRun, length); + return createArray(runs, length); + } + + private static int minRunLength(int n) { + int r = 0; // Becomes 1 if any 1 bits are shifted off + while (n >= MIN_MERGE) { + r |= (n & 1); + n >>= 1; + } + return n + r; + } + + private static int[] createArray(List runs, int length) { + int[] a = new int[length]; + Arrays.fill(a, 0); + int endRun = -1; + for (long len : runs) { + a[endRun += len] = 1; + } + a[length - 1] = 0; + return a; + } + + /** + * Fills runs with a sequence of run lengths of the form
    + * Y_n x_{n,1} x_{n,2} ... x_{n,l_n}
    + * Y_{n-1} x_{n-1,1} x_{n-1,2} ... x_{n-1,l_{n-1}}
    + * ...
    + * Y_1 x_{1,1} x_{1,2} ... x_{1,l_1}
    + * The Y_i's are chosen to satisfy the invariant throughout execution, + * but the x_{i,j}'s are merged (by TimSort.mergeCollapse) + * into an X_i that violates the invariant. + * + * @param length The sum of all run lengths that will be added to runs. + */ + private static List runsJDKWorstCase(int minRun, int length) { + List runs = new ArrayList(); + + long runningTotal = 0, Y = minRun + 4, X = minRun; + + while (runningTotal + Y + X <= length) { + runningTotal += X + Y; + generateJDKWrongElem(runs, minRun, X); + runs.add(0, Y); + // X_{i+1} = Y_i + x_{i,1} + 1, since runs.get(1) = x_{i,1} + X = Y + runs.get(1) + 1; + // Y_{i+1} = X_{i+1} + Y_i + 1 + Y += X + 1; + } + + if (runningTotal + X <= length) { + runningTotal += X; + generateJDKWrongElem(runs, minRun, X); + } + + runs.add(length - runningTotal); + return runs; + } + + /** + * Adds a sequence x_1, ..., x_n of run lengths to runs such that:
    + * 1. X = x_1 + ... + x_n
    + * 2. x_j >= minRun for all j
    + * 3. x_1 + ... + x_{j-2} < x_j < x_1 + ... + x_{j-1} for all j
    + * These conditions guarantee that TimSort merges all x_j's one by one + * (resulting in X) using only merges on the second-to-last element. + * + * @param X The sum of the sequence that should be added to runs. + */ + private static void generateJDKWrongElem(List runs, int minRun, long X) { + for (long newTotal; X >= 2 * minRun + 1; X = newTotal) { + //Default strategy + newTotal = X / 2 + 1; + //Specialized strategies + if (3 * minRun + 3 <= X && X <= 4 * minRun + 1) { + // add x_1=MIN+1, x_2=MIN, x_3=X-newTotal to runs + newTotal = 2 * minRun + 1; + } else if (5 * minRun + 5 <= X && X <= 6 * minRun + 5) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=X-newTotal to runs + newTotal = 3 * minRun + 3; + } else if (8 * minRun + 9 <= X && X <= 10 * minRun + 9) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=X-newTotal to runs + newTotal = 5 * minRun + 5; + } else if (13 * minRun + 15 <= X && X <= 16 * minRun + 17) { + // add x_1=MIN+1, x_2=MIN, x_3=MIN+2, x_4=2MIN+2, x_5=3MIN+4, x_6=X-newTotal to runs + newTotal = 8 * minRun + 9; + } + runs.add(0, X - newTotal); + } + runs.add(0, X); + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 0cb1ed7397655..e0d6cc16bde05 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -65,6 +65,13 @@ class SorterSuite extends FunSuite { } } + // http://www.envisage-project.eu/timsort-specification-and-verification/ + test("SPARK-5984 TimSort bug") { + val data = TestTimSort.getTimSortBugTestSet(67108864) + new Sorter(new IntArraySortDataFormat).sort(data, 0, data.length, Ordering.Int) + (0 to data.length - 2).foreach(i => assert(data(i) <= data(i + 1))) + } + /** Runs an experiment several times. */ def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = { if (skip) { From 2df5f1f00661cd31b9fc37e80345a3f5f856c95f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 28 Feb 2015 22:51:01 -0800 Subject: [PATCH 300/817] [SPARK-6075] Fix bug in that caused lost accumulator updates: do not store WeakReferences in localAccums map This fixes a non-deterministic bug introduced in #4021 that could cause tasks' accumulator updates to be lost. The problem is that `localAccums` should not hold weak references: after the task finishes running there won't be any strong references to these local accumulators, so they can get garbage-collected before the executor reads the `localAccums` map. We don't need weak references here anyways, since this map is cleared at the end of each task. Author: Josh Rosen Closes #4835 from JoshRosen/SPARK-6075 and squashes the following commits: 4f4b5b2 [Josh Rosen] Remove defensive assertions that caused test failures in code unrelated to this change 120c7b0 [Josh Rosen] [SPARK-6075] Do not store WeakReferences in localAccums map --- .../scala/org/apache/spark/Accumulators.scala | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 30f0ccd73ccca..bcf832467f00b 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -280,15 +280,24 @@ object AccumulatorParam { // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right -private[spark] object Accumulators { - // Store a WeakReference instead of a StrongReference because this way accumulators can be - // appropriately garbage collected during long-running jobs and release memory - type WeakAcc = WeakReference[Accumulable[_, _]] - val originals = Map[Long, WeakAcc]() - val localAccums = new ThreadLocal[Map[Long, WeakAcc]]() { - override protected def initialValue() = Map[Long, WeakAcc]() +private[spark] object Accumulators extends Logging { + /** + * This global map holds the original accumulator objects that are created on the driver. + * It keeps weak references to these objects so that accumulators can be garbage-collected + * once the RDDs and user-code that reference them are cleaned up. + */ + val originals = Map[Long, WeakReference[Accumulable[_, _]]]() + + /** + * This thread-local map holds per-task copies of accumulators; it is used to collect the set + * of accumulator updates to send back to the driver when tasks complete. After tasks complete, + * this map is cleared by `Accumulators.clear()` (see Executor.scala). + */ + private val localAccums = new ThreadLocal[Map[Long, Accumulable[_, _]]]() { + override protected def initialValue() = Map[Long, Accumulable[_, _]]() } - var lastId: Long = 0 + + private var lastId: Long = 0 def newId(): Long = synchronized { lastId += 1 @@ -297,16 +306,16 @@ private[spark] object Accumulators { def register(a: Accumulable[_, _], original: Boolean): Unit = synchronized { if (original) { - originals(a.id) = new WeakAcc(a) + originals(a.id) = new WeakReference[Accumulable[_, _]](a) } else { - localAccums.get()(a.id) = new WeakAcc(a) + localAccums.get()(a.id) = a } } // Clear the local (non-original) accumulators for the current thread def clear() { synchronized { - localAccums.get.clear + localAccums.get.clear() } } @@ -320,12 +329,7 @@ private[spark] object Accumulators { def values: Map[Long, Any] = synchronized { val ret = Map[Long, Any]() for ((id, accum) <- localAccums.get) { - // Since we are now storing weak references, we must check whether the underlying data - // is valid. - ret(id) = accum.get match { - case Some(values) => values.localValue - case None => None - } + ret(id) = accum.localValue } return ret } @@ -341,6 +345,8 @@ private[spark] object Accumulators { case None => throw new IllegalAccessError("Attempted to access garbage collected Accumulator.") } + } else { + logWarning(s"Ignoring accumulator update for unknown accumulator id $id") } } } From fd8d283eeb98e310b1e85ef8c3a8af9e547ab5e0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sun, 1 Mar 2015 11:05:10 +0000 Subject: [PATCH 301/817] [SPARK-6074] [sql] Package pyspark sql bindings. This is needed for the SQL bindings to work on Yarn. Author: Marcelo Vanzin Closes #4822 from vanzin/SPARK-6074 and squashes the following commits: fb52001 [Marcelo Vanzin] [SPARK-6074] [sql] Package pyspark sql bindings. --- sql/core/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 03a5c9e7c24a0..e28baa512b95c 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -109,5 +109,13 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + ../../python + + pyspark/sql/*.py + + + From aedbbaa3dda9cbc154cd52c07f6d296b972b0eb2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 1 Mar 2015 16:26:57 -0800 Subject: [PATCH 302/817] [SPARK-6053][MLLIB] support save/load in PySpark's ALS A simple wrapper to save/load `MatrixFactorizationModel` in Python. jkbradley Author: Xiangrui Meng Closes #4811 from mengxr/SPARK-5991 and squashes the following commits: f135dac [Xiangrui Meng] update save doc 57e5200 [Xiangrui Meng] address comments 06140a4 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5991 282ec8d [Xiangrui Meng] support save/load in PySpark's ALS --- docs/mllib-collaborative-filtering.md | 8 ++- .../spark/mllib/util/modelSaveLoad.scala | 2 +- python/pyspark/mllib/recommendation.py | 20 ++++++- python/pyspark/mllib/util.py | 58 +++++++++++++++++++ 4 files changed, 82 insertions(+), 6 deletions(-) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 27aa4d38b7617..76140282a2dd0 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -200,10 +200,8 @@ In the following example we load rating data. Each row consists of a user, a pro We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation by measuring the Mean Squared Error of rating prediction. -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.recommendation import ALS, Rating +from pyspark.mllib.recommendation import ALS, MatrixFactorizationModel, Rating # Load and parse the data data = sc.textFile("data/mllib/als/test.data") @@ -220,6 +218,10 @@ predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2])) ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions) MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).reduce(lambda x, y: x + y) / ratesAndPreds.count() print("Mean Squared Error = " + str(MSE)) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = MatrixFactorizationModel.load(sc, "myModelPath") {% endhighlight %} If the rating matrix is derived from other source of information (i.e., it is inferred from other diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index 4458340497f0b..526d055c87387 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -48,7 +48,7 @@ trait Saveable { * * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. - * This directory and any intermediate directory will be created if needed. + * If the directory already exists, this method throws an exception. */ def save(sc: SparkContext, path: String): Unit diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 0d99e6dedfad9..03d7d011474cb 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -19,7 +19,8 @@ from pyspark import SparkContext from pyspark.rdd import RDD -from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc +from pyspark.mllib.util import Saveable, JavaLoader __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -39,7 +40,8 @@ def __reduce__(self): return Rating, (int(self.user), int(self.product), float(self.rating)) -class MatrixFactorizationModel(JavaModelWrapper): +@inherit_doc +class MatrixFactorizationModel(JavaModelWrapper, Saveable, JavaLoader): """A matrix factorisation model trained by regularized alternating least-squares. @@ -81,6 +83,17 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) >>> model.predict(2,2) 0.43... + + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> model.save(sc, path) + >>> sameModel = MatrixFactorizationModel.load(sc, path) + >>> sameModel.predict(2,2) + 0.43... + >>> try: + ... os.removedirs(path) + ... except: + ... pass """ def predict(self, user, product): return self._java_model.predict(int(user), int(product)) @@ -98,6 +111,9 @@ def userFeatures(self): def productFeatures(self): return self.call("getProductFeatures") + def save(self, sc, path): + self.call("save", sc._jsc.sc(), path) + class ALS(object): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 4ed978b45409c..17d43eadba12b 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -168,6 +168,64 @@ def loadLabeledPoints(sc, path, minPartitions=None): return callMLlibFunc("loadLabeledPoints", sc, path, minPartitions) +class Saveable(object): + """ + Mixin for models and transformers which may be saved as files. + """ + + def save(self, sc, path): + """ + Save this model to the given path. + + This saves: + * human-readable (JSON) model metadata to path/metadata/ + * Parquet formatted data to path/data/ + + The model may be loaded using py:meth:`Loader.load`. + + :param sc: Spark context used to save model data. + :param path: Path specifying the directory in which to save + this model. If the directory already exists, + this method throws an exception. + """ + raise NotImplementedError + + +class Loader(object): + """ + Mixin for classes which can load saved models from files. + """ + + @classmethod + def load(cls, sc, path): + """ + Load a model from the given path. The model should have been + saved using py:meth:`Saveable.save`. + + :param sc: Spark context used for loading model files. + :param path: Path specifying the directory to which the model + was saved. + :return: model instance + """ + raise NotImplemented + + +class JavaLoader(Loader): + """ + Mixin for classes which can load saved models using its Scala + implementation. + """ + + @classmethod + def load(cls, sc, path): + java_package = cls.__module__.replace("pyspark", "org.apache.spark") + java_class = ".".join([java_package, cls.__name__]) + java_obj = sc._jvm + for name in java_class.split("."): + java_obj = getattr(java_obj, name) + return cls(java_obj.load(sc._jsc.sc(), path)) + + def _test(): import doctest from pyspark.context import SparkContext From 3f00bb3ef1384fabf86a68180d40a1a515f6f5e3 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Sun, 1 Mar 2015 16:28:15 -0800 Subject: [PATCH 303/817] [SPARK-6083] [MLLib] [DOC] Make Python API example consistent in NaiveBayes Author: MechCoder Closes #4834 from MechCoder/spark-6083 and squashes the following commits: 1cdd7b5 [MechCoder] Add parse function 65bbbe9 [MechCoder] [SPARK-6083] Make Python API example consistent in NaiveBayes --- docs/mllib-naive-bayes.md | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 5224a0b49a991..55b8f2ce6c364 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -115,22 +115,28 @@ used for evaluation and prediction. Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import NaiveBayes +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.regression import LabeledPoint + +def parseLine(line): + parts = line.split(',') + label = float(parts[0]) + features = Vectors.dense([float(x) for x in parts[1].split(' ')]) + return LabeledPoint(label, features) + +data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine) -# an RDD of LabeledPoint -data = sc.parallelize([ - LabeledPoint(0.0, [0.0, 0.0]) - ... # more labeled points -]) +# Split data aproximately into training (60%) and test (40%) +training, test = data.randomSplit([0.6, 0.4], seed = 0) # Train a naive Bayes model. -model = NaiveBayes.train(data, 1.0) +model = NaiveBayes.train(training, 1.0) -# Make prediction. -prediction = model.predict([0.0, 0.0]) +# Make prediction and test accuracy. +predictionAndLabel = test.map(lambda p : (model.predict(p.features), p.label)) +accuracy = 1.0 * predictionAndLabel.filter(lambda (x, v): x == v).count() / test.count() {% endhighlight %} From d8fb40edea7c8c811814f1ff288d59178928964b Mon Sep 17 00:00:00 2001 From: Saisai Shao Date: Mon, 2 Mar 2015 08:49:19 +0000 Subject: [PATCH 304/817] [Streaming][Minor]Fix some error docs in streaming examples Small changes, please help to review, thanks a lot. Author: Saisai Shao Closes #4837 from jerryshao/doc-fix and squashes the following commits: 545291a [Saisai Shao] Fix some error docs in streaming examples --- .../apache/spark/examples/streaming/DirectKafkaWordCount.scala | 3 ++- examples/src/main/python/streaming/kafka_wordcount.py | 2 +- python/pyspark/streaming/kafka.py | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index deb08fd57b8c7..1c8a20bf8f1ae 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -30,7 +30,8 @@ import org.apache.spark.SparkConf * is a list of one or more kafka topics to consume from * * Example: - * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 + * $ bin/run-example streaming.DirectKafkaWordCount broker1-host:port,broker2-host:port \ + * topic1,topic2 */ object DirectKafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index ed398a82b8bb0..f82f161cb8844 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -23,7 +23,7 @@ http://kafka.apache.org/documentation.html#quickstart and then run the example - `$ bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/\ + `$ bin/spark-submit --jars external/kafka-assembly/target/scala-*/\ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 0002dc10e8a17..f083ed149effb 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -82,7 +82,7 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, Group Id = org.apache.spark, Artifact Id = spark-streaming-kafka-assembly, Version = %s. - Then, innclude the jar in the spark-submit command as + Then, include the jar in the spark-submit command as $ bin/spark-submit --jars ... From 948c2390ab004ad5cf3876d87c05d3e43a9aa9e0 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 2 Mar 2015 08:51:03 +0000 Subject: [PATCH 305/817] SPARK-3357 [CORE] Internal log messages should be set at DEBUG level instead of INFO Demote some 'noisy' log messages to debug level. I added a few more, to include everything that gets logged in stanzas like this: ``` 15/03/01 00:03:54 INFO BlockManager: Removing broadcast 0 15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0_piece0 15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0_piece0 of size 839 dropped from memory (free 277976091) 15/03/01 00:03:54 INFO BlockManagerInfo: Removed broadcast_0_piece0 on localhost:49524 in memory (size: 839.0 B, free: 265.1 MB) 15/03/01 00:03:54 INFO BlockManagerMaster: Updated info of block broadcast_0_piece0 15/03/01 00:03:54 INFO BlockManager: Removing block broadcast_0 15/03/01 00:03:54 INFO MemoryStore: Block broadcast_0 of size 1088 dropped from memory (free 277977179) 15/03/01 00:03:54 INFO ContextCleaner: Cleaned broadcast 0 ``` as well as regular messages like ``` 15/03/01 00:02:33 INFO MemoryStore: ensureFreeSpace(2640) called with curMem=47322, maxMem=278019440 ``` WDYT? good or should some be left alone? CC mengxr who suggested some of this. Author: Sean Owen Closes #4838 from srowen/SPARK-3357 and squashes the following commits: dce75c1 [Sean Owen] Back out some debug level changes d9b784d [Sean Owen] Demote some 'noisy' log messages to debug level --- core/src/main/scala/org/apache/spark/ContextCleaner.scala | 4 ++-- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- .../scala/org/apache/spark/storage/BlockManagerMaster.scala | 2 +- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 434f1e47cf822..4a9d007353373 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -188,10 +188,10 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** Perform broadcast cleanup. */ def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { try { - logDebug("Cleaning broadcast " + broadcastId) + logDebug(s"Cleaning broadcast $broadcastId") broadcastManager.unbroadcast(broadcastId, true, blocking) listeners.foreach(_.broadcastCleaned(broadcastId)) - logInfo("Cleaned broadcast " + broadcastId) + logDebug(s"Cleaned broadcast $broadcastId") } catch { case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 86dbd89f0ffb8..c8b7763f03fb7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1074,7 +1074,7 @@ private[spark] class BlockManager( * Remove all blocks belonging to the given broadcast. */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { - logInfo(s"Removing broadcast $broadcastId") + logDebug(s"Removing broadcast $broadcastId") val blocksToRemove = blockInfo.keys.collect { case bid @ BroadcastBlockId(`broadcastId`, _) => bid } @@ -1086,7 +1086,7 @@ private[spark] class BlockManager( * Remove a block from both memory and disk. */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { - logInfo(s"Removing block $blockId") + logDebug(s"Removing block $blockId") val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index b63c7f191155c..654796f23c96e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -61,7 +61,7 @@ class BlockManagerMaster( tachyonSize: Long): Boolean = { val res = askDriverWithReply[Boolean]( UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) - logInfo("Updated info of block " + blockId) + logDebug(s"Updated info of block $blockId") res } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 71305a46bf570..7f4b6e8bd3683 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -184,7 +184,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val entry = entries.remove(blockId) if (entry != null) { currentMemory -= entry.size - logInfo(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)") + logDebug(s"Block $blockId of size ${entry.size} dropped from memory (free $freeMemory)") true } else { false From 49c7a8f6f33d64d7e6c35391f83121440844a41d Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Mon, 2 Mar 2015 09:06:56 +0000 Subject: [PATCH 306/817] [SPARK-6103][Graphx]remove unused class to import in EdgeRDDImpl Class TaskContext is unused in EdgeRDDImpl, so we need to remove it from import list. Author: Lianhui Wang Closes #4846 from lianhuiwang/SPARK-6103 and squashes the following commits: 31aed64 [Lianhui Wang] remove unused class to import in EdgeRDDImpl --- .../main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 56cb41661e300..43a3aea0f6196 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} -import org.apache.spark.{OneToOneDependency, HashPartitioner, TaskContext} +import org.apache.spark.{OneToOneDependency, HashPartitioner} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel From 39a54b40aff66816f8b8f5c6133eaaad6eaecae1 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 2 Mar 2015 22:42:18 +0800 Subject: [PATCH 307/817] [SPARK-6073][SQL] Need to refresh metastore cache after append data in CreateMetastoreDataSourceAsSelect JIRA: https://issues.apache.org/jira/browse/SPARK-6073 liancheng Author: Yin Huai Closes #4824 from yhuai/refreshCache and squashes the following commits: b9542ef [Yin Huai] Refresh metadata cache in the Catalog in CreateMetastoreDataSourceAsSelect. --- .../spark/sql/hive/execution/commands.scala | 2 + .../sql/hive/MetastoreDataSourcesSuite.scala | 52 +++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 9934a5d3c30a2..ffaef8eef1c74 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -248,6 +248,8 @@ case class CreateMetastoreDataSourceAsSelect( isExternal) } + // Refresh the cache of the table in the catalog. + hiveContext.refreshTable(tableName) Seq.empty[Row] } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 00306f1cd7f86..868c35f35ff59 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -612,4 +612,56 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { val actualSchema = table("wide_schema").schema assert(schema === actualSchema) } + + test("insert into a table") { + def createDF(from: Int, to: Int): DataFrame = + createDataFrame((from to to).map(i => Tuple2(i, s"str$i"))).toDF("c1", "c2") + + createDF(0, 9).saveAsTable("insertParquet", "parquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM insertParquet p WHERE p.c1 > 5"), + (6 to 9).map(i => Row(i, s"str$i"))) + + intercept[AnalysisException] { + createDF(10, 19).saveAsTable("insertParquet", "parquet") + } + + createDF(10, 19).saveAsTable("insertParquet", "parquet", SaveMode.Append) + checkAnswer( + sql("SELECT p.c1, p.c2 FROM insertParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + + createDF(20, 29).saveAsTable("insertParquet", "parquet", SaveMode.Append) + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p WHERE p.c1 > 5 AND p.c1 < 25"), + (6 to 24).map(i => Row(i, s"str$i"))) + + intercept[AnalysisException] { + createDF(30, 39).saveAsTable("insertParquet") + } + + createDF(30, 39).saveAsTable("insertParquet", SaveMode.Append) + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p WHERE p.c1 > 5 AND p.c1 < 35"), + (6 to 34).map(i => Row(i, s"str$i"))) + + createDF(40, 49).insertInto("insertParquet") + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p WHERE p.c1 > 5 AND p.c1 < 45"), + (6 to 44).map(i => Row(i, s"str$i"))) + + createDF(50, 59).saveAsTable("insertParquet", SaveMode.Overwrite) + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p WHERE p.c1 > 51 AND p.c1 < 55"), + (52 to 54).map(i => Row(i, s"str$i"))) + createDF(60, 69).saveAsTable("insertParquet", SaveMode.Ignore) + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p"), + (50 to 59).map(i => Row(i, s"str$i"))) + + createDF(70, 79).insertInto("insertParquet", overwrite = true) + checkAnswer( + sql("SELECT p.c1, c2 FROM insertParquet p"), + (70 to 79).map(i => Row(i, s"str$i"))) + } } From 3efd8bb6cf139ce094ff631c7a9c1eb93fdcd566 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 2 Mar 2015 23:18:07 +0800 Subject: [PATCH 308/817] [SPARK-6052][SQL]In JSON schema inference, we should always set containsNull of an ArrayType to true Always set `containsNull = true` when infer the schema of JSON datasets. If we set `containsNull` based on records we scanned, we may miss arrays with null values when we do sampling. Also, because future data can have arrays with null values, if we convert JSON data to parquet, always setting `containsNull = true` is a more robust way to go. JIRA: https://issues.apache.org/jira/browse/SPARK-6052 Author: Yin Huai Closes #4806 from yhuai/jsonArrayContainsNull and squashes the following commits: 05eab9d [Yin Huai] Change containsNull to true. --- .../org/apache/spark/sql/json/JsonRDD.scala | 9 ++--- .../org/apache/spark/sql/json/JsonSuite.scala | 38 +++++++++---------- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index d83bdc2f7ff9a..e54a2a3679272 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -199,13 +199,12 @@ private[sql] object JsonRDD extends Logging { * type conflicts. */ private def typeOfArray(l: Seq[Any]): ArrayType = { - val containsNull = l.exists(v => v == null) val elements = l.flatMap(v => Option(v)) if (elements.isEmpty) { // If this JSON array is empty, we use NullType as a placeholder. // If this array is not empty in other JSON objects, we can resolve // the type after we have passed through all JSON objects. - ArrayType(NullType, containsNull) + ArrayType(NullType, containsNull = true) } else { val elementType = elements.map { e => e match { @@ -217,7 +216,7 @@ private[sql] object JsonRDD extends Logging { } }.reduce((type1: DataType, type2: DataType) => compatibleType(type1, type2)) - ArrayType(elementType, containsNull) + ArrayType(elementType, containsNull = true) } } @@ -245,7 +244,7 @@ private[sql] object JsonRDD extends Logging { // The value associated with the key is an array. // Handle inner structs of an array. def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { - case ArrayType(e: StructType, containsNull) => { + case ArrayType(e: StructType, _) => { // The elements of this arrays are structs. v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap { element => allKeysWithValueTypes(element) @@ -253,7 +252,7 @@ private[sql] object JsonRDD extends Logging { case (k, t) => (s"$key.$k", t) } } - case ArrayType(t1, containsNull) => + case ArrayType(t1, _) => v.asInstanceOf[Seq[Any]].flatMap(Option(_)).flatMap { element => buildKeyPathForInnerStructs(element, t1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 005f20b96df79..9d94d3406acfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -248,26 +248,26 @@ class JsonSuite extends QueryTest { val jsonDF = jsonRDD(complexFieldAndType1) val expectedSchema = StructType( - StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: - StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: - StructField("arrayOfBigInteger", ArrayType(DecimalType.Unlimited, false), true) :: - StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: - StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: - StructField("arrayOfInteger", ArrayType(LongType, false), true) :: - StructField("arrayOfLong", ArrayType(LongType, false), true) :: + StructField("arrayOfArray1", ArrayType(ArrayType(StringType, true), true), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, true), true), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType.Unlimited, true), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, true), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, true), true) :: + StructField("arrayOfInteger", ArrayType(LongType, true), true) :: + StructField("arrayOfLong", ArrayType(LongType, true), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: - StructField("arrayOfString", ArrayType(StringType, false), true) :: + StructField("arrayOfString", ArrayType(StringType, true), true) :: StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: StructField("field2", StringType, true) :: - StructField("field3", StringType, true) :: Nil), false), true) :: + StructField("field3", StringType, true) :: Nil), true), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: StructField("field2", DecimalType.Unlimited, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( - StructField("field1", ArrayType(LongType, false), true) :: - StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) + StructField("field1", ArrayType(LongType, true), true) :: + StructField("field2", ArrayType(StringType, true), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonDF.schema) @@ -487,7 +487,7 @@ class JsonSuite extends QueryTest { val jsonDF = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(LongType, false), true) :: + StructField("array", ArrayType(LongType, true), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -513,8 +513,8 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil), false), true) :: - StructField("array3", ArrayType(StringType, false), true) :: Nil) + StructField("field", LongType, true) :: Nil), true), true) :: + StructField("array3", ArrayType(StringType, true), true) :: Nil) assert(expectedSchema === jsonDF.schema) @@ -541,7 +541,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(LongType, false), true) :: + StructField("c", ArrayType(LongType, true), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: Nil) @@ -835,15 +835,15 @@ class JsonSuite extends QueryTest { val schema = StructType( StructField("field1", - ArrayType(ArrayType(ArrayType(ArrayType(StringType, false), false), true), false), true) :: + ArrayType(ArrayType(ArrayType(ArrayType(StringType, true), true), true), true), true) :: StructField("field2", ArrayType(ArrayType( - StructType(StructField("Test", LongType, true) :: Nil), false), true), true) :: + StructType(StructField("Test", LongType, true) :: Nil), true), true), true) :: StructField("field3", ArrayType(ArrayType( - StructType(StructField("Test", StringType, true) :: Nil), true), false), true) :: + StructType(StructField("Test", StringType, true) :: Nil), true), true), true) :: StructField("field4", - ArrayType(ArrayType(ArrayType(LongType, false), true), false), true) :: Nil) + ArrayType(ArrayType(ArrayType(LongType, true), true), true), true) :: Nil) assert(schema === jsonDF.schema) From 95ac68bf127b5370c13d6bc15adbda78228829cc Mon Sep 17 00:00:00 2001 From: Kenneth Myers Date: Mon, 2 Mar 2015 17:25:24 +0000 Subject: [PATCH 309/817] [SPARK-6111] Fixed usage string in documentation. Usage info in documentation does not match actual usage info. Doc string usage says ```Usage: network_wordcount.py ``` whereas the actual usage is ```Usage: kafka_wordcount.py ``` Author: Kenneth Myers Closes #4852 from kennethmyers/kafka_wordcount_documentation_fix and squashes the following commits: 3855325 [Kenneth Myers] Fixed usage string in documentation. --- examples/src/main/python/streaming/kafka_wordcount.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index f82f161cb8844..51e1ff822fc55 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -17,7 +17,7 @@ """ Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - Usage: network_wordcount.py + Usage: kafka_wordcount.py To run this on your local machine, you need to setup Kafka and create a producer first, see http://kafka.apache.org/documentation.html#quickstart From 9ce12aaf283a2793e719bdc956dd858922636e8d Mon Sep 17 00:00:00 2001 From: q00251598 Date: Mon, 2 Mar 2015 10:13:11 -0800 Subject: [PATCH 310/817] [SPARK-5741][SQL] Support the path contains comma in HiveContext When run ```select * from nzhang_part where hr = 'file,';```, it throws exception ```java.lang.IllegalArgumentException: Can not create a Path from an empty string``` . Because the path of hdfs contains comma, and FileInputFormat.setInputPaths will split path by comma. ### SQL ``` set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; create table nzhang_part like srcpart; insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08'; insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08'; insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select * from ( select key, value, hr from srcpart where ds='2008-04-08' union all select '1' as key, '1' as value, 'file,' as hr from src limit 1) s; select * from nzhang_part where hr = 'file,'; ``` ### Error Log ``` 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part where hr = 'file,'] java.lang.IllegalArgumentException: Can not create a Path from an empty string at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127) at org.apache.hadoop.fs.Path.(Path.java:135) at org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241) at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400) at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251) at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229) at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172) at scala.Option.map(Option.scala:145) at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196) Author: q00251598 Closes #4532 from watermen/SPARK-5741 and squashes the following commits: 9758ab1 [q00251598] fix bug 1db1a1c [q00251598] use setInputPaths(Job job, Path... inputPaths) b788a72 [q00251598] change FileInputFormat.setInputPaths to jobConf.set and add test suite --- .../execution/HiveCompatibilitySuite.scala | 1 + .../apache/spark/sql/hive/TableReader.scala | 2 +- .../merge4-0-b12e5c70d6d29757471b900b6160fa8a | 1 + .../merge4-1-593999fae618b6b38322bc9ae4e0c027 | 1 + ...merge4-10-692a197bd688b48f762e72978f54aa32 | 0 .../merge4-11-f407e661307b23a5d52a08a3e7af19b | 1500 +++++++++++++++++ ...merge4-12-62541540a18d68a3cb8497a741061d11 | 0 ...merge4-13-ed1103f06609365b40e78d13c654cc71 | 0 ...merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 | 3 + ...merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a | 1 + .../merge4-2-43d53504df013e6b35f81811138a167a | 1 + .../merge4-3-a4fb8359a2179ec70777aad6366071b7 | 1 + .../merge4-4-16367c381d4b189b3640c92511244bfe | 1 + .../merge4-5-3d24d877366c42030f6d9a596665720d | 0 .../merge4-6-b3a76420183795720ab3a384046e5af | 0 .../merge4-7-631a45828eae3f5f562d992efe4cd56d | 0 .../merge4-8-f407e661307b23a5d52a08a3e7af19b | 1000 +++++++++++ .../merge4-9-ad3dc168c8b6f048717e39ab16b0a319 | 0 18 files changed, 2511 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 create mode 100644 sql/hive/src/test/resources/golden/merge4-11-f407e661307b23a5d52a08a3e7af19b create mode 100644 sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 create mode 100644 sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 create mode 100644 sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 create mode 100644 sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a create mode 100644 sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d create mode 100644 sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af create mode 100644 sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d create mode 100644 sql/hive/src/test/resources/golden/merge4-8-f407e661307b23a5d52a08a3e7af19b create mode 100644 sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index c6ead4562d51e..6126ce7130426 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -632,6 +632,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "mapreduce8", "merge1", "merge2", + "merge4", "mergejoins", "multiMapJoin1", "multiMapJoin2", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index effaa5a443512..f22c9eaeedc7d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -248,7 +248,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { * instantiate a HadoopRDD. */ def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { - FileInputFormat.setInputPaths(jobConf, path) + FileInputFormat.setInputPaths(jobConf, Seq[Path](new Path(path)): _*) if (tableDesc != null) { PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc) Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) diff --git a/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 b/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-11-f407e661307b23a5d52a08a3e7af19b b/sql/hive/src/test/resources/golden/merge4-11-f407e661307b23a5d52a08a3e7af19b new file mode 100644 index 0000000000000..5d2cddc42f272 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-11-f407e661307b23a5d52a08a3e7af19b @@ -0,0 +1,1500 @@ +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 12 +0 val_0 2010-08-15 12 +0 val_0 2010-08-15 12 +2 val_2 2010-08-15 11 +2 val_2 2010-08-15 11 +2 val_2 2010-08-15 12 +4 val_4 2010-08-15 11 +4 val_4 2010-08-15 11 +4 val_4 2010-08-15 12 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 12 +5 val_5 2010-08-15 12 +5 val_5 2010-08-15 12 +8 val_8 2010-08-15 11 +8 val_8 2010-08-15 11 +8 val_8 2010-08-15 12 +9 val_9 2010-08-15 11 +9 val_9 2010-08-15 11 +9 val_9 2010-08-15 12 +10 val_10 2010-08-15 11 +10 val_10 2010-08-15 11 +10 val_10 2010-08-15 12 +11 val_11 2010-08-15 11 +11 val_11 2010-08-15 11 +11 val_11 2010-08-15 12 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 12 +12 val_12 2010-08-15 12 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 12 +15 val_15 2010-08-15 12 +17 val_17 2010-08-15 11 +17 val_17 2010-08-15 11 +17 val_17 2010-08-15 12 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 12 +18 val_18 2010-08-15 12 +19 val_19 2010-08-15 11 +19 val_19 2010-08-15 11 +19 val_19 2010-08-15 12 +20 val_20 2010-08-15 11 +20 val_20 2010-08-15 11 +20 val_20 2010-08-15 12 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 12 +24 val_24 2010-08-15 12 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 12 +26 val_26 2010-08-15 12 +27 val_27 2010-08-15 11 +27 val_27 2010-08-15 11 +27 val_27 2010-08-15 12 +28 val_28 2010-08-15 11 +28 val_28 2010-08-15 11 +28 val_28 2010-08-15 12 +30 val_30 2010-08-15 11 +30 val_30 2010-08-15 11 +30 val_30 2010-08-15 12 +33 val_33 2010-08-15 11 +33 val_33 2010-08-15 11 +33 val_33 2010-08-15 12 +34 val_34 2010-08-15 11 +34 val_34 2010-08-15 11 +34 val_34 2010-08-15 12 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 12 +35 val_35 2010-08-15 12 +35 val_35 2010-08-15 12 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 12 +37 val_37 2010-08-15 12 +41 val_41 2010-08-15 11 +41 val_41 2010-08-15 11 +41 val_41 2010-08-15 12 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 12 +42 val_42 2010-08-15 12 +43 val_43 2010-08-15 11 +43 val_43 2010-08-15 11 +43 val_43 2010-08-15 12 +44 val_44 2010-08-15 11 +44 val_44 2010-08-15 11 +44 val_44 2010-08-15 12 +47 val_47 2010-08-15 11 +47 val_47 2010-08-15 11 +47 val_47 2010-08-15 12 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 12 +51 val_51 2010-08-15 12 +53 val_53 2010-08-15 11 +53 val_53 2010-08-15 11 +53 val_53 2010-08-15 12 +54 val_54 2010-08-15 11 +54 val_54 2010-08-15 11 +54 val_54 2010-08-15 12 +57 val_57 2010-08-15 11 +57 val_57 2010-08-15 11 +57 val_57 2010-08-15 12 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 12 +58 val_58 2010-08-15 12 +64 val_64 2010-08-15 11 +64 val_64 2010-08-15 11 +64 val_64 2010-08-15 12 +65 val_65 2010-08-15 11 +65 val_65 2010-08-15 11 +65 val_65 2010-08-15 12 +66 val_66 2010-08-15 11 +66 val_66 2010-08-15 11 +66 val_66 2010-08-15 12 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 12 +67 val_67 2010-08-15 12 +69 val_69 2010-08-15 11 +69 val_69 2010-08-15 11 +69 val_69 2010-08-15 12 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 12 +70 val_70 2010-08-15 12 +70 val_70 2010-08-15 12 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 12 +72 val_72 2010-08-15 12 +74 val_74 2010-08-15 11 +74 val_74 2010-08-15 11 +74 val_74 2010-08-15 12 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 12 +76 val_76 2010-08-15 12 +77 val_77 2010-08-15 11 +77 val_77 2010-08-15 11 +77 val_77 2010-08-15 12 +78 val_78 2010-08-15 11 +78 val_78 2010-08-15 11 +78 val_78 2010-08-15 12 +80 val_80 2010-08-15 11 +80 val_80 2010-08-15 11 +80 val_80 2010-08-15 12 +82 val_82 2010-08-15 11 +82 val_82 2010-08-15 11 +82 val_82 2010-08-15 12 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 12 +83 val_83 2010-08-15 12 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 12 +84 val_84 2010-08-15 12 +85 val_85 2010-08-15 11 +85 val_85 2010-08-15 11 +85 val_85 2010-08-15 12 +86 val_86 2010-08-15 11 +86 val_86 2010-08-15 11 +86 val_86 2010-08-15 12 +87 val_87 2010-08-15 11 +87 val_87 2010-08-15 11 +87 val_87 2010-08-15 12 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 12 +90 val_90 2010-08-15 12 +90 val_90 2010-08-15 12 +92 val_92 2010-08-15 11 +92 val_92 2010-08-15 11 +92 val_92 2010-08-15 12 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 12 +95 val_95 2010-08-15 12 +96 val_96 2010-08-15 11 +96 val_96 2010-08-15 11 +96 val_96 2010-08-15 12 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 12 +97 val_97 2010-08-15 12 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 12 +98 val_98 2010-08-15 12 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 12 +100 val_100 2010-08-15 12 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 12 +103 val_103 2010-08-15 12 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 12 +104 val_104 2010-08-15 12 +105 val_105 2010-08-15 11 +105 val_105 2010-08-15 11 +105 val_105 2010-08-15 12 +111 val_111 2010-08-15 11 +111 val_111 2010-08-15 11 +111 val_111 2010-08-15 12 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 12 +113 val_113 2010-08-15 12 +114 val_114 2010-08-15 11 +114 val_114 2010-08-15 11 +114 val_114 2010-08-15 12 +116 val_116 2010-08-15 11 +116 val_116 2010-08-15 11 +116 val_116 2010-08-15 12 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 12 +118 val_118 2010-08-15 12 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 12 +119 val_119 2010-08-15 12 +119 val_119 2010-08-15 12 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 12 +120 val_120 2010-08-15 12 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 12 +125 val_125 2010-08-15 12 +126 val_126 2010-08-15 11 +126 val_126 2010-08-15 11 +126 val_126 2010-08-15 12 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 12 +128 val_128 2010-08-15 12 +128 val_128 2010-08-15 12 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 12 +129 val_129 2010-08-15 12 +131 val_131 2010-08-15 11 +131 val_131 2010-08-15 11 +131 val_131 2010-08-15 12 +133 val_133 2010-08-15 11 +133 val_133 2010-08-15 11 +133 val_133 2010-08-15 12 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 12 +134 val_134 2010-08-15 12 +136 val_136 2010-08-15 11 +136 val_136 2010-08-15 11 +136 val_136 2010-08-15 12 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 12 +137 val_137 2010-08-15 12 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +143 val_143 2010-08-15 11 +143 val_143 2010-08-15 11 +143 val_143 2010-08-15 12 +145 val_145 2010-08-15 11 +145 val_145 2010-08-15 11 +145 val_145 2010-08-15 12 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 12 +146 val_146 2010-08-15 12 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 12 +149 val_149 2010-08-15 12 +150 val_150 2010-08-15 11 +150 val_150 2010-08-15 11 +150 val_150 2010-08-15 12 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 12 +152 val_152 2010-08-15 12 +153 val_153 2010-08-15 11 +153 val_153 2010-08-15 11 +153 val_153 2010-08-15 12 +155 val_155 2010-08-15 11 +155 val_155 2010-08-15 11 +155 val_155 2010-08-15 12 +156 val_156 2010-08-15 11 +156 val_156 2010-08-15 11 +156 val_156 2010-08-15 12 +157 val_157 2010-08-15 11 +157 val_157 2010-08-15 11 +157 val_157 2010-08-15 12 +158 val_158 2010-08-15 11 +158 val_158 2010-08-15 11 +158 val_158 2010-08-15 12 +160 val_160 2010-08-15 11 +160 val_160 2010-08-15 11 +160 val_160 2010-08-15 12 +162 val_162 2010-08-15 11 +162 val_162 2010-08-15 11 +162 val_162 2010-08-15 12 +163 val_163 2010-08-15 11 +163 val_163 2010-08-15 11 +163 val_163 2010-08-15 12 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 12 +164 val_164 2010-08-15 12 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 12 +165 val_165 2010-08-15 12 +166 val_166 2010-08-15 11 +166 val_166 2010-08-15 11 +166 val_166 2010-08-15 12 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 12 +167 val_167 2010-08-15 12 +167 val_167 2010-08-15 12 +168 val_168 2010-08-15 11 +168 val_168 2010-08-15 11 +168 val_168 2010-08-15 12 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +170 val_170 2010-08-15 11 +170 val_170 2010-08-15 11 +170 val_170 2010-08-15 12 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 12 +172 val_172 2010-08-15 12 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 12 +174 val_174 2010-08-15 12 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 12 +175 val_175 2010-08-15 12 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 12 +176 val_176 2010-08-15 12 +177 val_177 2010-08-15 11 +177 val_177 2010-08-15 11 +177 val_177 2010-08-15 12 +178 val_178 2010-08-15 11 +178 val_178 2010-08-15 11 +178 val_178 2010-08-15 12 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 12 +179 val_179 2010-08-15 12 +180 val_180 2010-08-15 11 +180 val_180 2010-08-15 11 +180 val_180 2010-08-15 12 +181 val_181 2010-08-15 11 +181 val_181 2010-08-15 11 +181 val_181 2010-08-15 12 +183 val_183 2010-08-15 11 +183 val_183 2010-08-15 11 +183 val_183 2010-08-15 12 +186 val_186 2010-08-15 11 +186 val_186 2010-08-15 11 +186 val_186 2010-08-15 12 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 12 +187 val_187 2010-08-15 12 +187 val_187 2010-08-15 12 +189 val_189 2010-08-15 11 +189 val_189 2010-08-15 11 +189 val_189 2010-08-15 12 +190 val_190 2010-08-15 11 +190 val_190 2010-08-15 11 +190 val_190 2010-08-15 12 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 12 +191 val_191 2010-08-15 12 +192 val_192 2010-08-15 11 +192 val_192 2010-08-15 11 +192 val_192 2010-08-15 12 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 12 +193 val_193 2010-08-15 12 +193 val_193 2010-08-15 12 +194 val_194 2010-08-15 11 +194 val_194 2010-08-15 11 +194 val_194 2010-08-15 12 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 12 +195 val_195 2010-08-15 12 +196 val_196 2010-08-15 11 +196 val_196 2010-08-15 11 +196 val_196 2010-08-15 12 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 12 +197 val_197 2010-08-15 12 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 12 +199 val_199 2010-08-15 12 +199 val_199 2010-08-15 12 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 12 +200 val_200 2010-08-15 12 +201 val_201 2010-08-15 11 +201 val_201 2010-08-15 11 +201 val_201 2010-08-15 12 +202 val_202 2010-08-15 11 +202 val_202 2010-08-15 11 +202 val_202 2010-08-15 12 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 12 +203 val_203 2010-08-15 12 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 12 +205 val_205 2010-08-15 12 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 12 +207 val_207 2010-08-15 12 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 12 +208 val_208 2010-08-15 12 +208 val_208 2010-08-15 12 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 12 +209 val_209 2010-08-15 12 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 12 +213 val_213 2010-08-15 12 +214 val_214 2010-08-15 11 +214 val_214 2010-08-15 11 +214 val_214 2010-08-15 12 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 12 +216 val_216 2010-08-15 12 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 12 +217 val_217 2010-08-15 12 +218 val_218 2010-08-15 11 +218 val_218 2010-08-15 11 +218 val_218 2010-08-15 12 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 12 +219 val_219 2010-08-15 12 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 12 +221 val_221 2010-08-15 12 +222 val_222 2010-08-15 11 +222 val_222 2010-08-15 11 +222 val_222 2010-08-15 12 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 12 +223 val_223 2010-08-15 12 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 12 +224 val_224 2010-08-15 12 +226 val_226 2010-08-15 11 +226 val_226 2010-08-15 11 +226 val_226 2010-08-15 12 +228 val_228 2010-08-15 11 +228 val_228 2010-08-15 11 +228 val_228 2010-08-15 12 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 12 +229 val_229 2010-08-15 12 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 12 +233 val_233 2010-08-15 12 +235 val_235 2010-08-15 11 +235 val_235 2010-08-15 11 +235 val_235 2010-08-15 12 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 12 +237 val_237 2010-08-15 12 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 12 +238 val_238 2010-08-15 12 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 12 +239 val_239 2010-08-15 12 +241 val_241 2010-08-15 11 +241 val_241 2010-08-15 11 +241 val_241 2010-08-15 12 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 12 +242 val_242 2010-08-15 12 +244 val_244 2010-08-15 11 +244 val_244 2010-08-15 11 +244 val_244 2010-08-15 12 +247 val_247 2010-08-15 11 +247 val_247 2010-08-15 11 +247 val_247 2010-08-15 12 +248 val_248 2010-08-15 11 +248 val_248 2010-08-15 11 +248 val_248 2010-08-15 12 +249 val_249 2010-08-15 11 +249 val_249 2010-08-15 11 +249 val_249 2010-08-15 12 +252 val_252 2010-08-15 11 +252 val_252 2010-08-15 11 +252 val_252 2010-08-15 12 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 12 +255 val_255 2010-08-15 12 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 12 +256 val_256 2010-08-15 12 +257 val_257 2010-08-15 11 +257 val_257 2010-08-15 11 +257 val_257 2010-08-15 12 +258 val_258 2010-08-15 11 +258 val_258 2010-08-15 11 +258 val_258 2010-08-15 12 +260 val_260 2010-08-15 11 +260 val_260 2010-08-15 11 +260 val_260 2010-08-15 12 +262 val_262 2010-08-15 11 +262 val_262 2010-08-15 11 +262 val_262 2010-08-15 12 +263 val_263 2010-08-15 11 +263 val_263 2010-08-15 11 +263 val_263 2010-08-15 12 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 12 +265 val_265 2010-08-15 12 +266 val_266 2010-08-15 11 +266 val_266 2010-08-15 11 +266 val_266 2010-08-15 12 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 12 +272 val_272 2010-08-15 12 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 12 +273 val_273 2010-08-15 12 +273 val_273 2010-08-15 12 +274 val_274 2010-08-15 11 +274 val_274 2010-08-15 11 +274 val_274 2010-08-15 12 +275 val_275 2010-08-15 11 +275 val_275 2010-08-15 11 +275 val_275 2010-08-15 12 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 12 +278 val_278 2010-08-15 12 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 12 +280 val_280 2010-08-15 12 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 12 +281 val_281 2010-08-15 12 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 12 +282 val_282 2010-08-15 12 +283 val_283 2010-08-15 11 +283 val_283 2010-08-15 11 +283 val_283 2010-08-15 12 +284 val_284 2010-08-15 11 +284 val_284 2010-08-15 11 +284 val_284 2010-08-15 12 +285 val_285 2010-08-15 11 +285 val_285 2010-08-15 11 +285 val_285 2010-08-15 12 +286 val_286 2010-08-15 11 +286 val_286 2010-08-15 11 +286 val_286 2010-08-15 12 +287 val_287 2010-08-15 11 +287 val_287 2010-08-15 11 +287 val_287 2010-08-15 12 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 12 +288 val_288 2010-08-15 12 +289 val_289 2010-08-15 11 +289 val_289 2010-08-15 11 +289 val_289 2010-08-15 12 +291 val_291 2010-08-15 11 +291 val_291 2010-08-15 11 +291 val_291 2010-08-15 12 +292 val_292 2010-08-15 11 +292 val_292 2010-08-15 11 +292 val_292 2010-08-15 12 +296 val_296 2010-08-15 11 +296 val_296 2010-08-15 11 +296 val_296 2010-08-15 12 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 12 +298 val_298 2010-08-15 12 +298 val_298 2010-08-15 12 +302 val_302 2010-08-15 11 +302 val_302 2010-08-15 11 +302 val_302 2010-08-15 12 +305 val_305 2010-08-15 11 +305 val_305 2010-08-15 11 +305 val_305 2010-08-15 12 +306 val_306 2010-08-15 11 +306 val_306 2010-08-15 11 +306 val_306 2010-08-15 12 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 12 +307 val_307 2010-08-15 12 +308 val_308 2010-08-15 11 +308 val_308 2010-08-15 11 +308 val_308 2010-08-15 12 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 12 +309 val_309 2010-08-15 12 +310 val_310 2010-08-15 11 +310 val_310 2010-08-15 11 +310 val_310 2010-08-15 12 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 12 +311 val_311 2010-08-15 12 +311 val_311 2010-08-15 12 +315 val_315 2010-08-15 11 +315 val_315 2010-08-15 11 +315 val_315 2010-08-15 12 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 12 +316 val_316 2010-08-15 12 +316 val_316 2010-08-15 12 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 12 +317 val_317 2010-08-15 12 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 12 +318 val_318 2010-08-15 12 +318 val_318 2010-08-15 12 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 12 +321 val_321 2010-08-15 12 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 12 +322 val_322 2010-08-15 12 +323 val_323 2010-08-15 11 +323 val_323 2010-08-15 11 +323 val_323 2010-08-15 12 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 12 +325 val_325 2010-08-15 12 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 12 +327 val_327 2010-08-15 12 +327 val_327 2010-08-15 12 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 12 +331 val_331 2010-08-15 12 +332 val_332 2010-08-15 11 +332 val_332 2010-08-15 11 +332 val_332 2010-08-15 12 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 12 +333 val_333 2010-08-15 12 +335 val_335 2010-08-15 11 +335 val_335 2010-08-15 11 +335 val_335 2010-08-15 12 +336 val_336 2010-08-15 11 +336 val_336 2010-08-15 11 +336 val_336 2010-08-15 12 +338 val_338 2010-08-15 11 +338 val_338 2010-08-15 11 +338 val_338 2010-08-15 12 +339 val_339 2010-08-15 11 +339 val_339 2010-08-15 11 +339 val_339 2010-08-15 12 +341 val_341 2010-08-15 11 +341 val_341 2010-08-15 11 +341 val_341 2010-08-15 12 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 12 +342 val_342 2010-08-15 12 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 12 +344 val_344 2010-08-15 12 +345 val_345 2010-08-15 11 +345 val_345 2010-08-15 11 +345 val_345 2010-08-15 12 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +351 val_351 2010-08-15 11 +351 val_351 2010-08-15 11 +351 val_351 2010-08-15 12 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 12 +353 val_353 2010-08-15 12 +356 val_356 2010-08-15 11 +356 val_356 2010-08-15 11 +356 val_356 2010-08-15 12 +360 val_360 2010-08-15 11 +360 val_360 2010-08-15 11 +360 val_360 2010-08-15 12 +362 val_362 2010-08-15 11 +362 val_362 2010-08-15 11 +362 val_362 2010-08-15 12 +364 val_364 2010-08-15 11 +364 val_364 2010-08-15 11 +364 val_364 2010-08-15 12 +365 val_365 2010-08-15 11 +365 val_365 2010-08-15 11 +365 val_365 2010-08-15 12 +366 val_366 2010-08-15 11 +366 val_366 2010-08-15 11 +366 val_366 2010-08-15 12 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 12 +367 val_367 2010-08-15 12 +368 val_368 2010-08-15 11 +368 val_368 2010-08-15 11 +368 val_368 2010-08-15 12 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 12 +369 val_369 2010-08-15 12 +369 val_369 2010-08-15 12 +373 val_373 2010-08-15 11 +373 val_373 2010-08-15 11 +373 val_373 2010-08-15 12 +374 val_374 2010-08-15 11 +374 val_374 2010-08-15 11 +374 val_374 2010-08-15 12 +375 val_375 2010-08-15 11 +375 val_375 2010-08-15 11 +375 val_375 2010-08-15 12 +377 val_377 2010-08-15 11 +377 val_377 2010-08-15 11 +377 val_377 2010-08-15 12 +378 val_378 2010-08-15 11 +378 val_378 2010-08-15 11 +378 val_378 2010-08-15 12 +379 val_379 2010-08-15 11 +379 val_379 2010-08-15 11 +379 val_379 2010-08-15 12 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 12 +382 val_382 2010-08-15 12 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 12 +384 val_384 2010-08-15 12 +384 val_384 2010-08-15 12 +386 val_386 2010-08-15 11 +386 val_386 2010-08-15 11 +386 val_386 2010-08-15 12 +389 val_389 2010-08-15 11 +389 val_389 2010-08-15 11 +389 val_389 2010-08-15 12 +392 val_392 2010-08-15 11 +392 val_392 2010-08-15 11 +392 val_392 2010-08-15 12 +393 val_393 2010-08-15 11 +393 val_393 2010-08-15 11 +393 val_393 2010-08-15 12 +394 val_394 2010-08-15 11 +394 val_394 2010-08-15 11 +394 val_394 2010-08-15 12 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 12 +395 val_395 2010-08-15 12 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 12 +396 val_396 2010-08-15 12 +396 val_396 2010-08-15 12 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 12 +397 val_397 2010-08-15 12 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 12 +399 val_399 2010-08-15 12 +400 val_400 2010-08-15 11 +400 val_400 2010-08-15 11 +400 val_400 2010-08-15 12 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +402 val_402 2010-08-15 11 +402 val_402 2010-08-15 11 +402 val_402 2010-08-15 12 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 12 +403 val_403 2010-08-15 12 +403 val_403 2010-08-15 12 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 12 +404 val_404 2010-08-15 12 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +407 val_407 2010-08-15 11 +407 val_407 2010-08-15 11 +407 val_407 2010-08-15 12 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 12 +409 val_409 2010-08-15 12 +409 val_409 2010-08-15 12 +411 val_411 2010-08-15 11 +411 val_411 2010-08-15 11 +411 val_411 2010-08-15 12 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 12 +413 val_413 2010-08-15 12 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 12 +414 val_414 2010-08-15 12 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 12 +417 val_417 2010-08-15 12 +417 val_417 2010-08-15 12 +418 val_418 2010-08-15 11 +418 val_418 2010-08-15 11 +418 val_418 2010-08-15 12 +419 val_419 2010-08-15 11 +419 val_419 2010-08-15 11 +419 val_419 2010-08-15 12 +421 val_421 2010-08-15 11 +421 val_421 2010-08-15 11 +421 val_421 2010-08-15 12 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 12 +424 val_424 2010-08-15 12 +427 val_427 2010-08-15 11 +427 val_427 2010-08-15 11 +427 val_427 2010-08-15 12 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 12 +429 val_429 2010-08-15 12 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 12 +430 val_430 2010-08-15 12 +430 val_430 2010-08-15 12 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 12 +431 val_431 2010-08-15 12 +431 val_431 2010-08-15 12 +432 val_432 2010-08-15 11 +432 val_432 2010-08-15 11 +432 val_432 2010-08-15 12 +435 val_435 2010-08-15 11 +435 val_435 2010-08-15 11 +435 val_435 2010-08-15 12 +436 val_436 2010-08-15 11 +436 val_436 2010-08-15 11 +436 val_436 2010-08-15 12 +437 val_437 2010-08-15 11 +437 val_437 2010-08-15 11 +437 val_437 2010-08-15 12 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 12 +438 val_438 2010-08-15 12 +438 val_438 2010-08-15 12 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 12 +439 val_439 2010-08-15 12 +443 val_443 2010-08-15 11 +443 val_443 2010-08-15 11 +443 val_443 2010-08-15 12 +444 val_444 2010-08-15 11 +444 val_444 2010-08-15 11 +444 val_444 2010-08-15 12 +446 val_446 2010-08-15 11 +446 val_446 2010-08-15 11 +446 val_446 2010-08-15 12 +448 val_448 2010-08-15 11 +448 val_448 2010-08-15 11 +448 val_448 2010-08-15 12 +449 val_449 2010-08-15 11 +449 val_449 2010-08-15 11 +449 val_449 2010-08-15 12 +452 val_452 2010-08-15 11 +452 val_452 2010-08-15 11 +452 val_452 2010-08-15 12 +453 val_453 2010-08-15 11 +453 val_453 2010-08-15 11 +453 val_453 2010-08-15 12 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 12 +454 val_454 2010-08-15 12 +454 val_454 2010-08-15 12 +455 val_455 2010-08-15 11 +455 val_455 2010-08-15 11 +455 val_455 2010-08-15 12 +457 val_457 2010-08-15 11 +457 val_457 2010-08-15 11 +457 val_457 2010-08-15 12 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 12 +458 val_458 2010-08-15 12 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 12 +459 val_459 2010-08-15 12 +460 val_460 2010-08-15 11 +460 val_460 2010-08-15 11 +460 val_460 2010-08-15 12 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 12 +462 val_462 2010-08-15 12 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 12 +463 val_463 2010-08-15 12 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 12 +466 val_466 2010-08-15 12 +466 val_466 2010-08-15 12 +467 val_467 2010-08-15 11 +467 val_467 2010-08-15 11 +467 val_467 2010-08-15 12 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +470 val_470 2010-08-15 11 +470 val_470 2010-08-15 11 +470 val_470 2010-08-15 12 +472 val_472 2010-08-15 11 +472 val_472 2010-08-15 11 +472 val_472 2010-08-15 12 +475 val_475 2010-08-15 11 +475 val_475 2010-08-15 11 +475 val_475 2010-08-15 12 +477 val_477 2010-08-15 11 +477 val_477 2010-08-15 11 +477 val_477 2010-08-15 12 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 12 +478 val_478 2010-08-15 12 +479 val_479 2010-08-15 11 +479 val_479 2010-08-15 11 +479 val_479 2010-08-15 12 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 12 +480 val_480 2010-08-15 12 +480 val_480 2010-08-15 12 +481 val_481 2010-08-15 11 +481 val_481 2010-08-15 11 +481 val_481 2010-08-15 12 +482 val_482 2010-08-15 11 +482 val_482 2010-08-15 11 +482 val_482 2010-08-15 12 +483 val_483 2010-08-15 11 +483 val_483 2010-08-15 11 +483 val_483 2010-08-15 12 +484 val_484 2010-08-15 11 +484 val_484 2010-08-15 11 +484 val_484 2010-08-15 12 +485 val_485 2010-08-15 11 +485 val_485 2010-08-15 11 +485 val_485 2010-08-15 12 +487 val_487 2010-08-15 11 +487 val_487 2010-08-15 11 +487 val_487 2010-08-15 12 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +490 val_490 2010-08-15 11 +490 val_490 2010-08-15 11 +490 val_490 2010-08-15 12 +491 val_491 2010-08-15 11 +491 val_491 2010-08-15 11 +491 val_491 2010-08-15 12 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 12 +492 val_492 2010-08-15 12 +493 val_493 2010-08-15 11 +493 val_493 2010-08-15 11 +493 val_493 2010-08-15 12 +494 val_494 2010-08-15 11 +494 val_494 2010-08-15 11 +494 val_494 2010-08-15 12 +495 val_495 2010-08-15 11 +495 val_495 2010-08-15 11 +495 val_495 2010-08-15 12 +496 val_496 2010-08-15 11 +496 val_496 2010-08-15 11 +496 val_496 2010-08-15 12 +497 val_497 2010-08-15 11 +497 val_497 2010-08-15 11 +497 val_497 2010-08-15 12 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 12 +498 val_498 2010-08-15 12 +498 val_498 2010-08-15 12 diff --git a/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 b/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 b/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 b/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 new file mode 100644 index 0000000000000..30becc42d7b5a --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 @@ -0,0 +1,3 @@ +ds=2010-08-15/hr=11 +ds=2010-08-15/hr=12 +ds=2010-08-15/hr=file, diff --git a/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a b/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a new file mode 100644 index 0000000000000..4c867a5deff08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a @@ -0,0 +1 @@ +1 1 2010-08-15 file, diff --git a/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d b/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af b/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d b/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-8-f407e661307b23a5d52a08a3e7af19b b/sql/hive/src/test/resources/golden/merge4-8-f407e661307b23a5d52a08a3e7af19b new file mode 100644 index 0000000000000..aa972caa5665d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-8-f407e661307b23a5d52a08a3e7af19b @@ -0,0 +1,1000 @@ +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 11 +0 val_0 2010-08-15 12 +0 val_0 2010-08-15 12 +0 val_0 2010-08-15 12 +2 val_2 2010-08-15 11 +2 val_2 2010-08-15 12 +4 val_4 2010-08-15 11 +4 val_4 2010-08-15 12 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 11 +5 val_5 2010-08-15 12 +5 val_5 2010-08-15 12 +5 val_5 2010-08-15 12 +8 val_8 2010-08-15 11 +8 val_8 2010-08-15 12 +9 val_9 2010-08-15 11 +9 val_9 2010-08-15 12 +10 val_10 2010-08-15 11 +10 val_10 2010-08-15 12 +11 val_11 2010-08-15 11 +11 val_11 2010-08-15 12 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 11 +12 val_12 2010-08-15 12 +12 val_12 2010-08-15 12 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 11 +15 val_15 2010-08-15 12 +15 val_15 2010-08-15 12 +17 val_17 2010-08-15 11 +17 val_17 2010-08-15 12 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 11 +18 val_18 2010-08-15 12 +18 val_18 2010-08-15 12 +19 val_19 2010-08-15 11 +19 val_19 2010-08-15 12 +20 val_20 2010-08-15 11 +20 val_20 2010-08-15 12 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 11 +24 val_24 2010-08-15 12 +24 val_24 2010-08-15 12 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 11 +26 val_26 2010-08-15 12 +26 val_26 2010-08-15 12 +27 val_27 2010-08-15 11 +27 val_27 2010-08-15 12 +28 val_28 2010-08-15 11 +28 val_28 2010-08-15 12 +30 val_30 2010-08-15 11 +30 val_30 2010-08-15 12 +33 val_33 2010-08-15 11 +33 val_33 2010-08-15 12 +34 val_34 2010-08-15 11 +34 val_34 2010-08-15 12 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 11 +35 val_35 2010-08-15 12 +35 val_35 2010-08-15 12 +35 val_35 2010-08-15 12 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 11 +37 val_37 2010-08-15 12 +37 val_37 2010-08-15 12 +41 val_41 2010-08-15 11 +41 val_41 2010-08-15 12 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 11 +42 val_42 2010-08-15 12 +42 val_42 2010-08-15 12 +43 val_43 2010-08-15 11 +43 val_43 2010-08-15 12 +44 val_44 2010-08-15 11 +44 val_44 2010-08-15 12 +47 val_47 2010-08-15 11 +47 val_47 2010-08-15 12 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 11 +51 val_51 2010-08-15 12 +51 val_51 2010-08-15 12 +53 val_53 2010-08-15 11 +53 val_53 2010-08-15 12 +54 val_54 2010-08-15 11 +54 val_54 2010-08-15 12 +57 val_57 2010-08-15 11 +57 val_57 2010-08-15 12 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 11 +58 val_58 2010-08-15 12 +58 val_58 2010-08-15 12 +64 val_64 2010-08-15 11 +64 val_64 2010-08-15 12 +65 val_65 2010-08-15 11 +65 val_65 2010-08-15 12 +66 val_66 2010-08-15 11 +66 val_66 2010-08-15 12 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 11 +67 val_67 2010-08-15 12 +67 val_67 2010-08-15 12 +69 val_69 2010-08-15 11 +69 val_69 2010-08-15 12 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 11 +70 val_70 2010-08-15 12 +70 val_70 2010-08-15 12 +70 val_70 2010-08-15 12 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 11 +72 val_72 2010-08-15 12 +72 val_72 2010-08-15 12 +74 val_74 2010-08-15 11 +74 val_74 2010-08-15 12 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 11 +76 val_76 2010-08-15 12 +76 val_76 2010-08-15 12 +77 val_77 2010-08-15 11 +77 val_77 2010-08-15 12 +78 val_78 2010-08-15 11 +78 val_78 2010-08-15 12 +80 val_80 2010-08-15 11 +80 val_80 2010-08-15 12 +82 val_82 2010-08-15 11 +82 val_82 2010-08-15 12 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 11 +83 val_83 2010-08-15 12 +83 val_83 2010-08-15 12 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 11 +84 val_84 2010-08-15 12 +84 val_84 2010-08-15 12 +85 val_85 2010-08-15 11 +85 val_85 2010-08-15 12 +86 val_86 2010-08-15 11 +86 val_86 2010-08-15 12 +87 val_87 2010-08-15 11 +87 val_87 2010-08-15 12 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 11 +90 val_90 2010-08-15 12 +90 val_90 2010-08-15 12 +90 val_90 2010-08-15 12 +92 val_92 2010-08-15 11 +92 val_92 2010-08-15 12 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 11 +95 val_95 2010-08-15 12 +95 val_95 2010-08-15 12 +96 val_96 2010-08-15 11 +96 val_96 2010-08-15 12 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 11 +97 val_97 2010-08-15 12 +97 val_97 2010-08-15 12 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 11 +98 val_98 2010-08-15 12 +98 val_98 2010-08-15 12 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 11 +100 val_100 2010-08-15 12 +100 val_100 2010-08-15 12 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 11 +103 val_103 2010-08-15 12 +103 val_103 2010-08-15 12 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 11 +104 val_104 2010-08-15 12 +104 val_104 2010-08-15 12 +105 val_105 2010-08-15 11 +105 val_105 2010-08-15 12 +111 val_111 2010-08-15 11 +111 val_111 2010-08-15 12 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 11 +113 val_113 2010-08-15 12 +113 val_113 2010-08-15 12 +114 val_114 2010-08-15 11 +114 val_114 2010-08-15 12 +116 val_116 2010-08-15 11 +116 val_116 2010-08-15 12 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 11 +118 val_118 2010-08-15 12 +118 val_118 2010-08-15 12 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 11 +119 val_119 2010-08-15 12 +119 val_119 2010-08-15 12 +119 val_119 2010-08-15 12 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 11 +120 val_120 2010-08-15 12 +120 val_120 2010-08-15 12 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 11 +125 val_125 2010-08-15 12 +125 val_125 2010-08-15 12 +126 val_126 2010-08-15 11 +126 val_126 2010-08-15 12 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 11 +128 val_128 2010-08-15 12 +128 val_128 2010-08-15 12 +128 val_128 2010-08-15 12 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 11 +129 val_129 2010-08-15 12 +129 val_129 2010-08-15 12 +131 val_131 2010-08-15 11 +131 val_131 2010-08-15 12 +133 val_133 2010-08-15 11 +133 val_133 2010-08-15 12 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 11 +134 val_134 2010-08-15 12 +134 val_134 2010-08-15 12 +136 val_136 2010-08-15 11 +136 val_136 2010-08-15 12 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 11 +137 val_137 2010-08-15 12 +137 val_137 2010-08-15 12 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 11 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +138 val_138 2010-08-15 12 +143 val_143 2010-08-15 11 +143 val_143 2010-08-15 12 +145 val_145 2010-08-15 11 +145 val_145 2010-08-15 12 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 11 +146 val_146 2010-08-15 12 +146 val_146 2010-08-15 12 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 11 +149 val_149 2010-08-15 12 +149 val_149 2010-08-15 12 +150 val_150 2010-08-15 11 +150 val_150 2010-08-15 12 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 11 +152 val_152 2010-08-15 12 +152 val_152 2010-08-15 12 +153 val_153 2010-08-15 11 +153 val_153 2010-08-15 12 +155 val_155 2010-08-15 11 +155 val_155 2010-08-15 12 +156 val_156 2010-08-15 11 +156 val_156 2010-08-15 12 +157 val_157 2010-08-15 11 +157 val_157 2010-08-15 12 +158 val_158 2010-08-15 11 +158 val_158 2010-08-15 12 +160 val_160 2010-08-15 11 +160 val_160 2010-08-15 12 +162 val_162 2010-08-15 11 +162 val_162 2010-08-15 12 +163 val_163 2010-08-15 11 +163 val_163 2010-08-15 12 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 11 +164 val_164 2010-08-15 12 +164 val_164 2010-08-15 12 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 11 +165 val_165 2010-08-15 12 +165 val_165 2010-08-15 12 +166 val_166 2010-08-15 11 +166 val_166 2010-08-15 12 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 11 +167 val_167 2010-08-15 12 +167 val_167 2010-08-15 12 +167 val_167 2010-08-15 12 +168 val_168 2010-08-15 11 +168 val_168 2010-08-15 12 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 11 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +169 val_169 2010-08-15 12 +170 val_170 2010-08-15 11 +170 val_170 2010-08-15 12 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 11 +172 val_172 2010-08-15 12 +172 val_172 2010-08-15 12 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 11 +174 val_174 2010-08-15 12 +174 val_174 2010-08-15 12 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 11 +175 val_175 2010-08-15 12 +175 val_175 2010-08-15 12 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 11 +176 val_176 2010-08-15 12 +176 val_176 2010-08-15 12 +177 val_177 2010-08-15 11 +177 val_177 2010-08-15 12 +178 val_178 2010-08-15 11 +178 val_178 2010-08-15 12 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 11 +179 val_179 2010-08-15 12 +179 val_179 2010-08-15 12 +180 val_180 2010-08-15 11 +180 val_180 2010-08-15 12 +181 val_181 2010-08-15 11 +181 val_181 2010-08-15 12 +183 val_183 2010-08-15 11 +183 val_183 2010-08-15 12 +186 val_186 2010-08-15 11 +186 val_186 2010-08-15 12 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 11 +187 val_187 2010-08-15 12 +187 val_187 2010-08-15 12 +187 val_187 2010-08-15 12 +189 val_189 2010-08-15 11 +189 val_189 2010-08-15 12 +190 val_190 2010-08-15 11 +190 val_190 2010-08-15 12 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 11 +191 val_191 2010-08-15 12 +191 val_191 2010-08-15 12 +192 val_192 2010-08-15 11 +192 val_192 2010-08-15 12 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 11 +193 val_193 2010-08-15 12 +193 val_193 2010-08-15 12 +193 val_193 2010-08-15 12 +194 val_194 2010-08-15 11 +194 val_194 2010-08-15 12 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 11 +195 val_195 2010-08-15 12 +195 val_195 2010-08-15 12 +196 val_196 2010-08-15 11 +196 val_196 2010-08-15 12 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 11 +197 val_197 2010-08-15 12 +197 val_197 2010-08-15 12 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 11 +199 val_199 2010-08-15 12 +199 val_199 2010-08-15 12 +199 val_199 2010-08-15 12 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 11 +200 val_200 2010-08-15 12 +200 val_200 2010-08-15 12 +201 val_201 2010-08-15 11 +201 val_201 2010-08-15 12 +202 val_202 2010-08-15 11 +202 val_202 2010-08-15 12 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 11 +203 val_203 2010-08-15 12 +203 val_203 2010-08-15 12 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 11 +205 val_205 2010-08-15 12 +205 val_205 2010-08-15 12 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 11 +207 val_207 2010-08-15 12 +207 val_207 2010-08-15 12 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 11 +208 val_208 2010-08-15 12 +208 val_208 2010-08-15 12 +208 val_208 2010-08-15 12 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 11 +209 val_209 2010-08-15 12 +209 val_209 2010-08-15 12 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 11 +213 val_213 2010-08-15 12 +213 val_213 2010-08-15 12 +214 val_214 2010-08-15 11 +214 val_214 2010-08-15 12 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 11 +216 val_216 2010-08-15 12 +216 val_216 2010-08-15 12 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 11 +217 val_217 2010-08-15 12 +217 val_217 2010-08-15 12 +218 val_218 2010-08-15 11 +218 val_218 2010-08-15 12 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 11 +219 val_219 2010-08-15 12 +219 val_219 2010-08-15 12 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 11 +221 val_221 2010-08-15 12 +221 val_221 2010-08-15 12 +222 val_222 2010-08-15 11 +222 val_222 2010-08-15 12 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 11 +223 val_223 2010-08-15 12 +223 val_223 2010-08-15 12 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 11 +224 val_224 2010-08-15 12 +224 val_224 2010-08-15 12 +226 val_226 2010-08-15 11 +226 val_226 2010-08-15 12 +228 val_228 2010-08-15 11 +228 val_228 2010-08-15 12 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 11 +229 val_229 2010-08-15 12 +229 val_229 2010-08-15 12 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 11 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +230 val_230 2010-08-15 12 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 11 +233 val_233 2010-08-15 12 +233 val_233 2010-08-15 12 +235 val_235 2010-08-15 11 +235 val_235 2010-08-15 12 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 11 +237 val_237 2010-08-15 12 +237 val_237 2010-08-15 12 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 11 +238 val_238 2010-08-15 12 +238 val_238 2010-08-15 12 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 11 +239 val_239 2010-08-15 12 +239 val_239 2010-08-15 12 +241 val_241 2010-08-15 11 +241 val_241 2010-08-15 12 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 11 +242 val_242 2010-08-15 12 +242 val_242 2010-08-15 12 +244 val_244 2010-08-15 11 +244 val_244 2010-08-15 12 +247 val_247 2010-08-15 11 +247 val_247 2010-08-15 12 +248 val_248 2010-08-15 11 +248 val_248 2010-08-15 12 +249 val_249 2010-08-15 11 +249 val_249 2010-08-15 12 +252 val_252 2010-08-15 11 +252 val_252 2010-08-15 12 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 11 +255 val_255 2010-08-15 12 +255 val_255 2010-08-15 12 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 11 +256 val_256 2010-08-15 12 +256 val_256 2010-08-15 12 +257 val_257 2010-08-15 11 +257 val_257 2010-08-15 12 +258 val_258 2010-08-15 11 +258 val_258 2010-08-15 12 +260 val_260 2010-08-15 11 +260 val_260 2010-08-15 12 +262 val_262 2010-08-15 11 +262 val_262 2010-08-15 12 +263 val_263 2010-08-15 11 +263 val_263 2010-08-15 12 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 11 +265 val_265 2010-08-15 12 +265 val_265 2010-08-15 12 +266 val_266 2010-08-15 11 +266 val_266 2010-08-15 12 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 11 +272 val_272 2010-08-15 12 +272 val_272 2010-08-15 12 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 11 +273 val_273 2010-08-15 12 +273 val_273 2010-08-15 12 +273 val_273 2010-08-15 12 +274 val_274 2010-08-15 11 +274 val_274 2010-08-15 12 +275 val_275 2010-08-15 11 +275 val_275 2010-08-15 12 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 11 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +277 val_277 2010-08-15 12 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 11 +278 val_278 2010-08-15 12 +278 val_278 2010-08-15 12 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 11 +280 val_280 2010-08-15 12 +280 val_280 2010-08-15 12 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 11 +281 val_281 2010-08-15 12 +281 val_281 2010-08-15 12 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 11 +282 val_282 2010-08-15 12 +282 val_282 2010-08-15 12 +283 val_283 2010-08-15 11 +283 val_283 2010-08-15 12 +284 val_284 2010-08-15 11 +284 val_284 2010-08-15 12 +285 val_285 2010-08-15 11 +285 val_285 2010-08-15 12 +286 val_286 2010-08-15 11 +286 val_286 2010-08-15 12 +287 val_287 2010-08-15 11 +287 val_287 2010-08-15 12 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 11 +288 val_288 2010-08-15 12 +288 val_288 2010-08-15 12 +289 val_289 2010-08-15 11 +289 val_289 2010-08-15 12 +291 val_291 2010-08-15 11 +291 val_291 2010-08-15 12 +292 val_292 2010-08-15 11 +292 val_292 2010-08-15 12 +296 val_296 2010-08-15 11 +296 val_296 2010-08-15 12 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 11 +298 val_298 2010-08-15 12 +298 val_298 2010-08-15 12 +298 val_298 2010-08-15 12 +302 val_302 2010-08-15 11 +302 val_302 2010-08-15 12 +305 val_305 2010-08-15 11 +305 val_305 2010-08-15 12 +306 val_306 2010-08-15 11 +306 val_306 2010-08-15 12 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 11 +307 val_307 2010-08-15 12 +307 val_307 2010-08-15 12 +308 val_308 2010-08-15 11 +308 val_308 2010-08-15 12 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 11 +309 val_309 2010-08-15 12 +309 val_309 2010-08-15 12 +310 val_310 2010-08-15 11 +310 val_310 2010-08-15 12 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 11 +311 val_311 2010-08-15 12 +311 val_311 2010-08-15 12 +311 val_311 2010-08-15 12 +315 val_315 2010-08-15 11 +315 val_315 2010-08-15 12 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 11 +316 val_316 2010-08-15 12 +316 val_316 2010-08-15 12 +316 val_316 2010-08-15 12 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 11 +317 val_317 2010-08-15 12 +317 val_317 2010-08-15 12 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 11 +318 val_318 2010-08-15 12 +318 val_318 2010-08-15 12 +318 val_318 2010-08-15 12 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 11 +321 val_321 2010-08-15 12 +321 val_321 2010-08-15 12 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 11 +322 val_322 2010-08-15 12 +322 val_322 2010-08-15 12 +323 val_323 2010-08-15 11 +323 val_323 2010-08-15 12 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 11 +325 val_325 2010-08-15 12 +325 val_325 2010-08-15 12 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 11 +327 val_327 2010-08-15 12 +327 val_327 2010-08-15 12 +327 val_327 2010-08-15 12 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 11 +331 val_331 2010-08-15 12 +331 val_331 2010-08-15 12 +332 val_332 2010-08-15 11 +332 val_332 2010-08-15 12 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 11 +333 val_333 2010-08-15 12 +333 val_333 2010-08-15 12 +335 val_335 2010-08-15 11 +335 val_335 2010-08-15 12 +336 val_336 2010-08-15 11 +336 val_336 2010-08-15 12 +338 val_338 2010-08-15 11 +338 val_338 2010-08-15 12 +339 val_339 2010-08-15 11 +339 val_339 2010-08-15 12 +341 val_341 2010-08-15 11 +341 val_341 2010-08-15 12 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 11 +342 val_342 2010-08-15 12 +342 val_342 2010-08-15 12 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 11 +344 val_344 2010-08-15 12 +344 val_344 2010-08-15 12 +345 val_345 2010-08-15 11 +345 val_345 2010-08-15 12 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 11 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +348 val_348 2010-08-15 12 +351 val_351 2010-08-15 11 +351 val_351 2010-08-15 12 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 11 +353 val_353 2010-08-15 12 +353 val_353 2010-08-15 12 +356 val_356 2010-08-15 11 +356 val_356 2010-08-15 12 +360 val_360 2010-08-15 11 +360 val_360 2010-08-15 12 +362 val_362 2010-08-15 11 +362 val_362 2010-08-15 12 +364 val_364 2010-08-15 11 +364 val_364 2010-08-15 12 +365 val_365 2010-08-15 11 +365 val_365 2010-08-15 12 +366 val_366 2010-08-15 11 +366 val_366 2010-08-15 12 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 11 +367 val_367 2010-08-15 12 +367 val_367 2010-08-15 12 +368 val_368 2010-08-15 11 +368 val_368 2010-08-15 12 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 11 +369 val_369 2010-08-15 12 +369 val_369 2010-08-15 12 +369 val_369 2010-08-15 12 +373 val_373 2010-08-15 11 +373 val_373 2010-08-15 12 +374 val_374 2010-08-15 11 +374 val_374 2010-08-15 12 +375 val_375 2010-08-15 11 +375 val_375 2010-08-15 12 +377 val_377 2010-08-15 11 +377 val_377 2010-08-15 12 +378 val_378 2010-08-15 11 +378 val_378 2010-08-15 12 +379 val_379 2010-08-15 11 +379 val_379 2010-08-15 12 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 11 +382 val_382 2010-08-15 12 +382 val_382 2010-08-15 12 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 11 +384 val_384 2010-08-15 12 +384 val_384 2010-08-15 12 +384 val_384 2010-08-15 12 +386 val_386 2010-08-15 11 +386 val_386 2010-08-15 12 +389 val_389 2010-08-15 11 +389 val_389 2010-08-15 12 +392 val_392 2010-08-15 11 +392 val_392 2010-08-15 12 +393 val_393 2010-08-15 11 +393 val_393 2010-08-15 12 +394 val_394 2010-08-15 11 +394 val_394 2010-08-15 12 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 11 +395 val_395 2010-08-15 12 +395 val_395 2010-08-15 12 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 11 +396 val_396 2010-08-15 12 +396 val_396 2010-08-15 12 +396 val_396 2010-08-15 12 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 11 +397 val_397 2010-08-15 12 +397 val_397 2010-08-15 12 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 11 +399 val_399 2010-08-15 12 +399 val_399 2010-08-15 12 +400 val_400 2010-08-15 11 +400 val_400 2010-08-15 12 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 11 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +401 val_401 2010-08-15 12 +402 val_402 2010-08-15 11 +402 val_402 2010-08-15 12 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 11 +403 val_403 2010-08-15 12 +403 val_403 2010-08-15 12 +403 val_403 2010-08-15 12 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 11 +404 val_404 2010-08-15 12 +404 val_404 2010-08-15 12 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 11 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +406 val_406 2010-08-15 12 +407 val_407 2010-08-15 11 +407 val_407 2010-08-15 12 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 11 +409 val_409 2010-08-15 12 +409 val_409 2010-08-15 12 +409 val_409 2010-08-15 12 +411 val_411 2010-08-15 11 +411 val_411 2010-08-15 12 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 11 +413 val_413 2010-08-15 12 +413 val_413 2010-08-15 12 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 11 +414 val_414 2010-08-15 12 +414 val_414 2010-08-15 12 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 11 +417 val_417 2010-08-15 12 +417 val_417 2010-08-15 12 +417 val_417 2010-08-15 12 +418 val_418 2010-08-15 11 +418 val_418 2010-08-15 12 +419 val_419 2010-08-15 11 +419 val_419 2010-08-15 12 +421 val_421 2010-08-15 11 +421 val_421 2010-08-15 12 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 11 +424 val_424 2010-08-15 12 +424 val_424 2010-08-15 12 +427 val_427 2010-08-15 11 +427 val_427 2010-08-15 12 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 11 +429 val_429 2010-08-15 12 +429 val_429 2010-08-15 12 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 11 +430 val_430 2010-08-15 12 +430 val_430 2010-08-15 12 +430 val_430 2010-08-15 12 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 11 +431 val_431 2010-08-15 12 +431 val_431 2010-08-15 12 +431 val_431 2010-08-15 12 +432 val_432 2010-08-15 11 +432 val_432 2010-08-15 12 +435 val_435 2010-08-15 11 +435 val_435 2010-08-15 12 +436 val_436 2010-08-15 11 +436 val_436 2010-08-15 12 +437 val_437 2010-08-15 11 +437 val_437 2010-08-15 12 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 11 +438 val_438 2010-08-15 12 +438 val_438 2010-08-15 12 +438 val_438 2010-08-15 12 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 11 +439 val_439 2010-08-15 12 +439 val_439 2010-08-15 12 +443 val_443 2010-08-15 11 +443 val_443 2010-08-15 12 +444 val_444 2010-08-15 11 +444 val_444 2010-08-15 12 +446 val_446 2010-08-15 11 +446 val_446 2010-08-15 12 +448 val_448 2010-08-15 11 +448 val_448 2010-08-15 12 +449 val_449 2010-08-15 11 +449 val_449 2010-08-15 12 +452 val_452 2010-08-15 11 +452 val_452 2010-08-15 12 +453 val_453 2010-08-15 11 +453 val_453 2010-08-15 12 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 11 +454 val_454 2010-08-15 12 +454 val_454 2010-08-15 12 +454 val_454 2010-08-15 12 +455 val_455 2010-08-15 11 +455 val_455 2010-08-15 12 +457 val_457 2010-08-15 11 +457 val_457 2010-08-15 12 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 11 +458 val_458 2010-08-15 12 +458 val_458 2010-08-15 12 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 11 +459 val_459 2010-08-15 12 +459 val_459 2010-08-15 12 +460 val_460 2010-08-15 11 +460 val_460 2010-08-15 12 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 11 +462 val_462 2010-08-15 12 +462 val_462 2010-08-15 12 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 11 +463 val_463 2010-08-15 12 +463 val_463 2010-08-15 12 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 11 +466 val_466 2010-08-15 12 +466 val_466 2010-08-15 12 +466 val_466 2010-08-15 12 +467 val_467 2010-08-15 11 +467 val_467 2010-08-15 12 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 11 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +468 val_468 2010-08-15 12 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 11 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +469 val_469 2010-08-15 12 +470 val_470 2010-08-15 11 +470 val_470 2010-08-15 12 +472 val_472 2010-08-15 11 +472 val_472 2010-08-15 12 +475 val_475 2010-08-15 11 +475 val_475 2010-08-15 12 +477 val_477 2010-08-15 11 +477 val_477 2010-08-15 12 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 11 +478 val_478 2010-08-15 12 +478 val_478 2010-08-15 12 +479 val_479 2010-08-15 11 +479 val_479 2010-08-15 12 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 11 +480 val_480 2010-08-15 12 +480 val_480 2010-08-15 12 +480 val_480 2010-08-15 12 +481 val_481 2010-08-15 11 +481 val_481 2010-08-15 12 +482 val_482 2010-08-15 11 +482 val_482 2010-08-15 12 +483 val_483 2010-08-15 11 +483 val_483 2010-08-15 12 +484 val_484 2010-08-15 11 +484 val_484 2010-08-15 12 +485 val_485 2010-08-15 11 +485 val_485 2010-08-15 12 +487 val_487 2010-08-15 11 +487 val_487 2010-08-15 12 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 11 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +489 val_489 2010-08-15 12 +490 val_490 2010-08-15 11 +490 val_490 2010-08-15 12 +491 val_491 2010-08-15 11 +491 val_491 2010-08-15 12 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 11 +492 val_492 2010-08-15 12 +492 val_492 2010-08-15 12 +493 val_493 2010-08-15 11 +493 val_493 2010-08-15 12 +494 val_494 2010-08-15 11 +494 val_494 2010-08-15 12 +495 val_495 2010-08-15 11 +495 val_495 2010-08-15 12 +496 val_496 2010-08-15 11 +496 val_496 2010-08-15 12 +497 val_497 2010-08-15 11 +497 val_497 2010-08-15 12 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 11 +498 val_498 2010-08-15 12 +498 val_498 2010-08-15 12 +498 val_498 2010-08-15 12 diff --git a/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 b/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 new file mode 100644 index 0000000000000..e69de29bb2d1d From e7d8ae444fead27fe85a879f2f7a4cfdd8c47b16 Mon Sep 17 00:00:00 2001 From: DEBORAH SIEGEL Date: Mon, 2 Mar 2015 10:15:32 -0800 Subject: [PATCH 311/817] aggregateMessages example in graphX doc Examples illustrating difference between legacy mapReduceTriplets usage and aggregateMessages usage has type issues on the reduce for both operators. Being just an example- changed example to reduce the message String by concatenation. Although non-optimal for performance. Author: DEBORAH SIEGEL Closes #4853 from d3borah/master and squashes the following commits: db54173 [DEBORAH SIEGEL] fixed aggregateMessages example in graphX doc --- docs/graphx-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 28bdf81ca0ca5..c601d793a2e9a 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -663,7 +663,7 @@ val graph: Graph[Int, Float] = ... def msgFun(triplet: Triplet[Int, Float]): Iterator[(Int, String)] = { Iterator((triplet.dstId, "Hi")) } -def reduceFun(a: Int, b: Int): Int = a + b +def reduceFun(a: String, b: String): String = a + " " + b val result = graph.mapReduceTriplets[String](msgFun, reduceFun) {% endhighlight %} @@ -674,7 +674,7 @@ val graph: Graph[Int, Float] = ... def msgFun(triplet: EdgeContext[Int, Float, String]) { triplet.sendToDst("Hi") } -def reduceFun(a: Int, b: Int): Int = a + b +def reduceFun(a: String, b: String): String = a + " " + b val result = graph.aggregateMessages[String](msgFun, reduceFun) {% endhighlight %} From af2effdd7b54316af0c02e781911acfb148b962b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 2 Mar 2015 10:17:24 -0800 Subject: [PATCH 312/817] [SPARK-6080] [PySpark] correct LogisticRegressionWithLBFGS regType parameter for pyspark Currently LogisticRegressionWithLBFGS in python/pyspark/mllib/classification.py will invoke callMLlibFunc with a wrong "regType" parameter. It was assigned to "str(regType)" which translate None(Python) to "None"(Java/Scala). The right way should be translate None(Python) to null(Java/Scala) just as what we did at LogisticRegressionWithSGD. Author: Yanbo Liang Closes #4831 from yanboliang/pyspark_classification and squashes the following commits: 12db65a [Yanbo Liang] correct LogisticRegressionWithLBFGS regType parameter for pyspark --- python/pyspark/mllib/classification.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 00e2e76711e84..e4765173709e8 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -207,7 +207,7 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType """ def train(rdd, i): return callMLlibFunc("trainLogisticRegressionModelWithLBFGS", rdd, int(iterations), i, - float(regParam), str(regType), bool(intercept), int(corrections), + float(regParam), regType, bool(intercept), int(corrections), float(tolerance)) return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) From d9a8bae77826a0cc77df29d85883e914d0f0b4f3 Mon Sep 17 00:00:00 2001 From: Paul Power Date: Mon, 2 Mar 2015 13:08:47 -0800 Subject: [PATCH 313/817] [DOCS] Refactored Dataframe join comment to use correct parameter ordering The API signatire for join requires the JoinType to be the third parameter. The code examples provided for join show JoinType being provided as the 2nd parater resuling in errors (i.e. "df1.join(df2, "outer", $"df1Key" === $"df2Key") ). The correct sample code is df1.join(df2, $"df1Key" === $"df2Key", "outer") Author: Paul Power Closes #4847 from peerside/master and squashes the following commits: ebc1efa [Paul Power] Merge pull request #1 from peerside/peerside-patch-1 e353340 [Paul Power] Updated comments use correct sample code for Dataframe joins --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 060ab5e9a0cfa..f3aac0826ad23 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 @@ -337,11 +337,11 @@ class DataFrame protected[sql]( * {{{ * // Scala: * import org.apache.spark.sql.functions._ - * df1.join(df2, "outer", $"df1Key" === $"df2Key") + * df1.join(df2, $"df1Key" === $"df2Key", "outer") * * // Java: * import static org.apache.spark.sql.functions.*; - * df1.join(df2, "outer", col("df1Key") === col("df2Key")); + * df1.join(df2, col("df1Key").equalTo(col("df2Key")), "outer"); * }}} * * @param right Right side of the join. From 0b472f60cdf4984ab5e28e6dbf12615e8997a448 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 2 Mar 2015 21:10:08 +0000 Subject: [PATCH 314/817] SPARK-5390 [DOCS] Encourage users to post on Stack Overflow in Community Docs Point "Community" to main Spark Community page; mention SO tag apache-spark. Separately, the Apache site can be updated to mention, under Mailing Lists: "StackOverflow also has an apache-spark tag for Spark Q&A." or similar. Author: Sean Owen Closes #4843 from srowen/SPARK-5390 and squashes the following commits: 3508ac6 [Sean Owen] Point "Community" to main Spark Community page; mention SO tag apache-spark --- docs/index.md | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/docs/index.md b/docs/index.md index e006be640e582..0986398e6f744 100644 --- a/docs/index.md +++ b/docs/index.md @@ -115,6 +115,8 @@ options for deployment: * [Spark Homepage](http://spark.apache.org) * [Spark Wiki](https://cwiki.apache.org/confluence/display/SPARK) +* [Spark Community](http://spark.apache.org/community.html) resources, including local meetups +* [StackOverflow tag `apache-spark`](http://stackoverflow.com/questions/tagged/apache-spark) * [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and exercises about Spark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), @@ -123,11 +125,3 @@ options for deployment: * [Code Examples](http://spark.apache.org/examples.html): more are also available in the `examples` subfolder of Spark ([Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples), [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples), [Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python)) - -# Community - -To get help using Spark or keep up with Spark development, sign up for the [user mailing list](http://spark.apache.org/mailing-lists.html). - -If you're in the San Francisco Bay Area, there's a regular [Spark meetup](http://www.meetup.com/spark-users/) every few weeks. Come by to meet the developers and other users. - -Finally, if you'd like to contribute code to Spark, read [how to contribute](contributing-to-spark.html). From 3f9def81170c24f24f4a6b7ca7905de4f75e11e0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 2 Mar 2015 13:11:17 -0800 Subject: [PATCH 315/817] [Minor] Fix doc typo for describing primitiveTerm effectiveness condition It should be `true` instead of `false`? Author: Liang-Chi Hsieh Closes #4762 from viirya/doc_fix and squashes the following commits: 2e37482 [Liang-Chi Hsieh] Fix doc. --- .../spark/sql/catalyst/expressions/codegen/CodeGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1f80d84b744a1..c347780924caf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -121,7 +121,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin * @param nullTerm A term that holds a boolean value representing whether the expression evaluated * to null. * @param primitiveTerm A term for a possible primitive value of the result of the evaluation. Not - * valid if `nullTerm` is set to `false`. + * valid if `nullTerm` is set to `true`. * @param objectTerm A possibly boxed version of the result of evaluating this expression. */ protected case class EvaluatedExpression( From 582e5a24c55e8c876733537c9910001affc8b29b Mon Sep 17 00:00:00 2001 From: q00251598 Date: Mon, 2 Mar 2015 13:16:29 -0800 Subject: [PATCH 316/817] [SPARK-6040][SQL] Fix the percent bug in tablesample HiveQL expression like `select count(1) from src tablesample(1 percent);` means take 1% sample to select. But it means 100% in the current version of the Spark. Author: q00251598 Closes #4789 from watermen/SPARK-6040 and squashes the following commits: 2453ebe [q00251598] check and adjust the fraction. --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 11 ++++++++++- .../spark/sql/hive/execution/HiveQuerySuite.scala | 1 + 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 98263f602e9ec..ced99cd082614 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.sources.DescribeCommand import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ +import org.apache.spark.util.random.RandomSampler /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -850,7 +851,15 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_TABLESPLITSAMPLE", Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) => - Sample(fraction.toDouble, withReplacement = false, (math.random * 1000).toInt, relation) + // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling + // function takes X PERCENT as the input and the range of X is [0, 100], we need to + // adjust the fraction. + require( + fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) + && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be on interval [0, 100]") + Sample(fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt, + relation) case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index bb0a67dc03e1d..c0d21bc9a89da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -467,6 +467,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("sampling") { sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") + sql("SELECT * FROM src TABLESAMPLE(100 PERCENT) s") } test("DataFrame toString") { From 6b348d90f475440c285a4b636134ffa9351580b9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 2 Mar 2015 16:41:43 -0600 Subject: [PATCH 317/817] [SPARK-6050] [yarn] Relax matching of vcore count in received containers. Some YARN configurations return a vcore count for allocated containers that does not match the requested resource. That means Spark would always ignore those containers. So relax the the matching of the vcore count to allow the Spark jobs to run. Author: Marcelo Vanzin Closes #4818 from vanzin/SPARK-6050 and squashes the following commits: 991c803 [Marcelo Vanzin] Remove config option, standardize on legacy behavior (no vcore matching). 8c9c346 [Marcelo Vanzin] Restrict lax matching to vcores only. 3359692 [Marcelo Vanzin] [SPARK-6050] [yarn] Add config option to do lax resource matching. --- .../org/apache/spark/deploy/yarn/YarnAllocator.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 12c62a659d799..55bfbcd9cb84b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -290,8 +290,14 @@ private[yarn] class YarnAllocator( location: String, containersToUse: ArrayBuffer[Container], remaining: ArrayBuffer[Container]): Unit = { + // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the + // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested + // memory, but use the asked vcore count for matching, effectively disabling matching on vcore + // count. + val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, + resource.getVirtualCores) val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, - allocatedContainer.getResource) + matchingResource) // Match the allocation to a request if (!matchingRequests.isEmpty) { @@ -318,7 +324,7 @@ private[yarn] class YarnAllocator( assert(container.getResource.getMemory >= resource.getMemory) logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) - executorIdToContainer(executorId) = container + executorIdToContainer(executorId) = container val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]) From 26c1c56dea5d4160913bb65bb743aeb63fee3240 Mon Sep 17 00:00:00 2001 From: guliangliang Date: Mon, 2 Mar 2015 15:33:23 -0800 Subject: [PATCH 318/817] [SPARK-5522] Accelerate the Histroty Server start When starting the history server, all the log files will be fetched and parsed in order to get the applications' meta data e.g. App Name, Start Time, Duration, etc. In our production cluster, there exist 2600 log files (160G) in HDFS and it costs 3 hours to restart the history server, which is a little bit too long for us. It would be better, if the history server can show logs with missing information during start-up and fill the missing information after fetching and parsing a log file. Author: guliangliang Closes #4525 from marsishandsome/Spark5522 and squashes the following commits: a865c11 [guliangliang] fix bug2 4340c2b [guliangliang] fix bug af92a5a [guliangliang] [SPARK-5522] Accelerate the Histroty Server start --- .../deploy/history/FsHistoryProvider.scala | 115 +++++++++++------- 1 file changed, 74 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 3e3d6ff29faf0..c5fab1d440250 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -18,22 +18,23 @@ package org.apache.spark.deploy.history import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream} -import java.util.concurrent.{Executors, TimeUnit} +import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable import scala.concurrent.duration.Duration import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.hadoop.fs.{FileStatus, Path} +import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.permission.AccessControlException - -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf} + /** * A class that provides application history from event logs stored in the file system. @@ -98,6 +99,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } + /** + * An Executor to fetch and parse log files. + */ + private val replayExecutor: ExecutorService = { + if (!conf.contains("spark.testing")) { + Executors.newSingleThreadExecutor(Utils.namedThreadFactory("log-replay-executor")) + } else { + MoreExecutors.sameThreadExecutor() + } + } + initialize() private def initialize(): Unit = { @@ -171,10 +183,10 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private[history] def checkForLogs(): Unit = { try { - var newLastModifiedTime = lastModifiedTime val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) .getOrElse(Seq[FileStatus]()) - val logInfos = statusList + var newLastModifiedTime = lastModifiedTime + val logInfos: Seq[FileStatus] = statusList .filter { entry => try { getModificationTime(entry).map { time => @@ -189,48 +201,69 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis false } } - .flatMap { entry => - try { - Some(replay(entry, new ReplayListenerBus())) - } catch { - case e: Exception => - logError(s"Failed to load application log data from $entry.", e) - None - } - } - .sortWith(compareAppInfo) + .flatMap { entry => Some(entry) } + .sortWith { case (entry1, entry2) => + val mod1 = getModificationTime(entry1).getOrElse(-1L) + val mod2 = getModificationTime(entry2).getOrElse(-1L) + mod1 >= mod2 + } + + logInfos.sliding(20, 20).foreach { batch => + replayExecutor.submit(new Runnable { + override def run(): Unit = mergeApplicationListing(batch) + }) + } lastModifiedTime = newLastModifiedTime + } catch { + case e: Exception => logError("Exception in checking for event log updates", e) + } + } - // When there are new logs, merge the new list with the existing one, maintaining - // the expected ordering (descending end time). Maintaining the order is important - // to avoid having to sort the list every time there is a request for the log list. - if (!logInfos.isEmpty) { - val newApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - def addIfAbsent(info: FsApplicationHistoryInfo) = { - if (!newApps.contains(info.id) || - newApps(info.id).logPath.endsWith(EventLoggingListener.IN_PROGRESS) && - !info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - newApps += (info.id -> info) - } + /** + * Replay the log files in the list and merge the list of old applications with new ones + */ + private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { + val bus = new ReplayListenerBus() + val newApps = logs.flatMap { fileStatus => + try { + val res = replay(fileStatus, bus) + logInfo(s"Application log ${res.logPath} loaded successfully.") + Some(res) + } catch { + case e: Exception => + logError( + s"Exception encountered when attempting to load application log ${fileStatus.getPath}") + None + } + }.toSeq.sortWith(compareAppInfo) + + // When there are new logs, merge the new list with the existing one, maintaining + // the expected ordering (descending end time). Maintaining the order is important + // to avoid having to sort the list every time there is a request for the log list. + if (newApps.nonEmpty) { + val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { + if (!mergedApps.contains(info.id) || + mergedApps(info.id).logPath.endsWith(EventLoggingListener.IN_PROGRESS) && + !info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { + mergedApps += (info.id -> info) } + } - val newIterator = logInfos.iterator.buffered - val oldIterator = applications.values.iterator.buffered - while (newIterator.hasNext && oldIterator.hasNext) { - if (compareAppInfo(newIterator.head, oldIterator.head)) { - addIfAbsent(newIterator.next) - } else { - addIfAbsent(oldIterator.next) - } + val newIterator = newApps.iterator.buffered + val oldIterator = applications.values.iterator.buffered + while (newIterator.hasNext && oldIterator.hasNext) { + if (compareAppInfo(newIterator.head, oldIterator.head)) { + addIfAbsent(newIterator.next()) + } else { + addIfAbsent(oldIterator.next()) } - newIterator.foreach(addIfAbsent) - oldIterator.foreach(addIfAbsent) - - applications = newApps } - } catch { - case e: Exception => logError("Exception in checking for event log updates", e) + newIterator.foreach(addIfAbsent) + oldIterator.foreach(addIfAbsent) + + applications = mergedApps } } From 8223ce6a81e4cc9fdf816892365fcdff4006c35e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 2 Mar 2015 16:10:54 -0800 Subject: [PATCH 319/817] [SPARK-6114][SQL] Avoid metastore conversions before plan is resolved Author: Michael Armbrust Closes #4855 from marmbrus/explodeBug and squashes the following commits: a712249 [Michael Armbrust] [SPARK-6114][SQL] Avoid metastore conversions before plan is resolved --- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 4 ++++ .../spark/sql/hive/execution/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 14 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d3ad364328265..74b4e767caed2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -444,6 +444,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ object ParquetConversions extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { + if (!plan.resolved) { + return plan + } + // Collects all `MetastoreRelation`s which should be replaced val toBeReplaced = plan.collect { // Write path diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index f2bc73bf3bdf9..22ea19bd82f86 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -31,6 +31,9 @@ case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) case class Nested3(f3: Int) +case class NestedArray2(b: Seq[Int]) +case class NestedArray1(a: NestedArray2) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -38,6 +41,13 @@ case class Nested3(f3: Int) */ class SQLQuerySuite extends QueryTest { + test("explode nested Field") { + Seq(NestedArray1(NestedArray2(Seq(1,2,3)))).toDF.registerTempTable("nestedArray") + checkAnswer( + sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), + Row(1) :: Row(2) :: Row(3) :: Nil) + } + test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { checkAnswer( sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"), From 1a49496b4a9df40c74739fc0fb8a21c88a477075 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 2 Mar 2015 16:18:00 -0800 Subject: [PATCH 320/817] [SPARK-6082] [SQL] Provides better error message for malformed rows when caching tables Constructs like Hive `TRANSFORM` may generate malformed rows (via badly authored external scripts for example). I'm a bit hesitant to have this feature, since it introduces per-tuple cost when caching tables. However, considering caching tables is usually a one-time cost, this is probably worth having. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4842) Author: Cheng Lian Closes #4842 from liancheng/spark-6082 and squashes the following commits: b05dbff [Cheng Lian] Provides better error message for malformed rows when caching tables --- .../sql/columnar/InMemoryColumnarTableScan.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 11d5943fb427f..8944a32bc3887 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -119,6 +119,17 @@ private[sql] case class InMemoryRelation( var rowCount = 0 while (rowIterator.hasNext && rowCount < batchSize) { val row = rowIterator.next() + + // Added for SPARK-6082. This assertion can be useful for scenarios when something + // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM + // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat + // hard to decipher. + assert( + row.size == columnBuilders.size, + s"""Row column number mismatch, expected ${output.size} columns, but got ${row.size}. + |Row content: $row + """.stripMargin) + var i = 0 while (i < row.length) { columnBuilders(i).appendFrom(row, i) From 6776cb33ea691f7843b956b3e80979282967e826 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 2 Mar 2015 16:34:32 -0800 Subject: [PATCH 321/817] [SPARK-6066] Make event log format easier to parse Some users have reported difficulty in parsing the new event log format. Since we embed the metadata in the beginning of the file, when we compress the event log we need to skip the metadata because we need that information to parse the log later. This means we'll end up with a partially compressed file if event logging compression is turned on. The old format looks like: ``` sparkVersion = 1.3.0 compressionCodec = org.apache.spark.io.LZFCompressionCodec === LOG_HEADER_END === // actual events, could be compressed bytes ``` The new format in this patch puts the compression codec in the log file name instead. It also removes the metadata header altogether along with the Spark version, which was not needed. The new file name looks something like: ``` app_without_compression app_123.lzf app_456.snappy ``` I tested this with and without compression, using different compression codecs and event logging directories. I verified that both the `Master` and the `HistoryServer` can render both compressed and uncompressed logs as before. Author: Andrew Or Closes #4821 from andrewor14/event-log-format and squashes the following commits: 8511141 [Andrew Or] Fix test 654883d [Andrew Or] Add back metadata with Spark version 7f537cd [Andrew Or] Address review feedback 7d6aa61 [Andrew Or] Make codec an extension 59abee9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format 27c9a6c [Andrew Or] Address review feedback 519e51a [Andrew Or] Address review feedback ef69276 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format 88a091d [Andrew Or] Add tests for new format and file name f32d8d2 [Andrew Or] Fix tests 8db5a06 [Andrew Or] Embed metadata in the event log file name instead --- .../scala/org/apache/spark/SparkContext.scala | 9 + .../spark/deploy/ApplicationDescription.scala | 10 +- .../deploy/history/FsHistoryProvider.scala | 22 +-- .../apache/spark/deploy/master/Master.scala | 8 +- .../apache/spark/io/CompressionCodec.scala | 21 ++- .../scheduler/EventLoggingListener.scala | 162 +++++++----------- .../spark/scheduler/ReplayListenerBus.scala | 3 +- .../spark/scheduler/SparkListener.scala | 5 + .../spark/scheduler/SparkListenerBus.scala | 1 + .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 14 ++ .../history/FsHistoryProviderSuite.scala | 69 +++++--- .../scheduler/EventLoggingListenerSuite.scala | 62 ++++--- .../spark/scheduler/ReplayListenerSuite.scala | 13 +- 14 files changed, 212 insertions(+), 189 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3cd0c218a36fd..e231e8369dbac 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -51,6 +51,7 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.executor.TriggerThreadDump import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} +import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -233,6 +234,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli None } } + private[spark] val eventLogCodec: Option[String] = { + val compress = conf.getBoolean("spark.eventLog.compress", false) + if (compress && isEventLogEnabled) { + Some(CompressionCodec.getCodecName(conf)).map(CompressionCodec.getShortName) + } else { + None + } + } // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index ae55b4ff40b74..3d0d68de8f495 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -23,7 +23,9 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, var appUiUrl: String, - val eventLogDir: Option[String] = None) + val eventLogDir: Option[String] = None, + // short name of compression codec used when writing event logs, if any (e.g. lzf) + val eventLogCodec: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") @@ -34,8 +36,10 @@ private[spark] class ApplicationDescription( memoryPerSlave: Int = memoryPerSlave, command: Command = command, appUiUrl: String = appUiUrl, - eventLogDir: Option[String] = eventLogDir): ApplicationDescription = - new ApplicationDescription(name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir) + eventLogDir: Option[String] = eventLogDir, + eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = + new ApplicationDescription( + name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index c5fab1d440250..16d88c17d1a76 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -83,8 +83,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Constants used to parse Spark 1.0.0 log directories. private[history] val LOG_PREFIX = "EVENT_LOG_" - private[history] val SPARK_VERSION_PREFIX = "SPARK_VERSION_" - private[history] val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + private[history] val SPARK_VERSION_PREFIX = EventLoggingListener.SPARK_VERSION_KEY + "_" + private[history] val COMPRESSION_CODEC_PREFIX = EventLoggingListener.COMPRESSION_CODEC_KEY + "_" private[history] val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" /** @@ -324,7 +324,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { val logPath = eventLog.getPath() logInfo(s"Replaying log path: $logPath") - val (logInput, sparkVersion) = + val logInput = if (isLegacyLogDirectory(eventLog)) { openLegacyEventLog(logPath) } else { @@ -333,7 +333,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val appListener = new ApplicationEventListener bus.addListener(appListener) - bus.replay(logInput, sparkVersion, logPath.toString) + bus.replay(logInput, logPath.toString) new FsApplicationHistoryInfo( logPath.getName(), appListener.appId.getOrElse(logPath.getName()), @@ -353,30 +353,24 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * log file (along with other metadata files), which is the case for directories generated by * the code in previous releases. * - * @return 2-tuple of (input stream of the events, version of Spark which wrote the log) + * @return input stream that holds one JSON record per line. */ - private[history] def openLegacyEventLog(dir: Path): (InputStream, String) = { + private[history] def openLegacyEventLog(dir: Path): InputStream = { val children = fs.listStatus(dir) var eventLogPath: Path = null var codecName: Option[String] = None - var sparkVersion: String = null children.foreach { child => child.getPath().getName() match { case name if name.startsWith(LOG_PREFIX) => eventLogPath = child.getPath() - case codec if codec.startsWith(COMPRESSION_CODEC_PREFIX) => codecName = Some(codec.substring(COMPRESSION_CODEC_PREFIX.length())) - - case version if version.startsWith(SPARK_VERSION_PREFIX) => - sparkVersion = version.substring(SPARK_VERSION_PREFIX.length()) - case _ => } } - if (eventLogPath == null || sparkVersion == null) { + if (eventLogPath == null) { throw new IllegalArgumentException(s"$dir is not a Spark application log directory.") } @@ -388,7 +382,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } val in = new BufferedInputStream(fs.open(eventLogPath)) - (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) + codec.map(_.compressedInputStream(in)).getOrElse(in) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8cc6ec1e8192c..148485cc11863 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -737,13 +737,13 @@ private[spark] class Master( val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { val eventLogFile = app.desc.eventLogDir - .map { dir => EventLoggingListener.getLogPath(dir, app.id) } + .map { dir => EventLoggingListener.getLogPath(dir, app.id, app.desc.eventLogCodec) } .getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath return false } - + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { @@ -756,12 +756,12 @@ private[spark] class Master( return false } - val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) + val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") try { - replayBus.replay(logInput, sparkVersion, eventLogFile) + replayBus.replay(logInput, eventLogFile) } finally { logInput.close() } diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index f856890d279f4..0709b6d689e86 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -26,7 +26,6 @@ import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils -import org.apache.spark.Logging /** * :: DeveloperApi :: @@ -53,8 +52,12 @@ private[spark] object CompressionCodec { "lzf" -> classOf[LZFCompressionCodec].getName, "snappy" -> classOf[SnappyCompressionCodec].getName) + def getCodecName(conf: SparkConf): String = { + conf.get(configKey, DEFAULT_COMPRESSION_CODEC) + } + def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC)) + createCodec(conf, getCodecName(conf)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { @@ -71,6 +74,20 @@ private[spark] object CompressionCodec { s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC")) } + /** + * Return the short version of the given codec name. + * If it is already a short name, just return it. + */ + def getShortName(codecName: String): String = { + if (shortCompressionCodecNames.contains(codecName)) { + codecName + } else { + shortCompressionCodecNames + .collectFirst { case (k, v) if v == codecName => k } + .getOrElse { throw new IllegalArgumentException(s"No short name for codec $codecName.") } + } + } + val FALLBACK_COMPRESSION_CODEC = "lzf" val DEFAULT_COMPRESSION_CODEC = "snappy" val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 30075c172bdb1..2091a9fe8d0d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -62,6 +62,15 @@ private[spark] class EventLoggingListener( private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf) + private val compressionCodec = + if (shouldCompress) { + Some(CompressionCodec.createCodec(sparkConf)) + } else { + None + } + private val compressionCodecName = compressionCodec.map { c => + CompressionCodec.getShortName(c.getClass.getName) + } // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -80,7 +89,7 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId) + private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName) /** * Creates the log file in the configured log directory. @@ -111,19 +120,19 @@ private[spark] class EventLoggingListener( hadoopDataStream.get } - val compressionCodec = - if (shouldCompress) { - Some(CompressionCodec.createCodec(sparkConf)) - } else { - None - } - - fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) - val logStream = initEventLog(new BufferedOutputStream(dstream, outputBufferSize), - compressionCodec) - writer = Some(new PrintWriter(logStream)) + try { + val cstream = compressionCodec.map(_.compressedOutputStream(dstream)).getOrElse(dstream) + val bstream = new BufferedOutputStream(cstream, outputBufferSize) - logInfo("Logging events to %s".format(logPath)) + EventLoggingListener.initEventLog(bstream) + fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) + writer = Some(new PrintWriter(bstream)) + logInfo("Logging events to %s".format(logPath)) + } catch { + case e: Exception => + dstream.close() + throw e + } } /** Log the event as JSON. */ @@ -201,77 +210,57 @@ private[spark] object EventLoggingListener extends Logging { // Suffix applied to the names of files still being written by applications. val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" + val SPARK_VERSION_KEY = "SPARK_VERSION" + val COMPRESSION_CODEC_KEY = "COMPRESSION_CODEC" private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - // Marker for the end of header data in a log file. After this marker, log data, potentially - // compressed, will be found. - private val HEADER_END_MARKER = "=== LOG_HEADER_END ===" - - // To avoid corrupted files causing the heap to fill up. Value is arbitrary. - private val MAX_HEADER_LINE_LENGTH = 4096 - // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] /** - * Write metadata about the event log to the given stream. - * - * The header is a serialized version of a map, except it does not use Java serialization to - * avoid incompatibilities between different JDKs. It writes one map entry per line, in - * "key=value" format. + * Write metadata about an event log to the given stream. + * The metadata is encoded in the first line of the event log as JSON. * - * The very last entry in the header is the `HEADER_END_MARKER` marker, so that the parsing code - * can know when to stop. - * - * The format needs to be kept in sync with the openEventLog() method below. Also, it cannot - * change in new Spark versions without some other way of detecting the change (like some - * metadata encoded in the file name). - * - * @param logStream Raw output stream to the even log file. - * @param compressionCodec Optional compression codec to use. - * @return A stream where to write event log data. This may be a wrapper around the original - * stream (for example, when compression is enabled). + * @param logStream Raw output stream to the event log file. */ - def initEventLog( - logStream: OutputStream, - compressionCodec: Option[CompressionCodec]): OutputStream = { - val meta = mutable.HashMap(("version" -> SPARK_VERSION)) - compressionCodec.foreach { codec => - meta += ("compressionCodec" -> codec.getClass().getName()) - } - - def write(entry: String) = { - val bytes = entry.getBytes(Charsets.UTF_8) - if (bytes.length > MAX_HEADER_LINE_LENGTH) { - throw new IOException(s"Header entry too long: ${entry}") - } - logStream.write(bytes, 0, bytes.length) - } - - meta.foreach { case (k, v) => write(s"$k=$v\n") } - write(s"$HEADER_END_MARKER\n") - compressionCodec.map(_.compressedOutputStream(logStream)).getOrElse(logStream) + def initEventLog(logStream: OutputStream): Unit = { + val metadata = SparkListenerLogStart(SPARK_VERSION) + val metadataJson = compact(JsonProtocol.logStartToJson(metadata)) + "\n" + logStream.write(metadataJson.getBytes(Charsets.UTF_8)) } /** * Return a file-system-safe path to the log file for the given application. * + * Note that because we currently only create a single log file for each application, + * we must encode all the information needed to parse this event log in the file name + * instead of within the file itself. Otherwise, if the file is compressed, for instance, + * we won't know which codec to use to decompress the metadata needed to open the file in + * the first place. + * * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. + * @param compressionCodecName Name to identify the codec used to compress the contents + * of the log, or None if compression is not enabled. * @return A path which consists of file-system-safe characters. */ - def getLogPath(logBaseDir: String, appId: String): String = { - val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase - Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + def getLogPath( + logBaseDir: String, + appId: String, + compressionCodecName: Option[String] = None): String = { + val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + // e.g. app_123, app_123.lzf + val logName = sanitizedAppId + compressionCodecName.map { "." + _ }.getOrElse("") + Utils.resolveURI(logBaseDir).toString.stripSuffix("/") + "/" + logName } /** - * Opens an event log file and returns an input stream to the event data. + * Opens an event log file and returns an input stream that contains the event data. * - * @return 2-tuple (event input stream, Spark version of event data) + * @return input stream that holds one JSON record per line. */ - def openEventLog(log: Path, fs: FileSystem): (InputStream, String) = { + def openEventLog(log: Path, fs: FileSystem): InputStream = { // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain // IOException when a file does not exist, so try our best to throw a proper exception. if (!fs.exists(log)) { @@ -279,52 +268,17 @@ private[spark] object EventLoggingListener extends Logging { } val in = new BufferedInputStream(fs.open(log)) - // Read a single line from the input stream without buffering. - // We cannot use BufferedReader because we must avoid reading - // beyond the end of the header, after which the content of the - // file may be compressed. - def readLine(): String = { - val bytes = new ByteArrayOutputStream() - var next = in.read() - var count = 0 - while (next != '\n') { - if (next == -1) { - throw new IOException("Unexpected end of file.") - } - bytes.write(next) - count = count + 1 - if (count > MAX_HEADER_LINE_LENGTH) { - throw new IOException("Maximum header line length exceeded.") - } - next = in.read() - } - new String(bytes.toByteArray(), Charsets.UTF_8) + + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logName = log.getName.stripSuffix(IN_PROGRESS) + val codecName: Option[String] = logName.split("\\.").tail.lastOption + val codec = codecName.map { c => + codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) } - // Parse the header metadata in the form of k=v pairs - // This assumes that every line before the header end marker follows this format try { - val meta = new mutable.HashMap[String, String]() - var foundEndMarker = false - while (!foundEndMarker) { - readLine() match { - case HEADER_END_MARKER => - foundEndMarker = true - case entry => - val prop = entry.split("=", 2) - if (prop.length != 2) { - throw new IllegalArgumentException("Invalid metadata in log file.") - } - meta += (prop(0) -> prop(1)) - } - } - - val sparkVersion = meta.get("version").getOrElse( - throw new IllegalArgumentException("Missing Spark version in log metadata.")) - val codec = meta.get("compressionCodec").map { codecName => - codecMap.getOrElseUpdate(codecName, CompressionCodec.createCodec(new SparkConf, codecName)) - } - (codec.map(_.compressedInputStream(in)).getOrElse(in), sparkVersion) + codec.map(_.compressedInputStream(in)).getOrElse(in) } catch { case e: Exception => in.close() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index d9c3a10dc5413..95273c716b3e2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -39,10 +39,9 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * error is thrown by this method. * * @param logData Stream containing event log data. - * @param version Spark version that generated the events. * @param sourceName Filename (or other source identifier) from whence @logData is being read */ - def replay(logData: InputStream, version: String, sourceName: String) { + def replay(logData: InputStream, sourceName: String): Unit = { var currentLine: String = null var lineNumber: Int = 1 try { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index dd28ddb31de1f..52720d48ca67f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -116,6 +116,11 @@ case class SparkListenerApplicationStart(appName: String, appId: Option[String], @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent +/** + * An internal class that describes the metadata of an event log. + * This event is not meant to be posted to listeners downstream. + */ +private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index fe8a19a2c0cb9..61e69ecc08387 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -58,6 +58,7 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => listener.onExecutorRemoved(executorRemoved) + case logStart: SparkListenerLogStart => // ignore event log metadata } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index a0aa555f6244f..ffd4825705755 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -85,7 +85,7 @@ private[spark] class SparkDeploySchedulerBackend( args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, sc.eventLogDir) + appUIAddress, sc.eventLogDir, sc.eventLogCodec) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 8e20864db5673..474f79fb756f6 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -89,6 +89,8 @@ private[spark] object JsonProtocol { executorAddedToJson(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => executorRemovedToJson(executorRemoved) + case logStart: SparkListenerLogStart => + logStartToJson(logStart) // These aren't used, but keeps compiler happy case SparkListenerExecutorMetricsUpdate(_, _) => JNothing } @@ -214,6 +216,11 @@ private[spark] object JsonProtocol { ("Removed Reason" -> executorRemoved.reason) } + def logStartToJson(logStart: SparkListenerLogStart): JValue = { + ("Event" -> Utils.getFormattedClassName(logStart)) ~ + ("Spark Version" -> SPARK_VERSION) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | * -------------------------------------------------------------------- */ @@ -447,6 +454,7 @@ private[spark] object JsonProtocol { val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) val executorAdded = Utils.getFormattedClassName(SparkListenerExecutorAdded) val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) + val logStart = Utils.getFormattedClassName(SparkListenerLogStart) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -464,6 +472,7 @@ private[spark] object JsonProtocol { case `applicationEnd` => applicationEndFromJson(json) case `executorAdded` => executorAddedFromJson(json) case `executorRemoved` => executorRemovedFromJson(json) + case `logStart` => logStartFromJson(json) } } @@ -574,6 +583,11 @@ private[spark] object JsonProtocol { SparkListenerExecutorRemoved(time, executorId, reason) } + def logStartFromJson(json: JValue): SparkListenerLogStart = { + val sparkVersion = (json \ "Spark Version").extract[String] + SparkListenerLogStart(sparkVersion) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | * ---------------------------------------------------------------------- */ diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 85939eaadccc7..e908ba604ebed 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,18 +17,17 @@ package org.apache.spark.deploy.history -import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} +import java.net.URI import scala.io.Source -import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.Matchers import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.scheduler._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -45,18 +44,35 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers Utils.deleteRecursively(testDir) } + /** Create a fake log file using the new log format used in Spark 1.3+ */ + private def newLogFile( + appId: String, + inProgress: Boolean, + codec: Option[String] = None): File = { + val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" + val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId) + val logPath = new URI(logUri).getPath + ip + new File(logPath) + } + test("Parse new and old application logs") { val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val newAppComplete = new File(testDir, "new1") + val newAppComplete = newLogFile("new1", inProgress = false) writeFile(newAppComplete, true, None, SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), - SparkListenerApplicationEnd(4L) + SparkListenerApplicationEnd(5L) ) + // Write a new-style application log. + val newAppCompressedComplete = newLogFile("new1compressed", inProgress = false, Some("lzf")) + writeFile(newAppCompressedComplete, true, None, + SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"), + SparkListenerApplicationEnd(4L)) + // Write an unfinished app, new-style. - val newAppIncomplete = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) + val newAppIncomplete = newLogFile("new2", inProgress = true) writeFile(newAppIncomplete, true, None, SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") ) @@ -89,16 +105,18 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val list = provider.getListing().toSeq list should not be (null) - list.size should be (4) - list.count(e => e.completed) should be (2) + list.size should be (5) + list.count(_.completed) should be (3) - list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 4L, + list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, newAppComplete.lastModified(), "test", true)) - list(1) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, + list(1) should be (ApplicationHistoryInfo(newAppCompressedComplete.getName(), + "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) + list(2) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, oldAppComplete.lastModified(), "test", true)) - list(2) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, + list(3) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, -1L, oldAppIncomplete.lastModified(), "test", false)) - list(3) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, + list(4) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, -1L, newAppIncomplete.lastModified(), "test", false)) // Make sure the UI can be rendered. @@ -127,7 +145,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val logPath = new Path(logDir.getAbsolutePath()) try { - val (logInput, sparkVersion) = provider.openLegacyEventLog(logPath) + val logInput = provider.openLegacyEventLog(logPath) try { Source.fromInputStream(logInput).getLines().toSeq.size should be (2) } finally { @@ -141,12 +159,12 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-3697: ignore directories that cannot be read.") { - val logFile1 = new File(testDir, "new1") + val logFile1 = newLogFile("new1", inProgress = false) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1-1", None, 1L, "test"), SparkListenerApplicationEnd(2L) ) - val logFile2 = new File(testDir, "new2") + val logFile2 = newLogFile("new2", inProgress = false) writeFile(logFile2, true, None, SparkListenerApplicationStart("app1-2", None, 1L, "test"), SparkListenerApplicationEnd(2L) @@ -164,7 +182,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("history file is renamed from inprogress to completed") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) + val logFile1 = newLogFile("app1", inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), SparkListenerApplicationEnd(2L) @@ -174,7 +192,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers appListBeforeRename.size should be (1) appListBeforeRename.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) - logFile1.renameTo(new File(testDir, "app1")) + logFile1.renameTo(newLogFile("app1", inProgress = false)) provider.checkForLogs() val appListAfterRename = provider.getListing() appListAfterRename.size should be (1) @@ -184,7 +202,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers test("SPARK-5582: empty log directory") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) + val logFile1 = newLogFile("app1", inProgress = true) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), SparkListenerApplicationEnd(2L)) @@ -199,14 +217,13 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], events: SparkListenerEvent*) = { - val out = - if (isNewFormat) { - EventLoggingListener.initEventLog(new FileOutputStream(file), codec) - } else { - val fileStream = new FileOutputStream(file) - codec.map(_.compressedOutputStream(fileStream)).getOrElse(fileStream) - } - val writer = new OutputStreamWriter(out, "UTF-8") + val fstream = new FileOutputStream(file) + val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) + val bstream = new BufferedOutputStream(cstream) + if (isNewFormat) { + EventLoggingListener.initEventLog(new FileOutputStream(file)) + } + val writer = new OutputStreamWriter(bstream, "UTF-8") try { events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) } finally { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 437d8693c0b1f..992dde66f982f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} +import java.net.URI import scala.collection.mutable import scala.io.Source @@ -26,7 +27,7 @@ import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -78,7 +79,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin test("Basic event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => - testEventLogging(compressionCodec = Some(codec)) + testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) } } @@ -88,25 +89,35 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin test("End-to-end event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => - testApplicationEventLogging(compressionCodec = Some(codec)) + testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) } } test("Log overwriting") { - val log = new FileOutputStream(new File(testDir, "test")) - log.close() - try { - testEventLogging() - assert(false) - } catch { - case e: IOException => - // Expected, since we haven't enabled log overwrite. - } - + val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, "test") + val logPath = new URI(logUri).getPath + // Create file before writing the event log + new FileOutputStream(new File(logPath)).close() + // Expected IOException, since we haven't enabled log overwrite. + intercept[IOException] { testEventLogging() } // Try again, but enable overwriting. testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) } + test("Event log name") { + // without compression + assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath("/base-dir", "app1")) + // with compression + assert(s"file:/base-dir/app1.lzf" === + EventLoggingListener.getLogPath("/base-dir", "app1", Some("lzf"))) + // illegal characters in app ID + assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === + EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1")) + // illegal characters in app ID with compression + assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === + EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1", Some("lz4"))) + } + /* ----------------- * * Actual test logic * * ----------------- */ @@ -140,15 +151,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin eventLogger.stop() // Verify file contains exactly the two events logged - val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), - fileSystem) + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) try { val lines = readLines(logData) - assert(lines.size === 2) - assert(lines(0).contains("SparkListenerApplicationStart")) - assert(lines(1).contains("SparkListenerApplicationEnd")) - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) - assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + val logStart = SparkListenerLogStart(SPARK_VERSION) + assert(lines.size === 3) + assert(lines(0).contains("SparkListenerLogStart")) + assert(lines(1).contains("SparkListenerApplicationStart")) + assert(lines(2).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(2))) === applicationEnd) } finally { logData.close() } @@ -163,8 +176,10 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get + val eventLogPath = eventLogger.logPath val expectedLogDir = testDir.toURI().toString() - assert(eventLogger.logPath.startsWith(expectedLogDir + "/")) + assert(eventLogPath === EventLoggingListener.getLogPath( + expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -178,8 +193,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin eventExistenceListener.assertAllCallbacksInvoked() // Make sure expected events exist in the log file. - val (logData, version) = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), - fileSystem) + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logStart = SparkListenerLogStart(SPARK_VERSION) val lines = readLines(logData) val eventSet = mutable.Set( SparkListenerApplicationStart, @@ -204,6 +219,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin } } } + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 702c4cb3bdef9..601694f57aad0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -61,7 +61,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, SPARK_VERSION, logFilePath.toString) + replayer.replay(logData, logFilePath.toString) } finally { logData.close() } @@ -115,12 +115,12 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { assert(!eventLog.isDir) // Replay events - val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) + val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventMonster(conf) try { val replayer = new ReplayListenerBus() replayer.addListener(eventMonster) - replayer.replay(logData, version, eventLog.getPath().toString) + replayer.replay(logData, eventLog.getPath().toString) } finally { logData.close() } @@ -150,11 +150,4 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { override def start() { } } - - private def getCompressionCodec(codecName: String) = { - val conf = new SparkConf - conf.set("spark.io.compression.codec", codecName) - CompressionCodec.createCodec(conf) - } - } From 258d154c9f1afdd52dce19f03d81683ee34effac Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 2 Mar 2015 16:36:42 -0800 Subject: [PATCH 322/817] [SPARK-6048] SparkConf should not translate deprecated configs on set There are multiple issues with translating on set outlined in the JIRA. This PR reverts the translation logic added to `SparkConf`. In the future, after the 1.3.0 release we will figure out a way to reorganize the internal structure more elegantly. For now, let's preserve the existing semantics of `SparkConf` since it's a public interface. Unfortunately this means duplicating some code for now, but this is all internal and we can always clean it up later. Author: Andrew Or Closes #4799 from andrewor14/conf-set-translate and squashes the following commits: 11c525b [Andrew Or] Move warning to driver 10e77b5 [Andrew Or] Add documentation for deprecation precedence a369cb1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into conf-set-translate c26a9e3 [Andrew Or] Revert all translate logic in SparkConf fef6c9c [Andrew Or] Restore deprecation logic for spark.executor.userClassPathFirst 94b4dfa [Andrew Or] Translate on get, not set --- .../main/scala/org/apache/spark/SparkConf.scala | 15 +++++++++++---- .../org/apache/spark/executor/Executor.scala | 13 +++++++++---- .../scala/org/apache/spark/SparkConfSuite.scala | 12 ------------ docs/configuration.md | 4 +++- .../org/apache/spark/deploy/yarn/Client.scala | 3 ++- 5 files changed, 25 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 61b34d524a421..2ca19f53d2f07 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -68,7 +68,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (value == null) { throw new NullPointerException("null value for " + key) } - settings.put(translateConfKey(key, warn = true), value) + settings.put(key, value) this } @@ -140,7 +140,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { - settings.putIfAbsent(translateConfKey(key, warn = true), value) + settings.putIfAbsent(key, value) this } @@ -176,7 +176,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { - Option(settings.get(translateConfKey(key))) + Option(settings.get(key)) } /** Get all parameters as a list of pairs */ @@ -229,7 +229,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { def getAppId: String = get("spark.app.id") /** Does the configuration contain a given parameter? */ - def contains(key: String): Boolean = settings.containsKey(translateConfKey(key)) + def contains(key: String): Boolean = settings.containsKey(key) /** Copy this object */ override def clone: SparkConf = { @@ -343,6 +343,13 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } } + + // Warn against the use of deprecated configs + deprecatedConfigs.values.foreach { dc => + if (contains(dc.oldName)) { + dc.warn() + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b684fb704956b..bed0a08d4d515 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -92,6 +92,12 @@ private[spark] class Executor( private val executorActor = env.actorSystem.actorOf( Props(new ExecutorActor(executorId)), "ExecutorActor") + // Whether to load classes in user jars before those in Spark jars + private val userClassPathFirst: Boolean = { + conf.getBoolean("spark.executor.userClassPathFirst", + conf.getBoolean("spark.files.userClassPathFirst", false)) + } + // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager private val urlClassLoader = createClassLoader() @@ -309,7 +315,7 @@ private[spark] class Executor( val urls = userClassPath.toArray ++ currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL } - if (conf.getBoolean("spark.executor.userClassPathFirst", false)) { + if (userClassPathFirst) { new ChildFirstURLClassLoader(urls, currentLoader) } else { new MutableURLClassLoader(urls, currentLoader) @@ -324,14 +330,13 @@ private[spark] class Executor( val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) - val userClassPathFirst: java.lang.Boolean = - conf.getBoolean("spark.executor.userClassPathFirst", false) try { + val _userClassPathFirst: java.lang.Boolean = userClassPathFirst val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] val constructor = klass.getConstructor(classOf[SparkConf], classOf[String], classOf[ClassLoader], classOf[Boolean]) - constructor.newInstance(conf, classUri, parent, userClassPathFirst) + constructor.newInstance(conf, classUri, parent, _userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index ea6b73bc68b34..e08210ae60d17 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -197,18 +197,6 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro serializer.newInstance().serialize(new StringBuffer()) } - test("deprecated config keys") { - val conf = new SparkConf() - .set("spark.files.userClassPathFirst", "true") - .set("spark.yarn.user.classpath.first", "true") - assert(conf.contains("spark.files.userClassPathFirst")) - assert(conf.contains("spark.executor.userClassPathFirst")) - assert(conf.contains("spark.yarn.user.classpath.first")) - assert(conf.getBoolean("spark.files.userClassPathFirst", false)) - assert(conf.getBoolean("spark.executor.userClassPathFirst", false)) - assert(conf.getBoolean("spark.yarn.user.classpath.first", false)) - } - } class Class1 {} diff --git a/docs/configuration.md b/docs/configuration.md index c11787b17eb8c..ae90fe1f8f6b9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -70,7 +70,9 @@ each line consists of a key and a value separated by whitespace. For example: Any values specified as flags or in the properties file will be passed on to the application and merged with those specified through SparkConf. Properties set directly on the SparkConf take highest precedence, then flags passed to `spark-submit` or `spark-shell`, then options -in the `spark-defaults.conf` file. +in the `spark-defaults.conf` file. A few configuration keys have been renamed since earlier +versions of Spark; in such cases, the older key names are still accepted, but take lower +precedence than any instance of the newer key. ## Viewing Spark Properties diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 46d9df93488cb..61f8fc3f5a014 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -955,7 +955,8 @@ object Client extends Logging { if (isDriver) { conf.getBoolean("spark.driver.userClassPathFirst", false) } else { - conf.getBoolean("spark.executor.userClassPathFirst", false) + conf.getBoolean("spark.executor.userClassPathFirst", + conf.getBoolean("spark.files.userClassPathFirst", false)) } } From e3a88d1104ebdb858f0509f56d7bb536037e5f63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?hushan=5B=E8=83=A1=E7=8F=8A=5D?= Date: Mon, 2 Mar 2015 16:53:54 -0800 Subject: [PATCH 323/817] [SPARK-4777][CORE] Some block memory after unrollSafely not count into used memory(memoryStore.entrys or unrollMemory) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Some memory not count into memory used by memoryStore or unrollMemory. Thread A after unrollsafely memory, it will release 40MB unrollMemory(40MB will used by other threads). then ThreadA wait get accountingLock to tryToPut blockA(30MB). before Thread A get accountingLock, blockA memory size is not counting into unrollMemory or memoryStore.currentMemory. IIUC, freeMemory should minus that block memory So, put this release memory into pending, and release it in tryToPut before ensureSpace Author: hushan[胡珊] Closes #3629 from suyanNone/unroll-memory and squashes the following commits: 809cc41 [hushan[胡珊]] Refine 407b2c9 [hushan[胡珊]] Refine according comments 39960d0 [hushan[胡珊]] Refine comments 0fd0213 [hushan[胡珊]] add comments 0fc2bec [hushan[胡珊]] Release pending unroll memory after put block in memoryStore 3a3f2c8 [hushan[胡珊]] Refine blockManagerSuite unroll test 3323c45 [hushan[胡珊]] Refine getOrElse f664317 [hushan[胡珊]] Make sure not add pending in every releaseUnrollMemory call 08b32ba [hushan[胡珊]] Pending unroll memory for this block untill tryToPut --- .../apache/spark/storage/MemoryStore.scala | 52 ++++++++++++++++--- .../spark/storage/BlockManagerSuite.scala | 2 + 2 files changed, 47 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 7f4b6e8bd3683..1be860aea63d0 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -46,6 +46,14 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) // All accesses of this map are assumed to have manually synchronized on `accountingLock` private val unrollMemoryMap = mutable.HashMap[Long, Long]() + // Same as `unrollMemoryMap`, but for pending unroll memory as defined below. + // Pending unroll memory refers to the intermediate memory occupied by a thread + // after the unroll but before the actual putting of the block in the cache. + // This chunk of memory is expected to be released *as soon as* we finish + // caching the corresponding block as opposed to until after the task finishes. + // This is only used if a block is successfully unrolled in its entirety in + // memory (SPARK-4777). + private val pendingUnrollMemoryMap = mutable.HashMap[Long, Long]() /** * The amount of space ensured for unrolling values in memory, shared across all cores. @@ -283,12 +291,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } finally { - // If we return an array, the values returned do not depend on the underlying vector and - // we can immediately free up space for other threads. Otherwise, if we return an iterator, - // we release the memory claimed by this thread later on when the task finishes. + // If we return an array, the values returned will later be cached in `tryToPut`. + // In this case, we should release the memory after we cache the block there. + // Otherwise, if we return an iterator, we release the memory reserved here + // later when the task finishes. if (keepUnrolling) { - val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved - releaseUnrollMemoryForThisThread(amountToRelease) + accountingLock.synchronized { + val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved + releaseUnrollMemoryForThisThread(amountToRelease) + reservePendingUnrollMemoryForThisThread(amountToRelease) + } } } } @@ -353,6 +365,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } + // Release the unroll memory used because we no longer need the underlying Array + releasePendingUnrollMemoryForThisThread() } ResultWithDroppedBlocks(putSuccess, droppedBlocks) } @@ -381,7 +395,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Take into account the amount of memory currently occupied by unrolling blocks - val actualFreeMemory = freeMemory - currentUnrollMemory + // and minus the pending unroll memory for that block on current thread. + val threadId = Thread.currentThread().getId + val actualFreeMemory = freeMemory - currentUnrollMemory + + pendingUnrollMemoryMap.getOrElse(threadId, 0L) if (actualFreeMemory < space) { val rddToAdd = getRddId(blockIdToAdd) @@ -468,11 +485,32 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } + /** + * Reserve the unroll memory of current unroll successful block used by this thread + * until actually put the block into memory entry. + */ + def reservePendingUnrollMemoryForThisThread(memory: Long): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + pendingUnrollMemoryMap(threadId) = pendingUnrollMemoryMap.getOrElse(threadId, 0L) + memory + } + } + + /** + * Release pending unroll memory of current unroll successful block used by this thread + */ + def releasePendingUnrollMemoryForThisThread(): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + pendingUnrollMemoryMap.remove(threadId) + } + } + /** * Return the amount of memory currently occupied for unrolling blocks across all threads. */ def currentUnrollMemory: Long = accountingLock.synchronized { - unrollMemoryMap.values.sum + unrollMemoryMap.values.sum + pendingUnrollMemoryMap.values.sum } /** diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ffe6f039145ea..3fdbe99b5d02b 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1064,6 +1064,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.releasePendingUnrollMemoryForThisThread() // Unroll with not enough space. This should succeed after kicking out someBlock1. store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) @@ -1074,6 +1075,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(droppedBlocks.size === 1) assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) droppedBlocks.clear() + memoryStore.releasePendingUnrollMemoryForThisThread() // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. From 2db6a853a53b4c25e35983bc489510abb8a73e1d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Mar 2015 17:14:34 -0800 Subject: [PATCH 324/817] [SPARK-6121][SQL][MLLIB] simpleString for UDT `df.dtypes` shows `null` for UDTs. This PR uses `udt` by default and `VectorUDT` overwrites it with `vector`. jkbradley davies Author: Xiangrui Meng Closes #4858 from mengxr/SPARK-6121 and squashes the following commits: 34f0a77 [Xiangrui Meng] simpleString for UDT --- python/pyspark/mllib/linalg.py | 3 +++ python/pyspark/sql/types.py | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 597012b1c967c..f5aad28afda0f 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -152,6 +152,9 @@ def deserialize(self, datum): else: raise ValueError("do not recognize type %r" % tpe) + def simpleString(self): + return "vector" + class Vector(object): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 31a861e1feb46..0169028ccc4eb 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -468,7 +468,7 @@ def deserialize(self, datum): raise NotImplementedError("UDT must implement deserialize().") def simpleString(self): - return 'null' + return 'udt' def json(self): return json.dumps(self.jsonValue(), separators=(',', ':'), sort_keys=True) From 9d6c5aeebd3c7f8ff6defe3bccd8ff12ed918293 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Mar 2015 18:10:50 -0800 Subject: [PATCH 325/817] [SPARK-5537] Add user guide for multinomial logistic regression This is based on #4801 from dbtsai. The linear method guide is re-organized a little bit for this change. Closes #4801 Author: Xiangrui Meng Author: DB Tsai Closes #4861 from mengxr/SPARK-5537 and squashes the following commits: 47af0ac [Xiangrui Meng] update user guide for multinomial logistic regression cdc2e15 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into AlpineNow-mlor-doc 096d0ca [DB Tsai] first commit --- docs/mllib-linear-methods.md | 278 +++++++++++++++++++++++++++-------- 1 file changed, 217 insertions(+), 61 deletions(-) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index ffbd7ef1bff51..03f90d718ad8c 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -17,7 +17,7 @@ displayTitle: MLlib - Linear Methods \newcommand{\av}{\mathbf{\alpha}} \newcommand{\bv}{\mathbf{b}} \newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} +\newcommand{\id}{\mathbf{I}} \newcommand{\ind}{\mathbf{1}} \newcommand{\0}{\mathbf{0}} \newcommand{\unit}{\mathbf{e}} @@ -114,18 +114,26 @@ especially when the number of training examples is small. Under the hood, linear methods use convex optimization methods to optimize the objective functions. MLlib uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods. -## Binary classification - -[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) -aims to divide items into two categories: positive and negative. MLlib -supports two linear methods for binary classification: linear Support Vector -Machines (SVMs) and logistic regression. For both methods, MLlib supports -L1 and L2 regularized variants. The training data set is represented by an RDD -of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the -mathematical formulation in this guide, a training label $y$ is denoted as -either $+1$ (positive) or $-1$ (negative), which is convenient for the -formulation. *However*, the negative label is represented by $0$ in MLlib -instead of $-1$, to be consistent with multiclass labeling. +## Classification + +[Classification](http://en.wikipedia.org/wiki/Statistical_classification) aims to divide items into +categories. +The most common classification type is +[binary classificaion](http://en.wikipedia.org/wiki/Binary_classification), where there are two +categories, usually named positive and negative. +If there are more than two categories, it is called +[multiclass classification](http://en.wikipedia.org/wiki/Multiclass_classification). +MLlib supports two linear methods for classification: linear Support Vector Machines (SVMs) +and logistic regression. +Linear SVMs supports only binary classification, while logistic regression supports both binary and +multiclass classification problems. +For both methods, MLlib supports L1 and L2 regularized variants. +The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib, +where labels are class indices starting from zero: $0, 1, 2, \ldots$. +Note that, in the mathematical formulation in this guide, a binary label $y$ is denoted as either +$+1$ (positive) or $-1$ (negative), which is convenient for the formulation. +*However*, the negative label is represented by $0$ in MLlib instead of $-1$, to be consistent with +multiclass labeling. ### Linear Support Vector Machines (SVMs) @@ -144,41 +152,7 @@ denoted by $\x$, the model makes predictions based on the value of $\wv^T \x$. By the default, if $\wv^T \x \geq 0$ then the outcome is positive, and negative otherwise. -### Logistic regression - -[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a -binary response. -It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss -function in the formulation given by the logistic loss: -`\[ -L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). -\]` - -The logistic regression algorithm outputs a logistic regression model. Given a -new data point, denoted by $\x$, the model makes predictions by -applying the logistic function -`\[ -\mathrm{f}(z) = \frac{1}{1 + e^{-z}} -\]` -where $z = \wv^T \x$. -By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or -negative otherwise, though unlike linear SVMs, the raw output of the logistic regression -model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability -that $\x$ is positive). - -### Evaluation metrics - -MLlib supports common evaluation metrics for binary classification (not available in PySpark). -This -includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), -[receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), -precision-recall curve, and -[area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). -AUC is commonly used to compare the performance of various models while -precision/recall/F-measure can help determine the appropriate threshold to use -for prediction purposes. - -### Examples +**Examples**
    @@ -211,7 +185,7 @@ val model = SVMWithSGD.train(training, numIterations) // Clear the default threshold. model.clearThreshold() -// Compute raw scores on the test set. +// Compute raw scores on the test set. val scoreAndLabels = test.map { point => val score = model.predict(point.features) (score, point.label) @@ -247,8 +221,6 @@ svmAlg.optimizer. val modelL1 = svmAlg.run(training) {% endhighlight %} -[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) can be used in a similar fashion as `SVMWithSGD`. -
    @@ -284,11 +256,11 @@ public class SVMClassifier { JavaRDD training = data.sample(false, 0.6, 11L); training.cache(); JavaRDD test = data.subtract(training); - + // Run training algorithm to build the model. int numIterations = 100; final SVMModel model = SVMWithSGD.train(training.rdd(), numIterations); - + // Clear the default threshold. model.clearThreshold(); @@ -301,12 +273,12 @@ public class SVMClassifier { } } ); - + // Get evaluation metrics. - BinaryClassificationMetrics metrics = + BinaryClassificationMetrics metrics = new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels)); double auROC = metrics.areaUnderROC(); - + System.out.println("Area under ROC = " + auROC); // Save and load model @@ -372,7 +344,191 @@ print("Training Error = " + str(trainErr))
    -## Linear least squares, Lasso, and ridge regression +### Logistic regression + +[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a +binary response. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, +with the loss function in the formulation given by the logistic loss: +`\[ +L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). +\]` + +For binary classification problems, the algorithm outputs a binary logistic regression model. +Given a new data point, denoted by $\x$, the model makes predictions by +applying the logistic function +`\[ +\mathrm{f}(z) = \frac{1}{1 + e^{-z}} +\]` +where $z = \wv^T \x$. +By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or +negative otherwise, though unlike linear SVMs, the raw output of the logistic regression +model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability +that $\x$ is positive). + +Binary logistic regression can be generalized into +[multinomial logistic regression](http://en.wikipedia.org/wiki/Multinomial_logistic_regression) to +train and predict multiclass classification problems. +For example, for $K$ possible outcomes, one of the outcomes can be chosen as a "pivot", and the +other $K - 1$ outcomes can be separately regressed against the pivot outcome. +In MLlib, the first class $0$ is chosen as the "pivot" class. +See Section 4.4 of +[The Elements of Statistical Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for +references. +Here is an +[detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). + +For multiclass classification problems, the algorithm will outputs a multinomial logistic regression +model, which contains $K - 1$ binary logistic regression models regressed against the first class. +Given a new data points, $K - 1$ models will be run, and the class with largest probability will be +chosen as the predicted class. + +We implemented two algorithms to solve logistic regression: mini-batch gradient descent and L-BFGS. +We recommend L-BFGS over mini-batch gradient descent for faster convergence. + +**Examples** + +
    + +
    +The following code illustrates how to load a sample multiclass dataset, split it into train and +test, and use +[LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS) +to fit a logistic regression model. +Then the model is evaluated against the test dataset and saved to disk. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, LogisticRegressionModel} +import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +// Load training data in LIBSVM format. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0).cache() +val test = splits(1) + +// Run training algorithm to build the model +val model = new LogisticRegressionWithLBFGS() + .setNumClasses(10) + .run(training) + +// Compute raw scores on the test set. +val predictionAndLabels = test.map { case LabeledPoint(label, features) => + val prediction = model.predict(features) + (prediction, label) +} + +// Get evaluation metrics. +val metrics = new MulticlassMetrics(predictionAndLabels) +val precision = metrics.precision +println("Precision = " + precision) + +// Save and load model +model.save(sc, "myModelPath") +val sameModel = LogisticRegressionModel.load(sc, "myModelPath") +{% endhighlight %} + +
    + +
    +The following code illustrates how to load a sample multiclass dataset, split it into train and +test, and use +[LogisticRegressionWithLBFGS](api/java/org/apache/spark/mllib/classification/LogisticRegressionWithLBFGS.html) +to fit a logistic regression model. +Then the model is evaluated against the test dataset and saved to disk. + +{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; +import org.apache.spark.mllib.evaluation.MulticlassMetrics; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class MultinomialLogisticRegressionExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVM Classifier Example"); + SparkContext sc = new SparkContext(conf); + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L); + JavaRDD training = splits[0].cache(); + JavaRDD test = splits[1]; + + // Run training algorithm to build the model. + final LogisticRegressionModel model = new LogisticRegressionWithLBFGS() + .setNumClasses(10) + .run(training.rdd()); + + // Compute raw scores on the test set. + JavaRDD> predictionAndLabels = test.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + Double prediction = model.predict(p.features()); + return new Tuple2(prediction, p.label()); + } + } + ); + + // Get evaluation metrics. + MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); + double precision = metrics.precision(); + System.out.println("Precision = " + precision); + + // Save and load model + model.save(sc, "myModelPath"); + LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, "myModelPath"); + } +} +{% endhighlight %} +
    + +
    +The following example shows how to load a sample dataset, build Logistic Regression model, +and make predictions with the resulting model to compute the training error. + +Note that the Python API does not yet support multiclass classification and model save/load but +will in the future. + +{% highlight python %} +from pyspark.mllib.classification import LogisticRegressionWithLBFGS +from pyspark.mllib.regression import LabeledPoint +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("data/mllib/sample_svm_data.txt") +parsedData = data.map(parsePoint) + +# Build the model +model = LogisticRegressionWithLBFGS.train(parsedData) + +# Evaluating the model on training data +labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) +print("Training Error = " + str(trainErr)) +{% endhighlight %} +
    +
    + +# Regression + +### Linear least squares, Lasso, and ridge regression Linear least squares is the most common formulation for regression problems. @@ -390,7 +546,7 @@ regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) u regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -### Examples +**Examples**
    @@ -544,7 +700,7 @@ section of the Spark quick-start guide. Be sure to also include *spark-mllib* to your build file as a dependency. -## Streaming linear regression +###Streaming linear regression When data arrive in a streaming fashion, it is useful to fit regression models online, updating the parameters of the model as new data arrives. MLlib currently supports @@ -552,7 +708,7 @@ streaming linear regression using ordinary least squares. The fitting is similar to that performed offline, except fitting occurs on each batch of data, so that the model continually updates to reflect the data from the stream. -### Examples +**Examples** The following example demonstrates how to load training and testing data from two different input streams of text files, parse the streams as labeled points, fit a linear regression model @@ -619,7 +775,7 @@ will get better!
    -## Implementation (developer) +# Implementation (developer) Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent (SGD), building on the underlying gradient descent primitive (as described in the Date: Mon, 2 Mar 2015 18:40:46 -0800 Subject: [PATCH 326/817] [SPARK-6127][Streaming][Docs] Add Kafka to Python api docs davies Author: Tathagata Das Closes #4860 from tdas/SPARK-6127 and squashes the following commits: 82de92a [Tathagata Das] Add Kafka to Python api docs --- python/docs/pyspark.streaming.rst | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index f08185627d0bc..7890d9dcaac21 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -8,3 +8,10 @@ Module contents :members: :undoc-members: :show-inheritance: + +pyspark.streaming.kafka module +---------------------------- +.. automodule:: pyspark.streaming.kafka + :members: + :undoc-members: + :show-inheritance: From 12599942e69e4d73040f3a8611661a0862514ffc Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 2 Mar 2015 19:31:55 -0800 Subject: [PATCH 327/817] [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should work when using datasource api This PR contains the following changes: 1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However, the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values). 2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types. 3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings. 4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust. 5. Update the equality check of JSON relation. Since JSON does not really cares nullability, `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables. JIRA: https://issues.apache.org/jira/browse/SPARK-5950 Thanks viirya for the initial work in #4729. cc marmbrus liancheng Author: Yin Huai Closes #4826 from yhuai/insertNullabilityCheck and squashes the following commits: 3b61a04 [Yin Huai] Revert change on equals. 80e487e [Yin Huai] asNullable in UDT. 587d88b [Yin Huai] Make methods private. 0cb7ea2 [Yin Huai] marmbrus's comments. 3cec464 [Yin Huai] Cheng's comments. 486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck d3747d1 [Yin Huai] Remove unnecessary change. 8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck 8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check. 0eb5578 [Yin Huai] Fix tests. f6ed813 [Yin Huai] Update old parquet path. e4f397c [Yin Huai] Unit tests. b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check. 8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data. bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data. 0a703e7 [Yin Huai] Test failed again since we cannot read correct content. 9a26611 [Yin Huai] Make InsertIntoTable happy. 8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability 4ec17fd [Yin Huai] Failed test. --- .../apache/spark/mllib/linalg/Vectors.scala | 2 + .../spark/mllib/util/modelSaveLoad.scala | 2 +- .../plans/logical/basicOperators.scala | 3 +- .../apache/spark/sql/types/dataTypes.scala | 97 ++++++++++++++++++- .../spark/sql/types/DataTypeSuite.scala | 83 ++++++++++++++++ .../apache/spark/sql/json/JSONRelation.scala | 4 +- .../spark/sql/parquet/ParquetRelation.scala | 9 +- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../apache/spark/sql/parquet/newParquet.scala | 10 +- .../apache/spark/sql/sources/commands.scala | 7 +- .../org/apache/spark/sql/sources/rules.scala | 2 +- .../spark/sql/test/ExamplePointUDT.scala | 2 + .../spark/sql/UserDefinedTypeSuite.scala | 2 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 5 +- .../spark/sql/hive/execution/commands.scala | 33 +++---- .../sql/hive/MetastoreDataSourcesSuite.scala | 71 +++++++++++++- .../spark/sql/parquet/parquetSuites.scala | 29 ++++++ 17 files changed, 330 insertions(+), 36 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 4bdcb283da09c..e9d25dcb7e778 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -182,6 +182,8 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { case _ => false } } + + private[spark] override def asNullable: VectorUDT = this } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index 526d055c87387..30d642c754b7c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -110,7 +110,7 @@ private[mllib] object Loader { assert(loadedFields.contains(field.name), s"Unable to parse model data." + s" Expected field with name ${field.name} was missing in loaded schema:" + s" ${loadedFields.mkString(", ")}") - assert(loadedFields(field.name) == field.dataType, + assert(loadedFields(field.name).sameType(field.dataType), s"Unable to parse model data. Expected field $field but found field" + s" with different type: ${loadedFields(field.name)}") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 89544add74430..20cc8e90a71a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -120,7 +120,8 @@ case class InsertIntoTable( override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { - case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType + case (childAttr, tableAttr) => + DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 2abb1caee9cd9..92d322845f5c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -181,7 +181,7 @@ object DataType { /** * Compares two types, ignoring nullability of ArrayType, MapType, StructType. */ - private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { (left, right) match { case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => equalsIgnoreNullability(leftElementType, rightElementType) @@ -198,6 +198,43 @@ object DataType { case (left, right) => left == right } } + + /** + * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. + * + * Compatible nullability is defined as follows: + * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` + * if and only if `to.containsNull` is true, or both of `from.containsNull` and + * `to.containsNull` are false. + * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` + * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and + * `to.valueContainsNull` are false. + * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` + * if and only if for all every pair of fields, `to.nullable` is true, or both + * of `fromField.nullable` and `toField.nullable` are false. + */ + private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { + (from, to) match { + case (ArrayType(fromElement, fn), ArrayType(toElement, tn)) => + (tn || !fn) && equalsIgnoreCompatibleNullability(fromElement, toElement) + + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + (tn || !fn) && + equalsIgnoreCompatibleNullability(fromKey, toKey) && + equalsIgnoreCompatibleNullability(fromValue, toValue) + + case (StructType(fromFields), StructType(toFields)) => + fromFields.size == toFields.size && + fromFields.zip(toFields).forall { + case (fromField, toField) => + fromField.name == toField.name && + (toField.nullable || !fromField.nullable) && + equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) + } + + case (fromDataType, toDataType) => fromDataType == toDataType + } + } } @@ -230,6 +267,17 @@ abstract class DataType { def prettyJson: String = pretty(render(jsonValue)) def simpleString: String = typeName + + /** Check if `this` and `other` are the same data type when ignoring nullability + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def sameType(other: DataType): Boolean = + DataType.equalsIgnoreNullability(this, other) + + /** Returns the same data type but set all nullability fields are true + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def asNullable: DataType } /** @@ -245,6 +293,8 @@ class NullType private() extends DataType { // this type. Otherwise, the companion object would be of type "NullType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. override def defaultSize: Int = 1 + + private[spark] override def asNullable: NullType = this } case object NullType extends NullType @@ -310,6 +360,8 @@ class StringType private() extends NativeType with PrimitiveType { * The default size of a value of the StringType is 4096 bytes. */ override def defaultSize: Int = 4096 + + private[spark] override def asNullable: StringType = this } case object StringType extends StringType @@ -344,6 +396,8 @@ class BinaryType private() extends NativeType with PrimitiveType { * The default size of a value of the BinaryType is 4096 bytes. */ override def defaultSize: Int = 4096 + + private[spark] override def asNullable: BinaryType = this } case object BinaryType extends BinaryType @@ -369,6 +423,8 @@ class BooleanType private() extends NativeType with PrimitiveType { * The default size of a value of the BooleanType is 1 byte. */ override def defaultSize: Int = 1 + + private[spark] override def asNullable: BooleanType = this } case object BooleanType extends BooleanType @@ -399,6 +455,8 @@ class TimestampType private() extends NativeType { * The default size of a value of the TimestampType is 12 bytes. */ override def defaultSize: Int = 12 + + private[spark] override def asNullable: TimestampType = this } case object TimestampType extends TimestampType @@ -427,6 +485,8 @@ class DateType private() extends NativeType { * The default size of a value of the DateType is 4 bytes. */ override def defaultSize: Int = 4 + + private[spark] override def asNullable: DateType = this } case object DateType extends DateType @@ -485,6 +545,8 @@ class LongType private() extends IntegralType { override def defaultSize: Int = 8 override def simpleString = "bigint" + + private[spark] override def asNullable: LongType = this } case object LongType extends LongType @@ -514,6 +576,8 @@ class IntegerType private() extends IntegralType { override def defaultSize: Int = 4 override def simpleString = "int" + + private[spark] override def asNullable: IntegerType = this } case object IntegerType extends IntegerType @@ -543,6 +607,8 @@ class ShortType private() extends IntegralType { override def defaultSize: Int = 2 override def simpleString = "smallint" + + private[spark] override def asNullable: ShortType = this } case object ShortType extends ShortType @@ -572,6 +638,8 @@ class ByteType private() extends IntegralType { override def defaultSize: Int = 1 override def simpleString = "tinyint" + + private[spark] override def asNullable: ByteType = this } case object ByteType extends ByteType @@ -638,6 +706,8 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" case None => "decimal(10,0)" } + + private[spark] override def asNullable: DecimalType = this } @@ -696,6 +766,8 @@ class DoubleType private() extends FractionalType { * The default size of a value of the DoubleType is 8 bytes. */ override def defaultSize: Int = 8 + + private[spark] override def asNullable: DoubleType = this } case object DoubleType extends DoubleType @@ -724,6 +796,8 @@ class FloatType private() extends FractionalType { * The default size of a value of the FloatType is 4 bytes. */ override def defaultSize: Int = 4 + + private[spark] override def asNullable: FloatType = this } case object FloatType extends FloatType @@ -772,6 +846,9 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT override def defaultSize: Int = 100 * elementType.defaultSize override def simpleString = s"array<${elementType.simpleString}>" + + private[spark] override def asNullable: ArrayType = + ArrayType(elementType.asNullable, containsNull = true) } @@ -1017,6 +1094,15 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru */ private[sql] def merge(that: StructType): StructType = StructType.merge(this, that).asInstanceOf[StructType] + + private[spark] override def asNullable: StructType = { + val newFields = fields.map { + case StructField(name, dataType, nullable, metadata) => + StructField(name, dataType.asNullable, nullable = true, metadata) + } + + StructType(newFields) + } } @@ -1069,6 +1155,9 @@ case class MapType( override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) override def simpleString = s"map<${keyType.simpleString},${valueType.simpleString}>" + + private[spark] override def asNullable: MapType = + MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) } @@ -1122,4 +1211,10 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { * The default size of a value of the UserDefinedType is 4096 bytes. */ override def defaultSize: Int = 4096 + + /** + * For UDT, asNullable will not change the nullability of its internal sqlType and just returns + * itself. + */ + private[spark] override def asNullable: UserDefinedType[UserType] = this } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index c97e0bec3e3a2..a1341ea13d810 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -115,4 +115,87 @@ class DataTypeSuite extends FunSuite { checkDefaultSize(MapType(IntegerType, StringType, true), 410000) checkDefaultSize(MapType(IntegerType, ArrayType(DoubleType), false), 80400) checkDefaultSize(structType, 812) + + def checkEqualsIgnoreCompatibleNullability( + from: DataType, + to: DataType, + expected: Boolean): Unit = { + val testName = + s"equalsIgnoreCompatibleNullability: (from: ${from}, to: ${to})" + test(testName) { + assert(DataType.equalsIgnoreCompatibleNullability(from, to) === expected) + } + } + + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = true), + to = ArrayType(DoubleType, containsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(DoubleType, containsNull = false), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(DoubleType, containsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = true), + to = ArrayType(DoubleType, containsNull = false), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = ArrayType(DoubleType, containsNull = false), + to = ArrayType(StringType, containsNull = false), + expected = false) + + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = true), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = false), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = false), + to = MapType(StringType, DoubleType, valueContainsNull = true), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, DoubleType, valueContainsNull = true), + to = MapType(StringType, DoubleType, valueContainsNull = false), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, ArrayType(IntegerType, true), valueContainsNull = true), + to = MapType(StringType, ArrayType(IntegerType, false), valueContainsNull = true), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = MapType(StringType, ArrayType(IntegerType, false), valueContainsNull = true), + to = MapType(StringType, ArrayType(IntegerType, true), valueContainsNull = true), + expected = true) + + + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = false) :: Nil), + to = StructType(StructField("a", StringType, nullable = true) :: Nil), + expected = true) + checkEqualsIgnoreCompatibleNullability( + from = StructType(StructField("a", StringType, nullable = true) :: Nil), + to = StructType(StructField("a", StringType, nullable = false) :: Nil), + expected = false) + checkEqualsIgnoreCompatibleNullability( + from = StructType( + StructField("a", StringType, nullable = false) :: + StructField("b", StringType, nullable = true) :: Nil), + to = StructType( + StructField("a", StringType, nullable = false) :: + StructField("b", StringType, nullable = false) :: Nil), + expected = false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index 3b68b7c275016..f9d0ba2241d37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{DataType, StructType} private[sql] class DefaultSource @@ -131,7 +131,7 @@ private[sql] case class JSONRelation( override def equals(other: Any): Boolean = other match { case that: JSONRelation => - (this.path == that.path) && (this.schema == that.schema) + (this.path == that.path) && this.schema.sameType(that.schema) case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index a0d1005c0cae3..fd161bae128ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -23,6 +23,7 @@ import java.util.logging.Level import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.permission.FsAction +import org.apache.spark.sql.types.{StructType, DataType} import parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat} import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType @@ -172,9 +173,13 @@ private[sql] object ParquetRelation { sqlContext.conf.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED) .name()) ParquetRelation.enableLogForwarding() - ParquetTypesConverter.writeMetaData(attributes, path, conf) + // This is a hack. We always set nullable/containsNull/valueContainsNull to true + // for the schema of a parquet data. + val schema = StructType.fromAttributes(attributes).asNullable + val newAttributes = schema.toAttributes + ParquetTypesConverter.writeMetaData(newAttributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { - override val output = attributes + override val output = newAttributes } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 225ec6db7d553..62813a981e685 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -278,7 +278,10 @@ private[sql] case class InsertIntoParquetTable( ParquetOutputFormat.setWriteSupportClass(job, writeSupport) val conf = ContextUtil.getConfiguration(job) - RowWriteSupport.setSchema(relation.output, conf) + // This is a hack. We always set nullable/containsNull/valueContainsNull to true + // for the schema of a parquet data. + val schema = StructType.fromAttributes(relation.output).asNullable + RowWriteSupport.setSchema(schema.toAttributes, conf) val fspath = new Path(relation.path) val fs = fspath.getFileSystem(conf) 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 6d56be3ab8dd4..8d9585849327e 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 @@ -115,9 +115,15 @@ private[sql] class DefaultSource } val relation = if (doInsertion) { + // This is a hack. We always set nullable/containsNull/valueContainsNull to true + // for the schema of a parquet data. + val df = + sqlContext.createDataFrame( + data.queryExecution.toRdd, + data.schema.asNullable) val createdRelation = - createRelation(sqlContext, parameters, data.schema).asInstanceOf[ParquetRelation2] - createdRelation.insert(data, overwrite = mode == SaveMode.Overwrite) + createRelation(sqlContext, parameters, df.schema).asInstanceOf[ParquetRelation2] + createdRelation.insert(df, overwrite = mode == SaveMode.Overwrite) createdRelation } else { // If the save mode is Ignore, we will just create the relation based on existing data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index c9cd0e6e93829..0e540dad81283 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{LogicalRDD, RunnableCommand} +import org.apache.spark.sql.execution.RunnableCommand private[sql] case class InsertIntoDataSource( logicalRelation: LogicalRelation, @@ -29,7 +29,10 @@ private[sql] case class InsertIntoDataSource( override def run(sqlContext: SQLContext) = { val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] - relation.insert(DataFrame(sqlContext, query), overwrite) + val data = DataFrame(sqlContext, query) + // Apply the schema of the existing table to the new data. + val df = sqlContext.createDataFrame(data.queryExecution.toRdd, logicalRelation.schema) + relation.insert(df, overwrite) // Invalidate the cache. sqlContext.cacheManager.invalidateCache(logicalRelation) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 8440581074877..cfa58f1442218 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -56,7 +56,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { child: LogicalPlan) = { val newChildOutput = expectedOutput.zip(child.output).map { case (expected, actual) => - val needCast = !DataType.equalsIgnoreNullability(expected.dataType, actual.dataType) + val needCast = !expected.dataType.sameType(actual.dataType) // We want to make sure the filed names in the data to be inserted exactly match // names in the schema. val needRename = expected.name != actual.name diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index eb045e37bf5a9..c11d0ae5bf1cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -59,4 +59,6 @@ private[sql] class ExamplePointUDT extends UserDefinedType[ExamplePoint] { } override def userClass: Class[ExamplePoint] = classOf[ExamplePoint] + + private[spark] override def asNullable: ExamplePointUDT = this } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 47fdb5543235c..23f424c0bfc7c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -62,6 +62,8 @@ private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { } override def userClass = classOf[MyDenseVector] + + private[spark] override def asNullable: MyDenseVectorUDT = this } class UserDefinedTypeSuite extends QueryTest { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 74b4e767caed2..86fc6548f92c7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -638,7 +638,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with p } else if (childOutputDataTypes.size == tableOutputDataTypes.size && childOutputDataTypes.zip(tableOutputDataTypes) - .forall { case (left, right) => DataType.equalsIgnoreNullability(left, right) }) { + .forall { case (left, right) => left.sameType(right) }) { // If both types ignoring nullability of ArrayType, MapType, StructType are the same, // use InsertIntoHiveTable instead of InsertIntoTable. InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite) @@ -686,8 +686,7 @@ private[hive] case class InsertIntoHiveTable( override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { - case (childAttr, tableAttr) => - DataType.equalsIgnoreNullability(childAttr.dataType, tableAttr.dataType) + case (childAttr, tableAttr) => childAttr.dataType.sameType(tableAttr.dataType) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index ffaef8eef1c74..36bd3f8fe21f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -169,6 +169,7 @@ case class CreateMetastoreDataSourceAsSelect( options } + var existingSchema = None: Option[StructType] if (sqlContext.catalog.tableExists(Seq(tableName))) { // Check if we need to throw an exception or just return. mode match { @@ -188,22 +189,7 @@ case class CreateMetastoreDataSourceAsSelect( val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { case l @ LogicalRelation(i: InsertableRelation) => - if (l.schema != createdRelation.schema) { - val errorDescription = - s"Cannot append to table $tableName because the schema of this " + - s"DataFrame does not match the schema of table $tableName." - val errorMessage = - s""" - |$errorDescription - |== Schemas == - |${sideBySide( - s"== Expected Schema ==" +: - l.schema.treeString.split("\\\n"), - s"== Actual Schema ==" +: - createdRelation.schema.treeString.split("\\\n")).mkString("\n")} - """.stripMargin - throw new AnalysisException(errorMessage) - } else if (i != createdRelation.relation) { + if (i != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + s"match the existing relation of $tableName. " + @@ -221,6 +207,7 @@ case class CreateMetastoreDataSourceAsSelect( """.stripMargin throw new AnalysisException(errorMessage) } + existingSchema = Some(l.schema) case o => throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } @@ -234,15 +221,23 @@ case class CreateMetastoreDataSourceAsSelect( createMetastoreTable = true } - val df = DataFrame(hiveContext, query) + val data = DataFrame(hiveContext, query) + val df = existingSchema match { + // If we are inserting into an existing table, just use the existing schema. + case Some(schema) => sqlContext.createDataFrame(data.queryExecution.toRdd, schema) + case None => data + } // Create the relation based on the data of df. - ResolvedDataSource(sqlContext, provider, mode, optionsWithPath, df) + val resolved = ResolvedDataSource(sqlContext, provider, mode, optionsWithPath, df) if (createMetastoreTable) { + // We will use the schema of resolved.relation as the schema of the table (instead of + // the schema of df). It is important since the nullability may be changed by the relation + // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). hiveContext.catalog.createDataSourceTable( tableName, - Some(df.schema), + Some(resolved.relation.schema), provider, optionsWithPath, isExternal) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 868c35f35ff59..5d6a6f3b64f03 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -34,6 +34,8 @@ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.ParquetRelation2 import org.apache.spark.sql.sources.LogicalRelation +import scala.collection.mutable.ArrayBuffer + /** * Tests for persisting tables created though the data sources API into the metastore. */ @@ -581,7 +583,8 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { case LogicalRelation(p: ParquetRelation2) => // OK case _ => fail( - s"test_parquet_ctas should be converted to ${classOf[ParquetRelation2].getCanonicalName}") + "test_parquet_ctas should be converted to " + + s"${classOf[ParquetRelation2].getCanonicalName}") } // Clenup and reset confs. @@ -592,6 +595,72 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { } } + test("Pre insert nullability check (ArrayType)") { + val df1 = + createDataFrame(Tuple1(Seq(Int.box(1), null.asInstanceOf[Integer])) :: Nil).toDF("a") + val expectedSchema1 = + StructType( + StructField("a", ArrayType(IntegerType, containsNull = true), nullable = true) :: Nil) + assert(df1.schema === expectedSchema1) + df1.saveAsTable("arrayInParquet", "parquet", SaveMode.Overwrite) + + val df2 = + createDataFrame(Tuple1(Seq(2, 3)) :: Nil).toDF("a") + val expectedSchema2 = + StructType( + StructField("a", ArrayType(IntegerType, containsNull = false), nullable = true) :: Nil) + assert(df2.schema === expectedSchema2) + df2.insertInto("arrayInParquet", overwrite = false) + createDataFrame(Tuple1(Seq(4, 5)) :: Nil).toDF("a") + .saveAsTable("arrayInParquet", SaveMode.Append) // This one internally calls df2.insertInto. + createDataFrame(Tuple1(Seq(Int.box(6), null.asInstanceOf[Integer])) :: Nil).toDF("a") + .saveAsTable("arrayInParquet", "parquet", SaveMode.Append) + refreshTable("arrayInParquet") + + checkAnswer( + sql("SELECT a FROM arrayInParquet"), + Row(ArrayBuffer(1, null)) :: + Row(ArrayBuffer(2, 3)) :: + Row(ArrayBuffer(4, 5)) :: + Row(ArrayBuffer(6, null)) :: Nil) + + sql("DROP TABLE arrayInParquet") + } + + test("Pre insert nullability check (MapType)") { + val df1 = + createDataFrame(Tuple1(Map(1 -> null.asInstanceOf[Integer])) :: Nil).toDF("a") + val mapType1 = MapType(IntegerType, IntegerType, valueContainsNull = true) + val expectedSchema1 = + StructType( + StructField("a", mapType1, nullable = true) :: Nil) + assert(df1.schema === expectedSchema1) + df1.saveAsTable("mapInParquet", "parquet", SaveMode.Overwrite) + + val df2 = + createDataFrame(Tuple1(Map(2 -> 3)) :: Nil).toDF("a") + val mapType2 = MapType(IntegerType, IntegerType, valueContainsNull = false) + val expectedSchema2 = + StructType( + StructField("a", mapType2, nullable = true) :: Nil) + assert(df2.schema === expectedSchema2) + df2.insertInto("mapInParquet", overwrite = false) + createDataFrame(Tuple1(Map(4 -> 5)) :: Nil).toDF("a") + .saveAsTable("mapInParquet", SaveMode.Append) // This one internally calls df2.insertInto. + createDataFrame(Tuple1(Map(6 -> null.asInstanceOf[Integer])) :: Nil).toDF("a") + .saveAsTable("mapInParquet", "parquet", SaveMode.Append) + refreshTable("mapInParquet") + + checkAnswer( + sql("SELECT a FROM mapInParquet"), + Row(Map(1 -> null)) :: + Row(Map(2 -> 3)) :: + Row(Map(4 -> 5)) :: + Row(Map(6 -> null)) :: Nil) + + sql("DROP TABLE mapInParquet") + } + test("SPARK-6024 wide schema support") { // We will need 80 splits for this schema if the threshold is 4000. val schema = StructType((1 to 5000).map(i => StructField(s"c_${i}", StringType, true))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index c8da8eea4e646..89b943f008094 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.types._ // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -522,6 +523,34 @@ class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { super.afterAll() setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("values in arrays and maps stored in parquet are always nullable") { + val df = createDataFrame(Tuple2(Map(2 -> 3), Seq(4, 5, 6)) :: Nil).toDF("m", "a") + val mapType1 = MapType(IntegerType, IntegerType, valueContainsNull = false) + val arrayType1 = ArrayType(IntegerType, containsNull = false) + val expectedSchema1 = + StructType( + StructField("m", mapType1, nullable = true) :: + StructField("a", arrayType1, nullable = true) :: Nil) + assert(df.schema === expectedSchema1) + + df.saveAsTable("alwaysNullable", "parquet") + + val mapType2 = MapType(IntegerType, IntegerType, valueContainsNull = true) + val arrayType2 = ArrayType(IntegerType, containsNull = true) + val expectedSchema2 = + StructType( + StructField("m", mapType2, nullable = true) :: + StructField("a", arrayType2, nullable = true) :: Nil) + + assert(table("alwaysNullable").schema === expectedSchema2) + + checkAnswer( + sql("SELECT m, a FROM alwaysNullable"), + Row(Map(2 -> 3), Seq(4, 5, 6))) + + sql("DROP TABLE alwaysNullable") + } } class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { From 54d19689ff8d786acde5b8ada6741854ffadadea Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 Mar 2015 22:14:08 -0800 Subject: [PATCH 328/817] [SPARK-5310][SQL] Fixes to Docs and Datasources API - Various Fixes to docs - Make data source traits actually interfaces Based on #4862 but with fixed conflicts. Author: Reynold Xin Author: Michael Armbrust Closes #4868 from marmbrus/pr/4862 and squashes the following commits: fe091ea [Michael Armbrust] Merge remote-tracking branch 'origin/master' into pr/4862 0208497 [Reynold Xin] Test fixes. 34e0a28 [Reynold Xin] [SPARK-5310][SQL] Various fixes to Spark SQL docs. --- project/SparkBuild.scala | 29 +++++++------ .../org/apache/spark/sql/DataFrame.scala | 36 ++++++++++++++-- .../scala/org/apache/spark/sql/RDDApi.scala | 4 +- .../apache/spark/sql/jdbc/JDBCRelation.scala | 3 +- .../apache/spark/sql/json/JSONRelation.scala | 5 ++- .../apache/spark/sql/parquet/newParquet.scala | 3 +- .../apache/spark/sql/sources/interfaces.scala | 43 ++++++++----------- .../spark/sql/sources/DDLTestSuite.scala | 2 +- .../spark/sql/sources/FilteredScanSuite.scala | 3 +- .../spark/sql/sources/PrunedScanSuite.scala | 3 +- .../spark/sql/sources/TableScanSuite.scala | 11 ++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +- .../hive/execution/CreateTableAsSelect.scala | 3 +- .../execution/DescribeHiveTableCommand.scala | 4 +- .../hive/execution/HiveNativeCommand.scala | 6 +-- .../sql/hive/execution/HiveTableScan.scala | 4 +- .../hive/execution/InsertIntoHiveTable.scala | 6 +-- .../hive/execution/ScriptTransformation.scala | 15 ++----- .../spark/sql/hive/execution/commands.scala | 27 +++--------- .../spark/sql/hive/execution/package.scala | 25 ----------- .../{parquet => hive}/HiveParquetSuite.scala | 5 ++- .../sql/{parquet => hive}/parquetSuites.scala | 11 ++--- 22 files changed, 115 insertions(+), 136 deletions(-) delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala rename sql/hive/src/test/scala/org/apache/spark/sql/{parquet => hive}/HiveParquetSuite.scala (96%) rename sql/hive/src/test/scala/org/apache/spark/sql/{parquet => hive}/parquetSuites.scala (98%) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e4b1b96527fbd..4f17df59f4c1f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -357,6 +357,21 @@ object Unidoc { names.map(s => "org.apache.spark." + s).mkString(":") } + private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = { + packages + .map(_.filterNot(_.getName.contains("$"))) + .map(_.filterNot(_.getCanonicalPath.contains("akka"))) + .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) + .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) + .map(_.filterNot(_.getCanonicalPath.contains("sql/execution"))) + .map(_.filterNot(_.getCanonicalPath.contains("sql/hive/test"))) + } + lazy val settings = scalaJavaUnidocSettings ++ Seq ( publish := {}, @@ -368,22 +383,12 @@ object Unidoc { // Skip actual catalyst, but include the subproject. // Catalyst is not public API and contains quasiquotes which break scaladoc. unidocAllSources in (ScalaUnidoc, unidoc) := { - (unidocAllSources in (ScalaUnidoc, unidoc)).value - .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) + ignoreUndocumentedPackages((unidocAllSources in (ScalaUnidoc, unidoc)).value) }, // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { - (unidocAllSources in (JavaUnidoc, unidoc)).value - .map(_.filterNot(_.getName.contains("$"))) - .map(_.filterNot(_.getCanonicalPath.contains("akka"))) - .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) - .map(_.filterNot(_.getCanonicalPath.contains("network"))) - .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) - .map(_.filterNot(_.getCanonicalPath.contains("executor"))) - .map(_.filterNot(_.getCanonicalPath.contains("python"))) - .map(_.filterNot(_.getCanonicalPath.contains("collection"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) + ignoreUndocumentedPackages((unidocAllSources in (JavaUnidoc, unidoc)).value) }, // Javadoc options: create a window title, and group key packages on index page 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 f3aac0826ad23..46f50708a9184 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 @@ -64,7 +64,7 @@ private[sql] object DataFrame { * val people = sqlContext.parquetFile("...") * * // Create a DataFrame from data sources - * val df = + * val df = sqlContext.load("...", "json") * }}} * * Once created, it can be manipulated using the various domain-specific-language (DSL) functions @@ -80,9 +80,10 @@ private[sql] object DataFrame { * {{{ * // The following creates a new column that increases everybody's age by 10. * people("age") + 10 // in Scala + * people.col("age").plus(10); // in Java * }}} * - * A more concrete example: + * A more concrete example in Scala: * {{{ * // To create DataFrame using SQLContext * val people = sqlContext.parquetFile("...") @@ -94,6 +95,18 @@ private[sql] object DataFrame { * .agg(avg(people("salary")), max(people("age"))) * }}} * + * and in Java: + * {{{ + * // To create DataFrame using SQLContext + * DataFrame people = sqlContext.parquetFile("..."); + * DataFrame department = sqlContext.parquetFile("..."); + * + * people.filter("age".gt(30)) + * .join(department, people.col("deptId").equalTo(department("id"))) + * .groupBy(department.col("name"), "gender") + * .agg(avg(people.col("salary")), max(people.col("age"))); + * }}} + * * @groupname basic Basic DataFrame functions * @groupname dfops Language Integrated Queries * @groupname rdd RDD Operations @@ -102,7 +115,7 @@ private[sql] object DataFrame { */ // TODO: Improve documentation. @Experimental -class DataFrame protected[sql]( +class DataFrame private[sql]( @transient val sqlContext: SQLContext, @DeveloperApi @transient val queryExecution: SQLContext#QueryExecution) extends RDDApi[Row] with Serializable { @@ -295,12 +308,14 @@ class DataFrame protected[sql]( * 1984 04 0.450090 0.483521 * }}} * @param numRows Number of rows to show - * @group basic + * + * @group action */ def show(numRows: Int): Unit = println(showString(numRows)) /** * Displays the top 20 rows of [[DataFrame]] in a tabular form. + * @group action */ def show(): Unit = show(20) @@ -738,16 +753,19 @@ class DataFrame protected[sql]( /** * Returns the first `n` rows. + * @group action */ def head(n: Int): Array[Row] = limit(n).collect() /** * Returns the first row. + * @group action */ def head(): Row = head(1).head /** * Returns the first row. Alias for head(). + * @group action */ override def first(): Row = head() @@ -831,6 +849,11 @@ class DataFrame protected[sql]( this } + /** + * @group basic + */ + override def cache(): this.type = persist() + /** * @group basic */ @@ -847,6 +870,11 @@ class DataFrame protected[sql]( this } + /** + * @group basic + */ + override def unpersist(): this.type = unpersist(blocking = false) + ///////////////////////////////////////////////////////////////////////////// // I/O ///////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index df866fd1ad8ad..ba4373f0124b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -29,13 +29,13 @@ import org.apache.spark.storage.StorageLevel */ private[sql] trait RDDApi[T] { - def cache(): this.type = persist() + def cache(): this.type def persist(): this.type def persist(newLevel: StorageLevel): this.type - def unpersist(): this.type = unpersist(blocking = false) + def unpersist(): this.type def unpersist(blocking: Boolean): this.type diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index beb76f2c553c6..1778d39c42e2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -119,7 +119,8 @@ private[sql] case class JDBCRelation( url: String, table: String, parts: Array[Partition])(@transient val sqlContext: SQLContext) - extends PrunedFilteredScan { + extends BaseRelation + with PrunedFilteredScan { override val schema = JDBCRDD.resolveTable(url, table) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f9d0ba2241d37..b645199ded18c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -90,7 +90,10 @@ private[sql] case class JSONRelation( samplingRatio: Double, userSpecifiedSchema: Option[StructType])( @transient val sqlContext: SQLContext) - extends TableScan with InsertableRelation { + extends BaseRelation + with TableScan + with InsertableRelation { + // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) 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 8d9585849327e..234e6bb8443af 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 @@ -159,7 +159,8 @@ private[sql] case class ParquetRelation2( maybeSchema: Option[StructType] = None, maybePartitionSpec: Option[PartitionSpec] = None)( @transient val sqlContext: SQLContext) - extends CatalystScan + extends BaseRelation + with CatalystScan with InsertableRelation with SparkHadoopMapReduceUtil with Logging { 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 0c4b706eeebae..a046a48c1733d 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 @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.sources import org.apache.spark.annotation.{Experimental, DeveloperApi} @@ -90,12 +91,6 @@ trait CreatableRelationProvider { * existing data is expected to be overwritten by the contents of the DataFrame. * ErrorIfExists mode means that when saving a DataFrame to a data source, * if data already exists, an exception is expected to be thrown. - * - * @param sqlContext - * @param mode - * @param parameters - * @param data - * @return */ def createRelation( sqlContext: SQLContext, @@ -138,7 +133,7 @@ abstract class BaseRelation { * A BaseRelation that can produce all of its tuples as an RDD of Row objects. */ @DeveloperApi -trait TableScan extends BaseRelation { +trait TableScan { def buildScan(): RDD[Row] } @@ -148,7 +143,7 @@ trait TableScan extends BaseRelation { * containing all of its tuples as Row objects. */ @DeveloperApi -trait PrunedScan extends BaseRelation { +trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } @@ -162,24 +157,10 @@ trait PrunedScan extends BaseRelation { * as filtering partitions based on a bloom filter. */ @DeveloperApi -trait PrunedFilteredScan extends BaseRelation { +trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } -/** - * ::Experimental:: - * An interface for experimenting with a more direct connection to the query planner. Compared to - * [[PrunedFilteredScan]], this operator receives the raw expressions from the - * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. Unlike the other APIs this - * interface is not designed to be binary compatible across releases and thus should only be used - * for experimentation. - */ -@Experimental -trait CatalystScan extends BaseRelation { - def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] -} - -@DeveloperApi /** * ::DeveloperApi:: * A BaseRelation that can be used to insert data into it through the insert method. @@ -196,6 +177,20 @@ trait CatalystScan extends BaseRelation { * If a data source needs to check the actual nullability of a field, it needs to do it in the * insert method. */ -trait InsertableRelation extends BaseRelation { +@DeveloperApi +trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } + +/** + * ::Experimental:: + * An interface for experimenting with a more direct connection to the query planner. Compared to + * [[PrunedFilteredScan]], this operator receives the raw expressions from the + * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. Unlike the other APIs this + * interface is NOT designed to be binary compatible across releases and thus should only be used + * for experimentation. + */ +@Experimental +trait CatalystScan { + def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 0ec756bfeb7ef..54af50c6e10ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -29,7 +29,7 @@ class DDLScanSource extends RelationProvider { } case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) - extends TableScan { + extends BaseRelation with TableScan { override def schema = StructType(Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 41cd35683c196..ffeccf0b69394 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -32,7 +32,8 @@ class FilteredScanSource extends RelationProvider { } case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) - extends PrunedFilteredScan { + extends BaseRelation + with PrunedFilteredScan { override def schema = StructType( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index a33cf1172cac9..08fb5380dc026 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -31,7 +31,8 @@ class PrunedScanSource extends RelationProvider { } case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) - extends PrunedScan { + extends BaseRelation + with PrunedScan { override def schema = StructType( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 0a4d4b6342d4f..7928600ac2fb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -33,7 +33,7 @@ class SimpleScanSource extends RelationProvider { } case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) - extends TableScan { + extends BaseRelation with TableScan { override def schema = StructType(StructField("i", IntegerType, nullable = false) :: Nil) @@ -51,10 +51,11 @@ class AllDataTypesScanSource extends SchemaRelationProvider { } case class AllDataTypesScan( - from: Int, - to: Int, - userSpecifiedSchema: StructType)(@transient val sqlContext: SQLContext) - extends TableScan { + from: Int, + to: Int, + userSpecifiedSchema: StructType)(@transient val sqlContext: SQLContext) + extends BaseRelation + with TableScan { override def schema = userSpecifiedSchema diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 86fc6548f92c7..fe86bd206a71c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -777,7 +777,8 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) } -object HiveMetastoreTypes { + +private[hive] object HiveMetastoreTypes { protected val ddlParser = new DDLParser(HiveQl.parseSql(_)) def toDataType(metastoreType: String): DataType = synchronized { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index a547babcebfff..a0c91cbc4e86f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation /** - * :: Experimental :: * Create table and insert the query result into it. * @param database the database name of the new relation * @param tableName the table name of the new relation @@ -38,7 +37,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param desc the CreateTableDesc, which may contains serde, storage handler etc. */ -@Experimental +private[hive] case class CreateTableAsSelect( database: String, tableName: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 07b5a84fb6602..d0510aa342796 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -29,11 +29,9 @@ import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.SQLContext /** - * :: DeveloperApi :: - * * Implementation for "describe [extended] table". */ -@DeveloperApi +private[hive] case class DescribeHiveTableCommand( table: MetastoreRelation, override val output: Seq[Attribute], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala index 781a2e9164c82..9636da206087f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala @@ -17,17 +17,13 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.SQLContext import org.apache.spark.sql.types.StringType -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class HiveNativeCommand(sql: String) extends RunnableCommand { override def output = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index b56175fe76376..5b3cf2861e8ef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -26,21 +26,19 @@ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.types.{BooleanType, DataType} /** - * :: DeveloperApi :: * The Hive table scan operator. Column and partition pruning are both handled. * * @param requestedAttributes Attributes to be fetched from the Hive table. * @param relation The Hive table be be scanned. * @param partitionPruningPred An optional partition pruning predicate for partitioned table. */ -@DeveloperApi +private[hive] case class HiveTableScan( requestedAttributes: Seq[Attribute], relation: MetastoreRelation, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 91af35f0965c0..ba5c8e028a151 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{UnaryNode, SparkPlan} @@ -41,10 +40,7 @@ import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.{SerializableWritable, SparkException, TaskContext} -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index c54fbb6e24690..0c9aee33985bc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -21,15 +21,12 @@ import java.io.{BufferedReader, InputStreamReader} import java.io.{DataInputStream, DataOutputStream, EOFException} import java.util.Properties +import scala.collection.JavaConversions._ + import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe -import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ @@ -38,19 +35,14 @@ import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.util.Utils - -/* Implicit conversions */ -import scala.collection.JavaConversions._ - /** - * :: DeveloperApi :: * Transforms the input by forking and running the specified script. * * @param input the set of expression that should be passed to the script. * @param script the command that should be executed. * @param output the attributes that are produced by the script. */ -@DeveloperApi +private[hive] case class ScriptTransformation( input: Seq[Expression], script: String, @@ -175,6 +167,7 @@ case class ScriptTransformation( /** * The wrapper class of Hive input and output schema properties */ +private[hive] case class HiveScriptIOSchema ( inputRowFormat: Seq[(String, String)], outputRowFormat: Seq[(String, String)], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 36bd3f8fe21f2..63ad145a6a980 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ @@ -30,14 +29,13 @@ import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types.StructType /** - * :: DeveloperApi :: * Analyzes the given table in the current database to generate statistics, which will be * used in query optimizations. * * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ -@DeveloperApi +private[hive] case class AnalyzeTable(tableName: String) extends RunnableCommand { override def run(sqlContext: SQLContext) = { @@ -47,10 +45,9 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { } /** - * :: DeveloperApi :: * Drops a table from the metastore and removes it if it is cached. */ -@DeveloperApi +private[hive] case class DropTable( tableName: String, ifExists: Boolean) extends RunnableCommand { @@ -75,10 +72,7 @@ case class DropTable( } } -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class AddJar(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext) = { @@ -89,10 +83,7 @@ case class AddJar(path: String) extends RunnableCommand { } } -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class AddFile(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext) = { @@ -103,10 +94,7 @@ case class AddFile(path: String) extends RunnableCommand { } } -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class CreateMetastoreDataSource( tableName: String, userSpecifiedSchema: Option[StructType], @@ -146,10 +134,7 @@ case class CreateMetastoreDataSource( } } -/** - * :: DeveloperApi :: - */ -@DeveloperApi +private[hive] case class CreateMetastoreDataSourceAsSelect( tableName: String, provider: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala deleted file mode 100644 index 4989c42e964ec..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/package.scala +++ /dev/null @@ -1,25 +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 - -/** - * Physical execution operators used for running queries against data stored in Hive. These - * are not intended for use by users, but are documents so that it is easier to understand - * the output of EXPLAIN queries. - */ -package object execution diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala similarity index 96% rename from sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index e89b4489f15d1..7ff5719adb3ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.parquet +package org.apache.spark.sql.hive -import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.parquet.ParquetTest +import org.apache.spark.sql.{QueryTest, SQLConf} case class Cases(lower: String, UPPER: String) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala similarity index 98% rename from sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 89b943f008094..1904f5faef3a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -16,19 +16,20 @@ * limitations under the License. */ -package org.apache.spark.sql.parquet +package org.apache.spark.sql.hive import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.{QueryTest, SQLConf, SaveMode} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} -import org.apache.spark.sql.hive.execution.{InsertIntoHiveTable, HiveTableScan} +import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} +import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.SaveMode import org.apache.spark.sql.types._ @@ -393,7 +394,7 @@ class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") df.queryExecution.executedPlan match { - case insert: InsertIntoHiveTable => // OK + case insert: execution.InsertIntoHiveTable => // OK case o => fail(s"The SparkPlan should be ${classOf[InsertIntoHiveTable].getCanonicalName}. " + s"However, found ${o.toString}.") } @@ -422,7 +423,7 @@ class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") df.queryExecution.executedPlan match { - case insert: InsertIntoHiveTable => // OK + case insert: execution.InsertIntoHiveTable => // OK case o => fail(s"The SparkPlan should be ${classOf[InsertIntoHiveTable].getCanonicalName}. " + s"However, found ${o.toString}.") } From 7e53a79c30511dbd0e5d9878a4b8b0f5bc94e68b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Mar 2015 22:27:01 -0800 Subject: [PATCH 329/817] [SPARK-6097][MLLIB] Support tree model save/load in PySpark/MLlib Similar to `MatrixFactorizaionModel`, we only need wrappers to support save/load for tree models in Python. jkbradley Author: Xiangrui Meng Closes #4854 from mengxr/SPARK-6097 and squashes the following commits: 4586a4d [Xiangrui Meng] fix more typos 8ebcac2 [Xiangrui Meng] fix python style 91172d8 [Xiangrui Meng] fix typos 201b3b9 [Xiangrui Meng] update user guide b5158e2 [Xiangrui Meng] support tree model save/load in PySpark/MLlib --- docs/mllib-decision-tree.md | 16 ++++++----- docs/mllib-ensembles.md | 32 +++++++++++++--------- python/pyspark/mllib/recommendation.py | 9 +++---- python/pyspark/mllib/tests.py | 27 ++++++++++++++++++- python/pyspark/mllib/tree.py | 21 ++++++++++++--- python/pyspark/mllib/util.py | 37 +++++++++++++++++++++++--- 6 files changed, 109 insertions(+), 33 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 8e478ab035582..c1d0f8a6b1cd8 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -293,11 +293,9 @@ DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath");
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.tree import DecisionTree, DecisionTreeModel from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. @@ -317,6 +315,10 @@ testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(tes print('Test Error = ' + str(testErr)) print('Learned classification tree model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = DecisionTreeModel.load(sc, "myModelPath") {% endhighlight %}
    @@ -440,11 +442,9 @@ DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath");
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.tree import DecisionTree, DecisionTreeModel from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. @@ -464,6 +464,10 @@ testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / flo print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression tree model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = DecisionTreeModel.load(sc, "myModelPath") {% endhighlight %}
    diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index ec1ef38b453d3..cbfb682609af3 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -202,10 +202,8 @@ RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath");
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.tree import RandomForest +from pyspark.mllib.tree import RandomForest, RandomForestModel from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. @@ -228,6 +226,10 @@ testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(tes print('Test Error = ' + str(testErr)) print('Learned classification forest model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = RandomForestModel.load(sc, "myModelPath") {% endhighlight %}
    @@ -354,10 +356,8 @@ RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath");
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.tree import RandomForest +from pyspark.mllib.tree import RandomForest, RandomForestModel from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. @@ -380,6 +380,10 @@ testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / flo print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression forest model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = RandomForestModel.load(sc, "myModelPath") {% endhighlight %}
    @@ -581,10 +585,8 @@ GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "m
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.tree import GradientBoostedTrees +from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel from pyspark.mllib.util import MLUtils # Load and parse the data file. @@ -605,6 +607,10 @@ testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(tes print('Test Error = ' + str(testErr)) print('Learned classification GBT model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") {% endhighlight %}
    @@ -732,10 +738,8 @@ GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "m
    -Note that the Python API does not yet support model save/load but will in the future. - {% highlight python %} -from pyspark.mllib.tree import GradientBoostedTrees +from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel from pyspark.mllib.util import MLUtils # Load and parse the data file. @@ -756,6 +760,10 @@ testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / flo print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression GBT model:') print(model.toDebugString()) + +# Save and load model +model.save(sc, "myModelPath") +sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") {% endhighlight %}
    diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 03d7d011474cb..1a4527b12cef2 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -20,7 +20,7 @@ from pyspark import SparkContext from pyspark.rdd import RDD from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc -from pyspark.mllib.util import Saveable, JavaLoader +from pyspark.mllib.util import JavaLoader, JavaSaveable __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -41,7 +41,7 @@ def __reduce__(self): @inherit_doc -class MatrixFactorizationModel(JavaModelWrapper, Saveable, JavaLoader): +class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): """A matrix factorisation model trained by regularized alternating least-squares. @@ -92,7 +92,7 @@ class MatrixFactorizationModel(JavaModelWrapper, Saveable, JavaLoader): 0.43... >>> try: ... os.removedirs(path) - ... except: + ... except OSError: ... pass """ def predict(self, user, product): @@ -111,9 +111,6 @@ def userFeatures(self): def productFeatures(self): return self.call("getProductFeatures") - def save(self, sc, path): - self.call("save", sc._jsc.sc(), path) - class ALS(object): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 06207a076eece..5328d99b69684 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -19,7 +19,9 @@ Fuller unit tests for Python MLlib. """ +import os import sys +import tempfile import array as pyarray from numpy import array, array_equal @@ -195,7 +197,8 @@ def test_gmm_deterministic(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes - from pyspark.mllib.tree import DecisionTree, RandomForest, GradientBoostedTrees + from pyspark.mllib.tree import DecisionTree, DecisionTreeModel, RandomForest,\ + RandomForestModel, GradientBoostedTrees, GradientBoostedTreesModel data = [ LabeledPoint(0.0, [1, 0, 0]), LabeledPoint(1.0, [0, 1, 1]), @@ -205,6 +208,8 @@ def test_classification(self): rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] + temp_dir = tempfile.mkdtemp() + lr_model = LogisticRegressionWithSGD.train(rdd) self.assertTrue(lr_model.predict(features[0]) <= 0) self.assertTrue(lr_model.predict(features[1]) > 0) @@ -231,6 +236,11 @@ def test_classification(self): self.assertTrue(dt_model.predict(features[2]) <= 0) self.assertTrue(dt_model.predict(features[3]) > 0) + dt_model_dir = os.path.join(temp_dir, "dt") + dt_model.save(self.sc, dt_model_dir) + same_dt_model = DecisionTreeModel.load(self.sc, dt_model_dir) + self.assertEqual(same_dt_model.toDebugString(), dt_model.toDebugString()) + rf_model = RandomForest.trainClassifier( rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100) self.assertTrue(rf_model.predict(features[0]) <= 0) @@ -238,6 +248,11 @@ def test_classification(self): self.assertTrue(rf_model.predict(features[2]) <= 0) self.assertTrue(rf_model.predict(features[3]) > 0) + rf_model_dir = os.path.join(temp_dir, "rf") + rf_model.save(self.sc, rf_model_dir) + same_rf_model = RandomForestModel.load(self.sc, rf_model_dir) + self.assertEqual(same_rf_model.toDebugString(), rf_model.toDebugString()) + gbt_model = GradientBoostedTrees.trainClassifier( rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) self.assertTrue(gbt_model.predict(features[0]) <= 0) @@ -245,6 +260,16 @@ def test_classification(self): self.assertTrue(gbt_model.predict(features[2]) <= 0) self.assertTrue(gbt_model.predict(features[3]) > 0) + gbt_model_dir = os.path.join(temp_dir, "gbt") + gbt_model.save(self.sc, gbt_model_dir) + same_gbt_model = GradientBoostedTreesModel.load(self.sc, gbt_model_dir) + self.assertEqual(same_gbt_model.toDebugString(), gbt_model.toDebugString()) + + try: + os.removedirs(temp_dir) + except OSError: + pass + def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ RidgeRegressionWithSGD diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 73618f0449ad4..bf288d76447bd 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -23,12 +23,13 @@ from pyspark.mllib.common import callMLlibFunc, inherit_doc, JavaModelWrapper from pyspark.mllib.linalg import _convert_to_vector from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.util import JavaLoader, JavaSaveable __all__ = ['DecisionTreeModel', 'DecisionTree', 'RandomForestModel', 'RandomForest', 'GradientBoostedTreesModel', 'GradientBoostedTrees'] -class TreeEnsembleModel(JavaModelWrapper): +class TreeEnsembleModel(JavaModelWrapper, JavaSaveable): def predict(self, x): """ Predict values for a single data point or an RDD of points using @@ -66,7 +67,7 @@ def toDebugString(self): return self._java_model.toDebugString() -class DecisionTreeModel(JavaModelWrapper): +class DecisionTreeModel(JavaModelWrapper, JavaSaveable, JavaLoader): """ .. note:: Experimental @@ -103,6 +104,10 @@ def toDebugString(self): """ full model. """ return self._java_model.toDebugString() + @classmethod + def _java_loader_class(cls): + return "org.apache.spark.mllib.tree.model.DecisionTreeModel" + class DecisionTree(object): """ @@ -227,13 +232,17 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, @inherit_doc -class RandomForestModel(TreeEnsembleModel): +class RandomForestModel(TreeEnsembleModel, JavaLoader): """ .. note:: Experimental Represents a random forest model. """ + @classmethod + def _java_loader_class(cls): + return "org.apache.spark.mllib.tree.model.RandomForestModel" + class RandomForest(object): """ @@ -406,13 +415,17 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt @inherit_doc -class GradientBoostedTreesModel(TreeEnsembleModel): +class GradientBoostedTreesModel(TreeEnsembleModel, JavaLoader): """ .. note:: Experimental Represents a gradient-boosted tree model. """ + @classmethod + def _java_loader_class(cls): + return "org.apache.spark.mllib.tree.model.GradientBoostedTreesModel" + class GradientBoostedTrees(object): """ diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 17d43eadba12b..e877c720ac77a 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,7 +18,7 @@ import numpy as np import warnings -from pyspark.mllib.common import callMLlibFunc +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -191,6 +191,17 @@ def save(self, sc, path): raise NotImplementedError +@inherit_doc +class JavaSaveable(Saveable): + """ + Mixin for models that provide save() through their Scala + implementation. + """ + + def save(self, sc, path): + self._java_model.save(sc._jsc.sc(), path) + + class Loader(object): """ Mixin for classes which can load saved models from files. @@ -210,6 +221,7 @@ def load(cls, sc, path): raise NotImplemented +@inherit_doc class JavaLoader(Loader): """ Mixin for classes which can load saved models using its Scala @@ -217,13 +229,30 @@ class JavaLoader(Loader): """ @classmethod - def load(cls, sc, path): + def _java_loader_class(cls): + """ + Returns the full class name of the Java loader. The default + implementation replaces "pyspark" by "org.apache.spark" in + the Python full class name. + """ java_package = cls.__module__.replace("pyspark", "org.apache.spark") - java_class = ".".join([java_package, cls.__name__]) + return ".".join([java_package, cls.__name__]) + + @classmethod + def _load_java(cls, sc, path): + """ + Load a Java model from the given path. + """ + java_class = cls._java_loader_class() java_obj = sc._jvm for name in java_class.split("."): java_obj = getattr(java_obj, name) - return cls(java_obj.load(sc._jsc.sc(), path)) + return java_obj.load(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = cls._load_java(sc, path) + return cls(java_model) def _test(): From c2fe3a6ff1a48a9da54d2c2c4d80ecd06cdeebca Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Mar 2015 22:33:51 -0800 Subject: [PATCH 330/817] [SPARK-6120] [mllib] Warnings about memory in tree, ensemble model save Issue: When the Python DecisionTree example in the programming guide is run, it runs out of Java Heap Space when using the default memory settings for the spark shell. This prints a warning. CC: mengxr Author: Joseph K. Bradley Closes #4864 from jkbradley/dt-save-heap and squashes the following commits: 02e8daf [Joseph K. Bradley] fixed based on code review 7ecb1ed [Joseph K. Bradley] Added warnings about memory when calling tree and ensemble model save with too small a Java heap size --- .../mllib/tree/model/DecisionTreeModel.scala | 27 +++++++++++++++++-- .../mllib/tree/model/treeEnsembleModels.scala | 27 +++++++++++++++++-- 2 files changed, 50 insertions(+), 4 deletions(-) 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 060fd5b859a51..8a57ebc387d01 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 @@ -23,7 +23,7 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector @@ -32,6 +32,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.util.Utils /** * :: Experimental :: @@ -115,7 +116,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable override protected def formatVersion: String = "1.0" } -object DecisionTreeModel extends Loader[DecisionTreeModel] { +object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { private[tree] object SaveLoadV1_0 { @@ -187,6 +188,28 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] { val sqlContext = new SQLContext(sc) import sqlContext.implicits._ + // SPARK-6120: We do a hacky check here so users understand why save() is failing + // when they run the ML guide example. + // TODO: Fix this issue for real. + val memThreshold = 768 + if (sc.isLocal) { + val driverMemory = sc.getConf.getOption("spark.driver.memory") + .orElse(Option(System.getenv("SPARK_DRIVER_MEMORY"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + if (driverMemory <= memThreshold) { + logWarning(s"$thisClassName.save() was called, but it may fail because of too little" + + s" driver memory (${driverMemory}m)." + + s" If failure occurs, try setting driver-memory ${memThreshold}m (or larger).") + } + } else { + if (sc.executorMemory <= memThreshold) { + logWarning(s"$thisClassName.save() was called, but it may fail because of too little" + + s" executor memory (${sc.executorMemory}m)." + + s" If failure occurs try setting executor-memory ${memThreshold}m (or larger).") + } + } + // Create JSON metadata. val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ 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 4897906aea5b3..30a8f7ca301af 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 @@ -24,7 +24,7 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector @@ -34,6 +34,7 @@ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext +import org.apache.spark.util.Utils /** * :: Experimental :: @@ -250,7 +251,7 @@ private[tree] sealed class TreeEnsembleModel( def totalNumNodes: Int = trees.map(_.numNodes).sum } -private[tree] object TreeEnsembleModel { +private[tree] object TreeEnsembleModel extends Logging { object SaveLoadV1_0 { @@ -277,6 +278,28 @@ private[tree] object TreeEnsembleModel { val sqlContext = new SQLContext(sc) import sqlContext.implicits._ + // SPARK-6120: We do a hacky check here so users understand why save() is failing + // when they run the ML guide example. + // TODO: Fix this issue for real. + val memThreshold = 768 + if (sc.isLocal) { + val driverMemory = sc.getConf.getOption("spark.driver.memory") + .orElse(Option(System.getenv("SPARK_DRIVER_MEMORY"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + if (driverMemory <= memThreshold) { + logWarning(s"$className.save() was called, but it may fail because of too little" + + s" driver memory (${driverMemory}m)." + + s" If failure occurs, try setting driver-memory ${memThreshold}m (or larger).") + } + } else { + if (sc.executorMemory <= memThreshold) { + logWarning(s"$className.save() was called, but it may fail because of too little" + + s" executor memory (${sc.executorMemory}m)." + + s" If failure occurs try setting executor-memory ${memThreshold}m (or larger).") + } + } + // Create JSON metadata. implicit val format = DefaultFormats val ensembleMetadata = Metadata(model.algo.toString, model.trees(0).algo.toString, From b196056190c569505cc32669d1aec30ed9d70665 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 2 Mar 2015 22:37:12 -0800 Subject: [PATCH 331/817] [SPARK-5537][MLlib][Docs] Add user guide for multinomial logistic regression Adding more description on top of #4861. Author: DB Tsai Closes #4866 from dbtsai/doc and squashes the following commits: 37e9d07 [DB Tsai] doc --- docs/mllib-linear-methods.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 03f90d718ad8c..9270741d439d9 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -784,9 +784,19 @@ regularization parameter (`regParam`) along with various parameters associated w gradient descent (`stepSize`, `numIterations`, `miniBatchFraction`). For each of them, we support all three possible regularizations (none, L1 or L2). +For Logistic Regression, [L-BFGS](api/scala/index.html#org.apache.spark.mllib.optimization.LBFGS) +version is implemented under [LogisticRegressionWithLBFGS] +(api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS), and this +version supports both binary and multinomial Logistic Regression while SGD version only supports +binary Logistic Regression. However, L-BFGS version doesn't support L1 regularization but SGD one +supports L1 regularization. When L1 regularization is not required, L-BFGS version is strongly +recommended since it converges faster and more accurately compared to SGD by approximating the +inverse Hessian matrix using quasi-Newton method. + Algorithms are all implemented in Scala: * [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS) * [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) * [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) * [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) From 165ff364265a5328f797e0901fe0dc02f30a7e7a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 3 Mar 2015 01:38:07 -0800 Subject: [PATCH 332/817] HOTFIX: Bump HBase version in MapR profiles. After #2982 (SPARK-4048) we rely on the newer HBase packaging format. --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index bb355bf735bee..77d08d6ee826f 100644 --- a/pom.xml +++ b/pom.xml @@ -1605,7 +1605,7 @@ 1.0.3-mapr-3.0.3 2.4.1-mapr-1408 - 0.94.17-mapr-1405 + 0.98.4-mapr-1408 3.4.5-mapr-1406
    @@ -1615,7 +1615,7 @@ 2.4.1-mapr-1408 2.4.1-mapr-1408 - 0.94.17-mapr-1405-4.0.0-FCS + 0.98.4-mapr-1408 3.4.5-mapr-1406 From 0c9a8eaed74b2b381a1cc70e6e6a2783bac0912c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 3 Mar 2015 00:38:12 -0800 Subject: [PATCH 333/817] BUILD: Minor tweaks to internal build scripts This adds two features: 1. The ability to publish with a different maven version than that specified in the release source. 2. Forking of different Zinc instances during the parallel dist creation (to help with some stability issues). --- dev/create-release/create-release.sh | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index da15ce3e0e2f7..6f87fcd6d4eb4 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -34,6 +34,9 @@ ASF_PASSWORD=${ASF_PASSWORD:-XXX} GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} GIT_BRANCH=${GIT_BRANCH:-branch-1.0} RELEASE_VERSION=${RELEASE_VERSION:-1.2.0} +# Allows publishing under a different version identifier than +# was present in the actual release sources (e.g. rc-X) +PUBLISH_VERSION=${PUBLISH_VERSION:-$RELEASE_VERSION} NEXT_VERSION=${NEXT_VERSION:-1.2.1} RC_NAME=${RC_NAME:-rc2} @@ -97,30 +100,35 @@ if [[ ! "$@" =~ --skip-publish ]]; then pushd spark git checkout --force $GIT_TAG + # Substitute in case published version is different than released + old="^\( \{2,4\}\)${RELEASE_VERSION}<\/version>$" + new="\1${PUBLISH_VERSION}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" - repo_request="Apache Spark $GIT_TAG" + repo_request="Apache Spark $GIT_TAG (published as $PUBLISH_VERSION)" out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ -H "Content-Type:application/xml" -v \ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start) staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachespark-[0-9]\{4\}\).*/\1/") echo "Created Nexus staging repository: $staged_repo_id" - rm -rf $SPARK_REPO - - mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.11.sh - mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.10.sh + rm -rf $SPARK_REPO pushd $SPARK_REPO # Remove any extra files generated during install @@ -197,6 +205,12 @@ if [[ ! "$@" =~ --skip-package ]]; then ./dev/change-version-to-2.11.sh fi + # Create new Zinc instances for each binary release to avoid interference + # that causes OOM's and random compiler crashes. + zinc_port=${zinc_port:-3030} + zinc_port=$[$zinc_port + 1] + export ZINC_PORT=$zinc_port + ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . From 975643c256e548601bf9015c8840c947df5446bf Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 3 Mar 2015 10:32:57 +0000 Subject: [PATCH 334/817] [SPARK-6118] making package name of deploy.worker.CommandUtils and deploy.CommandUtilsSuite consistent https://issues.apache.org/jira/browse/SPARK-6118 I found that the object CommandUtils is placed under deploy.worker package, while CommandUtilsSuite is under deploy Conventionally, we put the implementation and unit test class under the same package here, to minimize the change, I move CommandUtilsSuite to worker package, **However, CommandUtils seems to contain some general methods (though only used by worker.* classes currently**, we may also consider to replace CommonUtils Author: CodingCat Closes #4856 from CodingCat/SPARK-6118 and squashes the following commits: cb93700 [CodingCat] making package name consistent --- .../apache/spark/deploy/{ => worker}/CommandUtilsSuite.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) rename core/src/test/scala/org/apache/spark/deploy/{ => worker}/CommandUtilsSuite.scala (94%) diff --git a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala similarity index 94% rename from core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala rename to core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala index 7915ee75d8778..1c27d83cf876c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.deploy +package org.apache.spark.deploy.worker -import org.apache.spark.deploy.worker.CommandUtils +import org.apache.spark.deploy.Command import org.apache.spark.util.Utils - import org.scalatest.{FunSuite, Matchers} class CommandUtilsSuite extends FunSuite with Matchers { From e359794cec7d30ece38752f62dc2a1d3d26b8feb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 3 Mar 2015 12:12:23 +0000 Subject: [PATCH 335/817] [SPARK-6138][CORE][minor] enhance the `toArray` method in `SizeTrackingVector` Use array copy instead of `Iterator#toArray` to make it more efficient. Author: Wenchen Fan Closes #4825 from cloud-fan/minor and squashes the following commits: c933ee5 [Wenchen Fan] make toArray method just in parent 946a35b [Wenchen Fan] minor enhance --- .../spark/util/collection/PrimitiveVector.scala | 15 ++++++++++++--- .../util/collection/SizeTrackingVector.scala | 7 ------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index 7e76d060d6000..b6c380a8eea9f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -71,12 +71,21 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: /** Resizes the array, dropping elements if the total length decreases. */ def resize(newLength: Int): PrimitiveVector[V] = { - val newArray = new Array[V](newLength) - _array.copyToArray(newArray) - _array = newArray + _array = copyArrayWithLength(newLength) if (newLength < _numElements) { _numElements = newLength } this } + + /** Return a trimmed version of the underlying array. */ + def toArray: Array[V] = { + copyArrayWithLength(size) + } + + private def copyArrayWithLength(length: Int): Array[V] = { + val copy = new Array[V](length) + _array.copyToArray(copy) + copy + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala index 65a7b4e0d497b..dfcfb66af8613 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala @@ -36,11 +36,4 @@ private[spark] class SizeTrackingVector[T: ClassTag] resetSamples() this } - - /** - * Return a trimmed version of the underlying array. - */ - def toArray: Array[T] = { - super.iterator.toArray - } } From 9af001749a37a86ccbf78063ec514a21801645fa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 Mar 2015 13:03:52 -0800 Subject: [PATCH 336/817] Revert "[SPARK-5423][Core] Cleanup resources in DiskMapIterator.finalize to ensure deleting the temp file" This reverts commit 90095bf3ce9304d09a32ceffaa99069079071b59. --- .../collection/ExternalAppendOnlyMap.scala | 52 ++++--------------- 1 file changed, 9 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index fc7e86e297540..8a0f5a602de12 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -387,15 +387,6 @@ class ExternalAppendOnlyMap[K, V, C]( private var batchIndex = 0 // Which batch we're in private var fileStream: FileInputStream = null - @volatile private var closed = false - - // A volatile variable to remember which DeserializationStream is using. Need to set it when we - // open a DeserializationStream. But we should use `deserializeStream` rather than - // `deserializeStreamToBeClosed` to read the content because touching a volatile variable will - // reduce the performance. It must be volatile so that we can see its correct value in the - // `finalize` method, which could run in any thread. - @volatile private var deserializeStreamToBeClosed: DeserializationStream = null - // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams private var deserializeStream = nextBatchStream() @@ -410,7 +401,6 @@ class ExternalAppendOnlyMap[K, V, C]( // we're still in a valid batch. if (batchIndex < batchOffsets.length - 1) { if (deserializeStream != null) { - deserializeStreamToBeClosed = null deserializeStream.close() fileStream.close() deserializeStream = null @@ -429,11 +419,7 @@ class ExternalAppendOnlyMap[K, V, C]( val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) val compressedStream = blockManager.wrapForCompression(blockId, bufferedStream) - // Before returning the stream, assign it to `deserializeStreamToBeClosed` so that we can - // close it in `finalize` and also avoid to touch the volatile `deserializeStreamToBeClosed` - // during reading the (K, C) pairs. - deserializeStreamToBeClosed = ser.deserializeStream(compressedStream) - deserializeStreamToBeClosed + ser.deserializeStream(compressedStream) } else { // No more batches left cleanup() @@ -482,34 +468,14 @@ class ExternalAppendOnlyMap[K, V, C]( item } - // TODO: Now only use `finalize` to ensure `close` gets called to clean up the resources. In the - // future, we need some mechanism to ensure this gets called once the resources are not used. - private def cleanup(): Unit = { - if (!closed) { - closed = true - batchIndex = batchOffsets.length // Prevent reading any other batch - fileStream = null - try { - val ds = deserializeStreamToBeClosed - deserializeStreamToBeClosed = null - deserializeStream = null - if (ds != null) { - ds.close() - } - } finally { - if (file.exists()) { - file.delete() - } - } - } - } - - override def finalize(): Unit = { - try { - cleanup() - } finally { - super.finalize() - } + // TODO: Ensure this gets called even if the iterator isn't drained. + private def cleanup() { + batchIndex = batchOffsets.length // Prevent reading any other batch + val ds = deserializeStream + deserializeStream = null + fileStream = null + ds.close() + file.delete() } } From e750a6bfddf1d7bf7d3e99a424ec2b83a18b40d9 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 3 Mar 2015 13:40:11 -0800 Subject: [PATCH 337/817] SPARK-1911 [DOCS] Warn users if their assembly jars are not built with Java 6 Add warning about building with Java 7+ and running the JAR on early Java 6. CC andrewor14 Author: Sean Owen Closes #4874 from srowen/SPARK-1911 and squashes the following commits: 79fa2f6 [Sean Owen] Add warning about building with Java 7+ and running the JAR on early Java 6. --- docs/building-spark.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/building-spark.md b/docs/building-spark.md index 4c3988e819ad8..57d0ca834f460 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,6 +9,10 @@ redirect_from: "building-with-maven.html" Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+. +**Note:** Building Spark with Java 7 or later can create JAR files that may not be +readable with early versions of Java 6, due to the large number of files in the JAR +archive. Build with Java 6 if this is an issue for your deployment. + # Building with `build/mvn` Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: From fe63e822918a01e1c1d741052b932e9944745fb6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 Mar 2015 13:44:05 -0800 Subject: [PATCH 338/817] [SPARK-6132] ContextCleaner race condition across SparkContexts The problem is that `ContextCleaner` may clean variables that belong to a different `SparkContext`. This can happen if the `SparkContext` to which the cleaner belongs stops, and a new one is started immediately afterwards in the same JVM. In this case, if the cleaner is in the middle of cleaning a broadcast, for instance, it will do so through `SparkEnv.get.blockManager`, which could be one that belongs to a different `SparkContext`. JoshRosen and I suspect that this is the cause of many flaky tests, most notably the `JavaAPISuite`. We were able to reproduce the failure locally (though it is not deterministic and very hard to reproduce). Author: Andrew Or Closes #4869 from andrewor14/cleaner-masquerade and squashes the following commits: 29168c0 [Andrew Or] Synchronize ContextCleaner stop --- .../org/apache/spark/ContextCleaner.scala | 39 ++++++++++++------- 1 file changed, 26 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 4a9d007353373..4dab886698e30 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -105,9 +105,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { cleaningThread.start() } - /** Stop the cleaner. */ + /** + * Stop the cleaning thread and wait until the thread has finished running its current task. + */ def stop() { stopped = true + // Interrupt the cleaning thread, but wait until the current task has finished before + // doing so. This guards against the race condition where a cleaning thread may + // potentially clean similarly named variables created by a different SparkContext, + // resulting in otherwise inexplicable block-not-found exceptions (SPARK-6132). + synchronized { + cleaningThread.interrupt() + } + cleaningThread.join() } /** Register a RDD for cleanup when it is garbage collected. */ @@ -140,18 +150,21 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) .map(_.asInstanceOf[CleanupTaskWeakReference]) - reference.map(_.task).foreach { task => - logDebug("Got cleaning task " + task) - referenceBuffer -= reference.get - task match { - case CleanRDD(rddId) => - doCleanupRDD(rddId, blocking = blockOnCleanupTasks) - case CleanShuffle(shuffleId) => - doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) - case CleanBroadcast(broadcastId) => - doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) - case CleanAccum(accId) => - doCleanupAccum(accId, blocking = blockOnCleanupTasks) + // Synchronize here to avoid being interrupted on stop() + synchronized { + reference.map(_.task).foreach { task => + logDebug("Got cleaning task " + task) + referenceBuffer -= reference.get + task match { + case CleanRDD(rddId) => + doCleanupRDD(rddId, blocking = blockOnCleanupTasks) + case CleanShuffle(shuffleId) => + doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) + case CleanBroadcast(broadcastId) => + doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) + case CleanAccum(accId) => + doCleanupAccum(accId, blocking = blockOnCleanupTasks) + } } } } catch { From 6c20f35290e220e4a659a0222d62575ff959d703 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 Mar 2015 15:09:57 -0800 Subject: [PATCH 339/817] [SPARK-6133] Make sc.stop() idempotent Before we would get the following (benign) error if we called `sc.stop()` twice. This is because the listener bus would try to post the end event again even after it has already stopped. This happens occasionally when flaky tests fail, usually as a result of other sources of error. Either way we shouldn't be logging this error when it is not the cause of the failure. ``` ERROR LiveListenerBus: SparkListenerBus has already stopped! Dropping event SparkListenerApplicationEnd(1425348445682) ``` Author: Andrew Or Closes #4871 from andrewor14/sc-stop and squashes the following commits: a14afc5 [Andrew Or] Move code after code 915db16 [Andrew Or] Move code into code --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e231e8369dbac..1a0bee4e3aea9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1392,10 +1392,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Shut down the SparkContext. */ def stop() { SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - postApplicationEnd() - ui.foreach(_.stop()) if (!stopped) { stopped = true + postApplicationEnd() + ui.foreach(_.stop()) env.metricsSystem.report() metadataCleaner.cancel() cleaner.foreach(_.stop()) From 1f1fccc5ceb0c5b7656a0594be3a67bd3b432e85 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 3 Mar 2015 15:33:19 -0800 Subject: [PATCH 340/817] [SPARK-5949] HighlyCompressedMapStatus needs more classes registered w/ kryo https://issues.apache.org/jira/browse/SPARK-5949 Author: Imran Rashid Closes #4877 from squito/SPARK-5949_register_roaring_bitmap and squashes the following commits: 7e13316 [Imran Rashid] style style style 5f6bb6d [Imran Rashid] more style 709bfe0 [Imran Rashid] style a5cb744 [Imran Rashid] update tests to cover both types of RoaringBitmapContainers 09610c6 [Imran Rashid] formatting f9a0b7c [Imran Rashid] put primitive array registrations together 97beaf8 [Imran Rashid] SPARK-5949 HighlyCompressedMapStatus needs more classes registered w/ kryo --- .../spark/serializer/KryoSerializer.scala | 15 +++++++++--- .../serializer/KryoSerializerSuite.scala | 23 +++++++++++++++++-- 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 02158aa0f866e..9ce64d41fbc40 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -20,22 +20,23 @@ package org.apache.spark.serializer import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import scala.reflect.ClassTag + import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} +import org.roaringbitmap.{ArrayContainer, BitmapContainer, RoaringArray, RoaringBitmap} import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast -import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} +import org.apache.spark.network.nio.{GetBlock, GotBlock, PutBlock} import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer -import scala.reflect.ClassTag - /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. * @@ -202,9 +203,17 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[CompressedMapStatus], classOf[HighlyCompressedMapStatus], + classOf[RoaringBitmap], + classOf[RoaringArray], + classOf[RoaringArray.Element], + classOf[Array[RoaringArray.Element]], + classOf[ArrayContainer], + classOf[BitmapContainer], classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Byte]], + classOf[Array[Short]], + classOf[Array[Long]], classOf[BoundedPriorityQueue[_]], classOf[SparkConf] ) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index a70f67af2e62e..523d898207447 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -23,9 +23,10 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SharedSparkContext} +import org.apache.spark.{SharedSparkContext, SparkConf} +import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ - +import org.apache.spark.storage.BlockManagerId class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") @@ -242,6 +243,24 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) } } + + test("registration of HighlyCompressedMapStatus") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 + // values, and they use a bitmap (dense) if they have more than 4096 values, and an + // array (sparse) if they use less. So we just create two cases, one sparse and one dense. + // and we use a roaring bitmap for the empty blocks, so we trigger the dense case w/ mostly + // empty blocks + + val ser = new KryoSerializer(conf).newInstance() + val denseBlockSizes = new Array[Long](5000) + val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) + Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => + ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + } + } } From d334bfbcf38d0c1c6835e72bc85e42196e295744 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 3 Mar 2015 20:49:45 -0800 Subject: [PATCH 341/817] [SPARK-6132][HOTFIX] ContextCleaner InterruptedException should be quiet If the cleaner is stopped, we shouldn't print a huge stack trace when the cleaner thread is interrupted because we purposefully did this. Author: Andrew Or Closes #4882 from andrewor14/cleaner-interrupt and squashes the following commits: 8652120 [Andrew Or] Just a hot fix --- core/src/main/scala/org/apache/spark/ContextCleaner.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 4dab886698e30..0c59a61e81393 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -168,6 +168,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } } catch { + case ie: InterruptedException if stopped => // ignore case e: Exception => logError("Error in cleaning thread", e) } } From 76e20a0a03cf2c02db35e00271924efb070eaaa5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 3 Mar 2015 23:52:02 -0800 Subject: [PATCH 342/817] [SPARK-6141][MLlib] Upgrade Breeze from 0.10 to 0.11 to fix convergence bug LBFGS and OWLQN in Breeze 0.10 has convergence check bug. This is fixed in 0.11, see the description in Breeze project for detail: https://github.com/scalanlp/breeze/pull/373#issuecomment-76879760 Author: Xiangrui Meng Author: DB Tsai Author: DB Tsai Closes #4879 from dbtsai/breeze and squashes the following commits: d848f65 [DB Tsai] Merge pull request #1 from mengxr/AlpineNow-breeze c2ca6ac [Xiangrui Meng] upgrade to breeze-0.11.1 35c2f26 [Xiangrui Meng] fix LRSuite 397a208 [DB Tsai] upgrade breeze --- mllib/pom.xml | 2 +- .../spark/mllib/classification/LogisticRegressionSuite.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index a8cee3d51a780..4c8f34417ca65 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -63,7 +63,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.10 + 0.11.1 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index d2b40f2cae020..aaa81da9e273c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -372,8 +372,12 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M testRDD2.cache() testRDD3.cache() + val numIteration = 10 + val lrA = new LogisticRegressionWithLBFGS().setIntercept(true) + lrA.optimizer.setNumIterations(numIteration) val lrB = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + lrB.optimizer.setNumIterations(numIteration) val modelA1 = lrA.run(testRDD1, initialWeights) val modelA2 = lrA.run(testRDD2, initialWeights) From 8d3e2414d430e1a0eb209eacba2cf739f3eab0c5 Mon Sep 17 00:00:00 2001 From: tedyu Date: Wed, 4 Mar 2015 11:00:52 +0000 Subject: [PATCH 343/817] SPARK-6085 Increase default value for memory overhead Author: tedyu Closes #4836 from tedyu/master and squashes the following commits: d65b495 [tedyu] SPARK-6085 Increase default value for memory overhead 1fdd4df [tedyu] SPARK-6085 Increase default value for memory overhead --- .../apache/spark/scheduler/cluster/mesos/MemoryUtils.scala | 2 +- docs/running-on-mesos.md | 2 +- docs/running-on-yarn.md | 2 +- .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala index 5101ec8352e79..705116cb13f54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkContext private[spark] object MemoryUtils { // These defaults copied from YARN - val OVERHEAD_FRACTION = 1.07 + val OVERHEAD_FRACTION = 1.10 val OVERHEAD_MINIMUM = 384 def calculateTotalMemory(sc: SparkContext) = { diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index db1173a06b0b1..e509e4bf37396 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -225,7 +225,7 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.executor.memoryOverhead - executor memory * 0.07, with minimum of 384 + executor memory * 0.10, with minimum of 384 This value is an additive for spark.executor.memory, specified in MiB, which is used to calculate the total Mesos task memory. A value of 384 diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 2b93eef6c26ed..68b1aeb8ebd01 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -113,7 +113,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.executor.memoryOverhead - executorMemory * 0.07, with minimum of 384 + executorMemory * 0.10, with minimum of 384 The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%). diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 146b2c0f1a302..5881dc5ffa3ad 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -86,10 +86,10 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { object YarnSparkHadoopUtil { // Additional memory overhead - // 7% was arrived at experimentally. In the interest of minimizing memory waste while covering + // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering // the common cases. Memory overhead tends to grow with container size. - val MEMORY_OVERHEAD_FACTOR = 0.07 + val MEMORY_OVERHEAD_FACTOR = 0.10 val MEMORY_OVERHEAD_MIN = 384 val ANY_HOST = "*" From 418f38d92f000b3f62fd363fbbc7ee22583d8ec3 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Wed, 4 Mar 2015 11:02:33 +0000 Subject: [PATCH 344/817] [SPARK-3355][Core]: Allow running maven tests in run-tests Added an AMPLAB_JENKINS_BUILD_TOOL env. variable to allow differentiation between maven and sbt build / test suites. The only issue I found with this is that, when running maven builds I wasn't able to get individual package tests running without running a `mvn install` first. Not sure what Jenkins is doing wrt its env., but figured its much better to just test everything than install packages in the "~/.m2/" directory and only test individual items, esp. if this is predominantly for the Jenkins build. Thoughts / comments would be great! Author: Brennon York Closes #4734 from brennonyork/SPARK-3355 and squashes the following commits: c813d32 [Brennon York] changed mvn call from 'clean compile 616ce30 [Brennon York] fixed merge conflicts 3540de9 [Brennon York] added an AMPLAB_JENKINS_BUILD_TOOL env. variable to allow differentiation between maven and sbt build / test suites --- dev/run-tests | 72 +++++++++++++++++++++++++++++++-------------------- 1 file changed, 44 insertions(+), 28 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 483958757a2dd..d6935a61c6d29 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -141,29 +141,41 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_BUILD { + HIVE_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" + HIVE_12_BUILD_ARGS="$HIVE_BUILD_ARGS -Phive-0.12.0" - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - # single argument! - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build - HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" echo "[info] Compile with Hive 0.12.0" - echo -e "q\n" \ - | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + [ -d "lib_managed" ] && rm -rf lib_managed + echo "[info] Building Spark with these arguments: $HIVE_12_BUILD_ARGS" + + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn $HIVE_12_BUILD_ARGS clean package -DskipTests + else + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + # single argument! + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi # Then build with default Hive version (0.13.1) because tests are based on this version echo "[info] Compile with Hive 0.13.1" - rm -rf lib_managed - echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ - " -Phive -Phive-thriftserver" - echo -e "q\n" \ - | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + [ -d "lib_managed" ] && rm -rf lib_managed + echo "[info] Building Spark with these arguments: $HIVE_BUILD_ARGS" + + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn $HIVE_BUILD_ARGS clean package -DskipTests + else + echo -e "q\n" \ + | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi } echo "" @@ -190,17 +202,21 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" - # NOTE: echo "q" is needed because sbt on encountering a build file with failure - # (either resolution or compilation) prompts the user for input either q, r, etc - # to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - # single argument! - # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. - # QUESTION: Why doesn't 'yes "q"' work? - # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - echo -e "q\n" \ - | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ - | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + if [ "${AMPLAB_JENKINS_BUILD_TOOL}" == "maven" ]; then + build/mvn test $SBT_MAVEN_PROFILES_ARGS --fail-at-end + else + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + fi } echo "" From 76b472f12a57bb5bec7b3791660eb47e9177da7f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 Mar 2015 19:39:02 +0800 Subject: [PATCH 345/817] [SPARK-6136] [SQL] Removed JDBC integration tests which depends on docker-client Integration test suites in the JDBC data source (`MySQLIntegration` and `PostgresIntegration`) depend on docker-client 2.7.5, which transitively depends on Guava 17.0. Unfortunately, Guava 17.0 is causing test runtime binary compatibility issues when Spark is compiled against Hive 0.12.0, or Hadoop 2.4. Considering `MySQLIntegration` and `PostgresIntegration` are ignored right now, I'd suggest moving them from the Spark project to the [Spark integration tests] [1] project. This PR removes both the JDBC data source integration tests and the docker-client test dependency. [1]: |https://github.com/databricks/spark-integration-tests [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4872) Author: Cheng Lian Closes #4872 from liancheng/remove-docker-client and squashes the following commits: 1f4169e [Cheng Lian] Removes DockerHacks 159b24a [Cheng Lian] Removed JDBC integration tests which depends on docker-client --- sql/core/pom.xml | 6 - .../apache/spark/sql/jdbc/DockerHacks.scala | 51 ---- .../spark/sql/jdbc/MySQLIntegration.scala | 228 ------------------ .../spark/sql/jdbc/PostgresIntegration.scala | 147 ----------- 4 files changed, 432 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e28baa512b95c..d4c8c687b67bd 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -99,12 +99,6 @@ 9.3-1102-jdbc41 test - - com.spotify - docker-client - 2.7.5 - test - target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala deleted file mode 100644 index f332cb389f339..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.jdbc - -import scala.collection.mutable.MutableList - -import com.spotify.docker.client._ - -/** - * A factory and morgue for DockerClient objects. In the DockerClient we use, - * calling close() closes the desired DockerClient but also renders all other - * DockerClients inoperable. This is inconvenient if we have more than one - * open, such as during tests. - */ -object DockerClientFactory { - var numClients: Int = 0 - val zombies = new MutableList[DockerClient]() - - def get(): DockerClient = { - this.synchronized { - numClients = numClients + 1 - DefaultDockerClient.fromEnv.build() - } - } - - def close(dc: DockerClient) { - this.synchronized { - numClients = numClients - 1 - zombies += dc - if (numClients == 0) { - zombies.foreach(_.close()) - zombies.clear() - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala deleted file mode 100644 index 5b8a76f461faf..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala +++ /dev/null @@ -1,228 +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.jdbc - -import java.math.BigDecimal -import java.sql.{Date, Timestamp} - -import com.spotify.docker.client.DockerClient -import com.spotify.docker.client.messages.ContainerConfig -import org.scalatest.{BeforeAndAfterAll, FunSuite, Ignore} - -import org.apache.spark.sql.test._ - -class MySQLDatabase { - val docker: DockerClient = DockerClientFactory.get() - val containerId = { - println("Pulling mysql") - docker.pull("mysql") - println("Configuring container") - val config = ContainerConfig.builder().image("mysql") - .env("MYSQL_ROOT_PASSWORD=rootpass") - .build() - println("Creating container") - val id = docker.createContainer(config).id - println("Starting container " + id) - docker.startContainer(id) - id - } - val ip = docker.inspectContainer(containerId).networkSettings.ipAddress - - def close() { - try { - println("Killing container " + containerId) - docker.killContainer(containerId) - println("Removing container " + containerId) - docker.removeContainer(containerId) - println("Closing docker client") - DockerClientFactory.close(docker) - } catch { - case e: Exception => - println(e) - println("You may need to clean this up manually.") - throw e - } - } -} - -@Ignore class MySQLIntegration extends FunSuite with BeforeAndAfterAll { - var ip: String = null - - def url(ip: String): String = url(ip, "mysql") - def url(ip: String, db: String): String = s"jdbc:mysql://$ip:3306/$db?user=root&password=rootpass" - - def waitForDatabase(ip: String, maxMillis: Long) { - println("Waiting for database to start up.") - val before = System.currentTimeMillis() - var lastException: java.sql.SQLException = null - while (true) { - if (System.currentTimeMillis() > before + maxMillis) { - throw new java.sql.SQLException(s"Database not up after $maxMillis ms.", lastException) - } - try { - val conn = java.sql.DriverManager.getConnection(url(ip)) - conn.close() - println("Database is up.") - return; - } catch { - case e: java.sql.SQLException => - lastException = e - java.lang.Thread.sleep(250) - } - } - } - - def setupDatabase(ip: String) { - val conn = java.sql.DriverManager.getConnection(url(ip)) - try { - conn.prepareStatement("CREATE DATABASE foo").executeUpdate() - conn.prepareStatement("CREATE TABLE foo.tbl (x INTEGER, y TEXT(8))").executeUpdate() - conn.prepareStatement("INSERT INTO foo.tbl VALUES (42,'fred')").executeUpdate() - conn.prepareStatement("INSERT INTO foo.tbl VALUES (17,'dave')").executeUpdate() - - conn.prepareStatement("CREATE TABLE foo.numbers (onebit BIT(1), tenbits BIT(10), " - + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, " - + "dbl DOUBLE)").executeUpdate() - conn.prepareStatement("INSERT INTO foo.numbers VALUES (b'0', b'1000100101', " - + "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, " - + "42.75, 1.0000000000000002)").executeUpdate() - - conn.prepareStatement("CREATE TABLE foo.dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, " - + "yr YEAR)").executeUpdate() - conn.prepareStatement("INSERT INTO foo.dates VALUES ('1991-11-09', '13:31:24', " - + "'1996-01-01 01:23:45', '2009-02-13 23:31:30', '2001')").executeUpdate() - - // TODO: Test locale conversion for strings. - conn.prepareStatement("CREATE TABLE foo.strings (a CHAR(10), b VARCHAR(10), c TINYTEXT, " - + "d TEXT, e MEDIUMTEXT, f LONGTEXT, g BINARY(4), h VARBINARY(10), i BLOB)" - ).executeUpdate() - conn.prepareStatement("INSERT INTO foo.strings VALUES ('the', 'quick', 'brown', 'fox', 'jumps', 'over', 'the', 'lazy', 'dog')").executeUpdate() - } finally { - conn.close() - } - } - - var db: MySQLDatabase = null - - override def beforeAll() { - // If you load the MySQL driver here, DriverManager will deadlock. The - // MySQL driver gets loaded when its jar gets loaded, unlike the Postgres - // and H2 drivers. - //Class.forName("com.mysql.jdbc.Driver") - - db = new MySQLDatabase() - waitForDatabase(db.ip, 60000) - setupDatabase(db.ip) - ip = db.ip - } - - override def afterAll() { - db.close() - } - - test("Basic test") { - val df = TestSQLContext.jdbc(url(ip, "foo"), "tbl") - val rows = df.collect() - assert(rows.length == 2) - val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 2) - assert(types(0).equals("class java.lang.Integer")) - assert(types(1).equals("class java.lang.String")) - } - - test("Numeric types") { - val df = TestSQLContext.jdbc(url(ip, "foo"), "numbers") - val rows = df.collect() - assert(rows.length == 1) - val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 9) - println(types(1)) - assert(types(0).equals("class java.lang.Boolean")) - assert(types(1).equals("class java.lang.Long")) - assert(types(2).equals("class java.lang.Integer")) - assert(types(3).equals("class java.lang.Integer")) - assert(types(4).equals("class java.lang.Integer")) - assert(types(5).equals("class java.lang.Long")) - assert(types(6).equals("class java.math.BigDecimal")) - assert(types(7).equals("class java.lang.Double")) - assert(types(8).equals("class java.lang.Double")) - assert(rows(0).getBoolean(0) == false) - assert(rows(0).getLong(1) == 0x225) - assert(rows(0).getInt(2) == 17) - assert(rows(0).getInt(3) == 77777) - assert(rows(0).getInt(4) == 123456789) - assert(rows(0).getLong(5) == 123456789012345L) - val bd = new BigDecimal("123456789012345.12345678901234500000") - assert(rows(0).getAs[BigDecimal](6).equals(bd)) - assert(rows(0).getDouble(7) == 42.75) - assert(rows(0).getDouble(8) == 1.0000000000000002) - } - - test("Date types") { - val df = TestSQLContext.jdbc(url(ip, "foo"), "dates") - val rows = df.collect() - assert(rows.length == 1) - val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 5) - assert(types(0).equals("class java.sql.Date")) - assert(types(1).equals("class java.sql.Timestamp")) - assert(types(2).equals("class java.sql.Timestamp")) - assert(types(3).equals("class java.sql.Timestamp")) - assert(types(4).equals("class java.sql.Date")) - assert(rows(0).getAs[Date](0).equals(new Date(91, 10, 9))) - assert(rows(0).getAs[Timestamp](1).equals(new Timestamp(70, 0, 1, 13, 31, 24, 0))) - assert(rows(0).getAs[Timestamp](2).equals(new Timestamp(96, 0, 1, 1, 23, 45, 0))) - assert(rows(0).getAs[Timestamp](3).equals(new Timestamp(109, 1, 13, 23, 31, 30, 0))) - assert(rows(0).getAs[Date](4).equals(new Date(101, 0, 1))) - } - - test("String types") { - val df = TestSQLContext.jdbc(url(ip, "foo"), "strings") - val rows = df.collect() - assert(rows.length == 1) - val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 9) - assert(types(0).equals("class java.lang.String")) - assert(types(1).equals("class java.lang.String")) - assert(types(2).equals("class java.lang.String")) - assert(types(3).equals("class java.lang.String")) - assert(types(4).equals("class java.lang.String")) - assert(types(5).equals("class java.lang.String")) - assert(types(6).equals("class [B")) - assert(types(7).equals("class [B")) - assert(types(8).equals("class [B")) - assert(rows(0).getString(0).equals("the")) - assert(rows(0).getString(1).equals("quick")) - assert(rows(0).getString(2).equals("brown")) - assert(rows(0).getString(3).equals("fox")) - assert(rows(0).getString(4).equals("jumps")) - assert(rows(0).getString(5).equals("over")) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](116, 104, 101, 0))) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](7), Array[Byte](108, 97, 122, 121))) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](8), Array[Byte](100, 111, 103))) - } - - test("Basic write test") { - val df1 = TestSQLContext.jdbc(url(ip, "foo"), "numbers") - val df2 = TestSQLContext.jdbc(url(ip, "foo"), "dates") - val df3 = TestSQLContext.jdbc(url(ip, "foo"), "strings") - df1.createJDBCTable(url(ip, "foo"), "numberscopy", false) - df2.createJDBCTable(url(ip, "foo"), "datescopy", false) - df3.createJDBCTable(url(ip, "foo"), "stringscopy", false) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala deleted file mode 100644 index e17be99ac31d5..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala +++ /dev/null @@ -1,147 +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.jdbc - -import java.sql.DriverManager - -import com.spotify.docker.client.DockerClient -import com.spotify.docker.client.messages.ContainerConfig -import org.scalatest.{BeforeAndAfterAll, FunSuite, Ignore} - -import org.apache.spark.sql.test._ - -class PostgresDatabase { - val docker: DockerClient = DockerClientFactory.get() - val containerId = { - println("Pulling postgres") - docker.pull("postgres") - println("Configuring container") - val config = ContainerConfig.builder().image("postgres") - .env("POSTGRES_PASSWORD=rootpass") - .build() - println("Creating container") - val id = docker.createContainer(config).id - println("Starting container " + id) - docker.startContainer(id) - id - } - val ip = docker.inspectContainer(containerId).networkSettings.ipAddress - - def close() { - try { - println("Killing container " + containerId) - docker.killContainer(containerId) - println("Removing container " + containerId) - docker.removeContainer(containerId) - println("Closing docker client") - DockerClientFactory.close(docker) - } catch { - case e: Exception => - println(e) - println("You may need to clean this up manually.") - throw e - } - } -} - -@Ignore class PostgresIntegration extends FunSuite with BeforeAndAfterAll { - lazy val db = new PostgresDatabase() - - def url(ip: String) = s"jdbc:postgresql://$ip:5432/postgres?user=postgres&password=rootpass" - - def waitForDatabase(ip: String, maxMillis: Long) { - val before = System.currentTimeMillis() - var lastException: java.sql.SQLException = null - while (true) { - if (System.currentTimeMillis() > before + maxMillis) { - throw new java.sql.SQLException(s"Database not up after $maxMillis ms.", - lastException) - } - try { - val conn = java.sql.DriverManager.getConnection(url(ip)) - conn.close() - println("Database is up.") - return; - } catch { - case e: java.sql.SQLException => - lastException = e - java.lang.Thread.sleep(250) - } - } - } - - def setupDatabase(ip: String) { - val conn = DriverManager.getConnection(url(ip)) - try { - conn.prepareStatement("CREATE DATABASE foo").executeUpdate() - conn.setCatalog("foo") - conn.prepareStatement("CREATE TABLE bar (a text, b integer, c double precision, d bigint, " - + "e bit(1), f bit(10), g bytea, h boolean, i inet, j cidr)").executeUpdate() - conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " - + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16')").executeUpdate() - } finally { - conn.close() - } - } - - override def beforeAll() { - println("Waiting for database to start up.") - waitForDatabase(db.ip, 60000) - println("Setting up database.") - setupDatabase(db.ip) - } - - override def afterAll() { - db.close() - } - - test("Type mapping for various types") { - val df = TestSQLContext.jdbc(url(db.ip), "public.bar") - val rows = df.collect() - assert(rows.length == 1) - val types = rows(0).toSeq.map(x => x.getClass.toString) - assert(types.length == 10) - assert(types(0).equals("class java.lang.String")) - assert(types(1).equals("class java.lang.Integer")) - assert(types(2).equals("class java.lang.Double")) - assert(types(3).equals("class java.lang.Long")) - assert(types(4).equals("class java.lang.Boolean")) - assert(types(5).equals("class [B")) - assert(types(6).equals("class [B")) - assert(types(7).equals("class java.lang.Boolean")) - assert(types(8).equals("class java.lang.String")) - assert(types(9).equals("class java.lang.String")) - assert(rows(0).getString(0).equals("hello")) - assert(rows(0).getInt(1) == 42) - assert(rows(0).getDouble(2) == 1.25) - assert(rows(0).getLong(3) == 123456789012345L) - assert(rows(0).getBoolean(4) == false) - // BIT(10)'s come back as ASCII strings of ten ASCII 0's and 1's... - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](5), Array[Byte](49,48,48,48,49,48,48,49,48,49))) - assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](0xDE.toByte, 0xAD.toByte, 0xBE.toByte, 0xEF.toByte))) - assert(rows(0).getBoolean(7) == true) - assert(rows(0).getString(8) == "172.16.0.42") - assert(rows(0).getString(9) == "192.168.0.0/16") - } - - test("Basic write test") { - val df = TestSQLContext.jdbc(url(db.ip), "public.bar") - df.createJDBCTable(url(db.ip), "public.barcopy", false) - // Test only that it doesn't bomb out. - } -} From aef8a84e42351419a67d56abaf1ee75a05eb11ea Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 4 Mar 2015 20:23:43 +0800 Subject: [PATCH 346/817] [SPARK-6134][SQL] Fix wrong datatype for casting FloatType and default LongType value in defaultPrimitive In `CodeGenerator`, the casting on `FloatType` should use `FloatType` instead of `IntegerType`. Besides, `defaultPrimitive` for `LongType` should be `-1L` instead of `1L`. Author: Liang-Chi Hsieh Closes #4870 from viirya/codegen_type and squashes the following commits: 76311dd [Liang-Chi Hsieh] Fix wrong datatype for casting on FloatType. Fix the wrong value for LongType in defaultPrimitive. --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index c347780924caf..e48b8cde20eda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -259,7 +259,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin child.castOrNull(c => q"$c.toDouble", DoubleType) case Cast(child @ NumericType(), FloatType) => - child.castOrNull(c => q"$c.toFloat", IntegerType) + child.castOrNull(c => q"$c.toFloat", FloatType) // Special handling required for timestamps in hive test cases since the toString function // does not match the expected output. @@ -626,7 +626,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case FloatType => ru.Literal(Constant(-1.0.toFloat)) case StringType => ru.Literal(Constant("")) case ShortType => ru.Literal(Constant(-1.toShort)) - case LongType => ru.Literal(Constant(1L)) + case LongType => ru.Literal(Constant(-1L)) case ByteType => ru.Literal(Constant(-1.toByte)) case DoubleType => ru.Literal(Constant(-1.toDouble)) case DecimalType() => q"org.apache.spark.sql.types.Decimal(-1)" From f6773edce05300faf1e673ea2d1782dfb9b8b998 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 4 Mar 2015 12:28:27 +0000 Subject: [PATCH 347/817] [SPARK-6107][CORE] Display inprogress application information for event log history for standalone mode when application is finished running abnormally (Ctrl + c for example), the history event log file is still ends with `.inprogress` suffix. And the application state can not be showed on webUI, User can only see "*Application history not foud xxxx, Application xxx is still in progress*". For application that not finished normally, the history will show: ![image](https://cloud.githubusercontent.com/assets/4716022/6437137/184f9fc0-c0f5-11e4-88cc-a2eb087e4561.png) Author: Zhang, Liye Closes #4848 from liyezhang556520/showLogInprogress and squashes the following commits: 03589ac [Zhang, Liye] change inprogress to in progress b55f19f [Zhang, Liye] scala modify after rebase 8aa66a2 [Zhang, Liye] use softer wording b030bd4 [Zhang, Liye] clean code 79c8cb1 [Zhang, Liye] fix some mistakes 11cdb68 [Zhang, Liye] add a missing space c29205b [Zhang, Liye] refine code according to sean owen's comments e9952a7 [Zhang, Liye] scala style fix again 150502d [Zhang, Liye] scala style fix f11a5da [Zhang, Liye] small fix for file path 22e878b [Zhang, Liye] enable in progress eventlog file --- .../apache/spark/deploy/master/Master.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 148485cc11863..4584b730e3420 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -736,30 +736,34 @@ private[spark] class Master( val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { - val eventLogFile = app.desc.eventLogDir - .map { dir => EventLoggingListener.getLogPath(dir, app.id, app.desc.eventLogCodec) } + val eventLogDir = app.desc.eventLogDir .getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath return false } - - val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) - - if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { + + val eventLogFilePrefix = EventLoggingListener.getLogPath( + eventLogDir, app.id, app.desc.eventLogCodec) + val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) + val inProgressExists = fs.exists(new Path(eventLogFilePrefix + + EventLoggingListener.IN_PROGRESS)) + + if (inProgressExists) { // Event logging is enabled for this application, but the application is still in progress - val title = s"Application history not found (${app.id})" - var msg = s"Application $appName is still in progress." - logWarning(msg) - msg = URLEncoder.encode(msg, "UTF-8") - app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - return false + logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") } - + + val (eventLogFile, status) = if (inProgressExists) { + (eventLogFilePrefix + EventLoggingListener.IN_PROGRESS, " (in progress)") + } else { + (eventLogFilePrefix, " (completed)") + } + val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") try { replayBus.replay(logInput, eventLogFile) } finally { @@ -774,7 +778,7 @@ private[spark] class Master( case fnf: FileNotFoundException => // Event logging is enabled for this application, but no event logs are found val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir}." + var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}." logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") From 3a35a0dfe940843c3f3a5f51acfe24def488faa9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 4 Mar 2015 12:58:39 -0800 Subject: [PATCH 348/817] [SPARK-6144] [core] Fix addFile when source files are on "hdfs:" The code failed in two modes: it complained when it tried to re-create a directory that already existed, and it was placing some files in the wrong parent directory. The patch fixes both issues. Author: Marcelo Vanzin Author: trystanleftwich Closes #4894 from vanzin/SPARK-6144 and squashes the following commits: 100b3a1 [Marcelo Vanzin] Style fix. 58266aa [Marcelo Vanzin] Fix fetchHcfs file for directories. 91733b7 [trystanleftwich] [SPARK-6144]When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail --- .../scala/org/apache/spark/util/Utils.scala | 28 +++--- .../org/apache/spark/util/UtilsSuite.scala | 85 ++++++++++--------- 2 files changed, 63 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4644088f19f4b..d3dc1d09cb7b4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -624,7 +624,8 @@ private[spark] object Utils extends Logging { case _ => val fs = getHadoopFileSystem(uri, hadoopConf) val path = new Path(uri) - fetchHcfsFile(path, new File(targetDir, path.getName), fs, conf, hadoopConf, fileOverwrite) + fetchHcfsFile(path, targetDir, fs, conf, hadoopConf, fileOverwrite, + filename = Some(filename)) } } @@ -639,19 +640,22 @@ private[spark] object Utils extends Logging { fs: FileSystem, conf: SparkConf, hadoopConf: Configuration, - fileOverwrite: Boolean): Unit = { - if (!targetDir.mkdir()) { + fileOverwrite: Boolean, + filename: Option[String] = None): Unit = { + if (!targetDir.exists() && !targetDir.mkdir()) { throw new IOException(s"Failed to create directory ${targetDir.getPath}") } - fs.listStatus(path).foreach { fileStatus => - val innerPath = fileStatus.getPath - if (fileStatus.isDir) { - fetchHcfsFile(innerPath, new File(targetDir, innerPath.getName), fs, conf, hadoopConf, - fileOverwrite) - } else { - val in = fs.open(innerPath) - val targetFile = new File(targetDir, innerPath.getName) - downloadFile(innerPath.toString, in, targetFile, fileOverwrite) + val dest = new File(targetDir, filename.getOrElse(path.getName)) + if (fs.isFile(path)) { + val in = fs.open(path) + try { + downloadFile(path.toString, in, dest, fileOverwrite) + } finally { + in.close() + } + } else { + fs.listStatus(path).foreach { fileStatus => + fetchHcfsFile(fileStatus.getPath(), dest, fs, conf, hadoopConf, fileOverwrite) } } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index fe2b644251157..fd77753c0d362 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -208,18 +208,18 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) // although child1 is old, child2 is still new so return true - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) child2.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) parent.setLastModified(System.currentTimeMillis - (1000 * 30)) // although parent and its immediate children are new, child3 is still old // we expect a full recursive search for new files. - assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) child3.setLastModified(System.currentTimeMillis - (1000 * 30)) - assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) } test("resolveURI") { @@ -339,21 +339,21 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(!tempDir1.exists()) val tempDir2 = Utils.createTempDir() - val tempFile1 = new File(tempDir2, "foo.txt") - Files.touch(tempFile1) - assert(tempFile1.exists()) - Utils.deleteRecursively(tempFile1) - assert(!tempFile1.exists()) + val sourceFile1 = new File(tempDir2, "foo.txt") + Files.touch(sourceFile1) + assert(sourceFile1.exists()) + Utils.deleteRecursively(sourceFile1) + assert(!sourceFile1.exists()) val tempDir3 = new File(tempDir2, "subdir") assert(tempDir3.mkdir()) - val tempFile2 = new File(tempDir3, "bar.txt") - Files.touch(tempFile2) - assert(tempFile2.exists()) + val sourceFile2 = new File(tempDir3, "bar.txt") + Files.touch(sourceFile2) + assert(sourceFile2.exists()) Utils.deleteRecursively(tempDir2) assert(!tempDir2.exists()) assert(!tempDir3.exists()) - assert(!tempFile2.exists()) + assert(!sourceFile2.exists()) } test("loading properties from file") { @@ -386,30 +386,39 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { } test("fetch hcfs dir") { - val tempDir = Utils.createTempDir() - val innerTempDir = Utils.createTempDir(tempDir.getPath) - val tempFile = File.createTempFile("someprefix", "somesuffix", innerTempDir) - val targetDir = new File("target-dir") - Files.write("some text", tempFile, UTF_8) - - try { - val path = new Path("file://" + tempDir.getAbsolutePath) - val conf = new Configuration() - val fs = Utils.getHadoopFileSystem(path.toString, conf) - Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) - assert(targetDir.exists()) - assert(targetDir.isDirectory()) - val newInnerDir = new File(targetDir, innerTempDir.getName) - println("inner temp dir: " + innerTempDir.getName) - targetDir.listFiles().map(_.getName).foreach(println) - assert(newInnerDir.exists()) - assert(newInnerDir.isDirectory()) - val newInnerFile = new File(newInnerDir, tempFile.getName) - assert(newInnerFile.exists()) - assert(newInnerFile.isFile()) - } finally { - Utils.deleteRecursively(tempDir) - Utils.deleteRecursively(targetDir) - } + val sourceDir = Utils.createTempDir() + val innerSourceDir = Utils.createTempDir(root=sourceDir.getPath) + val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) + val targetDir = new File(Utils.createTempDir(), "target-dir") + Files.write("some text", sourceFile, UTF_8) + + val path = new Path("file://" + sourceDir.getAbsolutePath) + val conf = new Configuration() + val fs = Utils.getHadoopFileSystem(path.toString, conf) + + assert(!targetDir.isDirectory()) + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + assert(targetDir.isDirectory()) + + // Copy again to make sure it doesn't error if the dir already exists. + Utils.fetchHcfsFile(path, targetDir, fs, new SparkConf(), conf, false) + + val destDir = new File(targetDir, sourceDir.getName()) + assert(destDir.isDirectory()) + + val destInnerDir = new File(destDir, innerSourceDir.getName) + assert(destInnerDir.isDirectory()) + + val destInnerFile = new File(destInnerDir, sourceFile.getName) + assert(destInnerFile.isFile()) + + val filePath = new Path("file://" + sourceFile.getAbsolutePath) + val testFileDir = new File("test-filename") + val testFileName = "testFName" + val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf) + Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(), + conf, false, Some(testFileName)) + val newFileName = new File(testFileDir, testFileName) + assert(newFileName.isFile()) } } From 1aa90e39e33caa497971544ee7643fb3ff048c12 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 4 Mar 2015 20:52:58 -0800 Subject: [PATCH 349/817] [SPARK-6149] [SQL] [Build] Excludes Guava 15 referenced by jackson-module-scala_2.10 This PR excludes Guava 15.0 from the SBT build, to make Spark SQL CLI (`bin/spark-sql`) work when compiled against Hive 0.12.0. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4890) Author: Cheng Lian Closes #4890 from liancheng/exclude-guava-15 and squashes the following commits: 91ae9fa [Cheng Lian] Moves Guava 15 exclusion from SBT build to POM 282bd2a [Cheng Lian] Excludes Guava 15 referenced by jackson-module-scala_2.10 --- pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/pom.xml b/pom.xml index 77d08d6ee826f..54fe784fe566f 100644 --- a/pom.xml +++ b/pom.xml @@ -583,10 +583,18 @@ jackson-databind ${fasterxml.jackson.version} + com.fasterxml.jackson.module jackson-module-scala_2.10 ${fasterxml.jackson.version} + + + com.google.guava + guava + + org.scala-lang From 7ac072f74b5a9a02339cede82ad5ffec5beed715 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 4 Mar 2015 21:00:51 -0800 Subject: [PATCH 350/817] SPARK-5143 [BUILD] [WIP] spark-network-yarn 2.11 depends on spark-network-shuffle 2.10 Update `` prop in POM when switching between Scala 2.10/2.11 ScrapCodes for review. This `sed` command is supposed to just replace the first occurrence, but it replaces them all. Are you more of a `sed` wizard than I? It may be a GNU/BSD thing that is throwing me off. Really, just the first instance should be replaced, hence the `[WIP]`. NB on OS X the original `sed` command here will create files like `pom.xml-e` through the source tree though it otherwise works. It's like `-e` is also the arg to `-i`. I couldn't get rid of that even with `-i""`. No biggie. Author: Sean Owen Closes #4876 from srowen/SPARK-5143 and squashes the following commits: b060c44 [Sean Owen] Oops, fixed reversed version numbers! e875d4a [Sean Owen] Add note about non-GNU sed; fix new pom.xml update to work as intended on GNU sed 703e1eb [Sean Owen] Update scala.binary.version prop in POM when switching between Scala 2.10/2.11 --- dev/change-version-to-2.10.sh | 8 +++++++- dev/change-version-to-2.11.sh | 7 ++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh index 7473c20d28e09..15e0c73b4295e 100755 --- a/dev/change-version-to-2.10.sh +++ b/dev/change-version-to-2.10.sh @@ -16,5 +16,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +# Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X) + find . -name 'pom.xml' | grep -v target \ - | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.11|\1_2.10|g' {} + | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.11/\1_2.10/g' {} + +# Also update in parent POM +sed -i -e '0,/2.112.10 in parent POM +sed -i -e '0,/2.102.11 Date: Thu, 5 Mar 2015 16:35:17 +0800 Subject: [PATCH 351/817] [SPARK-6153] [SQL] promote guava dep for hive-thriftserver For package thriftserver, guava is used at runtime. /cc pwendell Author: Daoyuan Wang Closes #4884 from adrian-wang/test and squashes the following commits: 4600ae7 [Daoyuan Wang] only promote for thriftserver 44dda18 [Daoyuan Wang] promote guava dep for hive --- sql/hive-thriftserver/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 123a1f629ab1c..279987f58313b 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -41,6 +41,11 @@ spark-hive_${scala.binary.version} ${project.version} + + com.google.guava + guava + runtime + ${hive.group} hive-cli From c9cfba0cebe3eb546e3e96f3e5b9b89a74c5b7de Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 5 Mar 2015 11:31:48 -0800 Subject: [PATCH 352/817] SPARK-6182 [BUILD] spark-parent pom needs to be published for both 2.10 and 2.11 Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11 Author: Sean Owen Closes #4912 from srowen/SPARK-6182.1 and squashes the following commits: eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11 --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-assembly/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/java8-tests/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- network/common/pom.xml | 2 +- network/shuffle/pom.xml | 2 +- network/yarn/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 28 files changed, 28 insertions(+), 28 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 3d1ed0dd8a7bd..cbf5b6c4aa8df 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 510e92640eff8..1fe61062b4606 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index c993781c0e0d6..fab776d142ef7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 8caad2bc2e27a..994071d94d0ad 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 0706f1ebf66e2..96c2787e35cd0 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 1f2681394c583..172d447b77cda 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 8daa7ed608f6a..5109b8ed87524 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index af96138d79405..369856187a244 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 560c8b9d18276..a344f000c5002 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index da6ffe7662f63..e95853f005ce2 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index e919c2c9b19ea..9b3475d7c3dc2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 0fb431808bacd..bc2f8be10c9ce 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 216661b8bc73a..7e49a71907336 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index f2f0aa78b0a4b..6eb29af03f833 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 8fac24b6ed86d..57e338c03ecf9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4c8f34417ca65..b5c949e155cfd 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/network/common/pom.xml b/network/common/pom.xml index 8f7c924d6b3a3..74437f37c47e4 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index c2d0300ecd904..a2bcca26d8344 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index 39b99f54f6dbc..cea7a20c223e2 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 54fe784fe566f..f99a83b9994ed 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ 14 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT pom Spark Project Parent POM diff --git a/repl/pom.xml b/repl/pom.xml index b883344bf0ceb..295f88ea3ecf9 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index a1947fb022e54..8ad026dbdf8ff 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index d4c8c687b67bd..3640104e497d4 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 279987f58313b..f466a3c0b5dc2 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 72c474d66055c..0e3f4eb98cbf7 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,7 +21,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1e92ba686a57d..0370b0e9e1aa3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -20,7 +20,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e7419ed2c607a..181236d1bcbf6 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 65344aa8738e0..c13534f0410a1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -19,7 +19,7 @@ 4.0.0 org.apache.spark - spark-parent + spark-parent_2.10 1.3.0-SNAPSHOT ../pom.xml From 0bfacd5c5dd7d10a69bcbcbda630f0843d1cf285 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 5 Mar 2015 11:50:09 -0800 Subject: [PATCH 353/817] [SPARK-6090][MLLIB] add a basic BinaryClassificationMetrics to PySpark/MLlib A simple wrapper around the Scala implementation. `DataFrame` is used for serialization/deserialization. Methods that return `RDD`s are not supported in this PR. davies If we recognize Scala's `Product`s in Py4J, we can easily add wrappers for Scala methods that returns `RDD[(Double, Double)]`. Is it easy to register serializer for `Product` in PySpark? Author: Xiangrui Meng Closes #4863 from mengxr/SPARK-6090 and squashes the following commits: 009a3a3 [Xiangrui Meng] provide schema dcddab5 [Xiangrui Meng] add a basic BinaryClassificationMetrics to PySpark/MLlib --- .../BinaryClassificationMetrics.scala | 8 ++ python/docs/pyspark.mllib.rst | 7 ++ python/pyspark/mllib/evaluation.py | 83 +++++++++++++++++++ python/run-tests | 1 + 4 files changed, 99 insertions(+) create mode 100644 python/pyspark/mllib/evaluation.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index ced042e2f96ca..c1d1a224817e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -22,6 +22,7 @@ import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.evaluation.binary._ import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.sql.DataFrame /** * :: Experimental :: @@ -53,6 +54,13 @@ class BinaryClassificationMetrics( */ def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0) + /** + * An auxiliary constructor taking a DataFrame. + * @param scoreAndLabels a DataFrame with two double columns: score and label + */ + private[mllib] def this(scoreAndLabels: DataFrame) = + this(scoreAndLabels.map(r => (r.getDouble(0), r.getDouble(1)))) + /** Unpersist intermediate RDDs used in the computation. */ def unpersist() { cumulativeCounts.unpersist() diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index b706c5e376ef4..15101470afc07 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -16,6 +16,13 @@ pyspark.mllib.clustering module :members: :undoc-members: +pyspark.mllib.evaluation module +------------------------------- + +.. automodule:: pyspark.mllib.evaluation + :members: + :undoc-members: + pyspark.mllib.feature module ------------------------------- diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py new file mode 100644 index 0000000000000..16cb49cc0cfff --- /dev/null +++ b/python/pyspark/mllib/evaluation.py @@ -0,0 +1,83 @@ +# +# 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. +# + +from pyspark.mllib.common import JavaModelWrapper +from pyspark.sql import SQLContext +from pyspark.sql.types import StructField, StructType, DoubleType + + +class BinaryClassificationMetrics(JavaModelWrapper): + """ + Evaluator for binary classification. + + >>> scoreAndLabels = sc.parallelize([ + ... (0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)], 2) + >>> metrics = BinaryClassificationMetrics(scoreAndLabels) + >>> metrics.areaUnderROC() + 0.70... + >>> metrics.areaUnderPR() + 0.83... + >>> metrics.unpersist() + """ + + def __init__(self, scoreAndLabels): + """ + :param scoreAndLabels: an RDD of (score, label) pairs + """ + sc = scoreAndLabels.ctx + sql_ctx = SQLContext(sc) + df = sql_ctx.createDataFrame(scoreAndLabels, schema=StructType([ + StructField("score", DoubleType(), nullable=False), + StructField("label", DoubleType(), nullable=False)])) + java_class = sc._jvm.org.apache.spark.mllib.evaluation.BinaryClassificationMetrics + java_model = java_class(df._jdf) + super(BinaryClassificationMetrics, self).__init__(java_model) + + def areaUnderROC(self): + """ + Computes the area under the receiver operating characteristic + (ROC) curve. + """ + return self.call("areaUnderROC") + + def areaUnderPR(self): + """ + Computes the area under the precision-recall curve. + """ + return self.call("areaUnderPR") + + def unpersist(self): + """ + Unpersists intermediate RDDs used in the computation. + """ + self.call("unpersist") + + +def _test(): + import doctest + from pyspark import SparkContext + import pyspark.mllib.evaluation + globs = pyspark.mllib.evaluation.__dict__.copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest') + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index a2c2f37a54eda..b7630c356cfae 100755 --- a/python/run-tests +++ b/python/run-tests @@ -75,6 +75,7 @@ function run_mllib_tests() { echo "Run mllib tests ..." run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" + run_test "pyspark/mllib/evaluation.py" run_test "pyspark/mllib/feature.py" run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/rand.py" From 424a86a1ed2a3e6dd54cf8b09fe2f13a1311b7e6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 5 Mar 2015 12:04:00 -0800 Subject: [PATCH 354/817] [SPARK-6175] Fix standalone executor log links when ephemeral ports or SPARK_PUBLIC_DNS are used This patch fixes two issues with the executor log viewing links added in Spark 1.3. In standalone mode, the log URLs might include a port value of 0 rather than the actual bound port of the UI, which broke the ability to view logs from workers whose web UIs had been configured to bind to ephemeral ports. In addition, the URLs used workers' local hostnames instead of respecting SPARK_PUBLIC_DNS, which prevented this feature from working properly on Spark EC2 clusters because the links would point to internal DNS names instead of external ones. I included tests for both of these bugs: - We now browse to the URLs and verify that they point to the expected pages. - To test SPARK_PUBLIC_DNS, I changed the code that reads the environment variable to do so via `SparkConf.getenv`, then used a custom SparkConf subclass to mock the environment variable (this pattern is used elsewhere in Spark's tests). Author: Josh Rosen Closes #4903 from JoshRosen/SPARK-6175 and squashes the following commits: 5577f41 [Josh Rosen] Remove println cfec135 [Josh Rosen] Use webUi.boundPort and publicAddress in log links 27918c7 [Josh Rosen] Add failing unit tests for standalone log URL viewing c250fbe [Josh Rosen] Respect SparkConf in local-cluster Workers. 422a2ef [Josh Rosen] Use conf.getenv to read SPARK_PUBLIC_DNS --- .../spark/deploy/LocalSparkCluster.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 2 +- .../spark/deploy/worker/ExecutorRunner.scala | 4 +- .../apache/spark/deploy/worker/Worker.scala | 9 ++-- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 54 +++++++++++++++---- .../deploy/worker/ExecutorRunnerTest.scala | 2 +- 8 files changed, 57 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 0401b15446a7b..3ab425aab84c8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -59,7 +59,7 @@ class LocalSparkCluster( /* Start the Workers */ for (workerNum <- 1 to numWorkers) { val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker, - memoryPerWorker, masters, null, Some(workerNum)) + memoryPerWorker, masters, null, Some(workerNum), _conf) workerActorSystems += workerSystem } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 4584b730e3420..15814293227ab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -96,7 +96,7 @@ private[spark] class Master( val webUi = new MasterWebUI(this, webUiPort) val masterPublicAddress = { - val envVar = System.getenv("SPARK_PUBLIC_DNS") + val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 066d46c4473eb..023f3c6269062 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -44,6 +44,7 @@ private[spark] class ExecutorRunner( val workerId: String, val host: String, val webUiPort: Int, + val publicAddress: String, val sparkHome: File, val executorDir: File, val workerUrl: String, @@ -140,7 +141,8 @@ private[spark] class ExecutorRunner( builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") // Add webUI log urls - val baseUrl = s"http://$host:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" + val baseUrl = + s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" builder.environment.put("SPARK_LOG_URL_STDERR", s"${baseUrl}stderr") builder.environment.put("SPARK_LOG_URL_STDOUT", s"${baseUrl}stdout") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 2473a90aa9309..f2e7418f4bf15 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -121,7 +121,7 @@ private[spark] class Worker( val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) val publicAddress = { - val envVar = System.getenv("SPARK_PUBLIC_DNS") + val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } var webUi: WorkerWebUI = null @@ -362,7 +362,8 @@ private[spark] class Worker( self, workerId, host, - webUiPort, + webUi.boundPort, + publicAddress, sparkHome, executorDir, akkaUrl, @@ -538,10 +539,10 @@ private[spark] object Worker extends Logging { memory: Int, masterUrls: Array[String], workDir: String, - workerNumber: Option[Int] = None): (ActorSystem, Int) = { + workerNumber: Option[Int] = None, + conf: SparkConf = new SparkConf): (ActorSystem, Int) = { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems - val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val actorName = "Worker" val securityMgr = new SecurityManager(conf) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 9be65a4a39a09..ec68837a1516c 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -47,7 +47,7 @@ private[spark] abstract class WebUI( protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None protected val localHostName = Utils.localHostName() - protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) def getBasePath: String = basePath diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index e955636cf5b59..68b5776fc6515 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -119,7 +119,7 @@ class JsonProtocolSuite extends FunSuite { def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", 123, - new File("sparkHome"), new File("workDir"), "akka://worker", + "publicAddress", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index f33bdc73e40ac..54dd7c9c45c61 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -17,35 +17,69 @@ package org.apache.spark.deploy +import java.net.URL + import scala.collection.mutable +import scala.io.Source -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.FunSuite import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener} -import org.apache.spark.{SparkContext, LocalSparkContext} +import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} -class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext with BeforeAndAfter { +class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { /** Length of time to wait while draining listener events. */ - val WAIT_TIMEOUT_MILLIS = 10000 + private val WAIT_TIMEOUT_MILLIS = 10000 - before { + test("verify that correct log urls get propagated from workers") { sc = new SparkContext("local-cluster[2,1,512]", "test") + + val listener = new SaveExecutorInfo + sc.addSparkListener(listener) + + // Trigger a job so that executors get added + sc.parallelize(1 to 100, 4).map(_.toString).count() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + // Browse to each URL to check that it's valid + info.logUrlMap.foreach { case (logType, logUrl) => + val html = Source.fromURL(logUrl).mkString + assert(html.contains(s"$logType log page")) + } + } } - test("verify log urls get propagated from workers") { + test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") { + val SPARK_PUBLIC_DNS = "public_dns" + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(getAll) + } + } + val conf = new MySparkConf() + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + val listener = new SaveExecutorInfo sc.addSparkListener(listener) - val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(_.toString) - rdd2.setName("Target RDD") - rdd2.count() + // Trigger a job so that executors get added + sc.parallelize(1 to 100, 4).map(_.toString).count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) + info.logUrlMap.values.foreach { logUrl => + assert(new URL(logUrl).getHost === SPARK_PUBLIC_DNS) + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 76511699e5ac5..6fca6321e5a1b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -33,7 +33,7 @@ class ExecutorRunnerTest extends FunSuite { val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, - new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) From 5873c713cc47af311f517ea33a6110993a410377 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 5 Mar 2015 14:49:01 -0800 Subject: [PATCH 355/817] [SPARK-6145][SQL] fix ORDER BY on nested fields Based on #4904 with style errors fixed. `LogicalPlan#resolve` will not only produce `Attribute`, but also "`GetField` chain". So in `ResolveSortReferences`, after resolve the ordering expressions, we should not just collect the `Attribute` results, but also `Attribute` at the bottom of "`GetField` chain". Author: Wenchen Fan Author: Michael Armbrust Closes #4918 from marmbrus/pr/4904 and squashes the following commits: 997f84e [Michael Armbrust] fix style 3eedbfc [Wenchen Fan] fix 6145 --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 5 +++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index c363a5efacde8..54ab13ca352d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -385,7 +385,7 @@ class SqlParser extends AbstractSparkSQLParser { protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { - case i1 ~ i2 ~ rest => UnresolvedAttribute(i1 + "." + i2 + rest.mkString(".", ".", "")) + case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) } protected lazy val dataType: Parser[DataType] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e4e542562f22d..7753331748d7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -310,7 +310,7 @@ class Analyzer(catalog: Catalog, } /** - * In many dialects of SQL is it valid to sort by attributes that are not present in the SELECT + * In many dialects of SQL it is valid to sort by attributes that are not present in the SELECT * clause. This rule detects such queries and adds the required attributes to the original * projection, so that they will be available during sorting. Another projection is added to * remove these attributes after sorting. @@ -321,7 +321,8 @@ class Analyzer(catalog: Catalog, if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolve(_, resolver)) - val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) + val requiredAttributes = + AttributeSet(resolved.flatMap(_.collect { case a: Attribute => a })) val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 097bf0dd23c89..4dedcd365f6cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1049,4 +1049,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { rdd.toDF().registerTempTable("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } + + test("SPARK-6145: ORDER BY test for nested fields") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder") + // These should be successfully analyzed + sql("SELECT 1 FROM nestedOrder ORDER BY a.b").queryExecution.analyzed + sql("SELECT a.b FROM nestedOrder ORDER BY a.b").queryExecution.analyzed + sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a").queryExecution.analyzed + sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d").queryExecution.analyzed + } } From 1b4bb25c10d72132d7f4f3835ef9a3b94b2349e0 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 5 Mar 2015 14:49:44 -0800 Subject: [PATCH 356/817] [SPARK-6163][SQL] jsonFile should be backed by the data source API jira: https://issues.apache.org/jira/browse/SPARK-6163 Author: Yin Huai Closes #4896 from yhuai/SPARK-6163 and squashes the following commits: 45e023e [Yin Huai] Address @chenghao-intel's comment. 2e8734e [Yin Huai] Use JSON data source for jsonFile. 92a4a33 [Yin Huai] Test. --- .../org/apache/spark/sql/SQLContext.scala | 12 +++----- .../org/apache/spark/sql/json/JsonSuite.scala | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index ce800e0754559..9c49e84bf9680 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -542,20 +542,16 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group specificdata */ @Experimental - def jsonFile(path: String, schema: StructType): DataFrame = { - val json = sparkContext.textFile(path) - jsonRDD(json, schema) - } + def jsonFile(path: String, schema: StructType): DataFrame = + load("json", schema, Map("path" -> path)) /** * :: Experimental :: * @group specificdata */ @Experimental - def jsonFile(path: String, samplingRatio: Double): DataFrame = { - val json = sparkContext.textFile(path) - jsonRDD(json, samplingRatio) - } + def jsonFile(path: String, samplingRatio: Double): DataFrame = + load("json", Map("path" -> path, "samplingRatio" -> samplingRatio.toString)) /** * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 9d94d3406acfb..0c21f725f0b49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.json import java.sql.{Date, Timestamp} +import org.scalactic.Tolerance._ + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions._ @@ -551,6 +553,32 @@ class JsonSuite extends QueryTest { jsonDF.registerTempTable("jsonTable") } + test("jsonFile should be based on JSONRelation") { + val file = getTempFilePath("json") + val path = file.toString + sparkContext.parallelize(1 to 100).map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path) + val jsonDF = jsonFile(path, 0.49) + + val analyzed = jsonDF.queryExecution.analyzed + assert( + analyzed.isInstanceOf[LogicalRelation], + "The DataFrame returned by jsonFile should be based on JSONRelation.") + val relation = analyzed.asInstanceOf[LogicalRelation].relation + assert( + relation.isInstanceOf[JSONRelation], + "The DataFrame returned by jsonFile should be based on JSONRelation.") + assert(relation.asInstanceOf[JSONRelation].path === path) + assert(relation.asInstanceOf[JSONRelation].samplingRatio === (0.49 +- 0.001)) + + val schema = StructType(StructField("a", LongType, true) :: Nil) + val logicalRelation = + jsonFile(path, schema).queryExecution.analyzed.asInstanceOf[LogicalRelation] + val relationWithSchema = logicalRelation.relation.asInstanceOf[JSONRelation] + assert(relationWithSchema.path === path) + assert(relationWithSchema.schema === schema) + assert(relationWithSchema.samplingRatio > 0.99) + } + test("Loading a JSON dataset from a text file") { val file = getTempFilePath("json") val path = file.toString From eb48fd6e9d55fb034c00e61374bb9c2a86a82fb8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 5 Mar 2015 14:50:25 -0800 Subject: [PATCH 357/817] [SQL] Make Strategies a public developer API Author: Michael Armbrust Closes #4920 from marmbrus/openStrategies and squashes the following commits: cbc35c0 [Michael Armbrust] [SQL] Make Strategies a public developer API --- sql/core/src/main/scala/org/apache/spark/sql/package.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 02e5b015e8ec2..3f97a11ceb97d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -34,10 +34,13 @@ import org.apache.spark.sql.execution.SparkPlan package object sql { /** - * Converts a logical plan into zero or more SparkPlans. + * Converts a logical plan into zero or more SparkPlans. This API is exposed for experimenting + * with the query planner and is not designed to be stable across spark releases. Developers + * writing libraries should instead consider using the stable APIs provided in + * [[org.apache.spark.sql.sources]] */ @DeveloperApi - protected[sql] type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] + type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] /** * Type alias for [[DataFrame]]. Kept here for backward source compatibility for Scala. From d8b3da9ddfe44a2886f3841ceef4ebf9fc318640 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 6 Mar 2015 09:34:07 +0000 Subject: [PATCH 358/817] [CORE, DEPLOY][minor] align arguments order with docs of worker The help message for starting `worker` is `Usage: Worker [options] `. While in `start-slaves.sh`, the format is not align with that, it is confusing for the fist glance. Author: Zhang, Liye Closes #4924 from liyezhang556520/startSlaves and squashes the following commits: 7fd5deb [Zhang, Liye] align arguments order with docs of worker --- sbin/start-slaves.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index ba1a84abc1fef..76316a3067c93 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -64,6 +64,6 @@ else SPARK_WORKER_WEBUI_PORT=8081 fi for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" done fi From cd7594ca6acf1226bf91f8a783606bf5c116f7df Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Mar 2015 09:43:24 +0000 Subject: [PATCH 359/817] [core] [minor] Don't pollute source directory when running UtilsSuite. Author: Marcelo Vanzin Closes #4921 from vanzin/utils-suite and squashes the following commits: 7795dd4 [Marcelo Vanzin] [core] [minor] Don't pollute source directory when running UtilsSuite. --- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index fd77753c0d362..b91428efadfd0 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -386,10 +386,11 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { } test("fetch hcfs dir") { - val sourceDir = Utils.createTempDir() + val tempDir = Utils.createTempDir() + val sourceDir = new File(tempDir, "source-dir") val innerSourceDir = Utils.createTempDir(root=sourceDir.getPath) val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) - val targetDir = new File(Utils.createTempDir(), "target-dir") + val targetDir = new File(tempDir, "target-dir") Files.write("some text", sourceFile, UTF_8) val path = new Path("file://" + sourceDir.getAbsolutePath) @@ -413,7 +414,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(destInnerFile.isFile()) val filePath = new Path("file://" + sourceFile.getAbsolutePath) - val testFileDir = new File("test-filename") + val testFileDir = new File(tempDir, "test-filename") val testFileName = "testFName" val testFilefs = Utils.getHadoopFileSystem(filePath.toString, conf) Utils.fetchHcfsFile(filePath, testFileDir, testFilefs, new SparkConf(), From 05cb6b34d8fc25114f3dd3e2bd156386c00eb391 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Fri, 6 Mar 2015 13:20:20 +0000 Subject: [PATCH 360/817] [Minor] Resolve sbt warnings: postfix operator second should be enabled Resolve sbt warnings: ``` [warn] spark/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala:155: postfix operator second should be enabled [warn] by making the implicit value scala.language.postfixOps visible. [warn] This can be achieved by adding the import clause 'import scala.language.postfixOps' [warn] or by setting the compiler option -language:postfixOps. [warn] See the Scala docs for value scala.language.postfixOps for a discussion [warn] why the feature should be explicitly enabled. [warn] Await.ready(f, 1 second) [warn] ^ ``` Author: GuoQiang Li Closes #4908 from witgo/sbt_warnings and squashes the following commits: 0629af4 [GuoQiang Li] Resolve sbt warnings: postfix operator second should be enabled --- .../scala/org/apache/spark/scheduler/local/LocalBackend.scala | 1 + .../org/apache/spark/streaming/util/WriteAheadLogManager.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 4676b828d3d89..d95426d918e19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer import scala.concurrent.duration._ +import scala.language.postfixOps import akka.actor.{Actor, ActorRef, Props} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala index 985ded9111f74..6bdfe45dc7f83 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -20,6 +20,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.{Await, ExecutionContext, Future} +import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path From dba0b2eadb441f41ded0f0b9706b720bcfa84881 Mon Sep 17 00:00:00 2001 From: Vinod K C Date: Fri, 6 Mar 2015 14:43:09 +0000 Subject: [PATCH 361/817] [SPARK-6178][Shuffle] Removed unused imports Author: Vinod K C Author: Vinod K C Closes #4900 from vinodkc/unused_imports and squashes the following commits: 5373456 [Vinod K C] Removed empty lines 9da7438 [Vinod K C] Changed order of import 594d471 [Vinod K C] Removed unused imports --- .../org/apache/spark/network/TransportContext.java | 1 - .../spark/network/protocol/ChunkFetchFailure.java | 1 - .../org/apache/spark/network/protocol/Encoders.java | 1 - .../org/apache/spark/network/protocol/RpcFailure.java | 1 - .../apache/spark/network/server/TransportServer.java | 1 - .../java/org/apache/spark/network/util/JavaUtils.java | 10 ++-------- .../java/org/apache/spark/network/util/NettyUtils.java | 1 - .../org/apache/spark/network/sasl/SaslRpcHandler.java | 3 --- .../spark/network/shuffle/OneForOneBlockFetcher.java | 1 - .../spark/network/shuffle/protocol/OpenBlocks.java | 1 - .../network/shuffle/protocol/RegisterExecutor.java | 1 - .../spark/network/shuffle/protocol/StreamHandle.java | 2 -- .../spark/network/shuffle/protocol/UploadBlock.java | 1 - .../org/apache/spark/network/sasl/SparkSaslSuite.java | 6 +++--- .../network/shuffle/OneForOneBlockFetcherSuite.java | 9 +++++++-- 15 files changed, 12 insertions(+), 28 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index 5bc6e5a2418a9..f0a89c9d9116c 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -35,7 +35,6 @@ import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.server.TransportRequestHandler; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index 986957c1509fd..f76bb49e874fc 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -17,7 +17,6 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Charsets; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 873c694250942..9162d0b977f83 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -20,7 +20,6 @@ import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; /** Provides a canonical set of Encoders for simple types. */ public class Encoders { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index ebd764eb5eb5f..6b991375fc486 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -17,7 +17,6 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Charsets; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index ef209991804b4..b7ce8541e565e 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -28,7 +28,6 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.SocketChannel; -import io.netty.util.internal.PlatformDependent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index bf8a1fc42fc6d..73da9b7346f4d 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -17,19 +17,13 @@ package org.apache.spark.network.util; -import java.nio.ByteBuffer; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.File; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; -import com.google.common.base.Preconditions; -import com.google.common.io.Closeables; import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index 2a4b88b64cdc9..dabd6261d2aa0 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -25,7 +25,6 @@ import io.netty.channel.Channel; import io.netty.channel.EventLoopGroup; import io.netty.channel.ServerChannel; -import io.netty.channel.epoll.Epoll; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.epoll.EpollServerSocketChannel; import io.netty.channel.epoll.EpollSocketChannel; diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java index 3777a18e33f78..026cbd260d16c 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java @@ -19,16 +19,13 @@ import java.util.concurrent.ConcurrentMap; -import com.google.common.base.Charsets; import com.google.common.collect.Maps; -import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.protocol.Encodable; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 8ed2e0b39ad23..e653f5cb147ee 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -29,7 +29,6 @@ import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.StreamHandle; -import org.apache.spark.network.util.JavaUtils; /** * Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java index 62fce9b0d16cd..60485bace643c 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java @@ -23,7 +23,6 @@ import io.netty.buffer.ByteBuf; import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** Request to read a set of blocks. Returns {@link StreamHandle}. */ public class OpenBlocks extends BlockTransferMessage { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java index 7eb4385044077..38acae3b31d64 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java @@ -21,7 +21,6 @@ import io.netty.buffer.ByteBuf; import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** * Initial registration message between an executor and its local shuffle server. diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java index bc9daa6158ba3..9a9220211a50c 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java @@ -20,8 +20,6 @@ import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; - /** * Identifier for a fixed number of chunks to read from a stream created by an "open blocks" * message. This is used by {@link org.apache.spark.network.shuffle.OneForOneBlockFetcher}. diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java index 0b23e112bd512..2ff9aaa650f92 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java @@ -23,7 +23,6 @@ import io.netty.buffer.ByteBuf; import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 67a07f38eb5a0..23b4e06f064e1 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -17,12 +17,12 @@ package org.apache.spark.network.sasl; -import java.util.Map; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; -import com.google.common.collect.ImmutableMap; import org.junit.Test; -import static org.junit.Assert.*; /** * Jointly tests SparkSaslClient and SparkSaslServer, as both are black boxes. diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 842741e3d354f..b35a6d685dd02 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -28,11 +28,16 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import static org.junit.Assert.*; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; From 48a723c98684c5bb3d185cada4888cae952791bd Mon Sep 17 00:00:00 2001 From: RobertZK Date: Sat, 7 Mar 2015 00:16:50 +0000 Subject: [PATCH 362/817] Fix python typo (+ Scala, Java typos) Author: RobertZK Author: Robert Krzyzanowski Closes #4840 from robertzk/patch-1 and squashes the following commits: d286215 [RobertZK] lambda fix per @laserson 5937989 [Robert Krzyzanowski] Fix python typo --- docs/programming-guide.md | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 7b0701828878e..b5e04bd0c610d 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1336,25 +1336,28 @@ Accumulators do not change the lazy evaluation model of Spark. If they are being
    {% highlight scala %} -val acc = sc.accumulator(0) -data.map(x => acc += x; f(x)) -// Here, acc is still 0 because no actions have cause the `map` to be computed. +val accum = sc.accumulator(0) +data.map { x => accum += x; f(x) } +// Here, accum is still 0 because no actions have caused the `map` to be computed. {% endhighlight %}
    {% highlight java %} Accumulator accum = sc.accumulator(0); -data.map(x -> accum.add(x); f(x);); -// Here, accum is still 0 because no actions have cause the `map` to be computed. +data.map(x -> { accum.add(x); return f(x); }); +// Here, accum is still 0 because no actions have caused the `map` to be computed. {% endhighlight %}
    {% highlight python %} accum = sc.accumulator(0) -data.map(lambda x => acc.add(x); f(x)) -# Here, acc is still 0 because no actions have cause the `map` to be computed. +def g(x): + accum.add(x) + return f(x) +data.map(g) +# Here, accum is still 0 because no actions have caused the `map` to be computed. {% endhighlight %}
    From 2646794ffb2970618087e2e964d9f4c953e17e6a Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 7 Mar 2015 12:33:41 +0000 Subject: [PATCH 363/817] [EC2] Reorder print statements on termination The PR reorders some print statements slightly on cluster termination so that they read better. For example, from this: ``` Are you sure you want to destroy the cluster spark-cluster-test? The following instances will be terminated: Searching for existing cluster spark-cluster-test in region us-west-2... Found 1 master(s), 2 slaves > ... ALL DATA ON ALL NODES WILL BE LOST!! Destroy cluster spark-cluster-test (y/N): ``` To this: ``` Searching for existing cluster spark-cluster-test in region us-west-2... Found 1 master(s), 2 slaves The following instances will be terminated: > ... ALL DATA ON ALL NODES WILL BE LOST!! Are you sure you want to destroy the cluster spark-cluster-test? (y/N) ``` Author: Nicholas Chammas Closes #4932 from nchammas/termination-print-order and squashes the following commits: c23711d [Nicholas Chammas] reorder prints on termination --- ec2/spark_ec2.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index c59ab565c6862..dabb9fce40d01 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1126,14 +1126,16 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, True) elif action == "destroy": - print "Are you sure you want to destroy the cluster %s?" % cluster_name - print "The following instances will be terminated:" (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - for inst in master_nodes + slave_nodes: - print "> %s" % inst.public_dns_name - msg = "ALL DATA ON ALL NODES WILL BE LOST!!\nDestroy cluster %s (y/N): " % cluster_name + if any(master_nodes + slave_nodes): + print "The following instances will be terminated:" + for inst in master_nodes + slave_nodes: + print "> %s" % inst.public_dns_name + print "ALL DATA ON ALL NODES WILL BE LOST!!" + + msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) response = raw_input(msg) if response == "y": print "Terminating master..." @@ -1145,7 +1147,6 @@ def real_main(): # Delete security groups as well if opts.delete_groups: - print "Deleting security groups (this will take some time)..." group_names = [cluster_name + "-master", cluster_name + "-slaves"] wait_for_cluster_state( conn=conn, @@ -1153,6 +1154,7 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated' ) + print "Deleting security groups (this will take some time)..." attempt = 1 while attempt <= 3: print "Attempt %d" % attempt From 729c05bda87c2383d1c54b31850ed10814617cde Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Sat, 7 Mar 2015 12:35:26 +0000 Subject: [PATCH 364/817] [Minor]fix the wrong description Found it by accident. I'm not gonna file jira for this as it is a very tiny fix. Author: WangTaoTheTonic Closes #4936 from WangTaoTheTonic/wrongdesc and squashes the following commits: fb8a8ec [WangTaoTheTonic] fix the wrong description aca5596 [WangTaoTheTonic] fix the wrong description --- sbin/stop-all.sh | 4 ++-- sbin/stop-master.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 971d5d49da664..1a9abe07db844 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -17,8 +17,8 @@ # limitations under the License. # -# Start all spark daemons. -# Run this on the master nde +# Stop all spark daemons. +# Run this on the master node. sbin="`dirname "$0"`" diff --git a/sbin/stop-master.sh b/sbin/stop-master.sh index b6bdaa4db373c..729702d92191e 100755 --- a/sbin/stop-master.sh +++ b/sbin/stop-master.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Starts the master on the machine this script is executed on. +# Stops the master on the machine this script is executed on. sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` From 334c5bd1ae50ac76770e545cab302361673f45de Mon Sep 17 00:00:00 2001 From: Florian Verhein Date: Sat, 7 Mar 2015 12:56:59 +0000 Subject: [PATCH 365/817] [SPARK-5641] [EC2] Allow spark_ec2.py to copy arbitrary files to cluster Give users an easy way to rcp a directory structure to the master's / as part of the cluster launch, at a useful point in the workflow (before setup.sh is called on the master). This is an alternative approach to meeting requirements discussed in https://github.com/apache/spark/pull/4487 Author: Florian Verhein Closes #4583 from florianverhein/master and squashes the following commits: 49dee88 [Florian Verhein] removed addition of trailing / in rsync to give user this option, added documentation in help 7b8e3d8 [Florian Verhein] remove unused args 87d922c [Florian Verhein] [SPARK-5641] [EC2] implement --deploy-root-dir --- ec2/spark_ec2.py | 42 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index dabb9fce40d01..b6e7c4c2af39b 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -159,6 +159,15 @@ def parse_args(): "--spark-ec2-git-branch", default=DEFAULT_SPARK_EC2_BRANCH, help="Github repo branch of spark-ec2 to use (default: %default)") + parser.add_option( + "--deploy-root-dir", + default=None, + help="A directory to copy into / on the first master. " + + "Must be absolute. Note that a trailing slash is handled as per rsync: " + + "If you omit it, the last directory of the --deploy-root-dir path will be created " + + "in / before copying its contents. If you append the trailing slash, " + + "the directory is not created and its contents are copied directly into /. " + + "(default: %default).") parser.add_option( "--hadoop-major-version", default="1", help="Major version of Hadoop (default: %default)") @@ -694,6 +703,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): modules=modules ) + if opts.deploy_root_dir is not None: + print "Deploying {s} to master...".format(s=opts.deploy_root_dir) + deploy_user_files( + root_dir=opts.deploy_root_dir, + opts=opts, + master_nodes=master_nodes + ) + print "Running setup on master..." setup_spark_cluster(master, opts) print "Done!" @@ -931,6 +948,23 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): shutil.rmtree(tmp_dir) +# Deploy a given local directory to a cluster, WITHOUT parameter substitution. +# Note that unlike deploy_files, this works for binary files. +# Also, it is up to the user to add (or not) the trailing slash in root_dir. +# Files are only deployed to the first master instance in the cluster. +# +# root_dir should be an absolute path. +def deploy_user_files(root_dir, opts, master_nodes): + active_master = master_nodes[0].public_dns_name + command = [ + 'rsync', '-rv', + '-e', stringify_command(ssh_command(opts)), + "%s" % root_dir, + "%s@%s:/" % (opts.user, active_master) + ] + subprocess.check_call(command) + + def stringify_command(parts): if isinstance(parts, str): return parts @@ -1099,6 +1133,14 @@ def real_main(): "Furthermore, we currently only support forks named spark-ec2." sys.exit(1) + if not (opts.deploy_root_dir is None or + (os.path.isabs(opts.deploy_root_dir) and + os.path.isdir(opts.deploy_root_dir) and + os.path.exists(opts.deploy_root_dir))): + print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \ + "on the local file system" + sys.exit(1) + try: conn = ec2.connect_to_region(opts.region) except Exception as e: From 52ed7da12e26c45734ce53a1be19ef148b2b953e Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sun, 8 Mar 2015 14:01:26 +0000 Subject: [PATCH 366/817] [SPARK-6193] [EC2] Push group filter up to EC2 When looking for a cluster, spark-ec2 currently pulls down [info for all instances](https://github.com/apache/spark/blob/eb48fd6e9d55fb034c00e61374bb9c2a86a82fb8/ec2/spark_ec2.py#L620) and filters locally. When working on an AWS account with hundreds of active instances, this step alone can take over 10 seconds. This PR improves how spark-ec2 searches for clusters by pushing the filter up to EC2. Basically, the problem (and solution) look like this: ```python >>> timeit.timeit('blah = conn.get_all_reservations()', setup='from __main__ import conn', number=10) 116.96390509605408 >>> timeit.timeit('blah = conn.get_all_reservations(filters={"instance.group-name": ["my-cluster-master"]})', setup='from __main__ import conn', number=10) 4.629754066467285 ``` Translated to a user-visible action, this looks like (against an AWS account with ~200 active instances): ```shell # master $ python -m timeit -n 3 --setup 'import subprocess' 'subprocess.call("./spark-ec2 get-master my-cluster --region us-west-2", shell=True)' ... 3 loops, best of 3: 9.83 sec per loop # this PR $ python -m timeit -n 3 --setup 'import subprocess' 'subprocess.call("./spark-ec2 get-master my-cluster --region us-west-2", shell=True)' ... 3 loops, best of 3: 1.47 sec per loop ``` This PR also refactors `get_existing_cluster()` to make it, I hope, simpler. Finally, this PR fixes some minor grammar issues related to printing status to the user. :tophat: :clap: Author: Nicholas Chammas Closes #4922 from nchammas/get-existing-cluster-faster and squashes the following commits: 18802f1 [Nicholas Chammas] ignore shutting-down f2a5b9f [Nicholas Chammas] fix grammar d96a489 [Nicholas Chammas] push group filter up to EC2 --- ec2/spark_ec2.py | 78 +++++++++++++++++++++++++----------------------- 1 file changed, 41 insertions(+), 37 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index b6e7c4c2af39b..5e636ddd17e94 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -22,6 +22,7 @@ from __future__ import with_statement import hashlib +import itertools import logging import os import os.path @@ -299,13 +300,6 @@ def get_validate_spark_version(version, repo): return version -# Check whether a given EC2 instance object is in a state we consider active, -# i.e. not terminating or terminated. We count both stopping and stopped as -# active since we can restart stopped clusters. -def is_active(instance): - return (instance.state in ['pending', 'running', 'stopping', 'stopped']) - - # Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ # Last Updated: 2014-06-20 # For easy maintainability, please keep this manually-inputted dictionary sorted by key. @@ -573,8 +567,11 @@ def launch_cluster(conn, opts, cluster_name): placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances - print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, - zone, slave_res.id) + print "Launched {s} slave{plural_s} in {z}, regid = {r}".format( + s=num_slaves_this_zone, + plural_s=('' if num_slaves_this_zone == 1 else 's'), + z=zone, + r=slave_res.id) i += 1 # Launch or resume masters @@ -621,40 +618,47 @@ def launch_cluster(conn, opts, cluster_name): return (master_nodes, slave_nodes) -# Get the EC2 instances in an existing cluster if available. -# Returns a tuple of lists of EC2 instance objects for the masters and slaves def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): - print "Searching for existing cluster " + cluster_name + " in region " \ - + opts.region + "..." - reservations = conn.get_all_reservations() - master_nodes = [] - slave_nodes = [] - for res in reservations: - active = [i for i in res.instances if is_active(i)] - for inst in active: - group_names = [g.name for g in inst.groups] - if (cluster_name + "-master") in group_names: - master_nodes.append(inst) - elif (cluster_name + "-slaves") in group_names: - slave_nodes.append(inst) - if any((master_nodes, slave_nodes)): - print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) - if master_nodes != [] or not die_on_error: - return (master_nodes, slave_nodes) - else: - if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name \ - + "-master" + " in region " + opts.region - else: - print >> sys.stderr, "ERROR: Could not find any existing cluster" \ - + " in region " + opts.region + """ + Get the EC2 instances in an existing cluster if available. + Returns a tuple of lists of EC2 instance objects for the masters and slaves. + """ + print "Searching for existing cluster {c} in region {r}...".format( + c=cluster_name, r=opts.region) + + def get_instances(group_names): + """ + Get all non-terminated instances that belong to any of the provided security groups. + + EC2 reservation filters and instance states are documented here: + http://docs.aws.amazon.com/cli/latest/reference/ec2/describe-instances.html#options + """ + reservations = conn.get_all_reservations( + filters={"instance.group-name": group_names}) + instances = itertools.chain.from_iterable(r.instances for r in reservations) + return [i for i in instances if i.state not in ["shutting-down", "terminated"]] + + master_instances = get_instances([cluster_name + "-master"]) + slave_instances = get_instances([cluster_name + "-slaves"]) + + if any((master_instances, slave_instances)): + print "Found {m} master{plural_m}, {s} slave{plural_s}.".format( + m=len(master_instances), + plural_m=('' if len(master_instances) == 1 else 's'), + s=len(slave_instances), + plural_s=('' if len(slave_instances) == 1 else 's')) + + if not master_instances and die_on_error: + print >> sys.stderr, \ + "ERROR: Could not find a master for cluster {c} in region {r}.".format( + c=cluster_name, r=opts.region) sys.exit(1) + return (master_instances, slave_instances) + # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. - - def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = master_nodes[0].public_dns_name if deploy_ssh_key: From f16b7b031feeb13ec9c17608bd99566f56431869 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 8 Mar 2015 14:09:40 +0000 Subject: [PATCH 367/817] SPARK-6205 [CORE] UISeleniumSuite fails for Hadoop 2.x test with NoClassDefFoundError Add xml-apis to core test deps to work aroudn UISeleniumSuite classpath issue Author: Sean Owen Closes #4933 from srowen/SPARK-6205 and squashes the following commits: ddd4d32 [Sean Owen] Add xml-apis to core test deps to work aroudn UISeleniumSuite classpath issue --- core/pom.xml | 6 ++++++ pom.xml | 7 +++++++ 2 files changed, 13 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index fab776d142ef7..dc0d07d806635 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -319,6 +319,12 @@ selenium-java test
    + + + xml-apis + xml-apis + test + org.mockito mockito-all diff --git a/pom.xml b/pom.xml index f99a83b9994ed..51bef30f9ca8f 100644 --- a/pom.xml +++ b/pom.xml @@ -422,6 +422,13 @@ 2.42.2 test + + + xml-apis + xml-apis + 1.4.01 + test + org.slf4j slf4j-api From 55b1b32dc8b9b25deea8e5864b53fe802bb92741 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 8 Mar 2015 19:47:35 +0000 Subject: [PATCH 368/817] [GraphX] Improve LiveJournalPageRank example 1. Removed unnecessary import 2. Modified usage print since user must specify the --numEPart parameter as it is required in Analytics.main Author: Jacky Li Closes #4917 from jackylk/import and squashes the following commits: 6c07682 [Jacky Li] fix comment c0df8f2 [Jacky Li] fix scalastyle b6235e6 [Jacky Li] fix for comment 87be83b [Jacky Li] remove default value description 5caae76 [Jacky Li] remove import and modify usage --- .../spark/examples/graphx/LiveJournalPageRank.scala | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index e809a65b79975..f6f8d9f90c275 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -17,11 +17,6 @@ package org.apache.spark.examples.graphx -import org.apache.spark.SparkContext._ -import org.apache.spark._ -import org.apache.spark.graphx._ - - /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from * http://snap.stanford.edu/data/soc-LiveJournal1.html. @@ -31,13 +26,13 @@ object LiveJournalPageRank { if (args.length < 1) { System.err.println( "Usage: LiveJournalPageRank \n" + + " --numEPart=\n" + + " The number of partitions for the graph's edge RDD.\n" + " [--tol=]\n" + " The tolerance allowed at convergence (smaller => more accurate). Default is " + "0.001.\n" + " [--output=]\n" + " If specified, the file to write the ranks to.\n" + - " [--numEPart=]\n" + - " The number of partitions for the graph's edge RDD. Default is 4.\n" + " [--partStrategy=RandomVertexCut | EdgePartition1D | EdgePartition2D | " + "CanonicalRandomVertexCut]\n" + " The way edges are assigned to edge partitions. Default is RandomVertexCut.") From f7c799204358bcc38c5972a29e5994b78b25b515 Mon Sep 17 00:00:00 2001 From: Theodore Vasiloudis Date: Mon, 9 Mar 2015 14:16:07 +0000 Subject: [PATCH 369/817] [EC2] [SPARK-6188] Instance types can be mislabeled when re-starting cluster with default arguments As described in https://issues.apache.org/jira/browse/SPARK-6188 and discovered in https://issues.apache.org/jira/browse/SPARK-5838. When re-starting a cluster, if the user does not provide the instance types, which is the recommended behavior in the docs currently, the instance will be assigned the default type m1.large. This then affects the setup of the machines. This solves this by getting the instance types from the existing instances, and overwriting the default options. EDIT: Further clarification of the issue: In short, while the instances themselves are the same as launched, their setup is done assuming the default instance type, m1.large. This means that the machines are assumed to have 2 disks, and that leads to problems that are described in in issue [5838](https://issues.apache.org/jira/browse/SPARK-5838), where machines that have one disk end up having shuffle spills in the in the small (8GB) snapshot partitions that quickly fills up and results in failing jobs due to "No space left on device" errors. Other instance specific settings that are set in the spark_ec2.py script are likely to be wrong as well. Author: Theodore Vasiloudis Author: Theodore Vasiloudis Closes #4916 from thvasilo/SPARK-6188]-Instance-types-can-be-mislabeled-when-re-starting-cluster-with-default-arguments and squashes the following commits: 6705b98 [Theodore Vasiloudis] Added comment to clarify setting master instance type to the empty string. a3d29fe [Theodore Vasiloudis] More trailing whitespace 7b32429 [Theodore Vasiloudis] Removed trailing whitespace 3ebd52a [Theodore Vasiloudis] Make sure that the instance type is correct when relaunching a cluster. --- ec2/spark_ec2.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5e636ddd17e94..b50b3816ff890 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1307,6 +1307,17 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='ssh-ready' ) + + # Determine types of running instances + existing_master_type = master_nodes[0].instance_type + existing_slave_type = slave_nodes[0].instance_type + # Setting opts.master_instance_type to the empty string indicates we + # have the same instance type for the master and the slaves + if existing_master_type == existing_slave_type: + existing_master_type = "" + opts.master_instance_type = existing_master_type + opts.instance_type = existing_slave_type + setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: From 70f88148bb04161a1a4968230d8e3fc7e3f8321a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 9 Mar 2015 13:29:19 -0700 Subject: [PATCH 370/817] [Docs] Replace references to SchemaRDD with DataFrame Author: Reynold Xin Closes #4952 from rxin/schemardd-df-reference and squashes the following commits: b2b1dbe [Reynold Xin] [Docs] Replace references to SchemaRDD with DataFrame --- python/pyspark/ml/pipeline.py | 4 ++-- python/pyspark/ml/wrapper.py | 2 +- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 5233c5801e2e6..83880a5afcd1d 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -39,7 +39,7 @@ def fit(self, dataset, params={}): Fits a model to the input dataset with optional parameters. :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` + :py:class:`pyspark.sql.DataFrame` :param params: an optional param map that overwrites embedded params :returns: fitted model @@ -62,7 +62,7 @@ def transform(self, dataset, params={}): Transforms the input dataset with optional parameters. :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` + :py:class:`pyspark.sql.DataFrame` :param params: an optional param map that overwrites embedded params :returns: transformed dataset diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 4bae96f678388..31a66b3d2f730 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -102,7 +102,7 @@ def _fit_java(self, dataset, params={}): """ Fits a Java model to the input dataset. :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` + :py:class:`pyspark.sql.DataFrame` :param params: additional params (overwriting embedded values) :return: fitted Java model """ diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f966f25c5a14c..ed9b207a86a0b 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -263,14 +263,14 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - test("SPARK-2576 importing SQLContext.createSchemaRDD.") { + test("SPARK-2576 importing SQLContext.createDataFrame.") { // We need to use local-cluster to test this case. val output = runInterpreter("local-cluster[1,1,512]", """ |val sqlContext = new org.apache.spark.sql.SQLContext(sc) - |import sqlContext.createSchemaRDD + |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toSchemaRDD.collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) From 3cac1991a1def0adaf42face2c578d3ab8c27025 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 9 Mar 2015 16:16:16 -0700 Subject: [PATCH 371/817] [SPARK-5310][Doc] Update SQL Programming Guide to include DataFrames. Author: Reynold Xin Closes #4954 from rxin/df-docs and squashes the following commits: c592c70 [Reynold Xin] [SPARK-5310][Doc] Update SQL Programming Guide to include DataFrames. --- docs/_layouts/global.html | 2 +- docs/index.md | 2 +- docs/sql-programming-guide.md | 404 ++++++++++++++++++++++++---------- 3 files changed, 286 insertions(+), 122 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index efc4c612937df..2e88b3093652d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -71,7 +71,7 @@
  • Spark Programming Guide
  • Spark Streaming
  • -
  • Spark SQL
  • +
  • DataFrames and SQL
  • MLlib (Machine Learning)
  • GraphX (Graph Processing)
  • Bagel (Pregel on Spark)
  • diff --git a/docs/index.md b/docs/index.md index 0986398e6f744..b5b016e34795e 100644 --- a/docs/index.md +++ b/docs/index.md @@ -74,7 +74,7 @@ options for deployment: in all supported languages (Scala, Java, Python) * Modules built on Spark: * [Spark Streaming](streaming-programming-guide.html): processing real-time data streams - * [Spark SQL](sql-programming-guide.html): support for structured data and relational queries + * [Spark SQL and DataFrames](sql-programming-guide.html): support for structured data and relational queries * [MLlib](mllib-guide.html): built-in machine learning library * [GraphX](graphx-programming-guide.html): Spark's new API for graph processing * [Bagel (Pregel on Spark)](bagel-programming-guide.html): older, simple graph processing model diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0146a4ed1b745..4fbdca7397951 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1,7 +1,7 @@ --- layout: global -displayTitle: Spark SQL Programming Guide -title: Spark SQL +displayTitle: Spark SQL and DataFrame Guide +title: Spark SQL and DataFrames --- * This will become a table of contents (this text will be scraped). @@ -9,55 +9,24 @@ title: Spark SQL # Overview -
    -
    - -Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using -Spark. At the core of this component is a new type of RDD, -[DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame). DataFrames are composed of -[Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with -a schema that describes the data types of each column in the row. A DataFrame is similar to a table -in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) -file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). - -All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`. - -
    +Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as a distributed query engine. -
    -Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using -Spark. At the core of this component is a new type of RDD, -[DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame). DataFrames are composed of -[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with -a schema that describes the data types of each column in the row. A DataFrame is similar to a table -in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) -file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). -
    -
    +# DataFrames -Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using -Spark. At the core of this component is a new type of RDD, -[DataFrame](api/python/pyspark.sql.html#pyspark.sql.DataFrame). DataFrames are composed of -[Row](api/python/pyspark.sql.Row-class.html) objects, along with -a schema that describes the data types of each column in the row. A DataFrame is similar to a table -in a traditional relational database. A DataFrame can be created from an existing RDD, a [Parquet](http://parquet.io) -file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). +A DataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing RDDs. -All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell. -
    -
    +The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), and [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame). -**Spark SQL is currently an alpha component. While we will minimize API changes, some APIs may change in future releases.** +All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell` or the `pyspark` shell. -*************************************************************************************************** -# Getting Started +## Starting Point: SQLContext
    -The entry point into all relational functionality in Spark is the +The entry point into all functionality in Spark SQL is the [SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its descendants. To create a basic SQLContext, all you need is a SparkContext. @@ -69,39 +38,19 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.implicits._ {% endhighlight %} -In addition to the basic SQLContext, you can also create a HiveContext, which provides a -superset of the functionality provided by the basic SQLContext. Additional features include -the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the -ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing Hive setup, and all of the data sources available to a SQLContext are still available. -HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to -feature parity with a HiveContext. -
    -The entry point into all relational functionality in Spark is the -[SQLContext](api/scala/index.html#org.apache.spark.sql.api.SQLContext) class, or one -of its descendants. To create a basic SQLContext, all you need is a JavaSparkContext. +The entry point into all functionality in Spark SQL is the +[SQLContext](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its +descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); {% endhighlight %} -In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict -super set of the functionality provided by the basic SQLContext. Additional features include -the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the -ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing Hive setup, and all of the data sources available to a SQLContext are still available. -HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to -feature parity with a HiveContext. -
    @@ -115,35 +64,266 @@ from pyspark.sql import SQLContext sqlContext = SQLContext(sc) {% endhighlight %} -In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict -super set of the functionality provided by the basic SQLContext. Additional features include -the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +
    +
    + +In addition to the basic SQLContext, you can also create a HiveContext, which provides a +superset of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to -feature parity with a HiveContext. - - - - +is recommended for the 1.3 release of Spark. Future releases will focus on bringing SQLContext up +to feature parity with a HiveContext. The specific variant of SQL that is used to parse queries can also be selected using the `spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, - this is recommended for most use cases. +this is recommended for most use cases. -# Data Sources -Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. -Registering a DataFrame as a table allows you to run SQL queries over its data. This section -describes the various methods for loading data into a DataFrame. +## Creating DataFrames + +With a `SQLContext`, applications can create `DataFrame`s from an existing `RDD`, from a Hive table, or from data sources. + +As an example, the following creates a `DataFrame` based on the content of a JSON file: + +
    +
    +{% highlight scala %} +val sc: SparkContext // An existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +val df = sqlContext.jsonFile("examples/src/main/resources/people.json") + +// Displays the content of the DataFrame to stdout +df.show() +{% endhighlight %} + +
    + +
    +{% highlight java %} +JavaSparkContext sc = ...; // An existing JavaSparkContext. +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); + +DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); + +// Displays the content of the DataFrame to stdout +df.show(); +{% endhighlight %} + +
    + +
    +{% highlight python %} +from pyspark.sql import SQLContext +sqlContext = SQLContext(sc) + +df = sqlContext.jsonFile("examples/src/main/resources/people.json") + +# Displays the content of the DataFrame to stdout +df.show() +{% endhighlight %} + +
    +
    + + +## DataFrame Operations + +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), and [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame). + +Here we include some basic examples of structured data processing using DataFrames: + + +
    +
    +{% highlight scala %} +val sc: SparkContext // An existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +// Create the DataFrame +val df = sqlContext.jsonFile("examples/src/main/resources/people.json") + +// Show the content of the DataFrame +df.show() +// age name +// null Michael +// 30 Andy +// 19 Justin + +// Print the schema in a tree format +df.printSchema() +// root +// |-- age: long (nullable = true) +// |-- name: string (nullable = true) + +// Select only the "name" column +df.select("name").show() +// name +// Michael +// Andy +// Justin + +// Select everybody, but increment the age by 1 +df.select("name", df("age") + 1).show() +// name (age + 1) +// Michael null +// Andy 31 +// Justin 20 + +// Select people older than 21 +df.filter(df("name") > 21).show() +// age name +// 30 Andy + +// Count people by age +df.groupBy("age").count().show() +// age count +// null 1 +// 19 1 +// 30 1 +{% endhighlight %} + +
    + +
    +{% highlight java %} +val sc: JavaSparkContext // An existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +// Create the DataFrame +DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); + +// Show the content of the DataFrame +df.show(); +// age name +// null Michael +// 30 Andy +// 19 Justin + +// Print the schema in a tree format +df.printSchema(); +// root +// |-- age: long (nullable = true) +// |-- name: string (nullable = true) + +// Select only the "name" column +df.select("name").show(); +// name +// Michael +// Andy +// Justin + +// Select everybody, but increment the age by 1 +df.select("name", df.col("age").plus(1)).show(); +// name (age + 1) +// Michael null +// Andy 31 +// Justin 20 + +// Select people older than 21 +df.filter(df("name") > 21).show(); +// age name +// 30 Andy + +// Count people by age +df.groupBy("age").count().show(); +// age count +// null 1 +// 19 1 +// 30 1 +{% endhighlight %} + +
    + +
    +{% highlight python %} +from pyspark.sql import SQLContext +sqlContext = SQLContext(sc) + +# Create the DataFrame +df = sqlContext.jsonFile("examples/src/main/resources/people.json") + +# Show the content of the DataFrame +df.show() +## age name +## null Michael +## 30 Andy +## 19 Justin + +# Print the schema in a tree format +df.printSchema() +## root +## |-- age: long (nullable = true) +## |-- name: string (nullable = true) + +# Select only the "name" column +df.select("name").show() +## name +## Michael +## Andy +## Justin + +# Select everybody, but increment the age by 1 +df.select("name", df.age + 1).show() +## name (age + 1) +## Michael null +## Andy 31 +## Justin 20 + +# Select people older than 21 +df.filter(df.name > 21).show() +## age name +## 30 Andy + +# Count people by age +df.groupBy("age").count().show() +## age count +## null 1 +## 19 1 +## 30 1 + +{% endhighlight %} + +
    +
    + + +## Running SQL Queries Programmatically + +The `sql` function on a `SQLContext` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +
    +
    +{% highlight scala %} +val sqlContext = ... // An existing SQLContext +val df = sqlContext.sql("SELECT * FROM table") +{% endhighlight %} +
    + +
    +{% highlight java %} +val sqlContext = ... // An existing SQLContext +val df = sqlContext.sql("SELECT * FROM table") +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark.sql import SQLContext +sqlContext = SQLContext(sc) +df = sqlContext.sql("SELECT * FROM table") +{% endhighlight %} +
    +
    + -## RDDs +## Interoperating with RDDs Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This @@ -373,12 +553,12 @@ by `SQLContext`. For example: {% highlight java %} // Import factory methods provided by DataType. -import org.apache.spark.sql.api.java.DataType +import org.apache.spark.sql.types.DataType; // Import StructType and StructField -import org.apache.spark.sql.api.java.StructType -import org.apache.spark.sql.api.java.StructField +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructField; // Import Row. -import org.apache.spark.sql.api.java.Row +import org.apache.spark.sql.Row; // sc is an existing JavaSparkContext. SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); @@ -472,11 +652,19 @@ for name in names.collect(): print name {% endhighlight %} - + +# Data Sources + +Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. +A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. +Registering a DataFrame as a table allows you to run SQL queries over its data. This section +describes the various methods for loading data into a DataFrame. + + ## Parquet Files [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. @@ -904,15 +1092,14 @@ that these options will be deprecated in future release as more optimizations ar -# Other SQL Interfaces +# Distributed Query Engine -Spark SQL also supports interfaces for running SQL queries directly without the need to write any -code. +Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, without the need to write any code. ## Running the Thrift JDBC/ODBC server The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. +in Hive 0.13. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.13. To start the JDBC/ODBC server, run the following in the Spark directory: @@ -982,7 +1169,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may run `./bin/spark-sql --help` for a complete list of all available options. -# Compatibility with Other Systems +# Migration Guide ## Migration Guide for Shark User @@ -1139,33 +1326,10 @@ releases of Spark SQL. Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that. -# Writing Language-Integrated Relational Queries - -**Language-Integrated queries are experimental and currently only supported in Scala.** - -Spark SQL also supports a domain specific language for writing queries. Once again, -using the data from the above examples: - -{% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import sqlContext._ -val people: RDD[Person] = ... // An RDD of case class objects, from the first example. - -// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' -val teenagers = people.where('age >= 10).where('age <= 19).select('name) -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} - -The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers -prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are -evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/scala/index.html#org.apache.spark.sql.DataFrame). - +# Data Types -# Spark SQL DataType Reference +Spark SQL and DataFrames support the following data types: * Numeric types - `ByteType`: Represents 1-byte signed integer numbers. @@ -1208,10 +1372,10 @@ evaluated by the SQL execution engine. A full list of the functions supported c
    -All data types of Spark SQL are located in the package `org.apache.spark.sql`. +All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. You can access them by doing {% highlight scala %} -import org.apache.spark.sql._ +import org.apache.spark.sql.types._ {% endhighlight %} @@ -1263,7 +1427,7 @@ import org.apache.spark.sql._ - + @@ -1457,7 +1621,7 @@ please use factory methods provided in - +
    DecimalType scala.math.BigDecimal java.math.BigDecimal DecimalType
    StructType org.apache.spark.sql.api.java.Row org.apache.spark.sql.Row DataTypes.createStructType(fields)
    Note: fields is a List or an array of StructFields. @@ -1478,10 +1642,10 @@ please use factory methods provided in
    -All data types of Spark SQL are located in the package of `pyspark.sql`. +All data types of Spark SQL are located in the package of `pyspark.sql.types`. You can access them by doing {% highlight python %} -from pyspark.sql import * +from pyspark.sql.types import * {% endhighlight %} From 8767565cef01d847f57b7293d8b63b2422009b90 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 9 Mar 2015 16:24:06 -0700 Subject: [PATCH 372/817] [SPARK-6194] [SPARK-677] [PySpark] fix memory leak in collect() Because circular reference between JavaObject and JavaMember, an Java object can not be released until Python GC kick in, then it will cause memory leak in collect(), which may consume lots of memory in JVM. This PR change the way we sending collected data back into Python from local file to socket, which could avoid any disk IO during collect, also avoid any referrers of Java object in Python. cc JoshRosen Author: Davies Liu Closes #4923 from davies/fix_collect and squashes the following commits: d730286 [Davies Liu] address comments 24c92a4 [Davies Liu] fix style ba54614 [Davies Liu] use socket to transfer data from JVM 9517c8f [Davies Liu] fix memory leak in collect() --- .../apache/spark/api/python/PythonRDD.scala | 76 ++++++++++++++----- python/pyspark/context.py | 13 ++-- python/pyspark/rdd.py | 30 ++++---- python/pyspark/sql/dataframe.py | 14 +--- 4 files changed, 82 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b1cec0f6472b0..8d4a53b4ca9b0 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,26 +19,27 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, UUID, Collections} - -import org.apache.spark.input.PortableDataStream +import java.util.{Collections, ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.mutable import scala.language.existentials import com.google.common.base.Charsets.UTF_8 - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} +import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} + import org.apache.spark._ -import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast +import org.apache.spark.input.PortableDataStream import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils +import scala.util.control.NonFatal + private[spark] class PythonRDD( @transient parent: RDD[_], command: Array[Byte], @@ -341,21 +342,33 @@ private[spark] object PythonRDD extends Logging { /** * Adapter for calling SparkContext#runJob from Python. * - * This method will return an iterator of an array that contains all elements in the RDD + * This method will serve an iterator of an array that contains all elements in the RDD * (effectively a collect()), but allows you to run on a certain subset of partitions, * or to enable local execution. + * + * @return the port number of a local socket which serves the data collected from this job. */ def runJob( sc: SparkContext, rdd: JavaRDD[Array[Byte]], partitions: JArrayList[Int], - allowLocal: Boolean): Iterator[Array[Byte]] = { + allowLocal: Boolean): Int = { type ByteArray = Array[Byte] type UnrolledPartition = Array[ByteArray] val allPartitions: Array[UnrolledPartition] = sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions, allowLocal) val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) - flattenedPartition.iterator + serveIterator(flattenedPartition.iterator, + s"serve RDD ${rdd.id} with partitions ${partitions.mkString(",")}") + } + + /** + * A helper function to collect an RDD as an iterator, then serve it via socket. + * + * @return the port number of a local socket which serves the data collected from this job. + */ + def collectAndServe[T](rdd: RDD[T]): Int = { + serveIterator(rdd.collect().iterator, s"serve RDD ${rdd.id}") } def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): @@ -575,15 +588,44 @@ private[spark] object PythonRDD extends Logging { dataOut.write(bytes) } - def writeToFile[T](items: java.util.Iterator[T], filename: String) { - import scala.collection.JavaConverters._ - writeToFile(items.asScala, filename) - } + /** + * Create a socket server and a background thread to serve the data in `items`, + * + * The socket server can only accept one connection, or close if no connection + * in 3 seconds. + * + * Once a connection comes in, it tries to serialize all the data in `items` + * and send them into this connection. + * + * The thread will terminate after all the data are sent or any exceptions happen. + */ + private def serveIterator[T](items: Iterator[T], threadName: String): Int = { + val serverSocket = new ServerSocket(0, 1) + serverSocket.setReuseAddress(true) + // Close the socket if no connection in 3 seconds + serverSocket.setSoTimeout(3000) + + new Thread(threadName) { + setDaemon(true) + override def run() { + try { + val sock = serverSocket.accept() + val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + try { + writeIteratorToStream(items, out) + } finally { + out.close() + } + } catch { + case NonFatal(e) => + logError(s"Error while sending iterator", e) + } finally { + serverSocket.close() + } + } + }.start() - def writeToFile[T](items: Iterator[T], filename: String) { - val file = new DataOutputStream(new FileOutputStream(filename)) - writeIteratorToStream(items, file) - file.close() + serverSocket.getLocalPort } private def getMergedConf(confAsMap: java.util.HashMap[String, String], diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6011caf9f1c5a..78dccc40470e3 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -21,6 +21,8 @@ from threading import Lock from tempfile import NamedTemporaryFile +from py4j.java_collections import ListConverter + from pyspark import accumulators from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast @@ -30,13 +32,11 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel -from pyspark.rdd import RDD +from pyspark.rdd import RDD, _load_from_socket from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler -from py4j.java_collections import ListConverter - __all__ = ['SparkContext'] @@ -59,7 +59,6 @@ class SparkContext(object): _gateway = None _jvm = None - _writeToFile = None _next_accum_id = 0 _active_spark_context = None _lock = Lock() @@ -221,7 +220,6 @@ def _ensure_initialized(cls, instance=None, gateway=None): if not SparkContext._gateway: SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm - SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile if instance: if (SparkContext._active_spark_context and @@ -840,8 +838,9 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): # by runJob() in order to avoid having to pass a Python lambda into # SparkContext#runJob. mappedRDD = rdd.mapPartitions(partitionFunc) - it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) - return list(mappedRDD._collect_iterator_through_file(it)) + port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, + allowLocal) + return list(_load_from_socket(port, mappedRDD._jrdd_deserializer)) def show_profiles(self): """ Print the profile stats to stdout """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index cb12fed98c53d..bf17f513c0bc3 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -19,7 +19,6 @@ from collections import defaultdict from itertools import chain, ifilter, imap import operator -import os import sys import shlex from subprocess import Popen, PIPE @@ -29,6 +28,7 @@ import heapq import bisect import random +import socket from math import sqrt, log, isinf, isnan, pow, ceil from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ @@ -111,6 +111,17 @@ def _parse_memory(s): return int(float(s[:-1]) * units[s[-1].lower()]) +def _load_from_socket(port, serializer): + sock = socket.socket() + try: + sock.connect(("localhost", port)) + rf = sock.makefile("rb", 65536) + for item in serializer.load_stream(rf): + yield item + finally: + sock.close() + + class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -698,21 +709,8 @@ def collect(self): Return a list that contains all of the elements in this RDD. """ with SCCallSiteSync(self.context) as css: - bytesInJava = self._jrdd.collect().iterator() - return list(self._collect_iterator_through_file(bytesInJava)) - - def _collect_iterator_through_file(self, iterator): - # Transferring lots of data through Py4J can be slow because - # socket.readline() is inefficient. Instead, we'll dump the data to a - # file and read it back. - tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir) - tempFile.close() - self.ctx._writeToFile(iterator, tempFile.name) - # Read the data into Python and deserialize it: - with open(tempFile.name, 'rb') as tempFile: - for item in self._jrdd_deserializer.load_stream(tempFile): - yield item - os.unlink(tempFile.name) + port = self.ctx._jvm.PythonRDD.collectAndServe(self._jrdd.rdd()) + return list(_load_from_socket(port, self._jrdd_deserializer)) def reduce(self, f): """ diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 5c3b7377c33b5..e8ce4547455a5 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -19,13 +19,11 @@ import itertools import warnings import random -import os -from tempfile import NamedTemporaryFile from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD +from pyspark.rdd import RDD, _load_from_socket from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -310,14 +308,8 @@ def collect(self): [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ with SCCallSiteSync(self._sc) as css: - bytesInJava = self._jdf.javaToPython().collect().iterator() - tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir) - tempFile.close() - self._sc._writeToFile(bytesInJava, tempFile.name) - # Read the data into Python and deserialize it: - with open(tempFile.name, 'rb') as tempFile: - rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile)) - os.unlink(tempFile.name) + port = self._sc._jvm.PythonRDD.collectAndServe(self._jdf.javaToPython().rdd()) + rs = list(_load_from_socket(port, BatchedSerializer(PickleSerializer()))) cls = _create_cls(self.schema) return [cls(r) for r in rs] From 9a0272fbb322042788f14e9cd99e2db86b456225 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Tue, 10 Mar 2015 10:51:44 +0000 Subject: [PATCH 373/817] [SPARK-6177][MLlib]Add note in LDA example to remind possible coalesce JIRA: https://issues.apache.org/jira/browse/SPARK-6177 Add comment to introduce coalesce to LDA example to avoid the possible massive partitions from `sc.textFile`. sc.textFile will create RDD with one partition for each file, and the possible massive partitions downgrades LDA performance. Author: Yuhao Yang Closes #4899 from hhbyyh/adjustPartition and squashes the following commits: a499630 [Yuhao Yang] update comment 9a2d7b6 [Yuhao Yang] move to comment f7fd5d4 [Yuhao Yang] Merge remote-tracking branch 'upstream/master' into adjustPartition 26a564a [Yuhao Yang] add coalesce to LDAExample --- .../scala/org/apache/spark/examples/mllib/LDAExample.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 11399a7633638..08a93595a2e17 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -173,7 +173,9 @@ object LDAExample { stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { // Get dataset of document texts - // One document per line in each text file. + // One document per line in each text file. If the input consists of many small files, + // this can result in a large number of small partitions, which can degrade performance. + // In this case, consider using coalesce() to create fewer, larger partitions. val textRDD: RDD[String] = sc.textFile(paths.mkString(",")) // Split text into words From c4c4b07bf61cab01d92fde4f902d8c06abdce240 Mon Sep 17 00:00:00 2001 From: Lev Khomich Date: Tue, 10 Mar 2015 10:55:42 +0000 Subject: [PATCH 374/817] [SPARK-6087][CORE] Provide actionable exception if Kryo buffer is not large enough A simple try-catch wrapping KryoException to be more informative. Author: Lev Khomich Closes #4947 from levkhomich/master and squashes the following commits: 0f7a947 [Lev Khomich] [SPARK-6087][CORE] Provide actionable exception if Kryo buffer is not large enough --- .../apache/spark/serializer/KryoSerializer.scala | 8 +++++++- .../spark/serializer/KryoSerializerSuite.scala | 14 ++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 9ce64d41fbc40..dc7aa99738c17 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -158,7 +158,13 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() - kryo.writeClassAndObject(output, t) + try { + kryo.writeClassAndObject(output, t) + } catch { + case e: KryoException if e.getMessage.startsWith("Buffer overflow") => + throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + + "increase spark.kryoserializer.buffer.max.mb value.") + } ByteBuffer.wrap(output.toBytes) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 523d898207447..6198df84fab3d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -261,6 +261,20 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) } } + + test("serialization buffer overflow reporting") { + import org.apache.spark.SparkException + val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max.mb" + + val largeObject = (1 to 1000000).toArray + + val conf = new SparkConf(false) + conf.set(kryoBufferMaxProperty, "1") + + val ser = new KryoSerializer(conf).newInstance() + val thrown = intercept[SparkException](ser.serialize(largeObject)) + assert(thrown.getMessage.contains(kryoBufferMaxProperty)) + } } From d14df06c05a6228fd6522914c39aa75898eddfc1 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 10 Mar 2015 10:58:31 +0000 Subject: [PATCH 375/817] [SPARK-6191] [EC2] Generalize ability to download libs Right now we have a method to specifically download boto. This PR generalizes it so it's easy to download additional libraries if we want. For example, adding new external libraries for spark-ec2 is now as simple as: ```python external_libs = [ { "name": "boto", "version": "2.34.0", "md5": "5556223d2d0cc4d06dd4829e671dcecd" }, { "name": "PyYAML", "version": "3.11", "md5": "f50e08ef0fe55178479d3a618efe21db" }, { "name": "argparse", "version": "1.3.0", "md5": "9bcf7f612190885c8c85e30ba41db3c7" } ] ``` Likely use cases: * Downloading PyYAML to allow spark-ec2 configs to be persisted as a YAML file. ([SPARK-925](https://issues.apache.org/jira/browse/SPARK-925)) * Downloading argparse to clean up / modernize our option parsing. First run output, with PyYAML and argparse added just for demonstration purposes: ```shell $ ./spark-ec2 --version Downloading external libraries that spark-ec2 needs from PyPI to /path/to/spark/ec2/lib... This should be a one-time operation. - Downloading boto... - Finished downloading boto. - Downloading PyYAML... - Finished downloading PyYAML. - Downloading argparse... - Finished downloading argparse. spark-ec2 1.2.1 ``` Output thereafter: ```shell $ ./spark-ec2 --version spark-ec2 1.2.1 ``` Author: Nicholas Chammas Closes #4919 from nchammas/setup-ec2-libs and squashes the following commits: a077955 [Nicholas Chammas] print default region c95fb7d [Nicholas Chammas] to docstring 5448845 [Nicholas Chammas] remove libs added for demo purposes 60d8c23 [Nicholas Chammas] generalize ability to download libs --- ec2/spark_ec2.py | 82 +++++++++++++++++++++++++++++++----------------- 1 file changed, 54 insertions(+), 28 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index b50b3816ff890..3acb5fea042df 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -70,34 +70,60 @@ DEFAULT_SPARK_EC2_BRANCH = "branch-1.3" -def setup_boto(): - # Download Boto if it's not already present in the SPARK_EC2_DIR/lib folder: - version = "boto-2.34.0" - md5 = "5556223d2d0cc4d06dd4829e671dcecd" - url = "https://pypi.python.org/packages/source/b/boto/%s.tar.gz" % version - lib_dir = os.path.join(SPARK_EC2_DIR, "lib") - if not os.path.exists(lib_dir): - os.mkdir(lib_dir) - boto_lib_dir = os.path.join(lib_dir, version) - if not os.path.isdir(boto_lib_dir): - tgz_file_path = os.path.join(lib_dir, "%s.tar.gz" % version) - print "Downloading Boto from PyPi" - download_stream = urllib2.urlopen(url) - with open(tgz_file_path, "wb") as tgz_file: - tgz_file.write(download_stream.read()) - with open(tgz_file_path) as tar: - if hashlib.md5(tar.read()).hexdigest() != md5: - print >> stderr, "ERROR: Got wrong md5sum for Boto" - sys.exit(1) - tar = tarfile.open(tgz_file_path) - tar.extractall(path=lib_dir) - tar.close() - os.remove(tgz_file_path) - print "Finished downloading Boto" - sys.path.insert(0, boto_lib_dir) +def setup_external_libs(libs): + """ + Download external libraries from PyPI to SPARK_EC2_DIR/lib/ and prepend them to our PATH. + """ + PYPI_URL_PREFIX = "https://pypi.python.org/packages/source" + SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") + + if not os.path.exists(SPARK_EC2_LIB_DIR): + print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( + path=SPARK_EC2_LIB_DIR + ) + print "This should be a one-time operation." + os.mkdir(SPARK_EC2_LIB_DIR) + + for lib in libs: + versioned_lib_name = "{n}-{v}".format(n=lib["name"], v=lib["version"]) + lib_dir = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name) + + if not os.path.isdir(lib_dir): + tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") + print " - Downloading {lib}...".format(lib=lib["name"]) + download_stream = urllib2.urlopen( + "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( + prefix=PYPI_URL_PREFIX, + first_letter=lib["name"][:1], + lib_name=lib["name"], + lib_version=lib["version"] + ) + ) + with open(tgz_file_path, "wb") as tgz_file: + tgz_file.write(download_stream.read()) + with open(tgz_file_path) as tar: + if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: + print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]) + sys.exit(1) + tar = tarfile.open(tgz_file_path) + tar.extractall(path=SPARK_EC2_LIB_DIR) + tar.close() + os.remove(tgz_file_path) + print " - Finished downloading {lib}.".format(lib=lib["name"]) + sys.path.insert(1, lib_dir) + + +# Only PyPI libraries are supported. +external_libs = [ + { + "name": "boto", + "version": "2.34.0", + "md5": "5556223d2d0cc4d06dd4829e671dcecd" + } +] +setup_external_libs(external_libs) -setup_boto() import boto from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 @@ -136,7 +162,7 @@ def parse_args(): help="Master instance type (leave empty for same as instance-type)") parser.add_option( "-r", "--region", default="us-east-1", - help="EC2 region used to launch instances in, or to find them in") + help="EC2 region used to launch instances in, or to find them in (default: %default)") parser.add_option( "-z", "--zone", default="", help="Availability zone to launch instances in, or 'all' to spread " + @@ -230,7 +256,7 @@ def parse_args(): "(e.g -Dspark.worker.timeout=180)") parser.add_option( "--user-data", type="string", default="", - help="Path to a user-data file (most AMI's interpret this as an initialization script)") + help="Path to a user-data file (most AMIs interpret this as an initialization script)") parser.add_option( "--authorized-address", type="string", default="0.0.0.0/0", help="Address to authorize on created security groups (default: %default)") From 7c7d2d5e093b0623edc75dd166ec1179b4e62062 Mon Sep 17 00:00:00 2001 From: cheng chang Date: Tue, 10 Mar 2015 11:02:12 +0000 Subject: [PATCH 376/817] [SPARK-6186] [EC2] Make Tachyon version configurable in EC2 deployment script This PR comes from Tachyon community to solve the issue: https://tachyon.atlassian.net/browse/TACHYON-11 An accompanying PR is in mesos/spark-ec2: https://github.com/mesos/spark-ec2/pull/101 Author: cheng chang Closes #4901 from uronce-cc/master and squashes the following commits: 313aa36 [cheng chang] minor re-wording fd2a48e [cheng chang] Remove Tachyon when deploying through git hash 1d53c5c [cheng chang] add default value to --tachyon-version 6f8887e [cheng chang] make tachyon version configurable --- .../root/spark-ec2/ec2-variables.sh | 3 ++- ec2/spark_ec2.py | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 740c267fd9866..0857657152ec7 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -26,9 +26,10 @@ export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" export MODULES="{{modules}}" export SPARK_VERSION="{{spark_version}}" export SHARK_VERSION="{{shark_version}}" +export TACHYON_VERSION="{{tachyon_version}}" export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}" export SPARK_MASTER_OPTS="{{spark_master_opts}}" export AWS_ACCESS_KEY_ID="{{aws_access_key_id}}" -export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}" \ No newline at end of file +export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 3acb5fea042df..f848874b0c775 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -62,6 +62,16 @@ "1.2.1", ]) +SPARK_TACHYON_MAP = { + "1.0.0": "0.4.1", + "1.0.1": "0.4.1", + "1.0.2": "0.4.1", + "1.1.0": "0.5.0", + "1.1.1": "0.5.0", + "1.2.0": "0.5.0", + "1.2.1": "0.5.0", +} + DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" @@ -370,6 +380,10 @@ def get_validate_spark_version(version, repo): } +def get_tachyon_version(spark_version): + return SPARK_TACHYON_MAP.get(spark_version, "") + + # Attempt to resolve an appropriate AMI given the architecture and region of the request. def get_spark_ami(opts): if opts.instance_type in EC2_INSTANCE_TYPES: @@ -919,9 +933,13 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): if "." in opts.spark_version: # Pre-built Spark deploy spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo) + tachyon_v = get_tachyon_version(spark_v) else: # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) + tachyon_v = "" + print "Deploying Spark via git hash; Tachyon won't be set up" + modules = filter(lambda x: x != "tachyon", modules) template_vars = { "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), @@ -934,6 +952,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "swap": str(opts.swap), "modules": '\n'.join(modules), "spark_version": spark_v, + "tachyon_version": tachyon_v, "hadoop_major_version": opts.hadoop_major_version, "spark_worker_instances": "%d" % opts.worker_instances, "spark_master_opts": opts.master_opts From 74fb433702b676225097e1d4d2c2b170915a5d19 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 10 Mar 2015 17:25:04 -0700 Subject: [PATCH 377/817] Minor doc: Remove the extra blank line in data types javadoc. The extra blank line is preventing the first lines from showing up in the package summary page. Author: Reynold Xin Closes #4955 from rxin/datatype-docs and squashes the following commits: 1621114 [Reynold Xin] Minor doc: Remove the extra blank line in data types javadoc. --- .../apache/spark/sql/types/dataTypes.scala | 24 +++++-------------- 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 92d322845f5c5..bf39603d13bd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -240,7 +240,6 @@ object DataType { /** * :: DeveloperApi :: - * * The base type of all Spark SQL data types. * * @group dataType @@ -282,7 +281,6 @@ abstract class DataType { /** * :: DeveloperApi :: - * * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. * * @group dataType @@ -342,7 +340,6 @@ protected[sql] abstract class NativeType extends DataType { /** * :: DeveloperApi :: - * * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. * * @group dataType @@ -369,7 +366,6 @@ case object StringType extends StringType /** * :: DeveloperApi :: - * * The data type representing `Array[Byte]` values. * Please use the singleton [[DataTypes.BinaryType]]. * @@ -405,7 +401,6 @@ case object BinaryType extends BinaryType /** * :: DeveloperApi :: - * * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. * *@group dataType @@ -432,7 +427,6 @@ case object BooleanType extends BooleanType /** * :: DeveloperApi :: - * * The data type representing `java.sql.Timestamp` values. * Please use the singleton [[DataTypes.TimestampType]]. * @@ -464,7 +458,6 @@ case object TimestampType extends TimestampType /** * :: DeveloperApi :: - * * The data type representing `java.sql.Date` values. * Please use the singleton [[DataTypes.DateType]]. * @@ -492,6 +485,12 @@ class DateType private() extends NativeType { case object DateType extends DateType +/** + * :: DeveloperApi :: + * Numeric data types. + * + * @group dataType + */ abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a @@ -523,7 +522,6 @@ protected[sql] sealed abstract class IntegralType extends NumericType { /** * :: DeveloperApi :: - * * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. * * @group dataType @@ -554,7 +552,6 @@ case object LongType extends LongType /** * :: DeveloperApi :: - * * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. * * @group dataType @@ -585,7 +582,6 @@ case object IntegerType extends IntegerType /** * :: DeveloperApi :: - * * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. * * @group dataType @@ -616,7 +612,6 @@ case object ShortType extends ShortType /** * :: DeveloperApi :: - * * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. * * @group dataType @@ -666,7 +661,6 @@ case class PrecisionInfo(precision: Int, scale: Int) /** * :: DeveloperApi :: - * * The data type representing `java.math.BigDecimal` values. * A Decimal that might have fixed precision and scale, or unlimited values for these. * @@ -745,7 +739,6 @@ object DecimalType { /** * :: DeveloperApi :: - * * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. * * @group dataType @@ -775,7 +768,6 @@ case object DoubleType extends DoubleType /** * :: DeveloperApi :: - * * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. * * @group dataType @@ -811,7 +803,6 @@ object ArrayType { /** * :: DeveloperApi :: - * * The data type for collections of multiple values. * Internally these are represented as columns that contain a ``scala.collection.Seq``. * @@ -854,7 +845,6 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT /** * A field inside a StructType. - * * @param name The name of this field. * @param dataType The data type of this field. * @param nullable Indicates if values of this field can be `null` values. @@ -949,7 +939,6 @@ object StructType { /** * :: DeveloperApi :: - * * A [[StructType]] object can be constructed by * {{{ * StructType(fields: Seq[StructField]) @@ -1118,7 +1107,6 @@ object MapType { /** * :: DeveloperApi :: - * * The data type for Maps. Keys in a map are not allowed to have `null` values. * * Please use [[DataTypes.createMapType()]] to create a specific instance. From 2672374110d58e45ffae2408e74b96613deddda3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 10 Mar 2015 18:13:09 -0700 Subject: [PATCH 378/817] [SPARK-5183][SQL] Update SQL Docs with JDBC and Migration Guide Author: Michael Armbrust Closes #4958 from marmbrus/sqlDocs and squashes the following commits: 9351dbc [Michael Armbrust] fix parquet example 6877e13 [Michael Armbrust] add sql examples d81b7e7 [Michael Armbrust] rxins comments e393528 [Michael Armbrust] fix order 19c2735 [Michael Armbrust] more on data source load/store 00d5914 [Michael Armbrust] Update SQL Docs with JDBC and Migration Guide --- docs/sql-programming-guide.md | 382 ++++++++++++++++++++++++++++++++-- 1 file changed, 370 insertions(+), 12 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4fbdca7397951..9c363bc87e890 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -9,7 +9,7 @@ title: Spark SQL and DataFrames # Overview -Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as a distributed query engine. +Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as distributed SQL query engine. # DataFrames @@ -662,8 +662,146 @@ for name in names.collect(): Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. Registering a DataFrame as a table allows you to run SQL queries over its data. This section -describes the various methods for loading data into a DataFrame. +describes the general methods for loading and saving data using the Spark Data Sources and then +goes into specific options that are available for the built-in data sources. +## Generic Load/Save Functions + +In the simplest form, the default data source (`parquet` unless otherwise configured by +`spark.sql.sources.default`) will be used for all operations. + +
    +
    + +{% highlight scala %} +val df = sqlContext.load("people.parquet") +df.select("name", "age").save("namesAndAges.parquet") +{% endhighlight %} + +
    + +
    + +{% highlight java %} + +DataFrame df = sqlContext.load("people.parquet"); +df.select("name", "age").save("namesAndAges.parquet"); + +{% endhighlight %} + +
    + +
    + +{% highlight python %} + +df = sqlContext.load("people.parquet") +df.select("name", "age").save("namesAndAges.parquet") + +{% endhighlight %} + +
    +
    + +### Manually Specifying Options + +You can also manually specify the data source that will be used along with any extra options +that you would like to pass to the data source. Data sources are specified by their fully qualified +name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use the shorted +name (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types +using this syntax. + +
    +
    + +{% highlight scala %} +val df = sqlContext.load("people.json", "json") +df.select("name", "age").save("namesAndAges.parquet", "parquet") +{% endhighlight %} + +
    + +
    + +{% highlight java %} + +DataFrame df = sqlContext.load("people.json", "json"); +df.select("name", "age").save("namesAndAges.parquet", "parquet"); + +{% endhighlight %} + +
    + +
    + +{% highlight python %} + +df = sqlContext.load("people.json", "json") +df.select("name", "age").save("namesAndAges.parquet", "parquet") + +{% endhighlight %} + +
    +
    + +### Save Modes + +Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if +present. It is important to realize that these save modes do not utilize any locking and are not +atomic. Thus, it is not safe to have multiple writers attempting to write to the same location. +Additionally, when performing a `Overwrite`, the data will be deleted before writing out the +new data. + +
    + + + + + + + + + + + + + + + + + + + + + +
    Scala/JavaPythonMeaning
    SaveMode.ErrorIfExists (default)"error" (default) + When saving a DataFrame to a data source, if data already exists, + an exception is expected to be thrown. +
    SaveMode.Append"append" + When saving a DataFrame to a data source, if data/table already exists, + contents of the DataFrame are expected to be appended to existing data. +
    SaveMode.Overwrite"overwrite" + Overwrite mode means that when saving a DataFrame to a data source, + if data/table already exists, existing data is expected to be overwritten by the contents of + the DataFrame. +
    SaveMode.Ignore"ignore" + Ignore mode means that when saving a DataFrame to a data source, if data already exists, + the save operation is expected to not save the contents of the DataFrame and to not + change the existing data. This is similar to a `CREATE TABLE IF NOT EXISTS` in SQL. +
    + +### Saving to Persistent Tables + +When working with a `HiveContext`, `DataFrames` can also be saved as persistent tables using the +`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the +contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables +will still exist even after your Spark program has restarted, as long as you maintain your connection +to the same metastore. A DataFrame for a persistent table can be created by calling the `table` +method on a SQLContext with the name of the table. + +By default `saveAsTable` will create a "managed table", meaning that the location of the data will +be controlled by the metastore. Managed tables will also have their data deleted automatically +when a table is dropped. ## Parquet Files @@ -751,6 +889,22 @@ for teenName in teenNames.collect():
    +
    + +{% highlight sql %} + +CREATE TEMPORARY TABLE parquetTable +USING org.apache.spark.sql.parquet +OPTIONS ( + path "examples/src/main/resources/people.parquet" +) + +SELECT * FROM parquetTable + +{% endhighlight %} + +
    + ### Configuration @@ -942,6 +1096,22 @@ anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) {% endhighlight %} +
    + +{% highlight sql %} + +CREATE TEMPORARY TABLE jsonTable +USING org.apache.spark.sql.json +OPTIONS ( + path "examples/src/main/resources/people.json" +) + +SELECT * FROM jsonTable + +{% endhighlight %} + +
    + ## Hive Tables @@ -1022,6 +1192,121 @@ results = sqlContext.sql("FROM src SELECT key, value").collect() +## JDBC To Other Databases + +Spark SQL also includes a data source that can read data from other databases using JDBC. This +functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). +This is because the results are returned +as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. +The JDBC data source is also easier to use from Java or Python as it does not require the user to +provide a ClassTag. +(Note that this is different than the Spark SQL JDBC server, which allows other applications to +run queries using Spark SQL). + +To get started you will need to include the JDBC driver for you particular database on the +spark classpath. For example, to connect to postgres from the Spark Shell you would run the +following command: + +{% highlight bash %} +SPARK_CLASSPATH=postgresql-9.3-1102-jdbc41.jar bin/spark-shell +{% endhighlight %} + +Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using +the Data Sources API. The following options are supported: + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    url + The JDBC URL to connect to. +
    dbtable + The JDBC table that should be read. Note that anything that is valid in a `FROM` clause of + a SQL query can be used. For example, instead of a full table you could also use a + subquery in parentheses. +
    driver + The class name of the JDBC driver needed to connect to this URL. This class with be loaded + on the master and workers before running an JDBC commands to allow the driver to + register itself with the JDBC subsystem. +
    partitionColumn, lowerBound, upperBound, numPartitions + These options must all be specified if any of them is specified. They describe how to + partition the table when reading in parallel from multiple workers. + partitionColumn must be a numeric column from the table in question. +
    + +
    + +
    + +{% highlight scala %} +val jdbcDF = sqlContext.load("jdbc", Map( + "url" -> "jdbc:postgresql:dbserver", + "dbtable" -> "schema.tablename")) +{% endhighlight %} + +
    + +
    + +{% highlight java %} + +Map options = new HashMap(); +options.put("url", "jdbc:postgresql:dbserver"); +options.put("dbtable", "schema.tablename"); + +DataFrame jdbcDF = sqlContext.load("jdbc", options) +{% endhighlight %} + + +
    + +
    + +{% highlight python %} + +df = sqlContext.load("jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") + +{% endhighlight %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY TABLE jdbcTable +USING org.apache.spark.sql.jdbc +OPTIONS ( + url "jdbc:postgresql:dbserver", + dbtable "schema.tablename" +) + +{% endhighlight %} + +
    +
    + +## Troubleshooting + + * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. + * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. + + # Performance Tuning For some workloads it is possible to improve performance by either caching data in memory, or by @@ -1092,7 +1377,7 @@ that these options will be deprecated in future release as more optimizations ar
    -# Distributed Query Engine +# Distributed SQL Engine Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, without the need to write any code. @@ -1171,6 +1456,87 @@ options. # Migration Guide +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general theses classes try to +use types that are usable from both languages (i.e. `Array` instead of language specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to sqlContext.udf (Java & Scala) + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight java %} + +sqlCtx.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlCtx.udf().register("strLen", (String s) -> { s.length(); }); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + +#### Python DataTypes No Longer Singletons + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. + ## Migration Guide for Shark User ### Scheduling @@ -1289,15 +1655,10 @@ in Hive deployments. * Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL doesn't support buckets yet. -**Esoteric Hive Features** -* Tables with partitions using different input formats: In Spark SQL, all table partitions need to - have the same input format. -* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions - (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. +**Esoteric Hive Features** * `UNION` type * Unique join -* Single query multi insert * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at the moment and only supports populating the sizeInBytes field of the hive metastore. @@ -1313,9 +1674,6 @@ less important due to Spark SQL's in-memory computational model. Others are slot releases of Spark SQL. * Block level bitmap indexes and virtual columns (used to build indexes) -* Automatically convert a join to map join: For joining a large table with multiple small tables, - Hive automatically converts the join into a map join. We are adding this auto conversion in the - next release. * Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". * Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still From 2d4e00efe2cf179935ae108a68f28edf6e5a1628 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Wed, 11 Mar 2015 00:24:55 -0700 Subject: [PATCH 379/817] [SPARK-5986][MLLib] Add save/load for k-means This PR adds save/load for K-means as described in SPARK-5986. Python version will be added in another PR. Author: Xusen Yin Closes #4951 from yinxusen/SPARK-5986 and squashes the following commits: 6dd74a0 [Xusen Yin] rewrite some functions and classes cd390fd [Xusen Yin] add indexed point b144216 [Xusen Yin] remove invalid comments dce7055 [Xusen Yin] add save/load for k-means for SPARK-5986 --- .../spark/mllib/clustering/KMeansModel.scala | 68 ++++++++++++++++++- .../spark/mllib/clustering/KMeansSuite.scala | 44 +++++++++++- 2 files changed, 108 insertions(+), 4 deletions(-) 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 3b95a9e6936e8..707da537d238f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,15 +17,22 @@ package org.apache.spark.mllib.clustering +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.{Loader, Saveable} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.Row /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length @@ -58,4 +65,59 @@ class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { private def clusterCentersWithNorm: Iterable[VectorWithNorm] = clusterCenters.map(new VectorWithNorm(_)) + + override def save(sc: SparkContext, path: String): Unit = { + KMeansModel.SaveLoadV1_0.save(sc, this, path) + } + + override protected def formatVersion: String = "1.0" +} + +object KMeansModel extends Loader[KMeansModel] { + override def load(sc: SparkContext, path: String): KMeansModel = { + KMeansModel.SaveLoadV1_0.load(sc, path) + } + + private case class Cluster(id: Int, point: Vector) + + private object Cluster { + def apply(r: Row): Cluster = { + Cluster(r.getInt(0), r.getAs[Vector](1)) + } + } + + private[clustering] + object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel" + + def save(sc: SparkContext, model: KMeansModel, path: String): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + val dataRDD = sc.parallelize(model.clusterCenters.zipWithIndex).map { case (point, id) => + Cluster(id, point) + }.toDF() + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): KMeansModel = { + implicit val formats = DefaultFormats + val sqlContext = new SQLContext(sc) + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + val k = (metadata \ "k").extract[Int] + val centriods = sqlContext.parquetFile(Loader.dataPath(path)) + Loader.checkSchema[Cluster](centriods.schema) + val localCentriods = centriods.map(Cluster.apply).collect() + assert(k == localCentriods.size) + new KMeansModel(localCentriods.sortBy(_.id).map(_.point)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index caee5917000aa..7bf250eb5a383 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -21,9 +21,10 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class KMeansSuite extends FunSuite with MLlibTestSparkContext { @@ -257,6 +258,47 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext { assert(predicts(0) != predicts(3)) } } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + Array(true, false).foreach { case selector => + val model = KMeansSuite.createModel(10, 3, selector) + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = KMeansModel.load(sc, path) + KMeansSuite.checkEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } +} + +object KMeansSuite extends FunSuite { + def createModel(dim: Int, k: Int, isSparse: Boolean): KMeansModel = { + val singlePoint = isSparse match { + case true => + Vectors.sparse(dim, Array.empty[Int], Array.empty[Double]) + case _ => + Vectors.dense(Array.fill[Double](dim)(0.0)) + } + new KMeansModel(Array.fill[Vector](k)(singlePoint)) + } + + def checkEqual(a: KMeansModel, b: KMeansModel): Unit = { + assert(a.k === b.k) + a.clusterCenters.zip(b.clusterCenters).foreach { + case (ca: SparseVector, cb: SparseVector) => + assert(ca === cb) + case (ca: DenseVector, cb: DenseVector) => + assert(ca === cb) + case _ => + throw new AssertionError("checkEqual failed since the two clusters were not identical.\n") + } + } } class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext { From 517975d89d40a77c7186f488547eed11f79c1e97 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 11 Mar 2015 01:03:01 -0700 Subject: [PATCH 380/817] [SPARK-4924] Add a library for launching Spark jobs programmatically. This change encapsulates all the logic involved in launching a Spark job into a small Java library that can be easily embedded into other applications. The overall goal of this change is twofold, as described in the bug: - Provide a public API for launching Spark processes. This is a common request from users and currently there's no good answer for it. - Remove a lot of the duplicated code and other coupling that exists in the different parts of Spark that deal with launching processes. A lot of the duplication was due to different code needed to build an application's classpath (and the bootstrapper needed to run the driver in certain situations), and also different code needed to parse spark-submit command line options in different contexts. The change centralizes those as much as possible so that all code paths can rely on the library for handling those appropriately. Author: Marcelo Vanzin Closes #3916 from vanzin/SPARK-4924 and squashes the following commits: 18c7e4d [Marcelo Vanzin] Fix make-distribution.sh. 2ce741f [Marcelo Vanzin] Add lots of quotes. 3b28a75 [Marcelo Vanzin] Update new pom. a1b8af1 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 897141f [Marcelo Vanzin] Review feedback. e2367d2 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 28cd35e [Marcelo Vanzin] Remove stale comment. b1d86b0 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 00505f9 [Marcelo Vanzin] Add blurb about new API in the programming guide. 5f4ddcc [Marcelo Vanzin] Better usage messages. 92a9cfb [Marcelo Vanzin] Fix Win32 launcher, usage. 6184c07 [Marcelo Vanzin] Rename field. 4c19196 [Marcelo Vanzin] Update comment. 7e66c18 [Marcelo Vanzin] Fix pyspark tests. 0031a8e [Marcelo Vanzin] Review feedback. c12d84b [Marcelo Vanzin] Review feedback. And fix spark-submit on Windows. e2d4d71 [Marcelo Vanzin] Simplify some code used to launch pyspark. 43008a7 [Marcelo Vanzin] Don't make builder extend SparkLauncher. b4d6912 [Marcelo Vanzin] Use spark-submit script in SparkLauncher. 28b1434 [Marcelo Vanzin] Add a comment. 304333a [Marcelo Vanzin] Fix propagation of properties file arg. bb67b93 [Marcelo Vanzin] Remove unrelated Yarn change (that is also wrong). 8ec0243 [Marcelo Vanzin] Add missing newline. 95ddfa8 [Marcelo Vanzin] Fix handling of --help for spark-class command builder. 72da7ec [Marcelo Vanzin] Rename SparkClassLauncher. 62978e4 [Marcelo Vanzin] Minor cleanup of Windows code path. 9cd5b44 [Marcelo Vanzin] Make all non-public APIs package-private. e4c80b6 [Marcelo Vanzin] Reorganize the code so that only SparkLauncher is public. e50dc5e [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 de81da2 [Marcelo Vanzin] Fix CommandUtils. 86a87bf [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 2061967 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 46d46da [Marcelo Vanzin] Clean up a test and make it more future-proof. b93692a [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 ad03c48 [Marcelo Vanzin] Revert "Fix a thread-safety issue in "local" mode." 0b509d0 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 23aa2a9 [Marcelo Vanzin] Read java-opts from conf dir, not spark home. 7cff919 [Marcelo Vanzin] Javadoc updates. eae4d8e [Marcelo Vanzin] Fix new unit tests on Windows. e570fb5 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 44cd5f7 [Marcelo Vanzin] Add package-info.java, clean up javadocs. f7cacff [Marcelo Vanzin] Remove "launch Spark in new thread" feature. 7ed8859 [Marcelo Vanzin] Some more feedback. 54cd4fd [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 61919df [Marcelo Vanzin] Clean leftover debug statement. aae5897 [Marcelo Vanzin] Use launcher classes instead of jars in non-release mode. e584fc3 [Marcelo Vanzin] Rework command building a little bit. 525ef5b [Marcelo Vanzin] Rework Unix spark-class to handle argument with newlines. 8ac4e92 [Marcelo Vanzin] Minor test cleanup. e946a99 [Marcelo Vanzin] Merge PySparkLauncher into SparkSubmitCliLauncher. c617539 [Marcelo Vanzin] Review feedback round 1. fc6a3e2 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 f26556b [Marcelo Vanzin] Fix a thread-safety issue in "local" mode. 2f4e8b4 [Marcelo Vanzin] Changes needed to make this work with SPARK-4048. 799fc20 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 bb5d324 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 53faef1 [Marcelo Vanzin] Merge branch 'master' into SPARK-4924 a7936ef [Marcelo Vanzin] Fix pyspark tests. 656374e [Marcelo Vanzin] Mima fixes. 4d511e7 [Marcelo Vanzin] Fix tools search code. 7a01e4a [Marcelo Vanzin] Fix pyspark on Yarn. 1b3f6e9 [Marcelo Vanzin] Call SparkSubmit from spark-class launcher for unknown classes. 25c5ae6 [Marcelo Vanzin] Centralize SparkSubmit command line parsing. 27be98a [Marcelo Vanzin] Modify Spark to use launcher lib. 6f70eea [Marcelo Vanzin] [SPARK-4924] Add a library for launching Spark jobs programatically. --- .gitignore | 1 + bin/compute-classpath.cmd | 124 ------ bin/compute-classpath.sh | 161 -------- bin/load-spark-env.sh | 8 +- bin/pyspark | 59 +-- bin/pyspark2.cmd | 57 +-- bin/run-example | 2 +- bin/spark-class | 180 ++------- bin/spark-class2.cmd | 141 ++----- bin/spark-shell | 23 +- bin/spark-shell2.cmd | 27 +- bin/spark-sql | 20 +- bin/spark-submit | 66 +--- bin/spark-submit2.cmd | 71 +--- bin/utils.sh | 60 --- bin/windows-utils.cmd | 60 --- core/pom.xml | 5 + .../spark/deploy/SparkSubmitArguments.scala | 157 ++++---- .../SparkSubmitDriverBootstrapper.scala | 170 -------- .../spark/deploy/worker/CommandUtils.scala | 46 +-- .../org/apache/spark/executor/Executor.scala | 1 - .../launcher/SparkSubmitArgumentsParser.scala | 25 ++ .../spark/launcher/WorkerCommandBuilder.scala | 50 +++ docs/programming-guide.md | 5 + launcher/pom.xml | 83 ++++ .../launcher/AbstractCommandBuilder.java | 362 ++++++++++++++++++ .../spark/launcher/CommandBuilderUtils.java | 296 ++++++++++++++ .../java/org/apache/spark/launcher/Main.java | 173 +++++++++ .../launcher/SparkClassCommandBuilder.java | 108 ++++++ .../apache/spark/launcher/SparkLauncher.java | 279 ++++++++++++++ .../launcher/SparkSubmitCommandBuilder.java | 327 ++++++++++++++++ .../launcher/SparkSubmitOptionParser.java | 224 +++++++++++ .../apache/spark/launcher/package-info.java | 45 +++ .../launcher/CommandBuilderUtilsSuite.java | 101 +++++ .../spark/launcher/SparkLauncherSuite.java | 94 +++++ .../SparkSubmitCommandBuilderSuite.java | 278 ++++++++++++++ .../SparkSubmitOptionParserSuite.java | 108 ++++++ launcher/src/test/resources/log4j.properties | 31 ++ make-distribution.sh | 2 + pom.xml | 3 +- project/SparkBuild.scala | 7 +- python/pyspark/java_gateway.py | 3 +- sbin/spark-daemon.sh | 84 ++-- sbin/start-thriftserver.sh | 2 +- 44 files changed, 2891 insertions(+), 1238 deletions(-) delete mode 100644 bin/compute-classpath.cmd delete mode 100755 bin/compute-classpath.sh delete mode 100755 bin/utils.sh delete mode 100644 bin/windows-utils.cmd delete mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala create mode 100644 core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala create mode 100644 core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala create mode 100644 launcher/pom.xml create mode 100644 launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/Main.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java create mode 100644 launcher/src/main/java/org/apache/spark/launcher/package-info.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java create mode 100644 launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java create mode 100644 launcher/src/test/resources/log4j.properties diff --git a/.gitignore b/.gitignore index 9757054a50f9e..d162fa9cca994 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ *.iml *.iws *.pyc +*.pyo .idea/ .idea_modules/ build/*.jar diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd deleted file mode 100644 index 088f993954d9e..0000000000000 --- a/bin/compute-classpath.cmd +++ /dev/null @@ -1,124 +0,0 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting -rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we -rem need to set it here because we use !datanucleus_jars! below. -if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion -setlocal enabledelayedexpansion -:skip_delayed_expansion - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% - -if not "x%SPARK_CONF_DIR%"=="x" ( - set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% -) else ( - set CLASSPATH=%CLASSPATH%;%FWDIR%conf -) - -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) - -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -rem When Hive support is needed, Datanucleus jars must be included on the classpath. -rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -rem built with Hive, so look for them there. -if exist "%FWDIR%RELEASE" ( - set datanucleus_dir=%FWDIR%lib -) else ( - set datanucleus_dir=%FWDIR%lib_managed\jars -) -set "datanucleus_jars=" -for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( - set datanucleus_jars=!datanucleus_jars!;%%d -) -set CLASSPATH=%CLASSPATH%;%datanucleus_jars% - -set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes - -set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH - rem so that local compilation takes precedence over assembled jar - set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem To allow for distributions to append needed libraries to the classpath (e.g. when -rem using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and -rem append it to tbe final classpath. -if not "x%$SPARK_DIST_CLASSPATH%"=="x" ( - set CLASSPATH=%CLASSPATH%;%SPARK_DIST_CLASSPATH% -) - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh deleted file mode 100755 index f4f6b7b909490..0000000000000 --- a/bin/compute-classpath.sh +++ /dev/null @@ -1,161 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -# script and the ExecutorRunner in standalone cluster mode. - -# Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" - -. "$FWDIR"/bin/load-spark-env.sh - -if [ -n "$SPARK_CLASSPATH" ]; then - CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" -else - CLASSPATH="$SPARK_SUBMIT_CLASSPATH" -fi - -# Build up classpath -if [ -n "$SPARK_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$SPARK_CONF_DIR" -else - CLASSPATH="$CLASSPATH:$FWDIR/conf" -fi - -ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SPARK_SCALA_VERSION" - -if [ -n "$JAVA_HOME" ]; then - JAR_CMD="$JAVA_HOME/bin/jar" -else - JAR_CMD="jar" -fi - -# A developer option to prepend more recently compiled Spark classes -if [ -n "$SPARK_PREPEND_CLASSES" ]; then - echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ - "classes ahead of assembly." >&2 - # Spark classes - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SPARK_SCALA_VERSION/classes" - # Jars for shaded deps in their original form (copied here during build) - CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" -fi - -# Use spark-assembly jar from either RELEASE or assembly directory -if [ -f "$FWDIR/RELEASE" ]; then - assembly_folder="$FWDIR"/lib -else - assembly_folder="$ASSEMBLY_DIR" -fi - -num_jars=0 - -for f in "${assembly_folder}"/spark-assembly*hadoop*.jar; do - if [[ ! -e "$f" ]]; then - echo "Failed to find Spark assembly in $assembly_folder" 1>&2 - echo "You need to build Spark before running this program." 1>&2 - exit 1 - fi - ASSEMBLY_JAR="$f" - num_jars=$((num_jars+1)) -done - -if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $assembly_folder:" 1>&2 - ls "${assembly_folder}"/spark-assembly*hadoop*.jar 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 -fi - -# Verify that versions of java used to build the jars and run Spark are compatible -jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) -if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then - echo "Loading Spark jar with '$JAR_CMD' failed. " 1>&2 - echo "This is likely because Spark was compiled with Java 7 and run " 1>&2 - echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " 1>&2 - echo "or build Spark with Java 6." 1>&2 - exit 1 -fi - -CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" - -# When Hive support is needed, Datanucleus jars must be included on the classpath. -# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -# Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -# built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark -# assembly is built for Hive, before actually populating the CLASSPATH with the jars. -# Note that this check order is faster (by up to half a second) in the case where Hive is not used. -if [ -f "$FWDIR/RELEASE" ]; then - datanucleus_dir="$FWDIR"/lib -else - datanucleus_dir="$FWDIR"/lib_managed/jars -fi - -datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar$")" -datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" - -if [ -n "$datanucleus_jars" ]; then - hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) - if [ -n "$hive_files" ]; then - echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 - CLASSPATH="$CLASSPATH:$datanucleus_jars" - fi -fi - -# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 -if [[ $SPARK_TESTING == 1 ]]; then - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/test-classes" -fi - -# Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! -# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -# the configurtion files. -if [ -n "$HADOOP_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" -fi -if [ -n "$YARN_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" -fi - -# To allow for distributions to append needed libraries to the classpath (e.g. when -# using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and -# append it to tbe final classpath. -if [ -n "$SPARK_DIST_CLASSPATH" ]; then - CLASSPATH="$CLASSPATH:$SPARK_DIST_CLASSPATH" -fi - -echo "$CLASSPATH" diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 356b3d49b2ffe..2d7070c25d328 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -41,9 +41,9 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" - ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" - + ASSEMBLY_DIR2="$SPARK_HOME/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$SPARK_HOME/assembly/target/scala-2.10" + if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 @@ -54,5 +54,5 @@ if [ -z "$SPARK_SCALA_VERSION" ]; then export SPARK_SCALA_VERSION="2.11" else export SPARK_SCALA_VERSION="2.10" - fi + fi fi diff --git a/bin/pyspark b/bin/pyspark index 0b4f695dd06dd..e7f6a1a072c2a 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,36 +18,24 @@ # # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -# Export this as SPARK_HOME -export SPARK_HOME="$FWDIR" - -source "$FWDIR/bin/utils.sh" - -source "$FWDIR"/bin/load-spark-env.sh +source "$SPARK_HOME"/bin/load-spark-env.sh function usage() { + if [ -n "$1" ]; then + echo $1 + fi echo "Usage: ./bin/pyspark [options]" 1>&2 - "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit $2 } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi -# Exit if the user hasn't compiled Spark -if [ ! -f "$FWDIR/RELEASE" ]; then - # Exit if the user hasn't compiled Spark - ls "$FWDIR"/assembly/target/scala-$SPARK_SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null - if [[ $? != 0 ]]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target" 1>&2 - echo "You need to build Spark before running this program" 1>&2 - exit 1 - fi -fi - # In Spark <= 1.1, setting IPYTHON=1 would cause the driver to be launched using the `ipython` # executable, while the worker would still be launched using PYSPARK_PYTHON. # @@ -95,26 +83,13 @@ export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" -export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" - -# Build up arguments list manually to preserve quotes and backslashes. -# We export Spark submit arguments as an environment variable because shell.py must run as a -# PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" -PYSPARK_SUBMIT_ARGS="" -whitespace="[[:space:]]" -for i in "${SUBMISSION_OPTS[@]}"; do - if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi - if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi - PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" -done -export PYSPARK_SUBMIT_ARGS +export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" # For pyspark tests if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYSPARK_SUBMIT_ARGS=pyspark-shell if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else @@ -123,14 +98,6 @@ if [[ -n "$SPARK_TESTING" ]]; then exit fi -# If a python file is provided, directly run spark-submit. -if [[ "$1" =~ \.py$ ]]; then - echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 - echo -e "Use ./bin/spark-submit \n" 1>&2 - primary="$1" - shift - gatherSparkSubmitOpts "$@" - exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}" -else - exec "$PYSPARK_DRIVER_PYTHON" $PYSPARK_DRIVER_PYTHON_OPTS -fi +export PYSPARK_DRIVER_PYTHON +export PYSPARK_DRIVER_PYTHON_OPTS +exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main "$@" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a542ec80b49d6..4f5eb5e20614d 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -17,59 +17,22 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -set SCALA_VERSION=2.10 - rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Export this as SPARK_HOME -set SPARK_HOME=%FWDIR% - -rem Test whether the user has built Spark -if exist "%FWDIR%RELEASE" goto skip_build_test -set FOUND_JAR=0 -for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set FOUND_JAR=1 -) -if [%FOUND_JAR%] == [0] ( - echo Failed to find Spark assembly JAR. - echo You need to build Spark before running this program. - goto exit -) -:skip_build_test +set SPARK_HOME=%~dp0.. rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" rem Figure out which Python to use. -if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python +if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( + set PYSPARK_DRIVER_PYTHON=python + if not [%PYSPARK_PYTHON%] == [] set PYSPARK_DRIVER_PYTHON=%PYSPARK_PYTHON% +) -set PYTHONPATH=%FWDIR%python;%PYTHONPATH% -set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% -set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py -set PYSPARK_SUBMIT_ARGS=%* - -echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% - -rem Check whether the argument is a file -for /f %%i in ('echo %1^| findstr /R "\.py"') do ( - set PYTHON_FILE=%%i -) - -if [%PYTHON_FILE%] == [] ( - if [%IPYTHON%] == [1] ( - ipython %IPYTHON_OPTS% - ) else ( - %PYSPARK_PYTHON% - ) -) else ( - echo. - echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. - echo Use ./bin/spark-submit ^ - echo. - "%FWDIR%\bin\spark-submit.cmd" %PYSPARK_SUBMIT_ARGS% -) +set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py -:exit +call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main %* diff --git a/bin/run-example b/bin/run-example index a106411392e06..798e2caeb88ce 100755 --- a/bin/run-example +++ b/bin/run-example @@ -67,7 +67,7 @@ if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" fi -"$FWDIR"/bin/spark-submit \ +exec "$FWDIR"/bin/spark-submit \ --master $EXAMPLE_MASTER \ --class $EXAMPLE_CLASS \ "$SPARK_EXAMPLES_JAR" \ diff --git a/bin/spark-class b/bin/spark-class index 2f0441bb3c1c2..e29b234afaf96 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -16,89 +16,18 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -# NOTE: Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -cygwin=false -case "`uname`" in - CYGWIN*) cygwin=true;; -esac +set -e # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -# Export this as SPARK_HOME -export SPARK_HOME="$FWDIR" -export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"$SPARK_HOME/conf"}" - -. "$FWDIR"/bin/load-spark-env.sh +. "$SPARK_HOME"/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" 1>&2 exit 1 fi -if [ -n "$SPARK_MEM" ]; then - echo -e "Warning: SPARK_MEM is deprecated, please use a more specific config option" 1>&2 - echo -e "(e.g., spark.executor.memory or spark.driver.memory)." 1>&2 -fi - -# Use SPARK_MEM or 512m as the default memory, to be overridden by specific options -DEFAULT_MEM=${SPARK_MEM:-512m} - -SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - -# Add java opts and memory settings for master, worker, history server, executors, and repl. -case "$1" in - # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. - 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.deploy.history.HistoryServer') - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" - OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} - ;; - - # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. - 'org.apache.spark.executor.CoarseGrainedExecutorBackend') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} - ;; - 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" - OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} - export PYTHONPATH="$FWDIR/python:$PYTHONPATH" - export PYTHONPATH="$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" - ;; - - # Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + - # SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. - 'org.apache.spark.deploy.SparkSubmit') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" - OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} - if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then - if [[ $OSTYPE == darwin* ]]; then - export DYLD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$DYLD_LIBRARY_PATH" - else - export LD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$LD_LIBRARY_PATH" - fi - fi - if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then - OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" - fi - ;; - - *) - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" - OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} - ;; -esac - # Find the java binary if [ -n "${JAVA_HOME}" ]; then RUNNER="${JAVA_HOME}/bin/java" @@ -110,83 +39,48 @@ else exit 1 fi fi -JAVA_VERSION=$("$RUNNER" -version 2>&1 | grep 'version' | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') - -# Set JAVA_OPTS to be able to load native libraries and to set heap size -if [ "$JAVA_VERSION" -ge 18 ]; then - JAVA_OPTS="$OUR_JAVA_OPTS" -else - JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" -fi -JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" - -# Load extra JAVA_OPTS from conf/java-opts, if it exists -if [ -e "$SPARK_CONF_DIR/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat "$SPARK_CONF_DIR"/java-opts`" -fi - -# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! - -TOOLS_DIR="$FWDIR"/tools -SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the SBT build - export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar`" -fi -if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then - # Use the JAR from the Maven build - # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`" -fi -# Compute classpath using external script -classpath_output=$("$FWDIR"/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" - exit 1 -else - CLASSPATH="$classpath_output" -fi +# Look for the launcher. In non-release mode, add the compiled classes directly to the classpath +# instead of looking for a jar file. +SPARK_LAUNCHER_CP= +if [ -f $SPARK_HOME/RELEASE ]; then + LAUNCHER_DIR="$SPARK_HOME/lib" + num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" + if [ "$num_jars" -eq "0" -a -z "$SPARK_LAUNCHER_CP" ]; then + echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 + fi -if [[ "$1" =~ org.apache.spark.tools.* ]]; then - if test -z "$SPARK_TOOLS_JAR"; then - echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/" 1>&2 - echo "You need to run \"build/sbt tools/package\" before running $1." 1>&2 + LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" || true)" + if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 + echo "$LAUNCHER_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 exit 1 fi - CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" -fi -if $cygwin; then - CLASSPATH="`cygpath -wp "$CLASSPATH"`" - if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR="`cygpath -w "$SPARK_TOOLS_JAR"`" + SPARK_LAUNCHER_CP="${LAUNCHER_DIR}/${LAUNCHER_JARS}" +else + LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" + if [ ! -d "$LAUNCHER_DIR/classes" ]; then + echo "Failed to find Spark launcher classes in $LAUNCHER_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 fi + SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes" fi -export CLASSPATH -# In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. -# Here we must parse the properties file for relevant "spark.driver.*" configs before launching -# the driver JVM itself. Instead of handling this complexity in Bash, we launch a separate JVM -# to prepare the launch environment of this driver JVM. +# The launcher library will print arguments separated by a NULL character, to allow arguments with +# characters that would be otherwise interpreted by the shell. Read that in a while loop, populating +# an array that will be used to exec the final command. +CMD=() +while IFS= read -d '' -r ARG; do + CMD+=("$ARG") +done < <("$RUNNER" -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") -if [ -n "$SPARK_SUBMIT_BOOTSTRAP_DRIVER" ]; then - # This is used only if the properties file actually contains these special configs - # Export the environment variables needed by SparkSubmitDriverBootstrapper - export RUNNER - export CLASSPATH - export JAVA_OPTS - export OUR_JAVA_MEM - export SPARK_CLASS=1 - shift # Ignore main class (org.apache.spark.deploy.SparkSubmit) and use our own - exec "$RUNNER" org.apache.spark.deploy.SparkSubmitDriverBootstrapper "$@" +if [ "${CMD[0]}" = "usage" ]; then + "${CMD[@]}" else - # Note: The format of this command is closely echoed in SparkSubmitDriverBootstrapper.scala - if [ -n "$SPARK_PRINT_LAUNCH_COMMAND" ]; then - echo -n "Spark Command: " 1>&2 - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 - echo -e "========================================\n" 1>&2 - fi - exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + exec "${CMD[@]}" fi - diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index da46543647efd..37d22215a0e7e 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,135 +17,54 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -setlocal enabledelayedexpansion - -set SCALA_VERSION=2.10 - rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Export this as SPARK_HOME -set SPARK_HOME=%FWDIR% +set SPARK_HOME=%~dp0.. rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" rem Test that an argument was given -if not "x%1"=="x" goto arg_given +if "x%1"=="x" ( echo Usage: spark-class ^ [^] - goto exit -:arg_given - -if not "x%SPARK_MEM%"=="x" ( - echo Warning: SPARK_MEM is deprecated, please use a more specific config option - echo e.g., spark.executor.memory or spark.driver.memory. + exit /b 1 ) -rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options -set OUR_JAVA_MEM=%SPARK_MEM% -if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m - -set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true - -rem Add java opts and memory settings for master, worker, history server, executors, and repl. -rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. -if "%1"=="org.apache.spark.deploy.master.Master" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% -) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% -) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( - set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% - if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% - -rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. -) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% - if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% -) else if "%1"=="org.apache.spark.executor.MesosExecutorBackend" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% - if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% +set LAUNCHER_CP=0 +if exist %SPARK_HOME%\RELEASE goto find_release_launcher -rem Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + -rem SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. -rem The repl also uses SPARK_REPL_OPTS. -) else if "%1"=="org.apache.spark.deploy.SparkSubmit" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_SUBMIT_OPTS% %SPARK_REPL_OPTS% - if not "x%SPARK_SUBMIT_LIBRARY_PATH%"=="x" ( - set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_SUBMIT_LIBRARY_PATH% - ) else if not "x%SPARK_LIBRARY_PATH%"=="x" ( - set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_LIBRARY_PATH% - ) - if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% - if not "x%SPARK_SUBMIT_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_SUBMIT_DRIVER_MEMORY% -) else ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% - if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% +rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use Scala so +rem it doesn't really matter which one is picked up. Add the compiled classes directly to the +rem classpath instead of looking for a jar file, since it's very common for people using sbt to use +rem the "assembly" target instead of "package". +set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.10\classes +if exist %LAUNCHER_CLASSES% ( + set LAUNCHER_CP=%LAUNCHER_CLASSES% ) - -rem Set JAVA_OPTS to be able to load native libraries and to set heap size -for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i -for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i -if "%jversion%" geq "1.8.0" ( - set JAVA_OPTS=%OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% -) else ( - set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.11\classes +if exist %LAUNCHER_CLASSES% ( + set LAUNCHER_CP=%LAUNCHER_CLASSES% ) -rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! +goto check_launcher -rem Test whether the user has built Spark -if exist "%FWDIR%RELEASE" goto skip_build_test -set FOUND_JAR=0 -for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set FOUND_JAR=1 -) -if "%FOUND_JAR%"=="0" ( - echo Failed to find Spark assembly JAR. - echo You need to build Spark before running this program. - goto exit +:find_release_launcher +for %%d in (%SPARK_HOME%\lib\spark-launcher*.jar) do ( + set LAUNCHER_CP=%%d ) -:skip_build_test -set TOOLS_DIR=%FWDIR%tools -set SPARK_TOOLS_JAR= -for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar") do ( - set SPARK_TOOLS_JAR=%%d +:check_launcher +if "%LAUNCHER_CP%"=="0" ( + echo Failed to find Spark launcher JAR. + echo You need to build Spark before running this program. + exit /b 1 ) -rem Compute classpath using external script -set DONT_PRINT_CLASSPATH=1 -call "%FWDIR%bin\compute-classpath.cmd" -set DONT_PRINT_CLASSPATH=0 -set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR% - rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -rem In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. -rem Here we must parse the properties file for relevant "spark.driver.*" configs before launching -rem the driver JVM itself. Instead of handling this complexity here, we launch a separate JVM -rem to prepare the launch environment of this driver JVM. - -rem In this case, leave out the main class (org.apache.spark.deploy.SparkSubmit) and use our own. -rem Leaving out the first argument is surprisingly difficult to do in Windows. Note that this must -rem be done here because the Windows "shift" command does not work in a conditional block. -set BOOTSTRAP_ARGS= -shift -:start_parse -if "%~1" == "" goto end_parse -set BOOTSTRAP_ARGS=%BOOTSTRAP_ARGS% %~1 -shift -goto start_parse -:end_parse - -if not [%SPARK_SUBMIT_BOOTSTRAP_DRIVER%] == [] ( - set SPARK_CLASS=1 - "%RUNNER%" org.apache.spark.deploy.SparkSubmitDriverBootstrapper %BOOTSTRAP_ARGS% -) else ( - "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +rem The launcher library prints the command to be executed in a single line suitable for being +rem executed by the batch interpreter. So read all the output of the launcher into a variable. +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_CP% org.apache.spark.launcher.Main %*"') do ( + set SPARK_CMD=%%i ) -:exit +%SPARK_CMD% diff --git a/bin/spark-shell b/bin/spark-shell index cca5aa0676123..b3761b5e1375b 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -28,25 +28,24 @@ esac # Enter posix mode for bash set -o posix -## Global script variables -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -function usage() { +usage() { + if [ -n "$1" ]; then + echo "$1" + fi echo "Usage: ./bin/spark-shell [options]" "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + exit "$2" } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage + usage "" 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" - # SPARK-4161: scala does not assume use of the java classpath, -# so we need to add the "-Dscala.usejavacp=true" flag mnually. We +# so we need to add the "-Dscala.usejavacp=true" flag manually. We # do this specifically for the Spark shell because the scala REPL # has its own class loader, and any additional classpath specified # through spark.driver.extraClassPath is not automatically propagated. @@ -61,11 +60,11 @@ function main() { # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@" fi } diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 1d1a40da315eb..02f51fe59a911 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -25,17 +25,28 @@ if %ERRORLEVEL% equ 0 ( exit /b 0 ) -call %SPARK_HOME%\bin\windows-utils.cmd %* -if %ERRORLEVEL% equ 1 ( +rem SPARK-4161: scala does not assume use of the java classpath, +rem so we need to add the "-Dscala.usejavacp=true" flag manually. We +rem do this specifically for the Spark shell because the scala REPL +rem has its own class loader, and any additional classpath specified +rem through spark.driver.extraClassPath is not automatically propagated. +if "x%SPARK_SUBMIT_OPTS%"=="x" ( + set SPARK_SUBMIT_OPTS=-Dscala.usejavacp=true + goto run_shell +) +set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true" + +:run_shell +call %SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main %* +set SPARK_ERROR_LEVEL=%ERRORLEVEL% +if not "x%SPARK_LAUNCHER_USAGE_ERROR%"=="x" ( call :usage exit /b 1 ) - -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %SUBMISSION_OPTS% spark-shell %APPLICATION_OPTS% - -exit /b 0 +exit /b %SPARK_ERROR_LEVEL% :usage +echo %SPARK_LAUNCHER_USAGE_ERROR% echo "Usage: .\bin\spark-shell.cmd [options]" >&2 -%SPARK_HOME%\bin\spark-submit --help 2>&1 | findstr /V "Usage" 1>&2 -exit /b 0 +call %SPARK_HOME%\bin\spark-submit2.cmd --help 2>&1 | findstr /V "Usage" 1>&2 +goto :eof diff --git a/bin/spark-sql b/bin/spark-sql index 3b6cc420fea81..ca1729f4cfcb4 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -25,12 +25,15 @@ set -o posix # NOTE: This exact class name is matched downstream by SparkSubmit. # Any changes need to be reflected there. -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +export CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" # Figure out where Spark is installed -FWDIR="$(cd "`dirname "$0"`"/..; pwd)" +export FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { + if [ -n "$1" ]; then + echo "$1" + fi echo "Usage: ./bin/spark-sql [options] [cli option]" pattern="usage" pattern+="\|Spark assembly has been built with Hive" @@ -42,16 +45,13 @@ function usage { "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "CLI options:" - "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class "$CLASS" --help 2>&1 | grep -v "$pattern" 1>&2 + exit "$2" } +export -f usage if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - usage - exit 0 + usage "" 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" - -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exec "$FWDIR"/bin/spark-submit --class "$CLASS" "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 3e5cbdbb24394..bcff78edd51ca 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,58 +17,18 @@ # limitations under the License. # -# NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" -ORIG_ARGS=("$@") - -# Set COLUMNS for progress bar -export COLUMNS=`tput cols` - -while (($#)); do - if [ "$1" = "--deploy-mode" ]; then - SPARK_SUBMIT_DEPLOY_MODE=$2 - elif [ "$1" = "--properties-file" ]; then - SPARK_SUBMIT_PROPERTIES_FILE=$2 - elif [ "$1" = "--driver-memory" ]; then - export SPARK_SUBMIT_DRIVER_MEMORY=$2 - elif [ "$1" = "--driver-library-path" ]; then - export SPARK_SUBMIT_LIBRARY_PATH=$2 - elif [ "$1" = "--driver-class-path" ]; then - export SPARK_SUBMIT_CLASSPATH=$2 - elif [ "$1" = "--driver-java-options" ]; then - export SPARK_SUBMIT_OPTS=$2 - elif [ "$1" = "--master" ]; then - export MASTER=$2 - fi - shift -done - -if [ -z "$SPARK_CONF_DIR" ]; then - export SPARK_CONF_DIR="$SPARK_HOME/conf" -fi -DEFAULT_PROPERTIES_FILE="$SPARK_CONF_DIR/spark-defaults.conf" -if [ "$MASTER" == "yarn-cluster" ]; then - SPARK_SUBMIT_DEPLOY_MODE=cluster +SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +# Only define a usage function if an upstream script hasn't done so. +if ! type -t usage >/dev/null 2>&1; then + usage() { + if [ -n "$1" ]; then + echo "$1" + fi + "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit --help + exit "$2" + } + export -f usage fi -export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"} -export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"} - -# For client mode, the driver will be launched in the same JVM that launches -# SparkSubmit, so we may need to read the properties file for any extra class -# paths, library paths, java options and memory early on. Otherwise, it will -# be too late by the time the driver JVM has started. - -if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FILE" ]]; then - # Parse the properties file only if the special configs exist - contains_special_configs=$( - grep -e "spark.driver.extra*\|spark.driver.memory" "$SPARK_SUBMIT_PROPERTIES_FILE" | \ - grep -v "^[[:space:]]*#" - ) - if [ -n "$contains_special_configs" ]; then - export SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 - fi -fi - -exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "$@" diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 446cbc74b74f9..08ddb185742d2 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -17,62 +17,19 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! - -set SPARK_HOME=%~dp0.. -set ORIG_ARGS=%* - -rem Reset the values of all variables used -set SPARK_SUBMIT_DEPLOY_MODE=client - -if [%SPARK_CONF_DIR%] == [] ( - set SPARK_CONF_DIR=%SPARK_HOME%\conf -) -set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_CONF_DIR%\spark-defaults.conf -set SPARK_SUBMIT_DRIVER_MEMORY= -set SPARK_SUBMIT_LIBRARY_PATH= -set SPARK_SUBMIT_CLASSPATH= -set SPARK_SUBMIT_OPTS= -set SPARK_SUBMIT_BOOTSTRAP_DRIVER= - -:loop -if [%1] == [] goto continue - if [%1] == [--deploy-mode] ( - set SPARK_SUBMIT_DEPLOY_MODE=%2 - ) else if [%1] == [--properties-file] ( - set SPARK_SUBMIT_PROPERTIES_FILE=%2 - ) else if [%1] == [--driver-memory] ( - set SPARK_SUBMIT_DRIVER_MEMORY=%2 - ) else if [%1] == [--driver-library-path] ( - set SPARK_SUBMIT_LIBRARY_PATH=%2 - ) else if [%1] == [--driver-class-path] ( - set SPARK_SUBMIT_CLASSPATH=%2 - ) else if [%1] == [--driver-java-options] ( - set SPARK_SUBMIT_OPTS=%2 - ) else if [%1] == [--master] ( - set MASTER=%2 - ) - shift -goto loop -:continue - -if [%MASTER%] == [yarn-cluster] ( - set SPARK_SUBMIT_DEPLOY_MODE=cluster -) - -rem For client mode, the driver will be launched in the same JVM that launches -rem SparkSubmit, so we may need to read the properties file for any extra class -rem paths, library paths, java options and memory early on. Otherwise, it will -rem be too late by the time the driver JVM has started. - -if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( - if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( - rem Parse the properties file only if the special configs exist - for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ - %SPARK_SUBMIT_PROPERTIES_FILE%') do ( - set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 - ) - ) +rem This is the entry point for running Spark submit. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +set CLASS=org.apache.spark.deploy.SparkSubmit +call %~dp0spark-class2.cmd %CLASS% %* +set SPARK_ERROR_LEVEL=%ERRORLEVEL% +if not "x%SPARK_LAUNCHER_USAGE_ERROR%"=="x" ( + call :usage + exit /b 1 ) +exit /b %SPARK_ERROR_LEVEL% -cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% +:usage +echo %SPARK_LAUNCHER_USAGE_ERROR% +call %SPARK_HOME%\bin\spark-class2.cmd %CLASS% --help +goto :eof diff --git a/bin/utils.sh b/bin/utils.sh deleted file mode 100755 index 748dbe345a74c..0000000000000 --- a/bin/utils.sh +++ /dev/null @@ -1,60 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Gather all spark-submit options into SUBMISSION_OPTS -function gatherSparkSubmitOpts() { - - if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then - echo "Function for printing usage of $0 is not set." 1>&2 - echo "Please set usage function to shell variable 'SUBMIT_USAGE_FUNCTION' in $0" 1>&2 - exit 1 - fi - - # NOTE: If you add or remove spark-submit options, - # modify NOT ONLY this script but also SparkSubmitArgument.scala - SUBMISSION_OPTS=() - APPLICATION_OPTS=() - while (($#)); do - case "$1" in - --master | --deploy-mode | --class | --name | --jars | --packages | --py-files | --files | \ - --conf | --repositories | --properties-file | --driver-memory | --driver-java-options | \ - --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ - --total-executor-cores | --executor-cores | --queue | --num-executors | --archives | \ - --proxy-user) - if [[ $# -lt 2 ]]; then - "$SUBMIT_USAGE_FUNCTION" - exit 1; - fi - SUBMISSION_OPTS+=("$1"); shift - SUBMISSION_OPTS+=("$1"); shift - ;; - - --verbose | -v | --supervise) - SUBMISSION_OPTS+=("$1"); shift - ;; - - *) - APPLICATION_OPTS+=("$1"); shift - ;; - esac - done - - export SUBMISSION_OPTS - export APPLICATION_OPTS -} diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd deleted file mode 100644 index 0cf9e87ca554b..0000000000000 --- a/bin/windows-utils.cmd +++ /dev/null @@ -1,60 +0,0 @@ -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem Gather all spark-submit options into SUBMISSION_OPTS - -set SUBMISSION_OPTS= -set APPLICATION_OPTS= - -rem NOTE: If you add or remove spark-sumbmit options, -rem modify NOT ONLY this script but also SparkSubmitArgument.scala - -:OptsLoop -if "x%1"=="x" ( - goto :OptsLoopEnd -) - -SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--py-files\> \<--files\>" -SET opts="%opts:~1,-1% \<--conf\> \<--properties-file\> \<--driver-memory\> \<--driver-java-options\>" -SET opts="%opts:~1,-1% \<--driver-library-path\> \<--driver-class-path\> \<--executor-memory\>" -SET opts="%opts:~1,-1% \<--driver-cores\> \<--total-executor-cores\> \<--executor-cores\> \<--queue\>" -SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\> \<--packages\> \<--repositories\>" -SET opts="%opts:~1,-1% \<--proxy-user\>" - -echo %1 | findstr %opts% >nul -if %ERRORLEVEL% equ 0 ( - if "x%2"=="x" ( - echo "%1" requires an argument. >&2 - exit /b 1 - ) - set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 %2 - shift - shift - goto :OptsLoop -) -echo %1 | findstr "\<--verbose\> \<-v\> \<--supervise\>" >nul -if %ERRORLEVEL% equ 0 ( - set SUBMISSION_OPTS=%SUBMISSION_OPTS% %1 - shift - goto :OptsLoop -) -set APPLICATION_OPTS=%APPLICATION_OPTS% %1 -shift -goto :OptsLoop - -:OptsLoopEnd -exit /b 0 diff --git a/core/pom.xml b/core/pom.xml index dc0d07d806635..4164a3a7208d4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -76,6 +76,11 @@ + + org.apache.spark + spark-launcher_${scala.binary.version} + ${project.version} + org.apache.spark spark-network-common_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 82e66a374249c..94e4bdbfb7d7b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -18,18 +18,22 @@ package org.apache.spark.deploy import java.net.URI +import java.util.{List => JList} import java.util.jar.JarFile +import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.deploy.SparkSubmitAction._ +import org.apache.spark.launcher.SparkSubmitArgumentsParser import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) { +private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) + extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -84,7 +88,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } // Set parameters from command line arguments - parseOpts(args.toList) + try { + parse(args.toList) + } catch { + case e: IllegalArgumentException => + SparkSubmit.printErrorAndExit(e.getMessage()) + } // Populate `sparkProperties` map from properties file mergeDefaultSparkProperties() // Use `sparkProperties` map along with env vars to fill in any missing parameters @@ -277,167 +286,139 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - /** - * Fill in values by parsing user options. - * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend. - */ - private def parseOpts(opts: Seq[String]): Unit = { - val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r - - // Delineates parsing of Spark options from parsing of user options. - parse(opts) - - /** - * NOTE: If you add or remove spark-submit options, - * modify NOT ONLY this file but also utils.sh - */ - def parse(opts: Seq[String]): Unit = opts match { - case ("--name") :: value :: tail => + /** Fill in values by parsing user options. */ + override protected def handle(opt: String, value: String): Boolean = { + opt match { + case NAME => name = value - parse(tail) - case ("--master") :: value :: tail => + case MASTER => master = value - parse(tail) - case ("--class") :: value :: tail => + case CLASS => mainClass = value - parse(tail) - case ("--deploy-mode") :: value :: tail => + case DEPLOY_MODE => if (value != "client" && value != "cluster") { SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") } deployMode = value - parse(tail) - case ("--num-executors") :: value :: tail => + case NUM_EXECUTORS => numExecutors = value - parse(tail) - case ("--total-executor-cores") :: value :: tail => + case TOTAL_EXECUTOR_CORES => totalExecutorCores = value - parse(tail) - case ("--executor-cores") :: value :: tail => + case EXECUTOR_CORES => executorCores = value - parse(tail) - case ("--executor-memory") :: value :: tail => + case EXECUTOR_MEMORY => executorMemory = value - parse(tail) - case ("--driver-memory") :: value :: tail => + case DRIVER_MEMORY => driverMemory = value - parse(tail) - case ("--driver-cores") :: value :: tail => + case DRIVER_CORES => driverCores = value - parse(tail) - case ("--driver-class-path") :: value :: tail => + case DRIVER_CLASS_PATH => driverExtraClassPath = value - parse(tail) - case ("--driver-java-options") :: value :: tail => + case DRIVER_JAVA_OPTIONS => driverExtraJavaOptions = value - parse(tail) - case ("--driver-library-path") :: value :: tail => + case DRIVER_LIBRARY_PATH => driverExtraLibraryPath = value - parse(tail) - case ("--properties-file") :: value :: tail => + case PROPERTIES_FILE => propertiesFile = value - parse(tail) - case ("--kill") :: value :: tail => + case KILL_SUBMISSION => submissionToKill = value if (action != null) { SparkSubmit.printErrorAndExit(s"Action cannot be both $action and $KILL.") } action = KILL - parse(tail) - case ("--status") :: value :: tail => + case STATUS => submissionToRequestStatusFor = value if (action != null) { SparkSubmit.printErrorAndExit(s"Action cannot be both $action and $REQUEST_STATUS.") } action = REQUEST_STATUS - parse(tail) - case ("--supervise") :: tail => + case SUPERVISE => supervise = true - parse(tail) - case ("--queue") :: value :: tail => + case QUEUE => queue = value - parse(tail) - case ("--files") :: value :: tail => + case FILES => files = Utils.resolveURIs(value) - parse(tail) - case ("--py-files") :: value :: tail => + case PY_FILES => pyFiles = Utils.resolveURIs(value) - parse(tail) - case ("--archives") :: value :: tail => + case ARCHIVES => archives = Utils.resolveURIs(value) - parse(tail) - case ("--jars") :: value :: tail => + case JARS => jars = Utils.resolveURIs(value) - parse(tail) - case ("--packages") :: value :: tail => + case PACKAGES => packages = value - parse(tail) - case ("--repositories") :: value :: tail => + case REPOSITORIES => repositories = value - parse(tail) - case ("--conf" | "-c") :: value :: tail => + case CONF => value.split("=", 2).toSeq match { case Seq(k, v) => sparkProperties(k) = v case _ => SparkSubmit.printErrorAndExit(s"Spark config without '=': $value") } - parse(tail) - case ("--proxy-user") :: value :: tail => + case PROXY_USER => proxyUser = value - parse(tail) - case ("--help" | "-h") :: tail => + case HELP => printUsageAndExit(0) - case ("--verbose" | "-v") :: tail => + case VERBOSE => verbose = true - parse(tail) - case ("--version") :: tail => + case VERSION => SparkSubmit.printVersionAndExit() - case EQ_SEPARATED_OPT(opt, value) :: tail => - parse(opt :: value :: tail) + case _ => + throw new IllegalArgumentException(s"Unexpected argument '$opt'.") + } + true + } - case value :: tail if value.startsWith("-") => - SparkSubmit.printErrorAndExit(s"Unrecognized option '$value'.") + /** + * Handle unrecognized command line options. + * + * The first unrecognized option is treated as the "primary resource". Everything else is + * treated as application arguments. + */ + override protected def handleUnknown(opt: String): Boolean = { + if (opt.startsWith("-")) { + SparkSubmit.printErrorAndExit(s"Unrecognized option '$opt'.") + } - case value :: tail => - primaryResource = - if (!SparkSubmit.isShell(value) && !SparkSubmit.isInternal(value)) { - Utils.resolveURI(value).toString - } else { - value - } - isPython = SparkSubmit.isPython(value) - childArgs ++= tail + primaryResource = + if (!SparkSubmit.isShell(opt) && !SparkSubmit.isInternal(opt)) { + Utils.resolveURI(opt).toString + } else { + opt + } + isPython = SparkSubmit.isPython(opt) + false + } - case Nil => - } + override protected def handleExtraArgs(extra: JList[String]): Unit = { + childArgs ++= extra } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala deleted file mode 100644 index 311048cdaa324..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy - -import scala.collection.JavaConversions._ - -import org.apache.spark.util.{RedirectThread, Utils} - -/** - * Launch an application through Spark submit in client mode with the appropriate classpath, - * library paths, java options and memory. These properties of the JVM must be set before the - * driver JVM is launched. The sole purpose of this class is to avoid handling the complexity - * of parsing the properties file for such relevant configs in Bash. - * - * Usage: org.apache.spark.deploy.SparkSubmitDriverBootstrapper - */ -private[spark] object SparkSubmitDriverBootstrapper { - - // Note: This class depends on the behavior of `bin/spark-class` and `bin/spark-submit`. - // Any changes made there must be reflected in this file. - - def main(args: Array[String]): Unit = { - - // This should be called only from `bin/spark-class` - if (!sys.env.contains("SPARK_CLASS")) { - System.err.println("SparkSubmitDriverBootstrapper must be called from `bin/spark-class`!") - System.exit(1) - } - - val submitArgs = args - val runner = sys.env("RUNNER") - val classpath = sys.env("CLASSPATH") - val javaOpts = sys.env("JAVA_OPTS") - val defaultDriverMemory = sys.env("OUR_JAVA_MEM") - - // Spark submit specific environment variables - val deployMode = sys.env("SPARK_SUBMIT_DEPLOY_MODE") - val propertiesFile = sys.env("SPARK_SUBMIT_PROPERTIES_FILE") - val bootstrapDriver = sys.env("SPARK_SUBMIT_BOOTSTRAP_DRIVER") - val submitDriverMemory = sys.env.get("SPARK_SUBMIT_DRIVER_MEMORY") - val submitLibraryPath = sys.env.get("SPARK_SUBMIT_LIBRARY_PATH") - val submitClasspath = sys.env.get("SPARK_SUBMIT_CLASSPATH") - val submitJavaOpts = sys.env.get("SPARK_SUBMIT_OPTS") - - assume(runner != null, "RUNNER must be set") - assume(classpath != null, "CLASSPATH must be set") - assume(javaOpts != null, "JAVA_OPTS must be set") - assume(defaultDriverMemory != null, "OUR_JAVA_MEM must be set") - assume(deployMode == "client", "SPARK_SUBMIT_DEPLOY_MODE must be \"client\"!") - assume(propertiesFile != null, "SPARK_SUBMIT_PROPERTIES_FILE must be set") - assume(bootstrapDriver != null, "SPARK_SUBMIT_BOOTSTRAP_DRIVER must be set") - - // Parse the properties file for the equivalent spark.driver.* configs - val properties = Utils.getPropertiesFromFile(propertiesFile) - val confDriverMemory = properties.get("spark.driver.memory") - val confLibraryPath = properties.get("spark.driver.extraLibraryPath") - val confClasspath = properties.get("spark.driver.extraClassPath") - val confJavaOpts = properties.get("spark.driver.extraJavaOptions") - - // Favor Spark submit arguments over the equivalent configs in the properties file. - // Note that we do not actually use the Spark submit values for library path, classpath, - // and Java opts here, because we have already captured them in Bash. - - val newDriverMemory = submitDriverMemory - .orElse(confDriverMemory) - .getOrElse(defaultDriverMemory) - - val newClasspath = - if (submitClasspath.isDefined) { - classpath - } else { - classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") - } - - val newJavaOpts = - if (submitJavaOpts.isDefined) { - // SPARK_SUBMIT_OPTS is already captured in JAVA_OPTS - javaOpts - } else { - javaOpts + confJavaOpts.map(" " + _).getOrElse("") - } - - val filteredJavaOpts = Utils.splitCommandString(newJavaOpts) - .filterNot(_.startsWith("-Xms")) - .filterNot(_.startsWith("-Xmx")) - - // Build up command - val command: Seq[String] = - Seq(runner) ++ - Seq("-cp", newClasspath) ++ - filteredJavaOpts ++ - Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ - Seq("org.apache.spark.deploy.SparkSubmit") ++ - submitArgs - - // Print the launch command. This follows closely the format used in `bin/spark-class`. - if (sys.env.contains("SPARK_PRINT_LAUNCH_COMMAND")) { - System.err.print("Spark Command: ") - System.err.println(command.mkString(" ")) - System.err.println("========================================\n") - } - - // Start the driver JVM - val filteredCommand = command.filter(_.nonEmpty) - val builder = new ProcessBuilder(filteredCommand) - val env = builder.environment() - - if (submitLibraryPath.isEmpty && confLibraryPath.nonEmpty) { - val libraryPaths = confLibraryPath ++ sys.env.get(Utils.libraryPathEnvName) - env.put(Utils.libraryPathEnvName, libraryPaths.mkString(sys.props("path.separator"))) - } - - val process = builder.start() - - // If we kill an app while it's running, its sub-process should be killed too. - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - if (process != null) { - process.destroy() - process.waitFor() - } - } - }) - - // Redirect stdout and stderr from the child JVM - val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") - val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") - stdoutThread.start() - stderrThread.start() - - // Redirect stdin to child JVM only if we're not running Windows. This is because the - // subprocess there already reads directly from our stdin, so we should avoid spawning a - // thread that contends with the subprocess in reading from System.in. - val isWindows = Utils.isWindows - val isSubprocess = sys.env.contains("IS_SUBPROCESS") - if (!isWindows) { - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin", - propagateEof = true) - stdinThread.start() - // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on - // broken pipe, signaling that the parent process has exited. This is the case if the - // application is launched directly from python, as in the PySpark shell. In Windows, - // the termination logic is handled in java_gateway.py - if (isSubprocess) { - stdinThread.join() - process.destroy() - } - } - val returnCode = process.waitFor() - stdoutThread.join() - stderrThread.join() - sys.exit(returnCode) - } - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 3e013c32096c5..83f78cf47306c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -20,10 +20,12 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ +import scala.collection.JavaConversions._ import scala.collection.Map import org.apache.spark.Logging import org.apache.spark.deploy.Command +import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils /** @@ -54,12 +56,10 @@ object CommandUtils extends Logging { } private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") - // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows - Seq(runner) ++ buildJavaOpts(command, memory, sparkHome) ++ Seq(command.mainClass) ++ - command.arguments + val cmd = new WorkerCommandBuilder(sparkHome, memory, command).buildCommand() + cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments } /** @@ -92,44 +92,6 @@ object CommandUtils extends Logging { command.javaOpts) } - /** - * Attention: this must always be aligned with the environment variables in the run scripts and - * the way the JAVA_OPTS are assembled there. - */ - private def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - - // Exists for backwards compatibility with older Spark versions - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) - .getOrElse(Nil) - if (workerLocalOpts.length > 0) { - logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.") - logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") - } - - // Figure out our classpath with the external compute-classpath script - val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" - val classPath = Utils.executeAndGetOutput( - Seq(sparkHome + "/bin/compute-classpath" + ext), - extraEnvironment = command.environment) - val userClassPath = command.classPathEntries ++ Seq(classPath) - - val javaVersion = System.getProperty("java.version") - - val javaOpts = workerLocalOpts ++ command.javaOpts - - val permGenOpt = - if (!javaVersion.startsWith("1.8") && !javaOpts.exists(_.startsWith("-XX:MaxPermSize="))) { - // do not specify -XX:MaxPermSize if it was already specified by user - Some("-XX:MaxPermSize=128m") - } else { - None - } - - Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ javaOpts ++ memoryOpts - } - /** Spawn a thread that will redirect a given stream to a file */ def redirectStream(in: InputStream, file: File) { val out = new FileOutputStream(file, true) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bed0a08d4d515..a897e532184ac 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -49,7 +49,6 @@ private[spark] class Executor( isLocal: Boolean = false) extends Logging { - logInfo(s"Starting executor ID $executorId on host $executorHostname") // Application dependencies (added through SparkContext) that we've fetched so far on this node. diff --git a/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala new file mode 100644 index 0000000000000..a835012531052 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/launcher/SparkSubmitArgumentsParser.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher + +/** + * This class makes SparkSubmitOptionParser visible for Spark code outside of the `launcher` + * package, since Java doesn't have a feature similar to `private[spark]`, and we don't want + * that class to be public. + */ +private[spark] abstract class SparkSubmitArgumentsParser extends SparkSubmitOptionParser diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala new file mode 100644 index 0000000000000..9be98723aed14 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher + +import java.io.File +import java.util.{HashMap => JHashMap, List => JList, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.spark.deploy.Command + +/** + * This class is used by CommandUtils. It uses some package-private APIs in SparkLauncher, and since + * Java doesn't have a feature similar to `private[spark]`, and we don't want that class to be + * public, needs to live in the same package as the rest of the library. + */ +private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command) + extends AbstractCommandBuilder { + + childEnv.putAll(command.environment) + childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sparkHome) + + override def buildCommand(env: JMap[String, String]): JList[String] = { + val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) + cmd.add(s"-Xms${memoryMb}M") + cmd.add(s"-Xmx${memoryMb}M") + command.javaOpts.foreach(cmd.add) + addPermGenSizeOpt(cmd) + addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) + cmd + } + + def buildCommand(): JList[String] = buildCommand(new JHashMap[String, String]()) + +} diff --git a/docs/programming-guide.md b/docs/programming-guide.md index b5e04bd0c610d..fa0b4e3705d6e 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1369,6 +1369,11 @@ The [application submission guide](submitting-applications.html) describes how t In short, once you package your application into a JAR (for Java/Scala) or a set of `.py` or `.zip` files (for Python), the `bin/spark-submit` script lets you submit it to any supported cluster manager. +# Launching Spark jobs from Java / Scala + +The [org.apache.spark.launcher](api/java/index.html?org/apache/spark/launcher/package-summary.html) +package provides classes for launching Spark jobs as child processes using a simple Java API. + # Unit Testing Spark is friendly to unit testing with any popular unit test framework. diff --git a/launcher/pom.xml b/launcher/pom.xml new file mode 100644 index 0000000000000..ccbd9d0419a98 --- /dev/null +++ b/launcher/pom.xml @@ -0,0 +1,83 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.10 + 1.3.0-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-launcher_2.10 + jar + Spark Launcher Project + http://spark.apache.org/ + + launcher + + + + + + log4j + log4j + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.slf4j + slf4j-api + test + + + org.slf4j + slf4j-log4j12 + test + + + + + org.apache.hadoop + hadoop-client + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java new file mode 100644 index 0000000000000..dc90e9e987234 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -0,0 +1,362 @@ +/* + * 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.launcher; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileFilter; +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.jar.JarFile; +import java.util.regex.Pattern; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Abstract Spark command builder that defines common functionality. + */ +abstract class AbstractCommandBuilder { + + boolean verbose; + String appName; + String appResource; + String deployMode; + String javaHome; + String mainClass; + String master; + String propertiesFile; + final List appArgs; + final List jars; + final List files; + final List pyFiles; + final Map childEnv; + final Map conf; + + public AbstractCommandBuilder() { + this.appArgs = new ArrayList(); + this.childEnv = new HashMap(); + this.conf = new HashMap(); + this.files = new ArrayList(); + this.jars = new ArrayList(); + this.pyFiles = new ArrayList(); + } + + /** + * Builds the command to execute. + * + * @param env A map containing environment variables for the child process. It may already contain + * entries defined by the user (such as SPARK_HOME, or those defined by the + * SparkLauncher constructor that takes an environment), and may be modified to + * include other variables needed by the process to be executed. + */ + abstract List buildCommand(Map env) throws IOException; + + /** + * Builds a list of arguments to run java. + * + * This method finds the java executable to use and appends JVM-specific options for running a + * class with Spark in the classpath. It also loads options from the "java-opts" file in the + * configuration directory being used. + * + * Callers should still add at least the class to run, as well as any arguments to pass to the + * class. + */ + List buildJavaCommand(String extraClassPath) throws IOException { + List cmd = new ArrayList(); + if (javaHome == null) { + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); + } else { + cmd.add(join(File.separator, javaHome, "bin", "java")); + } + + // Load extra JAVA_OPTS from conf/java-opts, if it exists. + File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); + if (javaOpts.isFile()) { + BufferedReader br = new BufferedReader(new InputStreamReader( + new FileInputStream(javaOpts), "UTF-8")); + try { + String line; + while ((line = br.readLine()) != null) { + addOptionString(cmd, line); + } + } finally { + br.close(); + } + } + + cmd.add("-cp"); + cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath))); + return cmd; + } + + /** + * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't + * set it. + */ + void addPermGenSizeOpt(List cmd) { + // Don't set MaxPermSize for Java 8 and later. + String[] version = System.getProperty("java.version").split("\\."); + if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { + return; + } + + for (String arg : cmd) { + if (arg.startsWith("-XX:MaxPermSize=")) { + return; + } + } + + cmd.add("-XX:MaxPermSize=128m"); + } + + void addOptionString(List cmd, String options) { + if (!isEmpty(options)) { + for (String opt : parseOptionString(options)) { + cmd.add(opt); + } + } + } + + /** + * Builds the classpath for the application. Returns a list with one classpath entry per element; + * each entry is formatted in the way expected by java.net.URLClassLoader (more + * specifically, with trailing slashes for directories). + */ + List buildClassPath(String appClassPath) throws IOException { + String sparkHome = getSparkHome(); + String scala = getScalaVersion(); + + List cp = new ArrayList(); + addToClassPath(cp, getenv("SPARK_CLASSPATH")); + addToClassPath(cp, appClassPath); + + addToClassPath(cp, getConfDir()); + + boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); + boolean isTesting = "1".equals(getenv("SPARK_TESTING")); + if (prependClasses || isTesting) { + List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", + "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", + "yarn", "launcher"); + if (prependClasses) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, + scala)); + } + } + if (isTesting) { + for (String project : projects) { + addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome, + project, scala)); + } + } + + // Add this path to include jars that are shaded in the final deliverable created during + // the maven build. These jars are copied to this directory during the build. + addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); + } + + String assembly = findAssembly(scala); + addToClassPath(cp, assembly); + + // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus + // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt + // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built + // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark + // assembly is built for Hive, before actually populating the CLASSPATH with the jars. + // + // This block also serves as a check for SPARK-1703, when the assembly jar is built with + // Java 7 and ends up with too many files, causing issues with other JDK versions. + boolean needsDataNucleus = false; + JarFile assemblyJar = null; + try { + assemblyJar = new JarFile(assembly); + needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; + } catch (IOException ioe) { + if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { + System.err.println( + "Loading Spark jar failed.\n" + + "This is likely because Spark was compiled with Java 7 and run\n" + + "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + + "or build Spark with Java 6."); + System.exit(1); + } else { + throw ioe; + } + } finally { + if (assemblyJar != null) { + try { + assemblyJar.close(); + } catch (IOException e) { + // Ignore. + } + } + } + + if (needsDataNucleus) { + System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + + "in classpath."); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); + } + + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); + } + } + } + + addToClassPath(cp, getenv("HADOOP_CONF_DIR")); + addToClassPath(cp, getenv("YARN_CONF_DIR")); + addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); + return cp; + } + + /** + * Adds entries to the classpath. + * + * @param cp List to which the new entries are appended. + * @param entries New classpath entries (separated by File.pathSeparator). + */ + private void addToClassPath(List cp, String entries) { + if (isEmpty(entries)) { + return; + } + String[] split = entries.split(Pattern.quote(File.pathSeparator)); + for (String entry : split) { + if (!isEmpty(entry)) { + if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) { + entry += File.separator; + } + cp.add(entry); + } + } + } + + String getScalaVersion() { + String scala = getenv("SPARK_SCALA_VERSION"); + if (scala != null) { + return scala; + } + + String sparkHome = getSparkHome(); + File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); + File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); + checkState(!scala210.isDirectory() || !scala211.isDirectory(), + "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + if (scala210.isDirectory()) { + return "2.10"; + } else { + checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); + return "2.11"; + } + } + + String getSparkHome() { + String path = getenv(ENV_SPARK_HOME); + checkState(path != null, + "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); + return path; + } + + /** + * Loads the configuration file for the application, if it exists. This is either the + * user-specified properties file, or the spark-defaults.conf file under the Spark configuration + * directory. + */ + Properties loadPropertiesFile() throws IOException { + Properties props = new Properties(); + File propsFile; + if (propertiesFile != null) { + propsFile = new File(propertiesFile); + checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile); + } else { + propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE); + } + + if (propsFile.isFile()) { + FileInputStream fd = null; + try { + fd = new FileInputStream(propsFile); + props.load(new InputStreamReader(fd, "UTF-8")); + } finally { + if (fd != null) { + try { + fd.close(); + } catch (IOException e) { + // Ignore. + } + } + } + } + + return props; + } + + String getenv(String key) { + return firstNonEmpty(childEnv.get(key), System.getenv(key)); + } + + private String findAssembly(String scalaVersion) { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + + private String getConfDir() { + String confDir = getenv("SPARK_CONF_DIR"); + return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java new file mode 100644 index 0000000000000..9b04732afee14 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -0,0 +1,296 @@ +/* + * 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.launcher; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Helper methods for command builders. + */ +class CommandBuilderUtils { + + static final String DEFAULT_MEM = "512m"; + static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; + static final String ENV_SPARK_HOME = "SPARK_HOME"; + + /** Returns whether the given string is null or empty. */ + static boolean isEmpty(String s) { + return s == null || s.isEmpty(); + } + + /** Joins a list of strings using the given separator. */ + static String join(String sep, String... elements) { + StringBuilder sb = new StringBuilder(); + for (String e : elements) { + if (e != null) { + if (sb.length() > 0) { + sb.append(sep); + } + sb.append(e); + } + } + return sb.toString(); + } + + /** Joins a list of strings using the given separator. */ + static String join(String sep, Iterable elements) { + StringBuilder sb = new StringBuilder(); + for (String e : elements) { + if (e != null) { + if (sb.length() > 0) { + sb.append(sep); + } + sb.append(e); + } + } + return sb.toString(); + } + + /** + * Returns the first non-empty value mapped to the given key in the given maps, or null otherwise. + */ + static String firstNonEmptyValue(String key, Map... maps) { + for (Map map : maps) { + String value = (String) map.get(key); + if (!isEmpty(value)) { + return value; + } + } + return null; + } + + /** Returns the first non-empty, non-null string in the given list, or null otherwise. */ + static String firstNonEmpty(String... candidates) { + for (String s : candidates) { + if (!isEmpty(s)) { + return s; + } + } + return null; + } + + /** Returns the name of the env variable that holds the native library path. */ + static String getLibPathEnvName() { + if (isWindows()) { + return "PATH"; + } + + String os = System.getProperty("os.name"); + if (os.startsWith("Mac OS X")) { + return "DYLD_LIBRARY_PATH"; + } else { + return "LD_LIBRARY_PATH"; + } + } + + /** Returns whether the OS is Windows. */ + static boolean isWindows() { + String os = System.getProperty("os.name"); + return os.startsWith("Windows"); + } + + /** + * Updates the user environment, appending the given pathList to the existing value of the given + * environment variable (or setting it if it hasn't yet been set). + */ + static void mergeEnvPathList(Map userEnv, String envKey, String pathList) { + if (!isEmpty(pathList)) { + String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey)); + userEnv.put(envKey, join(File.pathSeparator, current, pathList)); + } + } + + /** + * Parse a string as if it were a list of arguments, following bash semantics. + * For example: + * + * Input: "\"ab cd\" efgh 'i \" j'" + * Output: [ "ab cd", "efgh", "i \" j" ] + */ + static List parseOptionString(String s) { + List opts = new ArrayList(); + StringBuilder opt = new StringBuilder(); + boolean inOpt = false; + boolean inSingleQuote = false; + boolean inDoubleQuote = false; + boolean escapeNext = false; + + // This is needed to detect when a quoted empty string is used as an argument ("" or ''). + boolean hasData = false; + + for (int i = 0; i < s.length(); i++) { + int c = s.codePointAt(i); + if (escapeNext) { + opt.appendCodePoint(c); + escapeNext = false; + } else if (inOpt) { + switch (c) { + case '\\': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + escapeNext = true; + } + break; + case '\'': + if (inDoubleQuote) { + opt.appendCodePoint(c); + } else { + inSingleQuote = !inSingleQuote; + } + break; + case '"': + if (inSingleQuote) { + opt.appendCodePoint(c); + } else { + inDoubleQuote = !inDoubleQuote; + } + break; + default: + if (!Character.isWhitespace(c) || inSingleQuote || inDoubleQuote) { + opt.appendCodePoint(c); + } else { + opts.add(opt.toString()); + opt.setLength(0); + inOpt = false; + hasData = false; + } + } + } else { + switch (c) { + case '\'': + inSingleQuote = true; + inOpt = true; + hasData = true; + break; + case '"': + inDoubleQuote = true; + inOpt = true; + hasData = true; + break; + case '\\': + escapeNext = true; + inOpt = true; + hasData = true; + break; + default: + if (!Character.isWhitespace(c)) { + inOpt = true; + hasData = true; + opt.appendCodePoint(c); + } + } + } + } + + checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s); + if (hasData) { + opts.add(opt.toString()); + } + return opts; + } + + /** Throws IllegalArgumentException if the given object is null. */ + static void checkNotNull(Object o, String arg) { + if (o == null) { + throw new IllegalArgumentException(String.format("'%s' must not be null.", arg)); + } + } + + /** Throws IllegalArgumentException with the given message if the check is false. */ + static void checkArgument(boolean check, String msg, Object... args) { + if (!check) { + throw new IllegalArgumentException(String.format(msg, args)); + } + } + + /** Throws IllegalStateException with the given message if the check is false. */ + static void checkState(boolean check, String msg, Object... args) { + if (!check) { + throw new IllegalStateException(String.format(msg, args)); + } + } + + /** + * Quote a command argument for a command to be run by a Windows batch script, if the argument + * needs quoting. Arguments only seem to need quotes in batch scripts if they have certain + * special characters, some of which need extra (and different) escaping. + * + * For example: + * original single argument: ab="cde fgh" + * quoted: "ab^=""cde fgh""" + */ + static String quoteForBatchScript(String arg) { + + boolean needsQuotes = false; + for (int i = 0; i < arg.length(); i++) { + int c = arg.codePointAt(i); + if (Character.isWhitespace(c) || c == '"' || c == '=') { + needsQuotes = true; + break; + } + } + if (!needsQuotes) { + return arg; + } + StringBuilder quoted = new StringBuilder(); + quoted.append("\""); + for (int i = 0; i < arg.length(); i++) { + int cp = arg.codePointAt(i); + switch (cp) { + case '"': + quoted.append('"'); + break; + + case '=': + quoted.append('^'); + break; + + default: + break; + } + quoted.appendCodePoint(cp); + } + quoted.append("\""); + return quoted.toString(); + } + + /** + * Quotes a string so that it can be used in a command string and be parsed back into a single + * argument by python's "shlex.split()" function. + * + * Basically, just add simple escapes. E.g.: + * original single argument : ab "cd" ef + * after: "ab \"cd\" ef" + */ + static String quoteForPython(String s) { + StringBuilder quoted = new StringBuilder().append('"'); + for (int i = 0; i < s.length(); i++) { + int cp = s.codePointAt(i); + if (cp == '"' || cp == '\\') { + quoted.appendCodePoint('\\'); + } + quoted.appendCodePoint(cp); + } + return quoted.append('"').toString(); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java new file mode 100644 index 0000000000000..206acfb514d86 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Command line interface for the Spark launcher. Used internally by Spark scripts. + */ +class Main { + + /** + * Usage: Main [class] [class args] + *

    + * This CLI works in two different modes: + *

      + *
    • "spark-submit": if class is "org.apache.spark.deploy.SparkSubmit", the + * {@link SparkLauncher} class is used to launch a Spark application.
    • + *
    • "spark-class": if another class is provided, an internal Spark class is run.
    • + *
    + * + * This class works in tandem with the "bin/spark-class" script on Unix-like systems, and + * "bin/spark-class2.cmd" batch script on Windows to execute the final command. + *

    + * On Unix-like systems, the output is a list of command arguments, separated by the NULL + * character. On Windows, the output is a command line suitable for direct execution from the + * script. + */ + public static void main(String[] argsArray) throws Exception { + checkArgument(argsArray.length > 0, "Not enough arguments: missing class name."); + + List args = new ArrayList(Arrays.asList(argsArray)); + String className = args.remove(0); + + boolean printLaunchCommand; + boolean printUsage; + AbstractCommandBuilder builder; + try { + if (className.equals("org.apache.spark.deploy.SparkSubmit")) { + builder = new SparkSubmitCommandBuilder(args); + } else { + builder = new SparkClassCommandBuilder(className, args); + } + printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); + printUsage = false; + } catch (IllegalArgumentException e) { + builder = new UsageCommandBuilder(e.getMessage()); + printLaunchCommand = false; + printUsage = true; + } + + Map env = new HashMap(); + List cmd = builder.buildCommand(env); + if (printLaunchCommand) { + System.err.println("Spark Command: " + join(" ", cmd)); + System.err.println("========================================"); + } + + if (isWindows()) { + // When printing the usage message, we can't use "cmd /v" since that prevents the env + // variable from being seen in the caller script. So do not call prepareWindowsCommand(). + if (printUsage) { + System.out.println(join(" ", cmd)); + } else { + System.out.println(prepareWindowsCommand(cmd, env)); + } + } else { + // In bash, use NULL as the arg separator since it cannot be used in an argument. + List bashCmd = prepareBashCommand(cmd, env); + for (String c : bashCmd) { + System.out.print(c); + System.out.print('\0'); + } + } + } + + /** + * Prepare a command line for execution from a Windows batch script. + * + * The method quotes all arguments so that spaces are handled as expected. Quotes within arguments + * are "double quoted" (which is batch for escaping a quote). This page has more details about + * quoting and other batch script fun stuff: http://ss64.com/nt/syntax-esc.html + * + * The command is executed using "cmd /c" and formatted in single line, since that's the + * easiest way to consume this from a batch script (see spark-class2.cmd). + */ + private static String prepareWindowsCommand(List cmd, Map childEnv) { + StringBuilder cmdline = new StringBuilder("cmd /c \""); + for (Map.Entry e : childEnv.entrySet()) { + cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue())); + cmdline.append(" && "); + } + for (String arg : cmd) { + cmdline.append(quoteForBatchScript(arg)); + cmdline.append(" "); + } + cmdline.append("\""); + return cmdline.toString(); + } + + /** + * Prepare the command for execution from a bash script. The final command will have commands to + * set up any needed environment variables needed by the child process. + */ + private static List prepareBashCommand(List cmd, Map childEnv) { + if (childEnv.isEmpty()) { + return cmd; + } + + List newCmd = new ArrayList(); + newCmd.add("env"); + + for (Map.Entry e : childEnv.entrySet()) { + newCmd.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + newCmd.addAll(cmd); + return newCmd; + } + + /** + * Internal builder used when command line parsing fails. This will behave differently depending + * on the platform: + * + * - On Unix-like systems, it will print a call to the "usage" function with two arguments: the + * the error string, and the exit code to use. The function is expected to print the command's + * usage and exit with the provided exit code. The script should use "export -f usage" after + * declaring a function called "usage", so that the function is available to downstream scripts. + * + * - On Windows it will set the variable "SPARK_LAUNCHER_USAGE_ERROR" to the usage error message. + * The batch script should check for this variable and print its usage, since batch scripts + * don't really support the "export -f" functionality used in bash. + */ + private static class UsageCommandBuilder extends AbstractCommandBuilder { + + private final String message; + + UsageCommandBuilder(String message) { + this.message = message; + } + + @Override + public List buildCommand(Map env) { + if (isWindows()) { + return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=" + message); + } else { + return Arrays.asList("usage", message, "1"); + } + } + + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java new file mode 100644 index 0000000000000..e601a0a19f368 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Command builder for internal Spark classes. + *

    + * This class handles building the command to launch all internal Spark classes except for + * SparkSubmit (which is handled by {@link SparkSubmitCommandBuilder} class. + */ +class SparkClassCommandBuilder extends AbstractCommandBuilder { + + private final String className; + private final List classArgs; + + SparkClassCommandBuilder(String className, List classArgs) { + this.className = className; + this.classArgs = classArgs; + } + + @Override + public List buildCommand(Map env) throws IOException { + List javaOptsKeys = new ArrayList(); + String memKey = null; + String extraClassPath = null; + + // Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + + // SPARK_DAEMON_MEMORY. + if (className.equals("org.apache.spark.deploy.master.Master")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_MASTER_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.deploy.worker.Worker")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_WORKER_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_HISTORY_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; + } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) { + javaOptsKeys.add("SPARK_JAVA_OPTS"); + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) { + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + } else if (className.startsWith("org.apache.spark.tools.")) { + String sparkHome = getSparkHome(); + File toolsDir = new File(join(File.separator, sparkHome, "tools", "target", + "scala-" + getScalaVersion())); + checkState(toolsDir.isDirectory(), "Cannot find tools build directory."); + + Pattern re = Pattern.compile("spark-tools_.*\\.jar"); + for (File f : toolsDir.listFiles()) { + if (re.matcher(f.getName()).matches()) { + extraClassPath = f.getAbsolutePath(); + break; + } + } + + checkState(extraClassPath != null, + "Failed to find Spark Tools Jar in %s.\n" + + "You need to run \"build/sbt tools/package\" before running %s.", + toolsDir.getAbsolutePath(), className); + + javaOptsKeys.add("SPARK_JAVA_OPTS"); + } + + List cmd = buildJavaCommand(extraClassPath); + for (String key : javaOptsKeys) { + addOptionString(cmd, System.getenv(key)); + } + + String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); + cmd.add("-Xms" + mem); + cmd.add("-Xmx" + mem); + addPermGenSizeOpt(cmd); + cmd.add(className); + cmd.addAll(classArgs); + return cmd; + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java new file mode 100644 index 0000000000000..b566507ee6061 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -0,0 +1,279 @@ +/* + * 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.launcher; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Launcher for Spark applications. + *

    + * Use this class to start Spark applications programmatically. The class uses a builder pattern + * to allow clients to configure the Spark application and launch it as a child process. + */ +public class SparkLauncher { + + /** The Spark master. */ + public static final String SPARK_MASTER = "spark.master"; + + /** Configuration key for the driver memory. */ + public static final String DRIVER_MEMORY = "spark.driver.memory"; + /** Configuration key for the driver class path. */ + public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath"; + /** Configuration key for the driver VM options. */ + public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions"; + /** Configuration key for the driver native library path. */ + public static final String DRIVER_EXTRA_LIBRARY_PATH = "spark.driver.extraLibraryPath"; + + /** Configuration key for the executor memory. */ + public static final String EXECUTOR_MEMORY = "spark.executor.memory"; + /** Configuration key for the executor class path. */ + public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath"; + /** Configuration key for the executor VM options. */ + public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; + /** Configuration key for the executor native library path. */ + public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + /** Configuration key for the number of executor CPU cores. */ + public static final String EXECUTOR_CORES = "spark.executor.cores"; + + private final SparkSubmitCommandBuilder builder; + + public SparkLauncher() { + this(null); + } + + /** + * Creates a launcher that will set the given environment variables in the child. + * + * @param env Environment variables to set. + */ + public SparkLauncher(Map env) { + this.builder = new SparkSubmitCommandBuilder(); + if (env != null) { + this.builder.childEnv.putAll(env); + } + } + + /** + * Set a custom JAVA_HOME for launching the Spark application. + * + * @param javaHome Path to the JAVA_HOME to use. + * @return This launcher. + */ + public SparkLauncher setJavaHome(String javaHome) { + checkNotNull(javaHome, "javaHome"); + builder.javaHome = javaHome; + return this; + } + + /** + * Set a custom Spark installation location for the application. + * + * @param sparkHome Path to the Spark installation to use. + * @return This launcher. + */ + public SparkLauncher setSparkHome(String sparkHome) { + checkNotNull(sparkHome, "sparkHome"); + builder.childEnv.put(ENV_SPARK_HOME, sparkHome); + return this; + } + + /** + * Set a custom properties file with Spark configuration for the application. + * + * @param path Path to custom properties file to use. + * @return This launcher. + */ + public SparkLauncher setPropertiesFile(String path) { + checkNotNull(path, "path"); + builder.propertiesFile = path; + return this; + } + + /** + * Set a single configuration value for the application. + * + * @param key Configuration key. + * @param value The value to use. + * @return This launcher. + */ + public SparkLauncher setConf(String key, String value) { + checkNotNull(key, "key"); + checkNotNull(value, "value"); + checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'"); + builder.conf.put(key, value); + return this; + } + + /** + * Set the application name. + * + * @param appName Application name. + * @return This launcher. + */ + public SparkLauncher setAppName(String appName) { + checkNotNull(appName, "appName"); + builder.appName = appName; + return this; + } + + /** + * Set the Spark master for the application. + * + * @param master Spark master. + * @return This launcher. + */ + public SparkLauncher setMaster(String master) { + checkNotNull(master, "master"); + builder.master = master; + return this; + } + + /** + * Set the deploy mode for the application. + * + * @param mode Deploy mode. + * @return This launcher. + */ + public SparkLauncher setDeployMode(String mode) { + checkNotNull(mode, "mode"); + builder.deployMode = mode; + return this; + } + + /** + * Set the main application resource. This should be the location of a jar file for Scala/Java + * applications, or a python script for PySpark applications. + * + * @param resource Path to the main application resource. + * @return This launcher. + */ + public SparkLauncher setAppResource(String resource) { + checkNotNull(resource, "resource"); + builder.appResource = resource; + return this; + } + + /** + * Sets the application class name for Java/Scala applications. + * + * @param mainClass Application's main class. + * @return This launcher. + */ + public SparkLauncher setMainClass(String mainClass) { + checkNotNull(mainClass, "mainClass"); + builder.mainClass = mainClass; + return this; + } + + /** + * Adds command line arguments for the application. + * + * @param args Arguments to pass to the application's main class. + * @return This launcher. + */ + public SparkLauncher addAppArgs(String... args) { + for (String arg : args) { + checkNotNull(arg, "arg"); + builder.appArgs.add(arg); + } + return this; + } + + /** + * Adds a jar file to be submitted with the application. + * + * @param jar Path to the jar file. + * @return This launcher. + */ + public SparkLauncher addJar(String jar) { + checkNotNull(jar, "jar"); + builder.jars.add(jar); + return this; + } + + /** + * Adds a file to be submitted with the application. + * + * @param file Path to the file. + * @return This launcher. + */ + public SparkLauncher addFile(String file) { + checkNotNull(file, "file"); + builder.files.add(file); + return this; + } + + /** + * Adds a python file / zip / egg to be submitted with the application. + * + * @param file Path to the file. + * @return This launcher. + */ + public SparkLauncher addPyFile(String file) { + checkNotNull(file, "file"); + builder.pyFiles.add(file); + return this; + } + + /** + * Enables verbose reporting for SparkSubmit. + * + * @param verbose Whether to enable verbose output. + * @return This launcher. + */ + public SparkLauncher setVerbose(boolean verbose) { + builder.verbose = verbose; + return this; + } + + /** + * Launches a sub-process that will start the configured Spark application. + * + * @return A process handle for the Spark app. + */ + public Process launch() throws IOException { + List cmd = new ArrayList(); + String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; + cmd.add(join(File.separator, builder.getSparkHome(), "bin", script)); + cmd.addAll(builder.buildSparkSubmitArgs()); + + // Since the child process is a batch script, let's quote things so that special characters are + // preserved, otherwise the batch interpreter will mess up the arguments. Batch scripts are + // weird. + if (isWindows()) { + List winCmd = new ArrayList(); + for (String arg : cmd) { + winCmd.add(quoteForBatchScript(arg)); + } + cmd = winCmd; + } + + ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()])); + for (Map.Entry e : builder.childEnv.entrySet()) { + pb.environment().put(e.getKey(), e.getValue()); + } + return pb.start(); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java new file mode 100644 index 0000000000000..6ffdff63d3c78 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -0,0 +1,327 @@ +/* + * 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.launcher; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +/** + * Special command builder for handling a CLI invocation of SparkSubmit. + *

    + * This builder adds command line parsing compatible with SparkSubmit. It handles setting + * driver-side options and special parsing behavior needed for the special-casing certain internal + * Spark applications. + *

    + * This class has also some special features to aid launching pyspark. + */ +class SparkSubmitCommandBuilder extends AbstractCommandBuilder { + + /** + * Name of the app resource used to identify the PySpark shell. The command line parser expects + * the resource name to be the very first argument to spark-submit in this case. + * + * NOTE: this cannot be "pyspark-shell" since that identifies the PySpark shell to SparkSubmit + * (see java_gateway.py), and can cause this code to enter into an infinite loop. + */ + static final String PYSPARK_SHELL = "pyspark-shell-main"; + + /** + * This is the actual resource name that identifies the PySpark shell to SparkSubmit. + */ + static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell"; + + /** + * This map must match the class names for available special classes, since this modifies the way + * command line parsing works. This maps the class name to the resource to use when calling + * spark-submit. + */ + private static final Map specialClasses = new HashMap(); + static { + specialClasses.put("org.apache.spark.repl.Main", "spark-shell"); + specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", + "spark-internal"); + specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2", + "spark-internal"); + } + + private final List sparkArgs; + + /** + * Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed + * to parse the command lines for things like bin/spark-shell, which allows users to mix and + * match arguments (e.g. "bin/spark-shell SparkShellArg --master foo"). + */ + private boolean allowsMixedArguments; + + SparkSubmitCommandBuilder() { + this.sparkArgs = new ArrayList(); + } + + SparkSubmitCommandBuilder(List args) { + this(); + List submitArgs = args; + if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { + this.allowsMixedArguments = true; + appResource = PYSPARK_SHELL_RESOURCE; + submitArgs = args.subList(1, args.size()); + } else { + this.allowsMixedArguments = false; + } + + new OptionParser().parse(submitArgs); + } + + @Override + public List buildCommand(Map env) throws IOException { + if (PYSPARK_SHELL_RESOURCE.equals(appResource)) { + return buildPySparkShellCommand(env); + } else { + return buildSparkSubmitCommand(env); + } + } + + List buildSparkSubmitArgs() { + List args = new ArrayList(); + SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); + + if (verbose) { + args.add(parser.VERBOSE); + } + + if (master != null) { + args.add(parser.MASTER); + args.add(master); + } + + if (deployMode != null) { + args.add(parser.DEPLOY_MODE); + args.add(deployMode); + } + + if (appName != null) { + args.add(parser.NAME); + args.add(appName); + } + + for (Map.Entry e : conf.entrySet()) { + args.add(parser.CONF); + args.add(String.format("%s=%s", e.getKey(), e.getValue())); + } + + if (propertiesFile != null) { + args.add(parser.PROPERTIES_FILE); + args.add(propertiesFile); + } + + if (!jars.isEmpty()) { + args.add(parser.JARS); + args.add(join(",", jars)); + } + + if (!files.isEmpty()) { + args.add(parser.FILES); + args.add(join(",", files)); + } + + if (!pyFiles.isEmpty()) { + args.add(parser.PY_FILES); + args.add(join(",", pyFiles)); + } + + if (mainClass != null) { + args.add(parser.CLASS); + args.add(mainClass); + } + + args.addAll(sparkArgs); + if (appResource != null) { + args.add(appResource); + } + args.addAll(appArgs); + + return args; + } + + private List buildSparkSubmitCommand(Map env) throws IOException { + // Load the properties file and check whether spark-submit will be running the app's driver + // or just launching a cluster app. When running the driver, the JVM's argument will be + // modified to cover the driver's configuration. + Properties props = loadPropertiesFile(); + boolean isClientMode = isClientMode(props); + String extraClassPath = isClientMode ? + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null; + + List cmd = buildJavaCommand(extraClassPath); + addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); + addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); + + if (isClientMode) { + // Figuring out where the memory value come from is a little tricky due to precedence. + // Precedence is observed in the following order: + // - explicit configuration (setConf()), which also covers --driver-memory cli argument. + // - properties file. + // - SPARK_DRIVER_MEMORY env variable + // - SPARK_MEM env variable + // - default value (512m) + String memory = firstNonEmpty(firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props), + System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); + cmd.add("-Xms" + memory); + cmd.add("-Xmx" + memory); + addOptionString(cmd, firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props)); + mergeEnvPathList(env, getLibPathEnvName(), + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + } + + addPermGenSizeOpt(cmd); + cmd.add("org.apache.spark.deploy.SparkSubmit"); + cmd.addAll(buildSparkSubmitArgs()); + return cmd; + } + + private List buildPySparkShellCommand(Map env) throws IOException { + // For backwards compatibility, if a script is specified in + // the pyspark command line, then run it using spark-submit. + if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { + System.err.println( + "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Use ./bin/spark-submit "); + appResource = appArgs.get(0); + appArgs.remove(0); + return buildCommand(env); + } + + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable + // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. + checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); + + Properties props = loadPropertiesFile(); + mergeEnvPathList(env, getLibPathEnvName(), + firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); + + // Store spark-submit arguments in an environment variable, since there's no way to pass + // them to shell.py on the comand line. + StringBuilder submitArgs = new StringBuilder(); + for (String arg : buildSparkSubmitArgs()) { + if (submitArgs.length() > 0) { + submitArgs.append(" "); + } + submitArgs.append(quoteForPython(arg)); + } + env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); + + List pyargs = new ArrayList(); + pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return pyargs; + } + + private boolean isClientMode(Properties userProps) { + String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); + // Default master is "local[*]", so assume client mode in that case. + return userMaster == null || + "client".equals(deployMode) || + (!userMaster.equals("yarn-cluster") && deployMode == null); + } + + private class OptionParser extends SparkSubmitOptionParser { + + private final List driverJvmKeys = Arrays.asList( + SparkLauncher.DRIVER_EXTRA_CLASSPATH, + SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, + SparkLauncher.DRIVER_MEMORY); + + @Override + protected boolean handle(String opt, String value) { + if (opt.equals(MASTER)) { + master = value; + } else if (opt.equals(DEPLOY_MODE)) { + deployMode = value; + } else if (opt.equals(PROPERTIES_FILE)) { + propertiesFile = value; + } else if (opt.equals(DRIVER_MEMORY)) { + conf.put(SparkLauncher.DRIVER_MEMORY, value); + } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { + conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); + } else if (opt.equals(DRIVER_LIBRARY_PATH)) { + conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); + } else if (opt.equals(DRIVER_CLASS_PATH)) { + conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); + } else if (opt.equals(CONF)) { + String[] setConf = value.split("=", 2); + checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); + if (driverJvmKeys.contains(setConf[0])) { + conf.put(setConf[0], setConf[1]); + } + } else if (opt.equals(CLASS)) { + // The special classes require some special command line handling, since they allow + // mixing spark-submit arguments with arguments that should be propagated to the shell + // itself. Note that for this to work, the "--class" argument must come before any + // non-spark-submit arguments. + mainClass = value; + if (specialClasses.containsKey(value)) { + allowsMixedArguments = true; + appResource = specialClasses.get(value); + } + } else { + sparkArgs.add(opt); + if (value != null) { + sparkArgs.add(value); + } + } + return true; + } + + @Override + protected boolean handleUnknown(String opt) { + // When mixing arguments, add unrecognized parameters directly to the user arguments list. In + // normal mode, any unrecognized parameter triggers the end of command line parsing, and the + // parameter itself will be interpreted by SparkSubmit as the application resource. The + // remaining params will be appended to the list of SparkSubmit arguments. + if (allowsMixedArguments) { + appArgs.add(opt); + return true; + } else { + sparkArgs.add(opt); + return false; + } + } + + @Override + protected void handleExtraArgs(List extra) { + for (String arg : extra) { + sparkArgs.add(arg); + } + } + + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java new file mode 100644 index 0000000000000..8526d2e7cfa3f --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -0,0 +1,224 @@ +/* + * 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.launcher; + +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Parser for spark-submit command line options. + *

    + * This class encapsulates the parsing code for spark-submit command line options, so that there + * is a single list of options that needs to be maintained (well, sort of, but it makes it harder + * to break things). + */ +class SparkSubmitOptionParser { + + // The following constants define the "main" name for the available options. They're defined + // to avoid copy & paste of the raw strings where they're needed. + // + // The fields are not static so that they're exposed to Scala code that uses this class. See + // SparkSubmitArguments.scala. That is also why this class is not abstract - to allow code to + // easily use these constants without having to create dummy implementations of this class. + protected final String CLASS = "--class"; + protected final String CONF = "--conf"; + protected final String DEPLOY_MODE = "--deploy-mode"; + protected final String DRIVER_CLASS_PATH = "--driver-class-path"; + protected final String DRIVER_CORES = "--driver-cores"; + protected final String DRIVER_JAVA_OPTIONS = "--driver-java-options"; + protected final String DRIVER_LIBRARY_PATH = "--driver-library-path"; + protected final String DRIVER_MEMORY = "--driver-memory"; + protected final String EXECUTOR_MEMORY = "--executor-memory"; + protected final String FILES = "--files"; + protected final String JARS = "--jars"; + protected final String KILL_SUBMISSION = "--kill"; + protected final String MASTER = "--master"; + protected final String NAME = "--name"; + protected final String PACKAGES = "--packages"; + protected final String PROPERTIES_FILE = "--properties-file"; + protected final String PROXY_USER = "--proxy-user"; + protected final String PY_FILES = "--py-files"; + protected final String REPOSITORIES = "--repositories"; + protected final String STATUS = "--status"; + protected final String TOTAL_EXECUTOR_CORES = "--total-executor-cores"; + + // Options that do not take arguments. + protected final String HELP = "--help"; + protected final String SUPERVISE = "--supervise"; + protected final String VERBOSE = "--verbose"; + protected final String VERSION = "--version"; + + // Standalone-only options. + + // YARN-only options. + protected final String ARCHIVES = "--archives"; + protected final String EXECUTOR_CORES = "--executor-cores"; + protected final String QUEUE = "--queue"; + protected final String NUM_EXECUTORS = "--num-executors"; + + /** + * This is the canonical list of spark-submit options. Each entry in the array contains the + * different aliases for the same option; the first element of each entry is the "official" + * name of the option, passed to {@link #handle(String, String)}. + *

    + * Options not listed here nor in the "switch" list below will result in a call to + * {@link $#handleUnknown(String)}. + *

    + * These two arrays are visible for tests. + */ + final String[][] opts = { + { ARCHIVES }, + { CLASS }, + { CONF, "-c" }, + { DEPLOY_MODE }, + { DRIVER_CLASS_PATH }, + { DRIVER_CORES }, + { DRIVER_JAVA_OPTIONS }, + { DRIVER_LIBRARY_PATH }, + { DRIVER_MEMORY }, + { EXECUTOR_CORES }, + { EXECUTOR_MEMORY }, + { FILES }, + { JARS }, + { KILL_SUBMISSION }, + { MASTER }, + { NAME }, + { NUM_EXECUTORS }, + { PACKAGES }, + { PROPERTIES_FILE }, + { PROXY_USER }, + { PY_FILES }, + { QUEUE }, + { REPOSITORIES }, + { STATUS }, + { TOTAL_EXECUTOR_CORES }, + }; + + /** + * List of switches (command line options that do not take parameters) recognized by spark-submit. + */ + final String[][] switches = { + { HELP, "-h" }, + { SUPERVISE }, + { VERBOSE, "-v" }, + { VERSION }, + }; + + /** + * Parse a list of spark-submit command line options. + *

    + * See SparkSubmitArguments.scala for a more formal description of available options. + * + * @throws IllegalArgumentException If an error is found during parsing. + */ + protected final void parse(List args) { + Pattern eqSeparatedOpt = Pattern.compile("(--[^=]+)=(.+)"); + + int idx = 0; + for (idx = 0; idx < args.size(); idx++) { + String arg = args.get(idx); + String value = null; + + Matcher m = eqSeparatedOpt.matcher(arg); + if (m.matches()) { + arg = m.group(1); + value = m.group(2); + } + + // Look for options with a value. + String name = findCliOption(arg, opts); + if (name != null) { + if (value == null) { + if (idx == args.size() - 1) { + throw new IllegalArgumentException( + String.format("Missing argument for option '%s'.", arg)); + } + idx++; + value = args.get(idx); + } + if (!handle(name, value)) { + break; + } + continue; + } + + // Look for a switch. + name = findCliOption(arg, switches); + if (name != null) { + if (!handle(name, null)) { + break; + } + continue; + } + + if (!handleUnknown(arg)) { + break; + } + } + + if (idx < args.size()) { + idx++; + } + handleExtraArgs(args.subList(idx, args.size())); + } + + /** + * Callback for when an option with an argument is parsed. + * + * @param opt The long name of the cli option (might differ from actual command line). + * @param value The value. This will be null if the option does not take a value. + * @return Whether to continue parsing the argument list. + */ + protected boolean handle(String opt, String value) { + throw new UnsupportedOperationException(); + } + + /** + * Callback for when an unrecognized option is parsed. + * + * @param opt Unrecognized option from the command line. + * @return Whether to continue parsing the argument list. + */ + protected boolean handleUnknown(String opt) { + throw new UnsupportedOperationException(); + } + + /** + * Callback for remaining command line arguments after either {@link #handle(String, String)} or + * {@link #handleUnknown(String)} return "false". This will be called at the end of parsing even + * when there are no remaining arguments. + * + * @param extra List of remaining arguments. + */ + protected void handleExtraArgs(List extra) { + throw new UnsupportedOperationException(); + } + + private String findCliOption(String name, String[][] available) { + for (String[] candidates : available) { + for (String candidate : candidates) { + if (candidate.equals(name)) { + return candidates[0]; + } + } + } + return null; + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java new file mode 100644 index 0000000000000..7ed756f4b8591 --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Library for launching Spark applications. + *

    + * This library allows applications to launch Spark programmatically. There's only one entry + * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class. + *

    + * To launch a Spark application, just instantiate a {@link org.apache.spark.launcher.SparkLauncher} + * and configure the application to run. For example: + * + *

    + * {@code
    + *   import org.apache.spark.launcher.SparkLauncher;
    + *
    + *   public class MyLauncher {
    + *     public static void main(String[] args) throws Exception {
    + *       Process spark = new SparkLauncher()
    + *         .setAppResource("/my/app.jar")
    + *         .setMainClass("my.spark.app.Main")
    + *         .setMaster("local")
    + *         .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
    + *         .launch();
    + *       spark.waitFor();
    + *     }
    + *   }
    + * }
    + * 
    + */ +package org.apache.spark.launcher; diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java new file mode 100644 index 0000000000000..dba0203867372 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.*; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +public class CommandBuilderUtilsSuite { + + @Test + public void testValidOptionStrings() { + testOpt("a b c d e", Arrays.asList("a", "b", "c", "d", "e")); + testOpt("a 'b c' \"d\" e", Arrays.asList("a", "b c", "d", "e")); + testOpt("a 'b\\\"c' \"'d'\" e", Arrays.asList("a", "b\\\"c", "'d'", "e")); + testOpt("a 'b\"c' \"\\\"d\\\"\" e", Arrays.asList("a", "b\"c", "\"d\"", "e")); + testOpt(" a b c \\\\ ", Arrays.asList("a", "b", "c", "\\")); + + // Following tests ported from UtilsSuite.scala. + testOpt("", new ArrayList()); + testOpt("a", Arrays.asList("a")); + testOpt("aaa", Arrays.asList("aaa")); + testOpt("a b c", Arrays.asList("a", "b", "c")); + testOpt(" a b\t c ", Arrays.asList("a", "b", "c")); + testOpt("a 'b c'", Arrays.asList("a", "b c")); + testOpt("a 'b c' d", Arrays.asList("a", "b c", "d")); + testOpt("'b c'", Arrays.asList("b c")); + testOpt("a \"b c\"", Arrays.asList("a", "b c")); + testOpt("a \"b c\" d", Arrays.asList("a", "b c", "d")); + testOpt("\"b c\"", Arrays.asList("b c")); + testOpt("a 'b\" c' \"d' e\"", Arrays.asList("a", "b\" c", "d' e")); + testOpt("a\t'b\nc'\nd", Arrays.asList("a", "b\nc", "d")); + testOpt("a \"b\\\\c\"", Arrays.asList("a", "b\\c")); + testOpt("a \"b\\\"c\"", Arrays.asList("a", "b\"c")); + testOpt("a 'b\\\"c'", Arrays.asList("a", "b\\\"c")); + testOpt("'a'b", Arrays.asList("ab")); + testOpt("'a''b'", Arrays.asList("ab")); + testOpt("\"a\"b", Arrays.asList("ab")); + testOpt("\"a\"\"b\"", Arrays.asList("ab")); + testOpt("''", Arrays.asList("")); + testOpt("\"\"", Arrays.asList("")); + } + + @Test + public void testInvalidOptionStrings() { + testInvalidOpt("\\"); + testInvalidOpt("\"abcde"); + testInvalidOpt("'abcde"); + } + + @Test + public void testWindowsBatchQuoting() { + assertEquals("abc", quoteForBatchScript("abc")); + assertEquals("\"a b c\"", quoteForBatchScript("a b c")); + assertEquals("\"a \"\"b\"\" c\"", quoteForBatchScript("a \"b\" c")); + assertEquals("\"a\"\"b\"\"c\"", quoteForBatchScript("a\"b\"c")); + assertEquals("\"ab^=\"\"cd\"\"\"", quoteForBatchScript("ab=\"cd\"")); + } + + @Test + public void testPythonArgQuoting() { + assertEquals("\"abc\"", quoteForPython("abc")); + assertEquals("\"a b c\"", quoteForPython("a b c")); + assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c")); + } + + private void testOpt(String opts, List expected) { + assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), + expected, parseOptionString(opts)); + } + + private void testInvalidOpt(String opts) { + try { + parseOptionString(opts); + fail("Expected exception for invalid option string."); + } catch (IllegalArgumentException e) { + // pass. + } + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java new file mode 100644 index 0000000000000..252d5abae1ca3 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.io.BufferedReader; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static org.junit.Assert.*; + +/** + * These tests require the Spark assembly to be built before they can be run. + */ +public class SparkLauncherSuite { + + private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class); + + @Test + public void testChildProcLauncher() throws Exception { + Map env = new HashMap(); + env.put("SPARK_PRINT_LAUNCH_COMMAND", "1"); + + SparkLauncher launcher = new SparkLauncher(env) + .setSparkHome(System.getProperty("spark.test.home")) + .setMaster("local") + .setAppResource("spark-internal") + .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, + "-Dfoo=bar -Dtest.name=-testChildProcLauncher") + .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) + .setMainClass(SparkLauncherTestApp.class.getName()) + .addAppArgs("proc"); + final Process app = launcher.launch(); + new Redirector("stdout", app.getInputStream()).start(); + new Redirector("stderr", app.getErrorStream()).start(); + assertEquals(0, app.waitFor()); + } + + public static class SparkLauncherTestApp { + + public static void main(String[] args) throws Exception { + assertEquals(1, args.length); + assertEquals("proc", args[0]); + assertEquals("bar", System.getProperty("foo")); + assertEquals("local", System.getProperty(SparkLauncher.SPARK_MASTER)); + } + + } + + private static class Redirector extends Thread { + + private final InputStream in; + + Redirector(String name, InputStream in) { + this.in = in; + setName(name); + setDaemon(true); + } + + @Override + public void run() { + try { + BufferedReader reader = new BufferedReader(new InputStreamReader(in, "UTF-8")); + String line; + while ((line = reader.readLine()) != null) { + LOG.warn(line); + } + } catch (Exception e) { + LOG.error("Error reading process output.", e); + } + } + + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java new file mode 100644 index 0000000000000..815edc4e4971f --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -0,0 +1,278 @@ +/* + * 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.launcher; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; + +public class SparkSubmitCommandBuilderSuite { + + private static File dummyPropsFile; + private static SparkSubmitOptionParser parser; + + @BeforeClass + public static void setUp() throws Exception { + dummyPropsFile = File.createTempFile("spark", "properties"); + parser = new SparkSubmitOptionParser(); + } + + @AfterClass + public static void cleanUp() throws Exception { + dummyPropsFile.delete(); + } + + @Test + public void testDriverCmdBuilder() throws Exception { + testCmdBuilder(true); + } + + @Test + public void testClusterCmdBuilder() throws Exception { + testCmdBuilder(false); + } + + @Test + public void testCliParser() throws Exception { + List sparkSubmitArgs = Arrays.asList( + parser.MASTER, + "local", + parser.DRIVER_MEMORY, + "42g", + parser.DRIVER_CLASS_PATH, + "/driverCp", + parser.DRIVER_JAVA_OPTIONS, + "extraJavaOpt", + parser.CONF, + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); + Map env = new HashMap(); + List cmd = buildCommand(sparkSubmitArgs, env); + + assertTrue(findInStringList(env.get(CommandBuilderUtils.getLibPathEnvName()), + File.pathSeparator, "/driverLibPath")); + assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); + assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); + assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); + } + + @Test + public void testShellCliParser() throws Exception { + List sparkSubmitArgs = Arrays.asList( + parser.CLASS, + "org.apache.spark.repl.Main", + parser.MASTER, + "foo", + "--app-arg", + "bar", + "--app-switch", + parser.FILES, + "baz", + parser.NAME, + "appName"); + + List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); + assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); + } + + @Test + public void testAlternateSyntaxParsing() throws Exception { + List sparkSubmitArgs = Arrays.asList( + parser.CLASS + "=org.my.Class", + parser.MASTER + "=foo", + parser.DEPLOY_MODE + "=bar"); + + List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); + assertEquals("foo", findArgValue(cmd, parser.MASTER)); + assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); + } + + @Test + public void testPySparkLauncher() throws Exception { + List sparkSubmitArgs = Arrays.asList( + SparkSubmitCommandBuilder.PYSPARK_SHELL, + "--master=foo", + "--deploy-mode=bar"); + + Map env = new HashMap(); + List cmd = buildCommand(sparkSubmitArgs, env); + assertEquals("python", cmd.get(cmd.size() - 1)); + assertEquals( + String.format("\"%s\" \"foo\" \"%s\" \"bar\" \"%s\"", + parser.MASTER, parser.DEPLOY_MODE, SparkSubmitCommandBuilder.PYSPARK_SHELL_RESOURCE), + env.get("PYSPARK_SUBMIT_ARGS")); + } + + @Test + public void testPySparkFallback() throws Exception { + List sparkSubmitArgs = Arrays.asList( + "--master=foo", + "--deploy-mode=bar", + "script.py", + "arg1"); + + Map env = new HashMap(); + List cmd = buildCommand(sparkSubmitArgs, env); + + assertEquals("foo", findArgValue(cmd, "--master")); + assertEquals("bar", findArgValue(cmd, "--deploy-mode")); + assertEquals("script.py", cmd.get(cmd.size() - 2)); + assertEquals("arg1", cmd.get(cmd.size() - 1)); + } + + private void testCmdBuilder(boolean isDriver) throws Exception { + String deployMode = isDriver ? "client" : "cluster"; + + SparkSubmitCommandBuilder launcher = + new SparkSubmitCommandBuilder(Collections.emptyList()); + launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, + System.getProperty("spark.test.home")); + launcher.master = "yarn"; + launcher.deployMode = deployMode; + launcher.appResource = "/foo"; + launcher.appName = "MyApp"; + launcher.mainClass = "my.Class"; + launcher.propertiesFile = dummyPropsFile.getAbsolutePath(); + launcher.appArgs.add("foo"); + launcher.appArgs.add("bar"); + launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native"); + launcher.conf.put("spark.foo", "foo"); + + Map env = new HashMap(); + List cmd = launcher.buildCommand(env); + + // Checks below are different for driver and non-driver mode. + + if (isDriver) { + assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms1g")); + assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx1g")); + } else { + boolean found = false; + for (String arg : cmd) { + if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) { + found = true; + break; + } + } + assertFalse("Memory arguments should not be set.", found); + } + + for (String arg : cmd) { + if (arg.startsWith("-XX:MaxPermSize=")) { + if (isDriver) { + assertEquals("-XX:MaxPermSize=256m", arg); + } else { + assertEquals("-XX:MaxPermSize=128m", arg); + } + } + } + + String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); + if (isDriver) { + assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp)); + } else { + assertFalse("Driver classpath should not be in command.", contains("/driver", cp)); + } + + String libPath = env.get(CommandBuilderUtils.getLibPathEnvName()); + if (isDriver) { + assertNotNull("Native library path should be set.", libPath); + assertTrue("Native library path should contain provided entry.", + contains("/native", libPath.split(Pattern.quote(File.pathSeparator)))); + } else { + assertNull("Native library should not be set.", libPath); + } + + // Checks below are the same for both driver and non-driver mode. + assertEquals(dummyPropsFile.getAbsolutePath(), findArgValue(cmd, parser.PROPERTIES_FILE)); + assertEquals("yarn", findArgValue(cmd, parser.MASTER)); + assertEquals(deployMode, findArgValue(cmd, parser.DEPLOY_MODE)); + assertEquals("my.Class", findArgValue(cmd, parser.CLASS)); + assertEquals("MyApp", findArgValue(cmd, parser.NAME)); + + boolean appArgsOk = false; + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals("/foo")) { + assertEquals("foo", cmd.get(i + 1)); + assertEquals("bar", cmd.get(i + 2)); + assertEquals(cmd.size(), i + 3); + appArgsOk = true; + break; + } + } + assertTrue("App resource and args should be added to command.", appArgsOk); + + Map conf = parseConf(cmd, parser); + assertEquals("foo", conf.get("spark.foo")); + } + + private boolean contains(String needle, String[] haystack) { + for (String entry : haystack) { + if (entry.equals(needle)) { + return true; + } + } + return false; + } + + private Map parseConf(List cmd, SparkSubmitOptionParser parser) { + Map conf = new HashMap(); + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals(parser.CONF)) { + String[] val = cmd.get(i + 1).split("=", 2); + conf.put(val[0], val[1]); + i += 1; + } + } + return conf; + } + + private String findArgValue(List cmd, String name) { + for (int i = 0; i < cmd.size(); i++) { + if (cmd.get(i).equals(name)) { + return cmd.get(i + 1); + } + } + fail(String.format("arg '%s' not found", name)); + return null; + } + + private boolean findInStringList(String list, String sep, String needle) { + return contains(needle, list.split(sep)); + } + + private List buildCommand(List args, Map env) throws Exception { + SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); + return builder.buildCommand(env); + } + +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java new file mode 100644 index 0000000000000..f3d2109917056 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import static org.apache.spark.launcher.SparkSubmitOptionParser.*; + +public class SparkSubmitOptionParserSuite { + + private SparkSubmitOptionParser parser; + + @Before + public void setUp() { + parser = spy(new DummyParser()); + } + + @Test + public void testAllOptions() { + int count = 0; + for (String[] optNames : parser.opts) { + for (String optName : optNames) { + String value = optName + "-value"; + parser.parse(Arrays.asList(optName, value)); + count++; + verify(parser).handle(eq(optNames[0]), eq(value)); + verify(parser, times(count)).handle(anyString(), anyString()); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + } + } + + for (String[] switchNames : parser.switches) { + int switchCount = 0; + for (String name : switchNames) { + parser.parse(Arrays.asList(name)); + count++; + switchCount++; + verify(parser, times(switchCount)).handle(eq(switchNames[0]), same((String) null)); + verify(parser, times(count)).handle(anyString(), any(String.class)); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + } + } + } + + @Test + public void testExtraOptions() { + List args = Arrays.asList(parser.MASTER, parser.MASTER, "foo", "bar"); + parser.parse(args); + verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); + verify(parser).handleUnknown(eq("foo")); + verify(parser).handleExtraArgs(eq(Arrays.asList("bar"))); + } + + @Test(expected=IllegalArgumentException.class) + public void testMissingArg() { + parser.parse(Arrays.asList(parser.MASTER)); + } + + @Test + public void testEqualSeparatedOption() { + List args = Arrays.asList(parser.MASTER + "=" + parser.MASTER); + parser.parse(args); + verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); + verify(parser).handleExtraArgs(eq(Collections.emptyList())); + } + + private static class DummyParser extends SparkSubmitOptionParser { + + @Override + protected boolean handle(String opt, String value) { + return true; + } + + @Override + protected boolean handleUnknown(String opt) { + return false; + } + + @Override + protected void handleExtraArgs(List extra) { + + } + + } + +} diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..00c20ad69cd4d --- /dev/null +++ b/launcher/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false + +# Some tests will set "test.name" to avoid overwriting the main log file. +log4j.appender.file.file=target/unit-tests${test.name}.log + +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/make-distribution.sh b/make-distribution.sh index dd990d4b96e46..82d33408cd5e5 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -127,6 +127,7 @@ if [ ! $(command -v "$MVN") ] ; then fi VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null | grep -v "INFO" | tail -n 1) SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) @@ -196,6 +197,7 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +cp "$SPARK_HOME"/launcher/target/spark-launcher_$SCALA_VERSION-$VERSION.jar "$DISTDIR/lib/" # This will fail if the -Pyarn profile is not provided # In this case, silence the error and ignore the return code of this command cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || : diff --git a/pom.xml b/pom.xml index 51bef30f9ca8f..a19da73cf45b3 100644 --- a/pom.xml +++ b/pom.xml @@ -105,6 +105,7 @@ external/zeromq examples repl + launcher @@ -1195,7 +1196,7 @@ true - ${session.executionRootDirectory} + ${spark.test.home} 1 false false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4f17df59f4c1f..35e748f26bbaa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -34,11 +34,11 @@ object BuildCommons { val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, - streamingMqtt, streamingTwitter, streamingZeromq) = + streamingMqtt, streamingTwitter, streamingZeromq, launcher) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq").map(ProjectRef(buildLocation, _)) + "streaming-zeromq", "launcher").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", @@ -155,8 +155,9 @@ object SparkBuild extends PomBuild { (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) // TODO: Add Sql to mima checks + // TODO: remove launcher from this list after 1.3. allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - networkCommon, networkShuffle, networkYarn).contains(x)).foreach { + networkCommon, networkShuffle, networkYarn, launcher).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 936857e75c7e9..43d2cf5171880 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -41,7 +41,7 @@ def launch_gateway(): submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") submit_args = submit_args if submit_args is not None else "" submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"] + command = [os.path.join(SPARK_HOME, script)] + submit_args # Start a socket that will be used by PythonGatewayServer to communicate its port to us callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) @@ -58,7 +58,6 @@ def launch_gateway(): # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): signal.signal(signal.SIGINT, signal.SIG_IGN) - env["IS_SUBPROCESS"] = "1" # tell JVM to exit after python exits proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env) else: # preexec_fn not supported on Windows diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 5e812a1d91c6b..92e76a3fe6ca2 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -121,45 +121,63 @@ if [ "$SPARK_NICENESS" = "" ]; then export SPARK_NICENESS=0 fi +run_command() { + mode="$1" + shift -case $option in + mkdir -p "$SPARK_PID_DIR" - (start|spark-submit) + if [ -f "$pid" ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + echo "$command running as process $TARGET_ID. Stop it first." + exit 1 + fi + fi - mkdir -p "$SPARK_PID_DIR" + if [ "$SPARK_MASTER" != "" ]; then + echo rsync from "$SPARK_MASTER" + rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' "$SPARK_MASTER/" "$SPARK_HOME" + fi - if [ -f $pid ]; then - TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then - echo "$command running as process $TARGET_ID. Stop it first." - exit 1 - fi - fi + spark_rotate_log "$log" + echo "starting $command, logging to $log" + + case "$mode" in + (class) + nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid="$!" + ;; + + (submit) + nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & + newpid="$!" + ;; + + (*) + echo "unknown mode: $mode" + exit 1 + ;; + esac + + echo "$newpid" > "$pid" + sleep 2 + # Check if the process has died; in that case we'll tail the log so the user can see + if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then + echo "failed to launch $command:" + tail -2 "$log" | sed 's/^/ /' + echo "full log in $log" + fi +} - if [ "$SPARK_MASTER" != "" ]; then - echo rsync from "$SPARK_MASTER" - rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" - fi +case $option in - spark_rotate_log "$log" - echo "starting $command, logging to $log" - if [ $option == spark-submit ]; then - source "$SPARK_HOME"/bin/utils.sh - gatherSparkSubmitOpts "$@" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command \ - "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" >> "$log" 2>&1 < /dev/null & - else - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - fi - newpid=$! - echo $newpid > $pid - sleep 2 - # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then - echo "failed to launch $command:" - tail -2 "$log" | sed 's/^/ /' - echo "full log in $log" - fi + (submit) + run_command submit "$@" + ;; + + (start) + run_command class "$@" ;; (stop) diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 070cc7a87e6f2..5b0aeb177fff3 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -52,4 +52,4 @@ fi export SUBMIT_USAGE_FUNCTION=usage -exec "$FWDIR"/sbin/spark-daemon.sh spark-submit $CLASS 1 "$@" +exec "$FWDIR"/sbin/spark-daemon.sh submit $CLASS 1 "$@" From 35b25640a4debddd5a4498455888f6241caf6223 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Mar 2015 12:16:32 +0000 Subject: [PATCH 381/817] [MINOR] [DOCS] Fix map -> mapToPair in Streaming Java example Fix map -> mapToPair in Java example. (And zap some unneeded "throws Exception" while here) Author: Sean Owen Closes #4967 from srowen/MapToPairFix and squashes the following commits: ded2bc0 [Sean Owen] Fix map -> mapToPair in Java example. (And zap some unneeded "throws Exception" while here) --- docs/streaming-programming-guide.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 815c98713b738..062ac2648db30 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -189,15 +189,15 @@ Next, we want to count these words. {% highlight java %} // Count each word in each batch -JavaPairDStream pairs = words.map( +JavaPairDStream pairs = words.mapToPair( new PairFunction() { - @Override public Tuple2 call(String s) throws Exception { + @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); JavaPairDStream wordCounts = pairs.reduceByKey( new Function2() { - @Override public Integer call(Integer i1, Integer i2) throws Exception { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); @@ -1041,7 +1041,7 @@ val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Se {% highlight java %} // Reduce function adding two integers, defined separately for clarity Function2 reduceFunc = new Function2() { - @Override public Integer call(Integer i1, Integer i2) throws Exception { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } }; From 40f49795e2624326dd4e38eedbf63d35860ea00e Mon Sep 17 00:00:00 2001 From: Hongbo Liu Date: Wed, 11 Mar 2015 12:18:24 +0000 Subject: [PATCH 382/817] [SQL][Minor] fix typo in comments Removed an repeated "from" in the comments. Author: Hongbo Liu Closes #4976 from liuhb86/mine and squashes the following commits: e280e7c [Hongbo Liu] [SQL][Minor] fix typo in comments --- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 109671bdca361..7e191ad0315a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -63,7 +63,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { /** * Faster version of equality which short-circuits when two treeNodes are the same instance. - * We don't just override Object.Equals, as doing so prevents the scala compiler from from + * We don't just override Object.equals, as doing so prevents the scala compiler from * generating case class `equals` methods */ def fastEquals(other: TreeNode[_]): Boolean = { From ec30c17822329e6d2b8c85625b31ba8bd8679fcf Mon Sep 17 00:00:00 2001 From: zzcclp Date: Wed, 11 Mar 2015 12:22:24 +0000 Subject: [PATCH 383/817] [SPARK-6279][Streaming]In KafkaRDD.scala, Miss expressions flag "s" at logging string In KafkaRDD.scala, Miss expressions flag "s" at logging string In logging file, it print `Beginning offset $ {part.fromOffset} is the same as ending offset ` but not `Beginning offset 111 is the same as ending offset `. Author: zzcclp Closes #4979 from zzcclp/SPARK-6279 and squashes the following commits: 768f88e [zzcclp] Miss expressions flag "s" --- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index d56cc01be9514..6d465bcb6bfc0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -86,7 +86,7 @@ class KafkaRDD[ val part = thePart.asInstanceOf[KafkaRDDPartition] assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) if (part.fromOffset == part.untilOffset) { - log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " + + log.warn(s"Beginning offset ${part.fromOffset} is the same as ending offset " + s"skipping ${part.topic} ${part.partition}") Iterator.empty } else { From 6e94c4eadf443ac3d34eaae4c334c8386fdec960 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Mar 2015 13:15:19 +0000 Subject: [PATCH 384/817] SPARK-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 edition Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc. Author: Sean Owen Closes #4950 from srowen/SPARK-6225 and squashes the following commits: 3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scheduler/EventLoggingListener.scala | 2 +- .../spark/util/MutableURLClassLoader.scala | 2 -- .../OutputCommitCoordinatorSuite.scala | 2 +- .../util/MutableURLClassLoaderSuite.scala | 3 +-- .../JavaStatefulNetworkWordCount.java | 1 + .../org/apache/spark/examples/HBaseTest.scala | 4 ++-- .../kafka/JavaDirectKafkaStreamSuite.java | 21 +++++++------------ .../streaming/kafka/JavaKafkaRDDSuite.java | 21 +++++++------------ .../streaming/kafka/JavaKafkaStreamSuite.java | 14 ++++++------- .../MatrixFactorizationModel.scala | 8 +++---- .../org/apache/spark/sql/sources/ddl.scala | 1 + .../sql/ScalaReflectionRelationSuite.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 2 +- .../apache/spark/streaming/JavaAPISuite.java | 4 ++++ 15 files changed, 40 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1a0bee4e3aea9..8121aab3b0b34 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1104,7 +1104,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (!fs.exists(hadoopPath)) { throw new FileNotFoundException(s"Added file $hadoopPath does not exist.") } - val isDir = fs.isDirectory(hadoopPath) + val isDir = fs.getFileStatus(hadoopPath).isDir if (!isLocal && scheme == "file" && isDir) { throw new SparkException(s"addFile does not support local directories when not running " + "local mode.") diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 2091a9fe8d0d3..34fa6d27c3a45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -95,7 +95,7 @@ private[spark] class EventLoggingListener( * Creates the log file in the configured log directory. */ def start() { - if (!fileSystem.isDirectory(new Path(logBaseDir))) { + if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDir) { throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.") } diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index d9c7103b2f3bf..1e0ba5c28754a 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions._ -import org.apache.spark.util.ParentClassLoader - /** * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 3cc860caa1d9b..c8c957856247a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -153,7 +153,7 @@ class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { def resultHandler(x: Int, y: Unit): Unit = {} val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully, - 0 until rdd.partitions.size, resultHandler, 0) + 0 until rdd.partitions.size, resultHandler, () => Unit) // It's an error if the job completes successfully even though no committer was authorized, // so throw an exception if the job was allowed to complete. intercept[TimeoutException] { diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index 31e3b7e7bb71b..87de90bb0dfb0 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -21,8 +21,7 @@ import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} -import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkException, TestUtils} class MutableURLClassLoaderSuite extends FunSuite { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java index d46c7107c7a21..dbf2ef02d7b76 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -82,6 +82,7 @@ public Optional call(List values, Optional state) { ssc.checkpoint("."); // Initial RDD input to updateStateByKey + @SuppressWarnings("unchecked") List> tuples = Arrays.asList(new Tuple2("hello", 1), new Tuple2("world", 1)); JavaPairRDD initialRDD = ssc.sc().parallelizePairs(tuples); diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 822673347bdce..f4684b42b5d41 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples import org.apache.hadoop.hbase.client.HBaseAdmin -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, TableName} import org.apache.hadoop.hbase.mapreduce.TableInputFormat import org.apache.spark._ @@ -36,7 +36,7 @@ object HBaseTest { // Initialize hBase table if necessary val admin = new HBaseAdmin(conf) if (!admin.isTableAvailable(args(0))) { - val tableDesc = new HTableDescriptor(args(0)) + val tableDesc = new HTableDescriptor(TableName.valueOf(args(0))) admin.createTable(tableDesc) } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 1334cc8fd1b57..d6ca6d58b5665 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -20,32 +20,27 @@ import java.io.Serializable; import java.util.HashMap; import java.util.HashSet; -import java.util.Random; import java.util.Arrays; -import org.apache.spark.SparkConf; - import scala.Tuple2; -import junit.framework.Assert; - import kafka.common.TopicAndPartition; import kafka.message.MessageAndMetadata; import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; -import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaDirectKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private transient Random random = new Random(); private transient KafkaStreamSuiteBase suiteBase = null; @Before @@ -93,7 +88,7 @@ public void testKafkaStream() throws InterruptedException { ).map( new Function, String>() { @Override - public String call(scala.Tuple2 kv) throws Exception { + public String call(Tuple2 kv) throws Exception { return kv._2(); } } @@ -121,7 +116,7 @@ public String call(MessageAndMetadata msgAndMd) throws Exception unifiedStream.foreachRDD( new Function, Void>() { @Override - public Void call(org.apache.spark.api.java.JavaRDD rdd) throws Exception { + public Void call(JavaRDD rdd) throws Exception { result.addAll(rdd.collect()); return null; } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 9d2e1705c6c73..4477b81827c70 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -19,27 +19,22 @@ import java.io.Serializable; import java.util.HashMap; -import java.util.HashSet; -import java.util.Arrays; - -import org.apache.spark.SparkConf; import scala.Tuple2; -import junit.framework.Assert; - import kafka.common.TopicAndPartition; import kafka.message.MessageAndMetadata; import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaKafkaRDDSuite implements Serializable { private transient JavaSparkContext sc = null; private transient KafkaStreamSuiteBase suiteBase = null; @@ -78,8 +73,8 @@ public void testKafkaRDD() throws InterruptedException { OffsetRange.create(topic2, 0, 0, 1) }; - HashMap emptyLeaders = new HashMap(); - HashMap leaders = new HashMap(); + HashMap emptyLeaders = new HashMap(); + HashMap leaders = new HashMap(); String[] hostAndPort = suiteBase.brokerAddress().split(":"); Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); leaders.put(new TopicAndPartition(topic1, 0), broker); @@ -96,7 +91,7 @@ public void testKafkaRDD() throws InterruptedException { ).map( new Function, String>() { @Override - public String call(scala.Tuple2 kv) throws Exception { + public String call(Tuple2 kv) throws Exception { return kv._2(); } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 208cc51b29876..bad0a93eb2e84 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,27 +22,25 @@ import java.util.List; import java.util.Random; -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.Duration; import scala.Predef; import scala.Tuple2; import scala.collection.JavaConverters; -import junit.framework.Assert; - import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; private transient Random random = new Random(); 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 c399496568bfb..5f5a996a87b81 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 @@ -199,12 +199,12 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] val userFeatures = sqlContext.parquetFile(userPath(path)) - .map { case Row(id: Int, features: Seq[Double]) => - (id, features.toArray) + .map { case Row(id: Int, features: Seq[_]) => + (id, features.asInstanceOf[Seq[Double]].toArray) } val productFeatures = sqlContext.parquetFile(productPath(path)) - .map { case Row(id: Int, features: Seq[Double]) => - (id, features.toArray) + .map { case Row(id: Int, features: Seq[_]) => + (id, features.asInstanceOf[Seq[Double]].toArray) } new MatrixFactorizationModel(rank, userFeatures, productFeatures) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 5020689f7a105..76754a6ce4617 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import scala.language.existentials import scala.language.implicitConversions import org.apache.spark.Logging diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 23df6e7eac043..17e923ca48502 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -86,7 +86,7 @@ class ScalaReflectionRelationSuite extends FunSuite { assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), new Date(70, 0, 1), // This is 1970-01-01 + new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1,2,3))) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index 09bbd5c867e4e..3181cfe40016c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -75,7 +75,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { Literal(0.asInstanceOf[Float]) :: Literal(0.asInstanceOf[Double]) :: Literal("0") :: - Literal(new java.sql.Date(114, 8, 23)) :: + Literal(java.sql.Date.valueOf("2014-09-23")) :: Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 57302ff407183..90340753a4eed 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -316,6 +316,7 @@ public void testReduceByWindowWithoutInverse() { testReduceByWindow(false); } + @SuppressWarnings("unchecked") private void testReduceByWindow(boolean withInverse) { List> inputData = Arrays.asList( Arrays.asList(1,2,3), @@ -684,6 +685,7 @@ public void testStreamingContextTransform(){ JavaDStream transformed1 = ssc.transform( listOfDStreams1, new Function2>, Time, JavaRDD>() { + @Override public JavaRDD call(List> listOfRDDs, Time time) { Assert.assertEquals(2, listOfRDDs.size()); return null; @@ -697,6 +699,7 @@ public JavaRDD call(List> listOfRDDs, Time time) { JavaPairDStream> transformed2 = ssc.transformToPair( listOfDStreams2, new Function2>, Time, JavaPairRDD>>() { + @Override public JavaPairRDD> call(List> listOfRDDs, Time time) { Assert.assertEquals(3, listOfRDDs.size()); JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); @@ -1829,6 +1832,7 @@ private List> fileTestPrepare(File testDir) throws IOException { return expected; } + @SuppressWarnings("unchecked") // SPARK-5795: no logic assertions, just testing that intended API invocations compile private void compileSaveAsJavaAPI(JavaPairDStream pds) { pds.saveAsNewAPIHadoopFiles( From 5b335bdda3efb7c6a5b18b4eeff189064c11e6c3 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 11 Mar 2015 13:16:22 +0000 Subject: [PATCH 385/817] [SPARK-6228] [network] Move SASL classes from network/shuffle to network... .../common. No code changes. Left the shuffle-related files in the shuffle module. Author: Marcelo Vanzin Closes #4953 from vanzin/SPARK-6228 and squashes the following commits: 664ef30 [Marcelo Vanzin] [SPARK-6228] [network] Move SASL classes from network/shuffle to network/common. --- .../java/org/apache/spark/network/sasl/SaslClientBootstrap.java | 0 .../src/main/java/org/apache/spark/network/sasl/SaslMessage.java | 0 .../main/java/org/apache/spark/network/sasl/SaslRpcHandler.java | 0 .../main/java/org/apache/spark/network/sasl/SecretKeyHolder.java | 0 .../main/java/org/apache/spark/network/sasl/SparkSaslClient.java | 0 .../main/java/org/apache/spark/network/sasl/SparkSaslServer.java | 0 .../test/java/org/apache/spark/network/sasl/SparkSaslSuite.java | 0 7 files changed, 0 insertions(+), 0 deletions(-) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java (100%) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SaslMessage.java (100%) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java (100%) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java (100%) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java (100%) rename network/{shuffle => common}/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java (100%) rename network/{shuffle => common}/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java (100%) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java b/network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java similarity index 100% rename from network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java rename to network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java From 548643a9e4690b69e2a496cdcd0a426b6de8d8b5 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 11 Mar 2015 13:20:15 +0000 Subject: [PATCH 386/817] [SPARK-4423] Improve foreach() documentation to avoid confusion between local- and cluster-mode behavior Hi all - I've added a writeup on how closures work within Spark to help clarify the general case for this problem and similar problems. I hope this addresses the issue and would love any feedback. Author: Ilya Ganelin Closes #4696 from ilganeli/SPARK-4423 and squashes the following commits: c5dc498 [Ilya Ganelin] Fixed typo 07b78e8 [Ilya Ganelin] Updated to fix capitalization 48c1983 [Ilya Ganelin] Updated to fix capitalization and clarify wording 2fd2a07 [Ilya Ganelin] Incoporated a few more minor fixes. Fixed a bug in python code. Added semicolons for java 4772f99 [Ilya Ganelin] Incorporated latest feedback 448bd79 [Ilya Ganelin] Updated some verbage and added section links 5dbbda5 [Ilya Ganelin] Improved some wording d374d3a [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4423 2600668 [Ilya Ganelin] Minor edits c768ab2 [Ilya Ganelin] Updated documentation to add a section on closures. This helps understand confusing behavior of foreach and map functions when attempting to modify variables outside of the scope of an RDD action or transformation --- docs/programming-guide.md | 72 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 68 insertions(+), 4 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index fa0b4e3705d6e..c011a8404f7c9 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -725,7 +725,7 @@ class MyClass(object): def __init__(self): self.field = "Hello" def doStuff(self, rdd): - return rdd.map(lambda s: self.field + x) + return rdd.map(lambda s: self.field + s) {% endhighlight %} To avoid this issue, the simplest way is to copy `field` into a local variable instead @@ -734,13 +734,76 @@ of accessing it externally: {% highlight python %} def doStuff(self, rdd): field = self.field - return rdd.map(lambda s: field + x) + return rdd.map(lambda s: field + s) {% endhighlight %}
    +### Understanding closures +One of the harder things about Spark is understanding the scope and life cycle of variables and methods when executing code across a cluster. RDD operations that modify variables outside of their scope can be a frequent source of confusion. In the example below we'll look at code that uses `foreach()` to increment a counter, but similar issues can occur for other operations as well. + +#### Example + +Consider the naive RDD element sum below, which behaves completely differently depending on whether execution is happening within the same JVM. A common example of this is when running Spark in `local` mode (`--master = local[n]`) versus deploying a Spark application to a cluster (e.g. via spark-submit to YARN): + +
    + +
    +{% highlight scala %} +var counter = 0 +var rdd = sc.parallelize(data) + +// Wrong: Don't do this!! +rdd.foreach(x => counter += x) + +println("Counter value: " + counter) +{% endhighlight %} +
    + +
    +{% highlight java %} +int counter = 0; +JavaRDD rdd = sc.parallelize(data); + +// Wrong: Don't do this!! +rdd.foreach(x -> counter += x); + +println("Counter value: " + counter); +{% endhighlight %} +
    + +
    +{% highlight python %} +counter = 0 +rdd = sc.parallelize(data) + +# Wrong: Don't do this!! +rdd.foreach(lambda x: counter += x) + +print("Counter value: " + counter) + +{% endhighlight %} +
    + +
    + +#### Local vs. cluster modes + +The primary challenge is that the behavior of the above code is undefined. In local mode with a single JVM, the above code will sum the values within the RDD and store it in **counter**. This is because both the RDD and the variable **counter** are in the same memory space on the driver node. + +However, in `cluster` mode, what happens is more complicated, and the above may not work as intended. To execute jobs, Spark breaks up the processing of RDD operations into tasks - each of which is operated on by an executor. Prior to execution, Spark computes the **closure**. The closure is those variables and methods which must be visible for the executor to perform its computations on the RDD (in this case `foreach()`). This closure is serialized and sent to each executor. In `local` mode, there is only the one executors so everything shares the same closure. In other modes however, this is not the case and the executors running on seperate worker nodes each have their own copy of the closure. + +What is happening here is that the variables within the closure sent to each executor are now copies and thus, when **counter** is referenced within the `foreach` function, it's no longer the **counter** on the driver node. There is still a **counter** in the memory of the driver node but this is no longer visible to the executors! The executors only sees the copy from the serialized closure. Thus, the final value of **counter** will still be zero since all operations on **counter** were referencing the value within the serialized closure. + +To ensure well-defined behavior in these sorts of scenarios one should use an [`Accumulator`](#AccumLink). Accumulators in Spark are used specifically to provide a mechanism for safely updating a variable when execution is split up across worker nodes in a cluster. The Accumulators section of this guide discusses these in more detail. + +In general, closures - constructs like loops or locally defined methods, should not be used to mutate some global state. Spark does not define or guarantee the behavior of mutations to objects referenced from outside of closures. Some code that does this may work in local mode, but that's just by accident and such code will not behave as expected in distributed mode. Use an Accumulator instead if some global aggregation is needed. + +#### Printing elements of an RDD +Another common idiom is attempting to print out the elements of an RDD using `rdd.foreach(println)` or `rdd.map(println)`. On a single machine, this will generate the expected output and print all the RDD's elements. However, in `cluster` mode, the output to `stdout` being called by the executors is now writing to the executor's `stdout` instead, not the one on the driver, so `stdout` on the driver won't show these! To print all elements on the driver, one can use the `collect()` method to first bring the RDD to the driver node thus: `rdd.collect().foreach(println)`. This can cause the driver to run out of memory, though, because `collect()` fetches the entire RDD to a single machine; if you only need to print a few elements of the RDD, a safer approach is to use the `take()`: `rdd.take(100).foreach(println)`. + ### Working with Key-Value Pairs
    @@ -1018,7 +1081,8 @@ for details. foreach(func) - Run a function func on each element of the dataset. This is usually done for side effects such as updating an accumulator variable (see below) or interacting with external storage systems. + Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems. +
    Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details. @@ -1191,7 +1255,7 @@ run on the cluster so that `v` is not shipped to the nodes more than once. In ad `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later). -## Accumulators +## Accumulators Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in From 2d87a415f20c85487537d6791a73827ff537f2c0 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 11 Mar 2015 13:22:05 +0000 Subject: [PATCH 387/817] SPARK-3642. Document the nuances of shared variables. Author: Sandy Ryza Closes #2490 from sryza/sandy-spark-3642 and squashes the following commits: aae3340 [Sandy Ryza] SPARK-3642. Document the nuances of broadcast variables --- docs/programming-guide.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index c011a8404f7c9..eda3a95426182 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1207,6 +1207,12 @@ than shipping a copy of it with tasks. They can be used, for example, to give ev large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost. +Spark actions are executed through a set of stages, separated by distributed "shuffle" operations. +Spark automatically broadcasts the common data needed by tasks within each stage. The data +broadcasted this way is cached in serialized form and deserialized before running each task. This +means that explicitly creating broadcast variables is only useful when tasks across multiple stages +need the same data or when caching the data in deserialized form is important. + Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The code below shows this: From 55c4831d68c8326380086b5540244f984ea9ec27 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Mar 2015 14:09:09 +0000 Subject: [PATCH 388/817] SPARK-6245 [SQL] jsonRDD() of empty RDD results in exception Avoid `UnsupportedOperationException` from JsonRDD.inferSchema on empty RDD. Not sure if this is supposed to be an error (but a better one), but it seems like this case can come up if the input is down-sampled so much that nothing is sampled. Now stuff like this: ``` sqlContext.jsonRDD(sc.parallelize(List[String]())) ``` just results in ``` org.apache.spark.sql.DataFrame = [] ``` Author: Sean Owen Closes #4971 from srowen/SPARK-6245 and squashes the following commits: 3699964 [Sean Owen] Set() -> Set.empty 3c619e1 [Sean Owen] Avoid UnsupportedOperationException from JsonRDD.inferSchema on empty RDD --- .../src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 6 +++++- .../test/scala/org/apache/spark/sql/json/JsonSuite.scala | 7 +++++++ .../scala/org/apache/spark/sql/json/TestJsonData.scala | 3 +++ 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index e54a2a3679272..2b0358c4e2a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -48,7 +48,11 @@ private[sql] object JsonRDD extends Logging { require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = - parseJson(schemaData, columnNameOfCorruptRecords).map(allKeysWithValueTypes).reduce(_ ++ _) + if (schemaData.isEmpty()) { + Set.empty[(String,DataType)] + } else { + parseJson(schemaData, columnNameOfCorruptRecords).map(allKeysWithValueTypes).reduce(_ ++ _) + } createSchema(allKeys) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 0c21f725f0b49..320b80d80e997 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -1033,4 +1033,11 @@ class JsonSuite extends QueryTest { assert(!logicalRelation2.sameResult(logicalRelation3), s"$logicalRelation2 and $logicalRelation3 should be considered not having the same result.") } + + test("SPARK-6245 JsonRDD.inferSchema on empty RDD") { + // This is really a test that it doesn't throw an exception + val emptySchema = JsonRDD.inferSchema(empty, 1.0, "") + assert(StructType(Seq()) === emptySchema) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 15698f61e0837..47a97a49daabb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -185,4 +185,7 @@ object TestJsonData { """{"a":{, b:3}""" :: """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: """]""" :: Nil) + + val empty = + TestSQLContext.sparkContext.parallelize(Seq[String]()) } From 51a79a770a8356bd0ed244af5ca7f1c44c9437d2 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 11 Mar 2015 11:19:51 -0700 Subject: [PATCH 389/817] [SPARK-6274][Streaming][Examples] Added examples streaming + sql examples. Added Scala, Java and Python streaming examples showing DataFrame and SQL operations within streaming. Author: Tathagata Das Closes #4975 from tdas/streaming-sql-examples and squashes the following commits: 705cba1 [Tathagata Das] Fixed python lint error 75a3fad [Tathagata Das] Fixed python lint error 5fbf789 [Tathagata Das] Removed empty lines at the end 874b943 [Tathagata Das] Added examples streaming + sql examples. --- .../spark/examples/streaming/JavaRecord.java | 31 +++++ .../streaming/JavaSqlNetworkWordCount.java | 122 ++++++++++++++++++ .../python/streaming/sql_network_wordcount.py | 82 ++++++++++++ .../streaming/SqlNetworkWordCount.scala | 101 +++++++++++++++ 4 files changed, 336 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java create mode 100644 examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java create mode 100644 examples/src/main/python/streaming/sql_network_wordcount.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java new file mode 100644 index 0000000000000..e63697a79f23a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +/** Java Bean class to be used with the example JavaSqlNetworkWordCount. */ +public class JavaRecord implements java.io.Serializable { + private String word; + + public String getWord() { + return word; + } + + public void setWord(String word) { + this.word = word; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java new file mode 100644 index 0000000000000..46562ddbbcb57 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import java.util.regex.Pattern; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.api.java.StorageLevels; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.Time; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +/** + * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + * network every second. + * + * Usage: JavaSqlNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example org.apache.spark.examples.streaming.JavaSqlNetworkWordCount localhost 9999` + */ + +public final class JavaSqlNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Create the context with a 1 second batch size + SparkConf sparkConf = new SparkConf().setAppName("JavaSqlNetworkWordCount"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1)); + + // Create a JavaReceiverInputDStream on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + JavaReceiverInputDStream lines = ssc.socketTextStream( + args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + + // Convert RDDs of the words DStream to DataFrame and run SQL query + words.foreachRDD(new Function2, Time, Void>() { + @Override + public Void call(JavaRDD rdd, Time time) { + SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + + // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame + JavaRDD rowRDD = rdd.map(new Function() { + public JavaRecord call(String word) { + JavaRecord record = new JavaRecord(); + record.setWord(word); + return record; + } + }); + DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); + + // Register as table + wordsDataFrame.registerTempTable("words"); + + // Do word count on table using SQL and print it + DataFrame wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word"); + System.out.println("========= " + time + "========="); + wordCountsDataFrame.show(); + return null; + } + }); + + ssc.start(); + ssc.awaitTermination(); + } +} + +/** Lazily instantiated singleton instance of SQLContext */ +class JavaSQLContextSingleton { + static private transient SQLContext instance = null; + static public SQLContext getInstance(SparkContext sparkContext) { + if (instance == null) { + instance = new SQLContext(sparkContext); + } + return instance; + } +} diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py new file mode 100644 index 0000000000000..f89bc562d856b --- /dev/null +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + network every second. + + Usage: sql_network_wordcount.py + and describe the TCP server that Spark Streaming would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` +""" + +import os +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.sql import SQLContext, Row + + +def getSqlContextInstance(sparkContext): + if ('sqlContextSingletonInstance' not in globals()): + globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) + return globals()['sqlContextSingletonInstance'] + + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: sql_network_wordcount.py " + exit(-1) + host, port = sys.argv[1:] + sc = SparkContext(appName="PythonSqlNetworkWordCount") + ssc = StreamingContext(sc, 1) + + # Create a socket stream on target ip:port and count the + # words in input stream of \n delimited text (eg. generated by 'nc') + lines = ssc.socketTextStream(host, int(port)) + words = lines.flatMap(lambda line: line.split(" ")) + + # Convert RDDs of the words DStream to DataFrame and run SQL query + def process(time, rdd): + print "========= %s =========" % str(time) + + try: + # Get the singleton instance of SQLContext + sqlContext = getSqlContextInstance(rdd.context) + + # Convert RDD[String] to RDD[Row] to DataFrame + rowRdd = rdd.map(lambda w: Row(word=w)) + wordsDataFrame = sqlContext.createDataFrame(rowRdd) + + # Register as table + wordsDataFrame.registerTempTable("words") + + # Do word count on table using SQL and print it + wordCountsDataFrame = \ + sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() + except: + pass + + words.foreachRDD(process) + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala new file mode 100644 index 0000000000000..5a6b9216a3fbc --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Time, Seconds, StreamingContext} +import org.apache.spark.util.IntParam +import org.apache.spark.sql.SQLContext +import org.apache.spark.storage.StorageLevel + +/** + * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + * network every second. + * + * Usage: SqlNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example org.apache.spark.examples.streaming.SqlNetworkWordCount localhost 9999` + */ + +object SqlNetworkWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println("Usage: NetworkWordCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + // Create the context with a 2 second batch size + val sparkConf = new SparkConf().setAppName("SqlNetworkWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) + + // Create a socket stream on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) + val words = lines.flatMap(_.split(" ")) + + // Convert RDDs of the words DStream to DataFrame and run SQL query + words.foreachRDD((rdd: RDD[String], time: Time) => { + // Get the singleton instance of SQLContext + val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) + import sqlContext.implicits._ + + // Convert RDD[String] to RDD[case class] to DataFrame + val wordsDataFrame = rdd.map(w => Record(w)).toDF() + + // Register as table + wordsDataFrame.registerTempTable("words") + + // Do word count on table using SQL and print it + val wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word") + println(s"========= $time =========") + wordCountsDataFrame.show() + }) + + ssc.start() + ssc.awaitTermination() + } +} + + +/** Case class for converting RDD to DataFrame */ +case class Record(word: String) + + +/** Lazily instantiated singleton instance of SQLContext */ +object SQLContextSingleton { + + @transient private var instance: SQLContext = _ + + def getInstance(sparkContext: SparkContext): SQLContext = { + if (instance == null) { + instance = new SQLContext(sparkContext) + } + instance + } +} From cd3b68d93a01f11bd3d5a441b341cb33d227e900 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 11 Mar 2015 18:48:21 -0700 Subject: [PATCH 390/817] [SPARK-6128][Streaming][Documentation] Updates to Spark Streaming Programming Guide Updates to the documentation are as follows: - Added information on Kafka Direct API and Kafka Python API - Added joins to the main streaming guide - Improved details on the fault-tolerance semantics Generated docs located here http://people.apache.org/~tdas/spark-1.3.0-temp-docs/streaming-programming-guide.html#fault-tolerance-semantics More things to add: - Configuration for Kafka receive rate - May be add concurrentJobs Author: Tathagata Das Closes #4956 from tdas/streaming-guide-update-1.3 and squashes the following commits: 819408c [Tathagata Das] Minor fixes. debe484 [Tathagata Das] Added DataFrames and MLlib 380cf8d [Tathagata Das] Fix link 04167a6 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-guide-update-1.3 0b77486 [Tathagata Das] Updates based on Josh's comments. 86c4c2a [Tathagata Das] Updated streaming guides 82de92a [Tathagata Das] Add Kafka to Python api docs --- docs/configuration.md | 14 +- docs/streaming-flume-integration.md | 2 + docs/streaming-kafka-integration.md | 151 +++++++-- docs/streaming-programming-guide.md | 470 +++++++++++++++++++++++----- 4 files changed, 528 insertions(+), 109 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index ae90fe1f8f6b9..a7116fbece9bb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1345,9 +1345,9 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.receiver.maxRate - infinite + not set - Maximum number records per second at which each receiver will receive data. + Maximum rate (number of records per second) at which each receiver will receive data. Effectively, each stream will consume at most this number of records per second. Setting this configuration to 0 or a negative number will put no limit on the rate. See the deployment guide @@ -1375,6 +1375,16 @@ Apart from these, the following properties are also available, and may be useful higher memory usage in Spark. + + spark.streaming.kafka.maxRatePerPartition + not set + + Maximum rate (number of records per second) at which data will be read from each Kafka + partition when using the new Kafka direct stream API. See the + Kafka Integration guide + for more details. + + #### Cluster Managers diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 40e17246fea83..c8ab146bcae0a 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -5,6 +5,8 @@ title: Spark Streaming + Flume Integration Guide [Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. +Python API Flume is not yet available in the Python API. + ## Approach 1: Flume-style Push-based Approach Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 77c0abbbacbd0..64714f0b799fc 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -2,58 +2,155 @@ layout: global title: Spark Streaming + Kafka Integration Guide --- -[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. +[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. -1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +## Approach 1: Receiver-based Approach +This approach uses a Receiver to receive the data. The Received is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. + +However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming. To ensure zero data loss, enable the Write Ahead Logs (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} -2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows. + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
    import org.apache.spark.streaming.kafka._ - val kafkaStream = KafkaUtils.createStream( - streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) + val kafkaStream = KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
    import org.apache.spark.streaming.kafka.*; - JavaPairReceiverInputDStream kafkaStream = KafkaUtils.createStream( - streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]); + JavaPairReceiverInputDStream kafkaStream = + KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); - See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + + kafkaStream = KafkaUtils.createStream(streamingContext, \ + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py).
    - *Points to remember:* + **Points to remember:** - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. -3. **Deploying:** Package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - -Note that the Kafka receiver used by default is an -[*unreliable* receiver](streaming-programming-guide.html#receiver-reliability) section in the -programming guide). In Spark 1.2, we have added an experimental *reliable* Kafka receiver that -provides stronger -[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) of zero -data loss on failures. This receiver is automatically used when the write ahead log -(also introduced in Spark 1.2) is enabled -(see [Deployment](#deploying-applications.html) section in the programming guide). This -may reduce the receiving throughput of individual Kafka receivers compared to the unreliable -receivers, but this can be corrected by running -[more receivers in parallel](streaming-programming-guide.html#level-of-parallelism-in-data-receiving) -to increase aggregate throughput. Additionally, it is recommended that the replication of the -received data within Spark be disabled when the write ahead log is enabled as the log is already stored -in a replicated storage system. This can be done by setting the storage level for the input -stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use + - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use `KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_2.10%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Direct Approach (No Receivers) +This is a new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature in Spark 1.3 and is only available in the Scala and Java API. + +This approach has the following advantages over the received-based approach (i.e. Approach 1). + +- *Simplified Parallelism:* No need to create multiple input Kafka streams and union-ing them. With `directStream`, Spark Streaming will create as many RDD partitions as there is Kafka partitions to consume, which will all read data from Kafka in parallel. So there is one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. + +- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminate the problem as there is no receiver, and hence no need for Write Ahead Logs. + +- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper and offsets tracked only by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. + +Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val directKafkaStream = KafkaUtils.createDirectStream[ + [key class], [value class], [key decoder class], [value decoder class] ]( + streamingContext, [map of Kafka parameters], [set of topics to consume]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream directKafkaStream = + KafkaUtils.createDirectStream(streamingContext, + [key class], [value class], [key decoder class], [value decoder class], + [map of Kafka parameters], [set of topics to consume]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). + +
    +
    + + In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. + By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. + + You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. + +
    +
    + directKafkaStream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges] + // offsetRanges.length = # of Kafka partitions being consumed + ... + } +
    +
    + directKafkaStream.foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws IOException { + OffsetRange[] offsetRanges = ((HasOffsetRanges)rdd).offsetRanges + // offsetRanges.length = # of Kafka partitions being consumed + ... + return null; + } + } + ); +
    +
    + + You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. + + Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate at which each Kafka partition will be read by this direct API. + +3. **Deploying:** Similar to the first approach, you can package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR and the launch the application using `spark-submit`. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. \ No newline at end of file diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 062ac2648db30..6d6229625f3f9 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -432,7 +432,7 @@ some of the common ones are as follows. For an up-to-date list, please refer to the -[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) +[Maven repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) for the full list of supported sources and artifacts. *** @@ -662,8 +662,7 @@ methods for creating DStreams from files and Akka actors as input sources. For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. - Python API As of Spark 1.2, - `fileStream` is not available in the Python API, only `textFileStream` is available. + Python API `fileStream` is not available in the Python API, only `textFileStream` is available. - **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver @@ -682,8 +681,9 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea ### Advanced Sources {:.no_toc} -Python API As of Spark 1.2, -these sources are not available in the Python API. + +Python API As of Spark 1.3, +out of these sources, *only* Kafka is available in the Python API. We will add more advanced sources in the Python API in future. This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts @@ -723,6 +723,12 @@ and it in the classpath. Some of these advanced sources are as follows. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.1.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. + +- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. + +- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. + - **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using [Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by @@ -732,17 +738,10 @@ Some of these advanced sources are as follows. ([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala) and [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)). -- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can received data from Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. - -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can receive data from Kafka 0.8.0. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. - -- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. - ### Custom Sources {:.no_toc} -Python API As of Spark 1.2, -these sources are not available in the Python API. +Python API This is not yet supported in Python. Input DStreams can also be created out of custom data sources. All you have to do is implement an user-defined **receiver** (see next section to understand what that is) that can receive data from @@ -846,7 +845,7 @@ Some of the common ones are as follows. -The last two transformations are worth highlighting again. +A few of these transformations are worth discussing in more detail. #### UpdateStateByKey Operation {:.no_toc} @@ -997,7 +996,7 @@ In fact, you can also use [machine learning](mllib-guide.html) and #### Window Operations {:.no_toc} -Finally, Spark Streaming also provides *windowed computations*, which allow you to apply +Spark Streaming also provides *windowed computations*, which allow you to apply transformations over a sliding window of data. This following figure illustrates this sliding window. @@ -1120,6 +1119,100 @@ said two parameters - windowLength and slideInterval. +#### Join Operations +{:.no_toc} +Finally, its worth highlighting how easily you can perform different kinds of joins in Spark Streaming. + + +##### Stream-stream joins +{:.no_toc} +Streams can be very easily joined with other streams. + +
    +
    +{% highlight scala %} +val stream1: DStream[String, String] = ... +val stream2: DStream[String, String] = ... +val joinedStream = stream1.join(stream2) +{% endhighlight %} +
    +
    +{% highlight java %} +JavaPairDStream stream1 = ... +JavaPairDStream stream2 = ... +JavaPairDStream joinedStream = stream1.join(stream2); +{% endhighlight %} +
    +
    +{% highlight python %} +stream1 = ... +stream2 = ... +joinedStream = stream1.join(stream2) +{% endhighlight %} +
    +
    +Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well. + +
    +
    +{% highlight scala %} +val windowedStream1 = stream1.window(Seconds(20)) +val windowedStream2 = stream2.window(Minutes(1)) +val joinedStream = windowedStream1.join(windowedStream2) +{% endhighlight %} +
    +
    +{% highlight java %} +JavaPairDStream windowedStream1 = stream1.window(Durations.seconds(20)); +JavaPairDStream windowedStream2 = stream2.window(Durations.minutes(1)); +JavaPairDStream joinedStream = windowedStream1.join(windowedStream2); +{% endhighlight %} +
    +
    +{% highlight python %} +windowedStream1 = stream1.window(20) +windowedStream2 = stream2.window(60) +joinedStream = windowedStream1.join(windowedStream2) +{% endhighlight %} +
    +
    + +##### Stream-dataset joins +{:.no_toc} +This has already been shown earlier while explain `DStream.transform` operation. Here is yet another example of joining a windowed stream with a dataset. + +
    +
    +{% highlight scala %} +val dataset: RDD[String, String] = ... +val windowedStream = stream.window(Seconds(20))... +val joinedStream = windowedStream.transform { rdd => rdd.join(dataset) } +{% endhighlight %} +
    +
    +{% highlight java %} +JavaPairRDD dataset = ... +JavaPairDStream windowedStream = stream.window(Durations.seconds(20)); +JavaPairDStream joinedStream = windowedStream.transform( + new Function>, JavaRDD>>() { + @Override + public JavaRDD> call(JavaRDD> rdd) { + return rdd.join(dataset); + } + } +); +{% endhighlight %} +
    +
    +{% highlight python %} +dataset = ... # some RDD +windowedStream = stream.window(20) +joinedStream = windowedStream.transform(lambda rdd: rdd.join(dataset)) +{% endhighlight %} +
    +
    + +In fact, you can also dynamically change the dataset you want to join against. The function provided to `transform` is evaluated every batch interval and therefore will use the current dataset that `dataset` reference points to. The complete list of DStream transformations is available in the API documentation. For the Scala API, see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) @@ -1327,6 +1420,178 @@ Note that the connections in the pool should be lazily created on demand and tim *** +## DataFrame and SQL Operations +You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SQLContext using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SQLContext. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL. + +
    +
    +{% highlight scala %} + +/** Lazily instantiated singleton instance of SQLContext */ +object SQLContextSingleton { + @transient private var instance: SQLContext = null + + // Instantiate SQLContext on demand + def getInstance(sparkContext: SparkContext): SQLContext = synchronized { + if (instance == null) { + instance = new SQLContext(sparkContext) + } + instance + } +} + +... + +/** Case class for converting RDD to DataFrame */ +case class Row(word: String) + +... + +/** DataFrame operations inside your streaming program */ + +val words: DStream[String] = ... + +words.foreachRDD { rdd => + + // Get the singleton instance of SQLContext + val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) + import sqlContext.implicits._ + + // Convert RDD[String] to RDD[case class] to DataFrame + val wordsDataFrame = rdd.map(w => Row(w)).toDF() + + // Register as table + wordsDataFrame.registerTempTable("words") + + // Do word count on DataFrame using SQL and print it + val wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() +} + +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala). +
    +
    +{% highlight java %} + +/** Lazily instantiated singleton instance of SQLContext */ +class JavaSQLContextSingleton { + static private transient SQLContext instance = null; + static public SQLContext getInstance(SparkContext sparkContext) { + if (instance == null) { + instance = new SQLContext(sparkContext); + } + return instance; + } +} + +... + +/** Java Bean class for converting RDD to DataFrame */ +public class JavaRow implements java.io.Serializable { + private String word; + + public String getWord() { + return word; + } + + public void setWord(String word) { + this.word = word; + } +} + +... + +/** DataFrame operations inside your streaming program */ + +JavaDStream words = ... + +words.foreachRDD( + new Function2, Time, Void>() { + @Override + public Void call(JavaRDD rdd, Time time) { + SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + + // Convert RDD[String] to RDD[case class] to DataFrame + JavaRDD rowRDD = rdd.map(new Function() { + public JavaRow call(String word) { + JavaRow record = new JavaRow(); + record.setWord(word); + return record; + } + }); + DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); + + // Register as table + wordsDataFrame.registerTempTable("words"); + + // Do word count on table using SQL and print it + DataFrame wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word"); + wordCountsDataFrame.show(); + return null; + } + } +); +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java). +
    +
    +{% highlight python %} + +# Lazily instantiated global instance of SQLContext +def getSqlContextInstance(sparkContext): + if ('sqlContextSingletonInstance' not in globals()): + globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) + return globals()['sqlContextSingletonInstance'] + +... + +# DataFrame operations inside your streaming program + +words = ... # DStream of strings + +def process(time, rdd): + print "========= %s =========" % str(time) + try: + # Get the singleton instance of SQLContext + sqlContext = getSqlContextInstance(rdd.context) + + # Convert RDD[String] to RDD[Row] to DataFrame + rowRdd = rdd.map(lambda w: Row(word=w)) + wordsDataFrame = sqlContext.createDataFrame(rowRdd) + + # Register as table + wordsDataFrame.registerTempTable("words") + + # Do word count on table using SQL and print it + wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() + except: + pass + +words.foreachRDD(process) +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/sql_network_wordcount.py). + +
    +
    + +You can also run SQL queries on tables defined on streaming data from a different thread (that is, asynchronous to the running StreamingContext). Just make sure that you set the StreamingContext to remember sufficient amount of streaming data such that query can run. Otherwise the StreamingContext, which is unaware of the any asynchronous SQL queries, will delete off old streaming data before the query can complete. For example, if you want to query the last batch, but your query can take 5 minutes to run, then call `streamingContext.remember(Minutes(5))` (in Scala, or equivalent in other languages). + +See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more about DataFrames. + +*** + +## MLlib Operations +You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. (Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. + +*** + ## Caching / Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream will automatically persist every RDD of that DStream in @@ -1580,9 +1845,8 @@ To run a Spark Streaming applications, you need to have the following. + *Mesos* - [Marathon](https://github.com/mesosphere/marathon) has been used to achieve this with Mesos. - -- *[Experimental in Spark 1.2] Configuring write ahead logs* - In Spark 1.2, - we have introduced a new experimental feature of write ahead logs for achieving strong +- *[Since Spark 1.2] Configuring write ahead logs* - Since Spark 1.2, + we have introduced _write ahead logs_ for achieving strong fault-tolerance guarantees. If enabled, all the data received from a receiver gets written into a write ahead log in the configuration checkpoint directory. This prevents data loss on driver recovery, thus ensuring zero data loss (discussed in detail in the @@ -1668,7 +1932,7 @@ improve the performance of you application. At a high level, you need to conside 2. Setting the right batch size such that the batches of data can be processed as fast as they are received (that is, data processing keeps up with the data ingestion). -## Reducing the Processing Time of each Batch +## Reducing the Batch Processing Times There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. @@ -1740,16 +2004,15 @@ documentation), or set the `spark.default.parallelism` ### Data Serialization {:.no_toc} -The overhead of data serialization can be significant, especially when sub-second batch sizes are - to be achieved. There are two aspects to it. +The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized. + +* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. -* **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data - serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default - RDDs are persisted as serialized byte arrays to minimize pauses related to GC. +* **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads. -* **Serialization of input data**: To ingest external data into Spark, data received as bytes - (say, from the network) needs to deserialized from bytes and re-serialized into Spark's - serialization format. Hence, the deserialization overhead of input data may be a bottleneck. +In both cases, using Kryo serialization can reduce both CPU and memory overheads. See the [Spark Tuning Guide](tuning.html#data-serialization)) for more details. Consider registering custom classes, and disabling object reference tracking for Kryo (see Kryo-related configurations in the [Configuration Guide](configuration.html#compression-and-serialization)). + +In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads. ### Task Launching Overheads {:.no_toc} @@ -1769,7 +2032,7 @@ thus allowing sub-second batch size to be viable. *** -## Setting the Right Batch Size +## Setting the Right Batch Interval For a Spark Streaming application running on a cluster to be stable, the system should be able to process data as fast as it is being received. In other words, batches of data should be processed as fast as they are being generated. Whether this is true for an application can be found by @@ -1801,40 +2064,40 @@ temporary data rate increases maybe fine as long as the delay reduces back to a ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail -in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, -we highlight a few customizations that are strongly recommended to minimize GC related pauses -in Spark Streaming applications and achieving more consistent batch processing times. - -* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams -serializes the data in memory (that is, -[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for -DStream compared to -[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). -Even though keeping the data serialized incurs higher serialization/deserialization overheads, -it significantly reduces GC pauses. - -* **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will - be cleared from memory based on Spark's built-in policy (LRU). If `spark.cleaner.ttl` is set, - then persistent RDDs that are older than that value are periodically cleared. As mentioned - [earlier](#operation), this needs to be careful set based on operations used in the Spark - Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the - [configuration property](configuration.html#spark-properties) `spark.streaming.unpersist` to - `true`. This makes the system to figure out which RDDs are not necessary to be kept around and - unpersists them. This is likely to reduce - the RDD memory usage of Spark, potentially improving GC behavior as well. - -* **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further -minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the +in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended that you read that. In this section, we discuss a few tuning parameters specifically in the context of Spark Streaming applications. + +The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low. + +In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. + +Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. + +There are a few parameters that can help you tune the memory usage and GC overheads. + +* **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. + +* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. +Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. + +* **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more -consistent batch processing times. +consistent batch processing times. Make sure you set the CMS GC on both the driver (using `--driver-java-options` in `spark-submit`) and the executors (using [Spark configuration](configuration.html#runtime-environment) `spark.executor.extraJavaOptions`). + +* **Other tips**: To further reduce GC overheads, here are some more tips to try. + - Use Tachyon for off-heap storage of persisted RDDs. See more detail in the [Spark Programming Guide](programming-guide.html#rdd-persistence). + - Use more executors with smaller heap sizes. This will reduce the GC pressure within each JVM heap. + *************************************************************************************************** *************************************************************************************************** # Fault-tolerance Semantics In this section, we will discuss the behavior of Spark Streaming applications in the event -of node failures. To understand this, let us remember the basic fault-tolerance semantics of -Spark's RDDs. +of failures. + +## Background +{:.no_toc} +To understand the semantics provided by Spark Streaming, let us remember the basic fault-tolerance semantics of Spark's RDDs. 1. An RDD is an immutable, deterministically re-computable, distributed dataset. Each RDD remembers the lineage of deterministic operations that were used on a fault-tolerant input @@ -1868,13 +2131,43 @@ Furthermore, there are two kinds of failures that we should be concerned about: With this basic knowledge, let us understand the fault-tolerance semantics of Spark Streaming. -## Semantics with files as input source +## Definitions +{:.no_toc} +The semantics of streaming systems are often captured in terms of how many times each record can be processed by the system. There are three types of guarantees that a system can provide under all possible operating conditions (despite failures, etc.) + +1. *At most once*: Each record will be either processed once or not processed at all. +2. *At least once*: Each record will be processed one or more times. This is stronger than *at-most once* as it ensure that no data will be lost. But there may be duplicates. +3. *Exactly once*: Each record will be processed exactly once - no data will be lost and no data will be processed multiple times. This is obviously the strongest guarantee of the three. + +## Basic Semantics +{:.no_toc} +In any stream processing system, broadly speaking, there are three steps in processing the data. + +1. *Receiving the data*: The data is received from sources using Receivers or otherwise. + +1. *Transforming the data*: The data received data is transformed using DStream and RDD transformations. + +1. *Pushing out the data*: The final transformed data is pushed out to external systems like file systems, databases, dashboards, etc. + +If a streaming application has to achieve end-to-end exactly-once guarantees, then each step has to provide exactly-once guarantee. That is, each record must be received exactly once, transformed exactly once, and pushed to downstream systems exactly once. Let's understand the semantics of these steps in the context of Spark Streaming. + +1. *Receiving the data*: Different input sources provided different guarantees. This is discussed in detail in the next subsection. + +1. *Transforming the data*: All data that has been received will be processed _exactly once_, thanks to the guarantees that RDDs provide. Even if there are failures, as long as the received input data is accessible, the final transformed RDDs will always have the same contents. + +1. *Pushing out the data*: Output operations by default ensure _at-least once_ semantics because it depends on the type of output operation (idempotent, or not) and the semantics of the downstream system (supports transactions or not). But users can implement their own transaction mechanisms to achieve _exactly-once_ semantics. This is discussed in more details later in the section. + +## Semantics of Received Data +{:.no_toc} +Different input sources provide different guarantees, ranging from _at-least once_ to _exactly once_. Read for more details. + +### With Files {:.no_toc} If all of the input data is already present in a fault-tolerant files system like HDFS, Spark Streaming can always recover from any failure and process all the data. This gives *exactly-once* semantics, that all the data will be processed exactly once no matter what fails. -## Semantics with input sources based on receivers +### With Receiver-based Sources {:.no_toc} For input sources based on receivers, the fault-tolerance semantics depend on both the failure scenario and the type of receiver. @@ -1893,10 +2186,9 @@ receivers, data received but not replicated can get lost. If the driver node fai then besides these losses, all the past data that was received and replicated in memory will be lost. This will affect the results of the stateful transformations. -To avoid this loss of past received data, Spark 1.2 introduces an experimental feature of _write +To avoid this loss of past received data, Spark 1.2 introduced _write ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs -enabled](#deploying-applications) and reliable receivers, there is zero data loss and -exactly-once semantics. +enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. The following table summarizes the semantics under failures: @@ -1908,23 +2200,30 @@ The following table summarizes the semantics under failures: - Spark 1.1 or earlier, or
    - Spark 1.2 without write ahead log + Spark 1.1 or earlier, OR
    + Spark 1.2 or later without write ahead logs Buffered data lost with unreliable receivers
    - Zero data loss with reliable receivers and files
    + Zero data loss with reliable receivers
    + At-least once semantics Buffered data lost with unreliable receivers
    Past data lost with all receivers
    - Zero data loss with files - + Undefined semantics + - Spark 1.2 with write ahead log - Zero data loss with reliable receivers and files - Zero data loss with reliable receivers and files + Spark 1.2 or later with write ahead logs + + Zero data loss with reliable receivers
    + At-least once semantics + + + Zero data loss with reliable receivers and files
    + At-least once semantics + @@ -1933,17 +2232,24 @@ The following table summarizes the semantics under failures: +### With Kafka Direct API +{:.no_toc} +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark 1.3) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). + ## Semantics of output operations {:.no_toc} -Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation - always leads to the same result. As a result, all DStream transformations are guaranteed to have - _exactly-once_ semantics. That is, the final transformed result will be same even if there were - was a worker node failure. However, output operations (like `foreachRDD`) have _at-least once_ - semantics, that is, the transformed data may get written to an external entity more than once in - the event of a worker failure. While this is acceptable for saving to HDFS using the - `saveAs***Files` operations (as the file will simply get over-written by the same data), - additional transactions-like mechanisms may be necessary to achieve exactly-once semantics - for output operations. +Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, +the transformed data may get written to an external entity more than once in +the event of a worker failure. While this is acceptable for saving to file systems using the +`saveAs***Files` operations (as the file will simply get overwritten with the same data), +additional effort may be necessary to achieve exactly-once semantics. There are two approaches. + +- *Idempotent updates*: Multiple attempts always write the same data. For example, `saveAs***Files` always writes the same data to the generated files. + +- *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following. + + - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application. + - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. *************************************************************************************************** @@ -2001,7 +2307,11 @@ package and renamed for better clarity. *************************************************************************************************** # Where to Go from Here - +* Additional guides + - [Kafka Integration Guide](streaming-kafka-integration.html) + - [Flume Integration Guide](streaming-flume-integration.html) + - [Kinesis Integration Guide](streaming-kinesis-integration.html) + - [Custom Receiver Guide](streaming-custom-receivers.html) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and @@ -2023,8 +2333,8 @@ package and renamed for better clarity. [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) - Python docs - * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) - * [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) + * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) + * [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) From e921a665c56950c03155f4b47500627265a4ba8e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 11 Mar 2015 22:24:08 -0700 Subject: [PATCH 391/817] BUILD: Adding more known contributor names --- dev/create-release/known_translations | 34 +++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index b74e4ee8a330b..0a599b5a65549 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -57,3 +57,37 @@ watermen - Yadong Qi witgo - Guoqiang Li xinyunh - Xinyun Huang zsxwing - Shixiong Zhu +Bilna - Bilna P +DoingDone9 - Doing Done +Earne - Ernest +FlytxtRnD - Meethu Mathew +GenTang - Gen TANG +JoshRosen - Josh Rosen +MechCoder - Manoj Kumar +OopsOutOfMemory - Sheng Li +Peishen-Jia - Peishen Jia +SaintBacchus - Huang Zhaowei +azagrebin - Andrey Zagrebin +bzz - Alexander Bezzubov +fjiang6 - Fan Jiang +gasparms - Gaspar Munoz +guowei2 - Guo Wei +hhbyyh - Yuhao Yang +hseagle - Peng Xu +javadba - Stephen Boesch +jbencook - Ben Cook +kul - Kuldeep +ligangty - Gang Li +marsishandsome - Liangliang Gu +medale - Markus Dale +nemccarthy - Nathan McCarthy +nxwhite-str - Nate Crosswhite +seayi - Xiaohua Yi +tianyi - Yi Tian +uncleGen - Uncle Gen +viper-kun - Xu Kun +x1- - Yuri Saito +zapletal-martin - Martin Zapletal +zuxqoj - Shekhar Bansal +mingyukim - Mingyu Kim +sigmoidanalytics - Mayur Rustagi From 25b71d8c15572f0f2b951c827c169f8c65f726ad Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Thu, 12 Mar 2015 00:55:26 -0700 Subject: [PATCH 392/817] [SPARK-6296] [SQL] Added equals to Column Author: Volodymyr Lyubinets Closes #4988 from vlyubin/columncomp and squashes the following commits: 92d7c8f [Volodymyr Lyubinets] Added equals to Column --- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 7 +++++++ .../scala/org/apache/spark/sql/ColumnExpressionSuite.scala | 5 +++++ 2 files changed, 12 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index a2cc9a9b93eb8..908c78a4d3f10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -59,6 +59,13 @@ class Column(protected[sql] val expr: Expression) { override def toString: String = expr.prettyString + override def equals(that: Any) = that match { + case that: Column => that.expr.equals(this.expr) + case _ => false + } + + override def hashCode: Int = this.expr.hashCode + /** * Unary minus, i.e. negate the expression. * {{{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 37c02aaa5460b..3036fbc05d021 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -313,4 +313,9 @@ class ColumnExpressionSuite extends QueryTest { test("lift alias out of cast") { assert(col("1234").as("name").cast("int").expr === col("1234").cast("int").as("name").expr) } + + test("columns can be compared") { + assert('key.desc == 'key.desc) + assert('key.desc != 'key.asc) + } } From 712679a7b447346a365b38574d7a86d56a93f767 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 12 Mar 2015 01:34:38 -0700 Subject: [PATCH 393/817] [SPARK-6294] fix hang when call take() in JVM on PythonRDD The Thread.interrupt() can not terminate the thread in some cases, so we should not wait for the writerThread of PythonRDD. This PR also ignore some exception during clean up. cc JoshRosen mengxr Author: Davies Liu Closes #4987 from davies/fix_take and squashes the following commits: 4488f1a [Davies Liu] fix hang when call take() in JVM on PythonRDD --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 9 ++++++--- python/pyspark/daemon.py | 5 ++++- python/pyspark/tests.py | 5 +++++ 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8d4a53b4ca9b0..4c71b69069eb3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -76,7 +76,6 @@ private[spark] class PythonRDD( context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() - writerThread.join() if (!reuse_worker || !released) { try { worker.close() @@ -248,13 +247,17 @@ private[spark] class PythonRDD( } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index f09587f211708..93885985fe377 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -61,7 +61,10 @@ def worker(sock): except SystemExit as exc: exit_code = compute_real_exit_code(exc.code) finally: - outfile.flush() + try: + outfile.flush() + except Exception: + pass return exit_code diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 06ba2b461d53e..dd8d3b1c53733 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -782,6 +782,11 @@ def test_narrow_dependency_in_join(self): jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) + # Regression test for SPARK-6294 + def test_take_on_jrdd(self): + rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) + rdd._jrdd.first() + class ProfilerTests(PySparkTestCase): From 0cba802adf15f5ab8da24dd1e8a5e7214cc4e148 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 12 Mar 2015 01:39:04 -0700 Subject: [PATCH 394/817] [SPARK-5814][MLLIB][GRAPHX] Remove JBLAS from runtime The issue is discussed in https://issues.apache.org/jira/browse/SPARK-5669. Replacing all JBLAS usage by netlib-java gives us a simpler dependency tree and less license issues to worry about. I didn't touch the test scope in this PR. The user guide is not modified to avoid merge conflicts with branch-1.3. srowen ankurdave pwendell Author: Xiangrui Meng Closes #4699 from mengxr/SPARK-5814 and squashes the following commits: 48635c6 [Xiangrui Meng] move netlib-java version to parent pom ca21c74 [Xiangrui Meng] remove jblas from ml-guide 5f7767a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5814 c5c4183 [Xiangrui Meng] merge master 0f20cad [Xiangrui Meng] add mima excludes e53e9f4 [Xiangrui Meng] remove jblas from mllib runtime ceaa14d [Xiangrui Meng] replace jblas by netlib-java in graphx fa7c2ca [Xiangrui Meng] move jblas to test scope --- assembly/pom.xml | 10 -- docs/mllib-guide.md | 5 - graphx/pom.xml | 11 ++- .../apache/spark/graphx/lib/SVDPlusPlus.scala | 96 ++++++++++++------- .../spark/graphx/lib/SVDPlusPlusSuite.scala | 6 +- mllib/pom.xml | 3 +- .../apache/spark/ml/recommendation/ALS.scala | 14 ++- .../spark/mllib/optimization/NNLS.scala | 86 +++++++++-------- .../MatrixFactorizationModel.scala | 15 ++- .../mllib/util/LinearDataGenerator.scala | 9 +- .../spark/mllib/util/MFDataGenerator.scala | 26 ++--- .../spark/mllib/util/SVMDataGenerator.scala | 7 +- .../spark/mllib/optimization/NNLSSuite.scala | 6 +- .../spark/mllib/stat/KernelDensitySuite.scala | 4 +- pom.xml | 1 + project/MimaExcludes.scala | 28 ++++++ 16 files changed, 183 insertions(+), 144 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index cbf5b6c4aa8df..d3bb4bde0c412 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -114,16 +114,6 @@ META-INF/*.RSA - - - org.jblas:jblas - - - lib/static/Linux/i386/** - lib/static/Mac OS X/** - lib/static/Windows/** - - diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4c7a7d9115ca1..598374f66df5e 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -80,11 +80,6 @@ include `netlib-java`'s native proxies by default. To configure [netlib-java](https://github.com/fommil/netlib-java) documentation for your platform's additional installation instructions. -MLlib also uses [jblas](https://github.com/mikiobraun/jblas) which -will require you to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. - To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. diff --git a/graphx/pom.xml b/graphx/pom.xml index 57e338c03ecf9..c0d534e185d7f 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -45,9 +45,14 @@ guava - org.jblas - jblas - ${jblas.version} + com.github.fommil.netlib + core + ${netlib.java.version} + + + net.sourceforge.f2j + arpack_combined_all + 0.1 org.scalacheck diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 3e4157a63fd1c..1a7178b82e3af 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -18,7 +18,9 @@ package org.apache.spark.graphx.lib import scala.util.Random -import org.jblas.DoubleMatrix + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + import org.apache.spark.rdd._ import org.apache.spark.graphx._ @@ -53,7 +55,7 @@ object SVDPlusPlus { * a Multifaceted Collaborative Filtering Model", * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. * - * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)), * see the details on page 6. * * @param edges edges for constructing the graph @@ -66,13 +68,10 @@ object SVDPlusPlus { : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { // Generate default vertex attribute - def defaultF(rank: Int): (DoubleMatrix, DoubleMatrix, Double, Double) = { - val v1 = new DoubleMatrix(rank) - val v2 = new DoubleMatrix(rank) - for (i <- 0 until rank) { - v1.put(i, Random.nextDouble()) - v2.put(i, Random.nextDouble()) - } + def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = { + // TODO: use a fixed random seed + val v1 = Array.fill(rank)(Random.nextDouble()) + val v2 = Array.fill(rank)(Random.nextDouble()) (v1, v2, 0.0, 0.0) } @@ -92,7 +91,7 @@ object SVDPlusPlus { (g1, g2) => (g1._1 + g2._1, g1._2 + g2._2)) val gJoinT0 = g.outerJoinVertices(t0) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() @@ -102,24 +101,28 @@ object SVDPlusPlus { def sendMsgTrainF(conf: Conf, u: Double) (ctx: EdgeContext[ - (DoubleMatrix, DoubleMatrix, Double, Double), + (Array[Double], Array[Double], Double, Double), Double, - (DoubleMatrix, DoubleMatrix, Double)]) { + (Array[Double], Array[Double], Double)]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + val rank = p.length + var pred = u + usr._3 + itm._3 + blas.ddot(rank, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = ctx.attr - pred - val updateP = q.mul(err) - .subColumnVector(p.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateQ = usr._2.mul(err) - .subColumnVector(q.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateY = q.mul(err * usr._4) - .subColumnVector(itm._2.mul(conf.gamma7)) - .mul(conf.gamma2) + // updateP = (err * q - conf.gamma7 * p) * conf.gamma2 + val updateP = q.clone() + blas.dscal(rank, err * conf.gamma2, updateP, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, p, 1, updateP, 1) + // updateQ = (err * usr._2 - conf.gamma7 * q) * conf.gamma2 + val updateQ = usr._2.clone() + blas.dscal(rank, err * conf.gamma2, updateQ, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, q, 1, updateQ, 1) + // updateY = (err * usr._4 * q - conf.gamma7 * itm._2) * conf.gamma2 + val updateY = q.clone() + blas.dscal(rank, err * usr._4 * conf.gamma2, updateY, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, itm._2, 1, updateY, 1) ctx.sendToSrc((updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)) ctx.sendToDst((updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1)) } @@ -127,14 +130,23 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes g.cache() - val t1 = g.aggregateMessages[DoubleMatrix]( + val t1 = g.aggregateMessages[Array[Double]]( ctx => ctx.sendToSrc(ctx.dstAttr._2), - (g1, g2) => g1.addColumnVector(g2)) + (g1, g2) => { + val out = g1.clone() + blas.daxpy(out.length, 1.0, g2, 1, out, 1) + out + }) val gJoinT1 = g.outerJoinVertices(t1) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[DoubleMatrix]) => - if (msg.isDefined) (vd._1, vd._1 - .addColumnVector(msg.get.mul(vd._4)), vd._3, vd._4) else vd + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), + msg: Option[Array[Double]]) => + if (msg.isDefined) { + val out = vd._1.clone() + blas.daxpy(out.length, vd._4, msg.get, 1, out, 1) + (vd._1, out, vd._3, vd._4) + } else { + vd + } }.cache() materialize(gJoinT1) g.unpersist() @@ -144,14 +156,24 @@ object SVDPlusPlus { g.cache() val t2 = g.aggregateMessages( sendMsgTrainF(conf, u), - (g1: (DoubleMatrix, DoubleMatrix, Double), g2: (DoubleMatrix, DoubleMatrix, Double)) => - (g1._1.addColumnVector(g2._1), g1._2.addColumnVector(g2._2), g1._3 + g2._3)) + (g1: (Array[Double], Array[Double], Double), g2: (Array[Double], Array[Double], Double)) => + { + val out1 = g1._1.clone() + blas.daxpy(out1.length, 1.0, g2._1, 1, out1, 1) + val out2 = g2._2.clone() + blas.daxpy(out2.length, 1.0, g2._2, 1, out2, 1) + (out1, out2, g1._3 + g2._3) + }) val gJoinT2 = g.outerJoinVertices(t2) { (vid: VertexId, - vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[(DoubleMatrix, DoubleMatrix, Double)]) => - (vd._1.addColumnVector(msg.get._1), vd._2.addColumnVector(msg.get._2), - vd._3 + msg.get._3, vd._4) + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[(Array[Double], Array[Double], Double)]) => { + val out1 = vd._1.clone() + blas.daxpy(out1.length, 1.0, msg.get._1, 1, out1, 1) + val out2 = vd._2.clone() + blas.daxpy(out2.length, 1.0, msg.get._2, 1, out2, 1) + (out1, out2, vd._3 + msg.get._3, vd._4) + } }.cache() materialize(gJoinT2) g.unpersist() @@ -160,10 +182,10 @@ object SVDPlusPlus { // calculate error on training set def sendMsgTestF(conf: Conf, u: Double) - (ctx: EdgeContext[(DoubleMatrix, DoubleMatrix, Double, Double), Double, Double]) { + (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + var pred = u + usr._3 + itm._3 + blas.ddot(q.length, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = (ctx.attr - pred) * (ctx.attr - pred) @@ -173,7 +195,7 @@ object SVDPlusPlus { g.cache() val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _) val gJoinT3 = g.outerJoinVertices(t3) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[Double]) => + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd }.cache() materialize(gJoinT3) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index 9987a4b1a3c25..7bd6b7f3c4ab2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -32,11 +32,11 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = SVDPlusPlus.runSVDPlusPlus(edges, conf) + val (graph, _) = SVDPlusPlus.run(edges, conf) graph.cache() - val err = graph.vertices.collect().map{ case (vid, vd) => + val err = graph.vertices.map { case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 - }.reduce(_ + _) / graph.triplets.collect().size + }.reduce(_ + _) / graph.numEdges assert(err <= svdppErr) } } diff --git a/mllib/pom.xml b/mllib/pom.xml index b5c949e155cfd..a76704a8c2c59 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -59,6 +59,7 @@ org.jblas jblas ${jblas.version} + test org.scalanlp @@ -116,7 +117,7 @@ com.github.fommil.netlib all - 1.1.2 + ${netlib.java.version} pom diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 7bb69df65362b..e3515ee81af3d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -26,7 +26,6 @@ import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} -import org.jblas.DoubleMatrix import org.netlib.util.intW import org.apache.spark.{Logging, Partitioner} @@ -361,14 +360,14 @@ object ALS extends Logging { private[recommendation] class NNLSSolver extends LeastSquaresNESolver { private var rank: Int = -1 private var workspace: NNLS.Workspace = _ - private var ata: DoubleMatrix = _ + private var ata: Array[Double] = _ private var initialized: Boolean = false private def initialize(rank: Int): Unit = { if (!initialized) { this.rank = rank workspace = NNLS.createWorkspace(rank) - ata = new DoubleMatrix(rank, rank) + ata = new Array[Double](rank * rank) initialized = true } else { require(this.rank == rank) @@ -385,7 +384,7 @@ object ALS extends Logging { val rank = ne.k initialize(rank) fillAtA(ne.ata, lambda * ne.n) - val x = NNLS.solve(ata, new DoubleMatrix(rank, 1, ne.atb: _*), workspace) + val x = NNLS.solve(ata, ne.atb, workspace) ne.reset() x.map(x => x.toFloat) } @@ -398,17 +397,16 @@ object ALS extends Logging { var i = 0 var pos = 0 var a = 0.0 - val data = ata.data while (i < rank) { var j = 0 while (j <= i) { a = triAtA(pos) - data(i * rank + j) = a - data(j * rank + i) = a + ata(i * rank + j) = a + ata(j * rank + i) = a pos += 1 j += 1 } - data(i * rank + i) += lambda + ata(i * rank + i) += lambda i += 1 } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index ccd93b318bc23..4766f7708295d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -17,7 +17,9 @@ package org.apache.spark.mllib.optimization -import org.jblas.{DoubleMatrix, SimpleBlas} +import java.{util => ju} + +import com.github.fommil.netlib.BLAS.{getInstance => blas} /** * Object used to solve nonnegative least squares problems using a modified @@ -25,20 +27,20 @@ import org.jblas.{DoubleMatrix, SimpleBlas} */ private[spark] object NNLS { class Workspace(val n: Int) { - val scratch = new DoubleMatrix(n, 1) - val grad = new DoubleMatrix(n, 1) - val x = new DoubleMatrix(n, 1) - val dir = new DoubleMatrix(n, 1) - val lastDir = new DoubleMatrix(n, 1) - val res = new DoubleMatrix(n, 1) - - def wipe() { - scratch.fill(0.0) - grad.fill(0.0) - x.fill(0.0) - dir.fill(0.0) - lastDir.fill(0.0) - res.fill(0.0) + val scratch = new Array[Double](n) + val grad = new Array[Double](n) + val x = new Array[Double](n) + val dir = new Array[Double](n) + val lastDir = new Array[Double](n) + val res = new Array[Double](n) + + def wipe(): Unit = { + ju.Arrays.fill(scratch, 0.0) + ju.Arrays.fill(grad, 0.0) + ju.Arrays.fill(x, 0.0) + ju.Arrays.fill(dir, 0.0) + ju.Arrays.fill(lastDir, 0.0) + ju.Arrays.fill(res, 0.0) } } @@ -60,18 +62,18 @@ private[spark] object NNLS { * direction, however, while this method only uses a conjugate gradient direction if the last * iteration did not cause a previously-inactive constraint to become active. */ - def solve(ata: DoubleMatrix, atb: DoubleMatrix, ws: Workspace): Array[Double] = { + def solve(ata: Array[Double], atb: Array[Double], ws: Workspace): Array[Double] = { ws.wipe() - val n = atb.rows + val n = atb.length val scratch = ws.scratch // find the optimal unconstrained step - def steplen(dir: DoubleMatrix, res: DoubleMatrix): Double = { - val top = SimpleBlas.dot(dir, res) - SimpleBlas.gemv(1.0, ata, dir, 0.0, scratch) + def steplen(dir: Array[Double], res: Array[Double]): Double = { + val top = blas.ddot(n, dir, 1, res, 1) + blas.dgemv("N", n, n, 1.0, ata, n, dir, 1, 0.0, scratch, 1) // Push the denominator upward very slightly to avoid infinities and silliness - top / (SimpleBlas.dot(scratch, dir) + 1e-20) + top / (blas.ddot(n, scratch, 1, dir, 1) + 1e-20) } // stopping condition @@ -96,52 +98,52 @@ private[spark] object NNLS { var i = 0 while (iterno < iterMax) { // find the residual - SimpleBlas.gemv(1.0, ata, x, 0.0, res) - SimpleBlas.axpy(-1.0, atb, res) - SimpleBlas.copy(res, grad) + blas.dgemv("N", n, n, 1.0, ata, n, x, 1, 0.0, res, 1) + blas.daxpy(n, -1.0, atb, 1, res, 1) + blas.dcopy(n, res, 1, grad, 1) // project the gradient i = 0 while (i < n) { - if (grad.data(i) > 0.0 && x.data(i) == 0.0) { - grad.data(i) = 0.0 + if (grad(i) > 0.0 && x(i) == 0.0) { + grad(i) = 0.0 } i = i + 1 } - val ngrad = SimpleBlas.dot(grad, grad) + val ngrad = blas.ddot(n, grad, 1, grad, 1) - SimpleBlas.copy(grad, dir) + blas.dcopy(n, grad, 1, dir, 1) // use a CG direction under certain conditions var step = steplen(grad, res) var ndir = 0.0 - val nx = SimpleBlas.dot(x, x) + val nx = blas.ddot(n, x, 1, x, 1) if (iterno > lastWall + 1) { val alpha = ngrad / lastNorm - SimpleBlas.axpy(alpha, lastDir, dir) + blas.daxpy(n, alpha, lastDir, 1, dir, 1) val dstep = steplen(dir, res) - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) if (stop(dstep, ndir, nx)) { // reject the CG step if it could lead to premature termination - SimpleBlas.copy(grad, dir) - ndir = SimpleBlas.dot(dir, dir) + blas.dcopy(n, grad, 1, dir, 1) + ndir = blas.ddot(n, dir, 1, dir, 1) } else { step = dstep } } else { - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) } // terminate? if (stop(step, ndir, nx)) { - return x.data.clone + return x.clone } // don't run through the walls i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i)) { - step = x.data(i) / dir.data(i) + if (step * dir(i) > x(i)) { + step = x(i) / dir(i) } i = i + 1 } @@ -149,19 +151,19 @@ private[spark] object NNLS { // take the step i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i) * (1 - 1e-14)) { - x.data(i) = 0 + if (step * dir(i) > x(i) * (1 - 1e-14)) { + x(i) = 0 lastWall = iterno } else { - x.data(i) -= step * dir.data(i) + x(i) -= step * dir(i) } i = i + 1 } iterno = iterno + 1 - SimpleBlas.copy(dir, lastDir) + blas.dcopy(n, dir, 1, lastDir, 1) lastNorm = ngrad } - x.data.clone + x.clone } } 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 5f5a996a87b81..36cbf060d9998 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 @@ -21,10 +21,10 @@ import java.io.IOException import java.lang.{Integer => JavaInteger} import org.apache.hadoop.fs.Path -import org.jblas.DoubleMatrix import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} @@ -70,9 +70,9 @@ class MatrixFactorizationModel( /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { - val userVector = new DoubleMatrix(userFeatures.lookup(user).head) - val productVector = new DoubleMatrix(productFeatures.lookup(product).head) - userVector.dot(productVector) + val userVector = userFeatures.lookup(user).head + val productVector = productFeatures.lookup(product).head + blas.ddot(userVector.length, userVector, 1, productVector, 1) } /** @@ -89,9 +89,7 @@ class MatrixFactorizationModel( } users.join(productFeatures).map { case (product, ((user, uFeatures), pFeatures)) => - val userVector = new DoubleMatrix(uFeatures) - val productVector = new DoubleMatrix(pFeatures) - Rating(user, product, userVector.dot(productVector)) + Rating(user, product, blas.ddot(uFeatures.length, uFeatures, 1, pFeatures, 1)) } } @@ -143,9 +141,8 @@ class MatrixFactorizationModel( recommendToFeatures: Array[Double], recommendableFeatures: RDD[(Int, Array[Double])], num: Int): Array[(Int, Double)] = { - val recommendToVector = new DoubleMatrix(recommendToFeatures) val scored = recommendableFeatures.map { case (id,features) => - (id, recommendToVector.dot(new DoubleMatrix(features))) + (id, blas.ddot(features.length, recommendToFeatures, 1, features, 1)) } scored.top(num)(Ordering.by(_._2)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 97f54aa62d31c..c9d33787b0bb5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.util import scala.collection.JavaConversions._ import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -72,11 +72,10 @@ object LinearDataGenerator { eps: Double = 0.1): Seq[LabeledPoint] = { val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) val y = x.map { xi => - new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian() + blas.ddot(weights.length, xi, 1, weights, 1) + intercept + eps * rnd.nextGaussian() } y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } @@ -100,9 +99,9 @@ object LinearDataGenerator { eps: Double, nparts: Int = 2, intercept: Double = 0.0) : RDD[LabeledPoint] = { - org.jblas.util.Random.seed(42) + val random = new Random(42) // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + val w = Array.fill(nfeatures)(random.nextDouble() - 0.5) val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => val seed = 42 + p diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index b76fbe89c3681..0c5b4f9d04a74 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.util +import java.{util => ju} + import scala.language.postfixOps import scala.util.Random -import org.jblas.DoubleMatrix - -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD /** @@ -72,24 +73,25 @@ object MFDataGenerator { val sc = new SparkContext(sparkMaster, "MFDataGenerator") - val A = DoubleMatrix.randn(m, rank) - val B = DoubleMatrix.randn(rank, n) - val z = 1 / scala.math.sqrt(scala.math.sqrt(rank)) - A.mmuli(z) - B.mmuli(z) - val fullData = A.mmul(B) + val random = new ju.Random(42L) + + val A = DenseMatrix.randn(m, rank, random) + val B = DenseMatrix.randn(rank, n, random) + val z = 1 / math.sqrt(rank) + val fullData = DenseMatrix.zeros(m, n) + BLAS.gemm(z, A, B, 1.0, fullData) val df = rank * (m + n - rank) val sampSize = scala.math.min(scala.math.round(trainSampFact * df), scala.math.round(.99 * m * n)).toInt val rand = new Random() val mn = m * n - val shuffled = rand.shuffle(1 to mn toList) + val shuffled = rand.shuffle((0 until mn).toList) val omega = shuffled.slice(0, sampSize) val ordered = omega.sortWith(_ < _).toArray val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) // optionally add gaussian noise if (noise) { @@ -105,7 +107,7 @@ object MFDataGenerator { val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) val testOrdered = testOmega.sortWith(_ < _).toArray val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 7db97e6bac688..a8e30cc9d730c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -51,8 +51,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures + 1, - Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) + val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -60,7 +59,7 @@ object SVMDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1 + val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 val y = if (yD < 0) 0.0 else 1.0 LabeledPoint(y, Vectors.dense(x)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index 82c327bd49fcd..22855e4e8f247 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -55,7 +55,7 @@ class NNLSSuite extends FunSuite { for (k <- 0 until 100) { val (ata, atb) = genOnesData(n, rand) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) assert(x.length === n) val answer = DoubleMatrix.ones(n, 1) SimpleBlas.axpy(-1.0, answer, x) @@ -79,7 +79,7 @@ class NNLSSuite extends FunSuite { val goodx = Array(0.13025, 0.54506, 0.2874, 0.0, 0.028628) val ws = NNLS.createWorkspace(n) - val x = NNLS.solve(ata, atb, ws) + val x = NNLS.solve(ata.data, atb.data, ws) for (i <- 0 until n) { assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) @@ -104,7 +104,7 @@ class NNLSSuite extends FunSuite { val ws = NNLS.createWorkspace(n) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) val obj = computeObjectiveValue(ata, atb, x) assert(obj < refObj + 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala index f6a1e19f50296..16ecae23dd9d4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -21,9 +21,9 @@ import org.scalatest.FunSuite import org.apache.commons.math3.distribution.NormalDistribution -import org.apache.spark.mllib.util.LocalClusterSparkContext +import org.apache.spark.mllib.util.MLlibTestSparkContext -class KernelDensitySuite extends FunSuite with LocalClusterSparkContext { +class KernelDensitySuite extends FunSuite with MLlibTestSparkContext { test("kernel density single sample") { val rdd = sc.parallelize(Array(5.0)) val evaluationPoints = Array(5.0, 6.0) diff --git a/pom.xml b/pom.xml index a19da73cf45b3..6fc56a86d44ac 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,7 @@ 1.8.8 2.4.4 1.1.1.6 + 1.1.2 [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5001) Author: Cheng Lian Closes #5001 from liancheng/parquet-doc and squashes the following commits: 89ad3db [Cheng Lian] Addresses @rxin's comments 7eb6955 [Cheng Lian] Docs for the new Parquet data source 415eefb [Cheng Lian] Some minor formatting improvements --- docs/sql-programming-guide.md | 237 ++++++++++++++++++++++++++-------- 1 file changed, 180 insertions(+), 57 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 76aa1a533d56e..11c29e20632ae 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -21,14 +21,14 @@ The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark. All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell` or the `pyspark` shell. -## Starting Point: SQLContext +## Starting Point: `SQLContext`
    The entry point into all functionality in Spark SQL is the -[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/scala/index.html#org.apache.spark.sql.`SQLContext`) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -43,8 +43,8 @@ import sqlContext.implicits._
    The entry point into all functionality in Spark SQL is the -[SQLContext](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. @@ -56,8 +56,8 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
    The entry point into all relational functionality in Spark is the -[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one -of its decedents. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight python %} from pyspark.sql import SQLContext @@ -67,20 +67,20 @@ sqlContext = SQLContext(sc)
    -In addition to the basic SQLContext, you can also create a HiveContext, which provides a -superset of the functionality provided by the basic SQLContext. Additional features include +In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a +superset of the functionality provided by the basic `SQLContext`. Additional features include the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the -ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing Hive setup, and all of the data sources available to a SQLContext are still available. -HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.3 release of Spark. Future releases will focus on bringing SQLContext up -to feature parity with a HiveContext. +ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an +existing Hive setup, and all of the data sources available to a `SQLContext` are still available. +`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using `HiveContext` +is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up +to feature parity with a `HiveContext`. The specific variant of SQL that is used to parse queries can also be selected using the `spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on -a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect -available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the +a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect +available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, this is recommended for most use cases. @@ -100,7 +100,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Displays the content of the DataFrame to stdout -df.show() +df.show() {% endhighlight %}
    @@ -151,10 +151,10 @@ val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Show the content of the DataFrame df.show() -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema() @@ -164,17 +164,17 @@ df.printSchema() // Select only the "name" column df.select("name").show() -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 df.select("name", df("age") + 1).show() // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 df.filter(df("name") > 21).show() @@ -201,10 +201,10 @@ DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); // Show the content of the DataFrame df.show(); -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema(); @@ -214,17 +214,17 @@ df.printSchema(); // Select only the "name" column df.select("name").show(); -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 df.select("name", df.col("age").plus(1)).show(); // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 df.filter(df("name") > 21).show(); @@ -251,10 +251,10 @@ df = sqlContext.jsonFile("examples/src/main/resources/people.json") # Show the content of the DataFrame df.show() -## age name +## age name ## null Michael -## 30 Andy -## 19 Justin +## 30 Andy +## 19 Justin # Print the schema in a tree format df.printSchema() @@ -264,17 +264,17 @@ df.printSchema() # Select only the "name" column df.select("name").show() -## name +## name ## Michael -## Andy -## Justin +## Andy +## Justin # Select everybody, but increment the age by 1 df.select("name", df.age + 1).show() ## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 +## Michael null +## Andy 31 +## Justin 20 # Select people older than 21 df.filter(df.name > 21).show() @@ -797,7 +797,7 @@ When working with a `HiveContext`, `DataFrames` can also be saved as persistent contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables will still exist even after your Spark program has restarted, as long as you maintain your connection to the same metastore. A DataFrame for a persistent table can be created by calling the `table` -method on a SQLContext with the name of the table. +method on a `SQLContext` with the name of the table. By default `saveAsTable` will create a "managed table", meaning that the location of the data will be controlled by the metastore. Managed tables will also have their data deleted automatically @@ -907,9 +907,132 @@ SELECT * FROM parquetTable
    +### Partition discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. The Parquet data source is now able to discover and infer +partitioning information automatically. For exmaple, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SQLContext.parquetFile` or `SQLContext.load`, Spark SQL will +automatically extract the partitioning information from the paths. Now the schema of the returned +DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types and string type are supported. + +### Schema merging + +Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +
    + +
    + +{% highlight scala %} +// sqlContext from the previous example is used in this example. +// This is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ + +// Create a simple DataFrame, stored into a partition directory +val df1 = sparkContext.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") +df1.saveAsParquetFile("data/test_table/key=1") + +// Create another DataFrame in a new partition directory, +// adding a new column and dropping an existing column +val df2 = sparkContext.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") +df2.saveAsParquetFile("data/test_table/key=2") + +// Read the partitioned table +val df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +// The final schema consists of all 3 columns in the Parquet files together +// with the partiioning column appeared in the partition directory paths. +// root +// |-- single: int (nullable = true) +// |-- double: int (nullable = true) +// |-- triple: int (nullable = true) +// |-- key : int (nullable = true) +{% endhighlight %} + +
    + +
    + +{% highlight python %} +# sqlContext from the previous example is used in this example. + +# Create a simple DataFrame, stored into a partition directory +df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ + .map(lambda i: Row(single=i, double=i * 2))) +df1.save("data/test_table/key=1", "parquet") + +# Create another DataFrame in a new partition directory, +# adding a new column and dropping an existing column +df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) + .map(lambda i: Row(single=i, triple=i * 3))) +df2.save("data/test_table/key=2", "parquet") + +# Read the partitioned table +df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +# The final schema consists of all 3 columns in the Parquet files together +# with the partiioning column appeared in the partition directory paths. +# root +# |-- single: int (nullable = true) +# |-- double: int (nullable = true) +# |-- triple: int (nullable = true) +# |-- key : int (nullable = true) +{% endhighlight %} + +
    + +
    + ### Configuration -Configuration of Parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL. @@ -972,7 +1095,7 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1014,7 +1137,7 @@ val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext : +This conversion can be done using one of two methods in a `SQLContext` : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1056,7 +1179,7 @@ DataFrame anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD);
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1085,7 +1208,7 @@ people.printSchema() # Register this DataFrame as a table. people.registerTempTable("people") -# SQL statements can be run by using the sql methods provided by sqlContext. +# SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a DataFrame can be created for a JSON dataset represented by @@ -1131,7 +1254,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a HiveContext. When not configured by the +not have an existing Hive deployment can still create a `HiveContext`. When not configured by the hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current directory. @@ -1318,7 +1441,7 @@ Spark SQL can cache tables using an in-memory columnar format by calling `sqlCon Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL.
    @@ -1429,10 +1552,10 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script that comes with Hive. -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - hive.server2.transport.mode - Set this to value: http + hive.server2.transport.mode - Set this to value: http hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 hive.server2.http.endpoint - HTTP endpoint; default is cliservice @@ -1506,7 +1629,7 @@ When using function inside of the DSL (now replaced with the `DataFrame` API) us Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users should instead import the classes in `org.apache.spark.sql.types` -#### UDF Registration Moved to sqlContext.udf (Java & Scala) +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been moved into the udf object in `SQLContext`. From 9048e8102e3f564842fa0dc6e82edce70b7dd3d7 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 13 Mar 2015 13:59:54 +0000 Subject: [PATCH 404/817] [SPARK-6197][CORE] handle json exception when hisotry file not finished writing For details, please refer to [SPARK-6197](https://issues.apache.org/jira/browse/SPARK-6197) Author: Zhang, Liye Closes #4927 from liyezhang556520/jsonParseError and squashes the following commits: 5cbdc82 [Zhang, Liye] without unnecessary wrap 2b48831 [Zhang, Liye] small changes with sean owen's comments 2973024 [Zhang, Liye] handle json exception when file not finished writing --- .../apache/spark/deploy/master/Master.scala | 3 ++- .../spark/scheduler/ReplayListenerBus.scala | 25 ++++++++++++++++--- 2 files changed, 23 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 15814293227ab..22935c9b1d394 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -764,8 +764,9 @@ private[spark] class Master( val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { - replayBus.replay(logInput, eventLogFile) + replayBus.replay(logInput, eventLogFile, maybeTruncated) } finally { logInput.close() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 95273c716b3e2..86f357abb8723 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -21,6 +21,7 @@ import java.io.{InputStream, IOException} import scala.io.Source +import com.fasterxml.jackson.core.JsonParseException import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging @@ -40,15 +41,31 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * * @param logData Stream containing event log data. * @param sourceName Filename (or other source identifier) from whence @logData is being read + * @param maybeTruncated Indicate whether log file might be truncated (some abnormal situations + * encountered, log file might not finished writing) or not */ - def replay(logData: InputStream, sourceName: String): Unit = { + def replay( + logData: InputStream, + sourceName: String, + maybeTruncated: Boolean = false): Unit = { var currentLine: String = null var lineNumber: Int = 1 try { val lines = Source.fromInputStream(logData).getLines() - lines.foreach { line => - currentLine = line - postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + while (lines.hasNext) { + currentLine = lines.next() + try { + postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine))) + } catch { + case jpe: JsonParseException => + // We can only ignore exception from last line of the file that might be truncated + if (!maybeTruncated || lines.hasNext) { + throw jpe + } else { + logWarning(s"Got JsonParseException from log file $sourceName" + + s" at line $lineNumber, the file might not have finished writing cleanly.") + } + } lineNumber += 1 } } catch { From ea3d2eed9b0a94b34543d9a9df87dc63a279deb1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 13 Mar 2015 14:08:56 +0000 Subject: [PATCH 405/817] [CORE][minor] remove unnecessary ClassTag in `DAGScheduler` This existed at the very beginning, but became unnecessary after [this commit](https://github.com/apache/spark/commit/37d8f37a8ec110416fba0d51d8ba70370ac380c1#diff-6a9ff7fb74fd490a50462d45db2d5e11L272). I think we should remove it if we don't plan to use it in the future. Author: Wenchen Fan Closes #4992 from cloud-fan/small and squashes the following commits: e857f2e [Wenchen Fan] remove unnecessary ClassTag --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bc84e2351ad74..e4170a55b7981 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps -import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.pattern.ask @@ -497,7 +496,7 @@ class DAGScheduler( waiter } - def runJob[T, U: ClassTag]( + def runJob[T, U]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], From dc4abd4dc40deacab39bfa9572b06bf0ea6daa6d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 13 Mar 2015 10:26:09 -0700 Subject: [PATCH 406/817] [SPARK-6252] [mllib] Added getLambda to Scala NaiveBayes Note: not relevant for Python API since it only has a static train method Author: Joseph K. Bradley Author: Joseph K. Bradley Closes #4969 from jkbradley/SPARK-6252 and squashes the following commits: a471d90 [Joseph K. Bradley] small edits from review 63eff48 [Joseph K. Bradley] Added getLambda to Scala NaiveBayes --- .../apache/spark/mllib/classification/NaiveBayes.scala | 3 +++ .../spark/mllib/classification/NaiveBayesSuite.scala | 8 ++++++++ 2 files changed, 11 insertions(+) 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 b11fd4f128c56..2ebc7fa5d4234 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 @@ -166,6 +166,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with this } + /** Get the smoothing parameter. Default: 1.0. */ + def getLambda: Double = lambda + /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 64dcc0fb9f82c..5a27c7d2309c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -85,6 +85,14 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(numOfPredictions < input.length / 5) } + test("get, set params") { + val nb = new NaiveBayes() + nb.setLambda(2.0) + assert(nb.getLambda === 2.0) + nb.setLambda(3.0) + assert(nb.getLambda === 3.0) + } + test("Naive Bayes") { val nPoints = 10000 From 7f13434a5c52b815c584ec773ab0e5df1a35ea86 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 13 Mar 2015 10:27:28 -0700 Subject: [PATCH 407/817] [SPARK-6278][MLLIB] Mention the change of objective in linear regression As discussed in the RC3 vote thread, we should mention the change of objective in linear regression in the migration guide. srowen Author: Xiangrui Meng Closes #4978 from mengxr/SPARK-6278 and squashes the following commits: fb3bbe6 [Xiangrui Meng] mention regularization parameter bfd6cff [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-6278 375fd09 [Xiangrui Meng] address Sean's comments f87ae71 [Xiangrui Meng] mention step size change --- docs/mllib-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 598374f66df5e..f8e879496c135 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -102,6 +102,8 @@ In the `spark.mllib` package, there were several breaking changes. The first ch * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. * `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. +* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. + So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. ## Previous Spark Versions From b943f5d907df0607ecffb729f2bccfa436438d7e Mon Sep 17 00:00:00 2001 From: Brennon York Date: Fri, 13 Mar 2015 18:48:31 +0000 Subject: [PATCH 408/817] [SPARK-4600][GraphX]: org.apache.spark.graphx.VertexRDD.diff does not work Turns out, per the [convo on the JIRA](https://issues.apache.org/jira/browse/SPARK-4600), `diff` is acting exactly as should. It became a large misconception as I thought it meant set difference, when in fact it does not. To that extent I merely updated the `diff` documentation to, hopefully, better reflect its true intentions moving forward. Author: Brennon York Closes #5015 from brennonyork/SPARK-4600 and squashes the following commits: 1e1d1e5 [Brennon York] reverted internal diff docs 92288f7 [Brennon York] reverted both the test suite and the diff function back to its origin functionality f428623 [Brennon York] updated diff documentation to better represent its function cc16d65 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4600 66818b9 [Brennon York] added small secondary diff test 99ad412 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4600 74b8c95 [Brennon York] corrected method by leveraging bitmask operations to correctly return only the portions of that are different from the calling VertexRDD 9717120 [Brennon York] updated diff impl to cause fewer objects to be created 710a21c [Brennon York] working diff given test case aa57f83 [Brennon York] updated to set ShortestPaths to run 'forward' rather than 'backward' --- .../src/main/scala/org/apache/spark/graphx/VertexRDD.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 09ae3f9f6c09b..40ecff7107109 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -122,8 +122,11 @@ abstract class VertexRDD[VD]( def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] /** - * Hides vertices that are the same between `this` and `other`; for vertices that are different, - * keeps the values from `other`. + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other VertexRDD with which to diff against. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] From cdc34ed9108688fea32ad170b1ba344fe047716b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 14 Mar 2015 07:09:53 +0800 Subject: [PATCH 409/817] [SPARK-6285] [SQL] Removes unused ParquetTestData and duplicated TestGroupWriteSupport All the contents in this file are not referenced anywhere and should have been removed in #4116 when I tried to get rid of the old Parquet test suites. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5010) Author: Cheng Lian Closes #5010 from liancheng/spark-6285 and squashes the following commits: 06ed057 [Cheng Lian] Removes unused ParquetTestData and duplicated TestGroupWriteSupport --- .../spark/sql/parquet/ParquetTestData.scala | 466 ------------------ 1 file changed, 466 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala deleted file mode 100644 index e4a10aa2ae6c3..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ /dev/null @@ -1,466 +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.parquet - -import java.io.File - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.mapreduce.Job -import org.apache.spark.sql.test.TestSQLContext - -import parquet.example.data.{GroupWriter, Group} -import parquet.example.data.simple.{NanoTime, SimpleGroup} -import parquet.hadoop.{ParquetReader, ParquetFileReader, ParquetWriter} -import parquet.hadoop.api.WriteSupport -import parquet.hadoop.api.WriteSupport.WriteContext -import parquet.hadoop.example.GroupReadSupport -import parquet.hadoop.util.ContextUtil -import parquet.io.api.RecordConsumer -import parquet.schema.{MessageType, MessageTypeParser} - -import org.apache.spark.util.Utils - -// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport -// with an empty configuration (it is after all not intended to be used in this way?) -// and members are private so we need to make our own in order to pass the schema -// to the writer. -private class TestGroupWriteSupport(schema: MessageType) extends WriteSupport[Group] { - var groupWriter: GroupWriter = null - override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { - groupWriter = new GroupWriter(recordConsumer, schema) - } - override def init(configuration: Configuration): WriteContext = { - new WriteContext(schema, new java.util.HashMap[String, String]()) - } - override def write(record: Group) { - groupWriter.write(record) - } -} - -private[sql] object ParquetTestData { - - val testSchema = - """message myrecord { - optional boolean myboolean; - optional int32 myint; - optional binary mystring (UTF8); - optional int64 mylong; - optional float myfloat; - optional double mydouble; - optional int96 mytimestamp; - }""" - - // field names for test assertion error messages - val testSchemaFieldNames = Seq( - "myboolean:Boolean", - "myint:Int", - "mystring:String", - "mylong:Long", - "myfloat:Float", - "mydouble:Double", - "mytimestamp:Timestamp" - ) - - val subTestSchema = - """ - message myrecord { - optional boolean myboolean; - optional int64 mylong; - } - """ - - val testFilterSchema = - """ - message myrecord { - required boolean myboolean; - required int32 myint; - required binary mystring (UTF8); - required int64 mylong; - required float myfloat; - required double mydouble; - optional boolean myoptboolean; - optional int32 myoptint; - optional binary myoptstring (UTF8); - optional int64 myoptlong; - optional float myoptfloat; - optional double myoptdouble; - optional int96 mytimestamp; - } - """ - - // field names for test assertion error messages - val subTestSchemaFieldNames = Seq( - "myboolean:Boolean", - "mylong:Long" - ) - - val testDir = Utils.createTempDir() - val testFilterDir = Utils.createTempDir() - - lazy val testData = new ParquetRelation(testDir.toURI.toString, None, TestSQLContext) - - val testNestedSchema1 = - // based on blogpost example, source: - // https://blog.twitter.com/2013/dremel-made-simple-with-parquet - // note: instead of string we have to use binary (?) otherwise - // Parquet gives us: - // IllegalArgumentException: expected one of [INT64, INT32, BOOLEAN, - // BINARY, FLOAT, DOUBLE, INT96, FIXED_LEN_BYTE_ARRAY] - // Also repeated primitives seem tricky to convert (AvroParquet - // only uses them in arrays?) so only use at most one in each group - // and nothing else in that group (-> is mapped to array)! - // The "values" inside ownerPhoneNumbers is a keyword currently - // so that array types can be translated correctly. - """ - message AddressBook { - required binary owner (UTF8); - optional group ownerPhoneNumbers { - repeated binary array (UTF8); - } - optional group contacts { - repeated group array { - required binary name (UTF8); - optional binary phoneNumber (UTF8); - } - } - } - """ - - - val testNestedSchema2 = - """ - message TestNested2 { - required int32 firstInt; - optional int32 secondInt; - optional group longs { - repeated int64 array; - } - required group entries { - repeated group array { - required double value; - optional boolean truth; - } - } - optional group outerouter { - repeated group array { - repeated group array { - repeated int32 array; - } - } - } - } - """ - - val testNestedSchema3 = - """ - message TestNested3 { - required int32 x; - optional group booleanNumberPairs { - repeated group array { - required int32 key; - optional group value { - repeated group array { - required double nestedValue; - optional boolean truth; - } - } - } - } - } - """ - - val testNestedSchema4 = - """ - message TestNested4 { - required int32 x; - optional group data1 { - repeated group map { - required binary key (UTF8); - required int32 value; - } - } - required group data2 { - repeated group map { - required binary key (UTF8); - required group value { - required int64 payload1; - optional binary payload2 (UTF8); - } - } - } - } - """ - - val testNestedDir1 = Utils.createTempDir() - val testNestedDir2 = Utils.createTempDir() - val testNestedDir3 = Utils.createTempDir() - val testNestedDir4 = Utils.createTempDir() - - lazy val testNestedData1 = - new ParquetRelation(testNestedDir1.toURI.toString, None, TestSQLContext) - lazy val testNestedData2 = - new ParquetRelation(testNestedDir2.toURI.toString, None, TestSQLContext) - - def writeFile() = { - testDir.delete() - val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) - val job = new Job() - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 until 15) { - val record = new SimpleGroup(schema) - if (i % 3 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - if (i % 5 == 0) { - record.add(1, 5) - } - record.add(2, "abc") - record.add(3, i.toLong << 33) - record.add(4, 2.5F) - record.add(5, 4.5D) - record.add(6, new NanoTime(1,2)) - writer.write(record) - } - writer.close() - } - - def writeFilterFile(records: Int = 200) = { - // for microbenchmark use: records = 300000000 - testFilterDir.delete - val path: Path = new Path(new Path(testFilterDir.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testFilterSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 to records) { - val record = new SimpleGroup(schema) - if (i % 4 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - record.add(1, i) - record.add(2, i.toString) - record.add(3, i.toLong) - record.add(4, i.toFloat + 0.5f) - record.add(5, i.toDouble + 0.5d) - if (i % 2 == 0) { - if (i % 3 == 0) { - record.add(6, true) - } else { - record.add(6, false) - } - record.add(7, i) - record.add(8, i.toString) - record.add(9, i.toLong) - record.add(10, i.toFloat + 0.5f) - record.add(11, i.toDouble + 0.5d) - } - - writer.write(record) - } - writer.close() - } - - def writeNestedFile1() { - // example data from https://blog.twitter.com/2013/dremel-made-simple-with-parquet - testNestedDir1.delete() - val path: Path = new Path(new Path(testNestedDir1.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) - - val r1 = new SimpleGroup(schema) - r1.add(0, "Julien Le Dem") - r1.addGroup(1) - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 123 4567") - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 666 1337") - .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "XXX XXX XXXX") - val contacts = r1.addGroup(2) - contacts.addGroup(0) - .append("name", "Dmitriy Ryaboy") - .append("phoneNumber", "555 987 6543") - contacts.addGroup(0) - .append("name", "Chris Aniszczyk") - - val r2 = new SimpleGroup(schema) - r2.add(0, "A. Nonymous") - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.write(r2) - writer.close() - } - - def writeNestedFile2() { - testNestedDir2.delete() - val path: Path = new Path(new Path(testNestedDir2.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema2) - - val r1 = new SimpleGroup(schema) - r1.add(0, 1) - r1.add(1, 7) - val longs = r1.addGroup(2) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME , 1.toLong << 32) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 33) - longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 34) - val booleanNumberPair = r1.addGroup(3).addGroup(0) - booleanNumberPair.add("value", 2.5) - booleanNumberPair.add("truth", false) - val top_level = r1.addGroup(4) - val second_level_a = top_level.addGroup(0) - val second_level_b = top_level.addGroup(0) - val third_level_aa = second_level_a.addGroup(0) - val third_level_ab = second_level_a.addGroup(0) - val third_level_c = second_level_b.addGroup(0) - third_level_aa.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 7) - third_level_ab.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 8) - third_level_c.add( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - 9) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - def writeNestedFile3() { - testNestedDir3.delete() - val path: Path = new Path(new Path(testNestedDir3.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema3) - - val r1 = new SimpleGroup(schema) - r1.add(0, 1) - val booleanNumberPairs = r1.addGroup(1) - val g1 = booleanNumberPairs.addGroup(0) - g1.add(0, 1) - val nested1 = g1.addGroup(1) - val ng1 = nested1.addGroup(0) - ng1.add(0, 1.5) - ng1.add(1, false) - val ng2 = nested1.addGroup(0) - ng2.add(0, 2.5) - ng2.add(1, true) - val g2 = booleanNumberPairs.addGroup(0) - g2.add(0, 2) - val ng3 = g2.addGroup(1) - .addGroup(0) - ng3.add(0, 3.5) - ng3.add(1, false) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - def writeNestedFile4() { - testNestedDir4.delete() - val path: Path = new Path(new Path(testNestedDir4.toURI), new Path("part-r-0.parquet")) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema4) - - val r1 = new SimpleGroup(schema) - r1.add(0, 7) - val map1 = r1.addGroup(1) - val keyValue1 = map1.addGroup(0) - keyValue1.add(0, "key1") - keyValue1.add(1, 1) - val keyValue2 = map1.addGroup(0) - keyValue2.add(0, "key2") - keyValue2.add(1, 2) - val map2 = r1.addGroup(2) - val keyValue3 = map2.addGroup(0) - // TODO: currently only string key type supported - keyValue3.add(0, "seven") - val valueGroup1 = keyValue3.addGroup(1) - valueGroup1.add(0, 42.toLong) - valueGroup1.add(1, "the answer") - val keyValue4 = map2.addGroup(0) - // TODO: currently only string key type supported - keyValue4.add(0, "eight") - val valueGroup2 = keyValue4.addGroup(1) - valueGroup2.add(0, 49.toLong) - - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - writer.write(r1) - writer.close() - } - - // TODO: this is not actually used anywhere but useful for debugging - /* def readNestedFile(file: File, schemaString: String): Unit = { - val configuration = new Configuration() - val path = new Path(new Path(file.toURI), new Path("part-r-0.parquet")) - val fs: FileSystem = path.getFileSystem(configuration) - val schema: MessageType = MessageTypeParser.parseMessageType(schemaString) - assert(schema != null) - val outputStatus: FileStatus = fs.getFileStatus(new Path(path.toString)) - val footers = ParquetFileReader.readFooter(configuration, outputStatus) - assert(footers != null) - val reader = new ParquetReader(new Path(path.toString), new GroupReadSupport()) - val first = reader.read() - assert(first != null) - } */ - - // to test golb pattern (wild card pattern matching for parquetFile input - val testGlobDir = Utils.createTempDir() - val testGlobSubDir1 = Utils.createTempDir(testGlobDir.getPath) - val testGlobSubDir2 = Utils.createTempDir(testGlobDir.getPath) - val testGlobSubDir3 = Utils.createTempDir(testGlobDir.getPath) - - def writeGlobFiles() = { - val subDirs = Array(testGlobSubDir1, testGlobSubDir2, testGlobSubDir3) - - subDirs.foreach { dir => - val path: Path = new Path(new Path(dir.toURI), new Path("part-r-0.parquet")) - val job = new Job() - val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) - - for(i <- 0 until 15) { - val record = new SimpleGroup(schema) - if(i % 3 == 0) { - record.add(0, true) - } else { - record.add(0, false) - } - if(i % 5 == 0) { - record.add(1, 5) - } - record.add(2, "abc") - record.add(3, i.toLong << 33) - record.add(4, 2.5F) - record.add(5, 4.5D) - writer.write(record) - } - writer.close() - } - } -} - From e360d5e4adf287444c10e72f8e4d57548839bf6e Mon Sep 17 00:00:00 2001 From: vinodkc Date: Sat, 14 Mar 2015 07:17:54 +0800 Subject: [PATCH 410/817] [SPARK-6317][SQL]Fixed HIVE console startup issue Author: vinodkc Author: Vinod K C Closes #5011 from vinodkc/HIVE_console_startupError and squashes the following commits: b43925f [vinodkc] Changed order of import b4f5453 [Vinod K C] Fixed HIVE console startup issue --- project/SparkBuild.scala | 4 ++-- sql/README.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4a06b9821bb98..f4c74c4051014 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -269,8 +269,8 @@ object SQL { |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.Dsl._ |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.test.TestSQLContext._ |import org.apache.spark.sql.types._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, @@ -300,8 +300,8 @@ object Hive { |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ |import org.apache.spark.sql.catalyst.util._ - |import org.apache.spark.sql.Dsl._ |import org.apache.spark.sql.execution + |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.types._ diff --git a/sql/README.md b/sql/README.md index a79249965ee67..48f83340e37b3 100644 --- a/sql/README.md +++ b/sql/README.md @@ -36,8 +36,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.execution +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ From b38e073fee794188d5267f1812b095e51874839e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 14 Mar 2015 00:43:33 -0700 Subject: [PATCH 411/817] [SPARK-6210] [SQL] use prettyString as column name in agg() use prettyString instead of toString() (which include id of expression) as column name in agg() Author: Davies Liu Closes #5006 from davies/prettystring and squashes the following commits: cb1fdcf [Davies Liu] use prettyString as column name in agg() --- python/pyspark/sql/dataframe.py | 32 +++++++++---------- .../spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../org/apache/spark/sql/GroupedData.scala | 8 ++--- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e8ce4547455a5..94001aec3774b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -631,11 +631,11 @@ def groupBy(self, *cols): for all the available aggregate functions. >>> df.groupBy().avg().collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age#0)=5.0), Row(name=u'Alice', AVG(age#0)=2.0)] + [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] """ jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) @@ -647,10 +647,10 @@ def agg(self, *exprs): (shorthand for df.groupBy.agg()). >>> df.agg({"age": "max"}).collect() - [Row(MAX(age#0)=5)] + [Row(MAX(age)=5)] >>> from pyspark.sql import functions as F >>> df.agg(F.min(df.age)).collect() - [Row(MIN(age#0)=2)] + [Row(MIN(age)=2)] """ return self.groupBy().agg(*exprs) @@ -766,7 +766,7 @@ def agg(self, *exprs): >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age#0)=5), Row(MIN(age#0)=2)] + [Row(MIN(age)=5), Row(MIN(age)=2)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): @@ -795,9 +795,9 @@ def mean(self, *cols): for each group. This is an alias for `avg`. >>> df.groupBy().mean('age').collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df3.groupBy().mean('age', 'height').collect() - [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] + [Row(AVG(age)=3.5, AVG(height)=82.5)] """ @df_varargs_api @@ -806,9 +806,9 @@ def avg(self, *cols): for each group. >>> df.groupBy().avg('age').collect() - [Row(AVG(age#0)=3.5)] + [Row(AVG(age)=3.5)] >>> df3.groupBy().avg('age', 'height').collect() - [Row(AVG(age#4L)=3.5, AVG(height#5L)=82.5)] + [Row(AVG(age)=3.5, AVG(height)=82.5)] """ @df_varargs_api @@ -817,9 +817,9 @@ def max(self, *cols): each group. >>> df.groupBy().max('age').collect() - [Row(MAX(age#0)=5)] + [Row(MAX(age)=5)] >>> df3.groupBy().max('age', 'height').collect() - [Row(MAX(age#4L)=5, MAX(height#5L)=85)] + [Row(MAX(age)=5, MAX(height)=85)] """ @df_varargs_api @@ -828,9 +828,9 @@ def min(self, *cols): each group. >>> df.groupBy().min('age').collect() - [Row(MIN(age#0)=2)] + [Row(MIN(age)=2)] >>> df3.groupBy().min('age', 'height').collect() - [Row(MIN(age#4L)=2, MIN(height#5L)=80)] + [Row(MIN(age)=2, MIN(height)=80)] """ @df_varargs_api @@ -839,9 +839,9 @@ def sum(self, *cols): group. >>> df.groupBy().sum('age').collect() - [Row(SUM(age#0)=7)] + [Row(SUM(age)=7)] >>> df3.groupBy().sum('age', 'height').collect() - [Row(SUM(age#4L)=7, SUM(height#5L)=165)] + [Row(SUM(age)=7, SUM(height)=165)] """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 7e191ad0315a5..f84ffe4e176cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -343,7 +343,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { }.mkString(", ") /** String representation of this node without any children */ - def simpleString = s"$nodeName $argString" + def simpleString = s"$nodeName $argString".trim override def toString: String = treeString 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 d00175265924c..45a63ae26ed71 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 @@ -37,7 +37,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.toString)() + case expr: Expression => Alias(expr, expr.prettyString)() } DataFrame( df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) @@ -63,7 +63,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) } columnExprs.map { c => val a = f(c) - Alias(a, a.toString)() + Alias(a, a.prettyString)() } } @@ -115,7 +115,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) def agg(exprs: Map[String, String]): DataFrame = { exprs.map { case (colName, expr) => val a = strToExpr(expr)(df(colName).expr) - Alias(a, a.toString)() + Alias(a, a.prettyString)() }.toSeq } @@ -159,7 +159,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) def agg(expr: Column, exprs: Column*): DataFrame = { val aggExprs = (expr +: exprs).map(_.expr).map { case expr: NamedExpression => expr - case expr: Expression => Alias(expr, expr.toString)() + case expr: Expression => Alias(expr, expr.prettyString)() } DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) } From ee15404a2b0009fc70119ac7af69137b54890d48 Mon Sep 17 00:00:00 2001 From: ArcherShao Date: Sat, 14 Mar 2015 08:27:18 +0000 Subject: [PATCH 412/817] [SQL]Delete some dupliate code in HiveThriftServer2 Author: ArcherShao Author: ArcherShao Closes #5007 from ArcherShao/20150313 and squashes the following commits: ae422ae [ArcherShao] Updated 459efbd [ArcherShao] [SQL]Delete some dupliate code in HiveThriftServer2 --- .../sql/hive/thriftserver/HiveThriftServer2.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 6e07df18b0e15..c3a3f8c0f41df 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -98,16 +98,14 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) - if (isHTTPTransportMode(hiveConf)) { - val thriftCliService = new ThriftHttpCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) + val thriftCliService = if (isHTTPTransportMode(hiveConf)) { + new ThriftHttpCLIService(sparkSqlCliService) } else { - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) + new ThriftBinaryCLIService(sparkSqlCliService) } + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) initCompositeService(hiveConf) } From 5be6b0e4f48aca12fcd47c1b77c4675ad651c332 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 14 Mar 2015 19:53:54 +0800 Subject: [PATCH 413/817] [SPARK-6195] [SQL] Adds in-memory column type for fixed-precision decimals This PR adds a specialized in-memory column type for fixed-precision decimals. For all other column types, a single integer column type ID is enough to determine which column type to use. However, this doesn't apply to fixed-precision decimal types with different precision and scale parameters. Moreover, according to the previous design, there seems no trivial way to encode precision and scale information into the columnar byte buffer. On the other hand, considering we always know the data type of the column to be built / scanned ahead of time. This PR no longer use column type ID to construct `ColumnBuilder`s and `ColumnAccessor`s, but resorts to the actual column data type. In this way, we can pass precision / scale information along the way. The column type ID is now not used anymore and can be removed in a future PR. ### Micro benchmark result The following micro benchmark builds a simple table with 2 million decimals (precision = 10, scale = 0), cache it in memory, then count all the rows. Code (simply paste it into Spark shell): ```scala import sc._ import sqlContext._ import sqlContext.implicits._ import org.apache.spark.sql.types._ import com.google.common.base.Stopwatch def benchmark(n: Int)(f: => Long) { val stopwatch = new Stopwatch() def run() = { stopwatch.reset() stopwatch.start() f stopwatch.stop() stopwatch.elapsedMillis() } val records = (0 until n).map(_ => run()) (0 until n).foreach(i => println(s"Round $i: ${records(i)} ms")) println(s"Average: ${records.sum / n.toDouble} ms") } // Explicit casting is required because ScalaReflection can't inspect decimal precision parallelize(1 to 2000000) .map(i => Tuple1(Decimal(i, 10, 0))) .toDF("dec") .select($"dec" cast DecimalType(10, 0)) .registerTempTable("dec") sql("CACHE TABLE dec") val df = table("dec") // Warm up df.count() df.count() benchmark(5) { df.count() } ``` With `FIXED_DECIMAL` column type: - Round 0: 75 ms - Round 1: 97 ms - Round 2: 75 ms - Round 3: 70 ms - Round 4: 72 ms - Average: 77.8 ms Without `FIXED_DECIMAL` column type: - Round 0: 1233 ms - Round 1: 1170 ms - Round 2: 1171 ms - Round 3: 1141 ms - Round 4: 1141 ms - Average: 1171.2 ms [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4938) Author: Cheng Lian Closes #4938 from liancheng/decimal-column-type and squashes the following commits: fef5338 [Cheng Lian] Updates fixed decimal column type related test cases e08ab5b [Cheng Lian] Only resorts to FIXED_DECIMAL when the value can be held in a long 4db713d [Cheng Lian] Adds in-memory column type for fixed-precision decimals --- .../spark/sql/columnar/ColumnAccessor.scala | 43 +++++++++------ .../spark/sql/columnar/ColumnBuilder.scala | 39 +++++++------ .../spark/sql/columnar/ColumnStats.scala | 17 ++++++ .../spark/sql/columnar/ColumnType.scala | 55 ++++++++++++++----- .../columnar/InMemoryColumnarTableScan.scala | 8 ++- .../spark/sql/columnar/ColumnStatsSuite.scala | 1 + .../spark/sql/columnar/ColumnTypeSuite.scala | 46 ++++++++++++---- .../sql/columnar/ColumnarTestUtils.scala | 23 ++++---- .../columnar/InMemoryColumnarQuerySuite.scala | 17 +++++- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- 11 files changed, 179 insertions(+), 76 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 91c4c105b14e6..b615eaa0dca0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -21,7 +21,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor -import org.apache.spark.sql.types.{BinaryType, DataType, NativeType} +import org.apache.spark.sql.types._ /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -89,6 +89,9 @@ private[sql] class DoubleColumnAccessor(buffer: ByteBuffer) private[sql] class FloatColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, FLOAT) +private[sql] class FixedDecimalColumnAccessor(buffer: ByteBuffer, precision: Int, scale: Int) + extends NativeColumnAccessor(buffer, FIXED_DECIMAL(precision, scale)) + private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) @@ -107,24 +110,28 @@ private[sql] class GenericColumnAccessor(buffer: ByteBuffer) with NullableColumnAccessor private[sql] object ColumnAccessor { - def apply(buffer: ByteBuffer): ColumnAccessor = { + def apply(dataType: DataType, buffer: ByteBuffer): ColumnAccessor = { val dup = buffer.duplicate().order(ByteOrder.nativeOrder) - // The first 4 bytes in the buffer indicate the column type. - val columnTypeId = dup.getInt() - - columnTypeId match { - case INT.typeId => new IntColumnAccessor(dup) - case LONG.typeId => new LongColumnAccessor(dup) - case FLOAT.typeId => new FloatColumnAccessor(dup) - case DOUBLE.typeId => new DoubleColumnAccessor(dup) - case BOOLEAN.typeId => new BooleanColumnAccessor(dup) - case BYTE.typeId => new ByteColumnAccessor(dup) - case SHORT.typeId => new ShortColumnAccessor(dup) - case STRING.typeId => new StringColumnAccessor(dup) - case DATE.typeId => new DateColumnAccessor(dup) - case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) - case BINARY.typeId => new BinaryColumnAccessor(dup) - case GENERIC.typeId => new GenericColumnAccessor(dup) + + // The first 4 bytes in the buffer indicate the column type. This field is not used now, + // because we always know the data type of the column ahead of time. + dup.getInt() + + dataType match { + case IntegerType => new IntColumnAccessor(dup) + case LongType => new LongColumnAccessor(dup) + case FloatType => new FloatColumnAccessor(dup) + case DoubleType => new DoubleColumnAccessor(dup) + case BooleanType => new BooleanColumnAccessor(dup) + case ByteType => new ByteColumnAccessor(dup) + case ShortType => new ShortColumnAccessor(dup) + case StringType => new StringColumnAccessor(dup) + case BinaryType => new BinaryColumnAccessor(dup) + case DateType => new DateColumnAccessor(dup) + case TimestampType => new TimestampColumnAccessor(dup) + case DecimalType.Fixed(precision, scale) if precision < 19 => + new FixedDecimalColumnAccessor(dup, precision, scale) + case _ => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 3a4977b836af7..d8d24a577347c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -106,6 +106,13 @@ private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleCol private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT) +private[sql] class FixedDecimalColumnBuilder( + precision: Int, + scale: Int) + extends NativeColumnBuilder( + new FixedDecimalColumnStats, + FIXED_DECIMAL(precision, scale)) + private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) private[sql] class DateColumnBuilder extends NativeColumnBuilder(new DateColumnStats, DATE) @@ -139,25 +146,25 @@ private[sql] object ColumnBuilder { } def apply( - typeId: Int, + dataType: DataType, initialSize: Int = 0, columnName: String = "", useCompression: Boolean = false): ColumnBuilder = { - - val builder = (typeId match { - case INT.typeId => new IntColumnBuilder - case LONG.typeId => new LongColumnBuilder - case FLOAT.typeId => new FloatColumnBuilder - case DOUBLE.typeId => new DoubleColumnBuilder - case BOOLEAN.typeId => new BooleanColumnBuilder - case BYTE.typeId => new ByteColumnBuilder - case SHORT.typeId => new ShortColumnBuilder - case STRING.typeId => new StringColumnBuilder - case BINARY.typeId => new BinaryColumnBuilder - case GENERIC.typeId => new GenericColumnBuilder - case DATE.typeId => new DateColumnBuilder - case TIMESTAMP.typeId => new TimestampColumnBuilder - }).asInstanceOf[ColumnBuilder] + val builder: ColumnBuilder = dataType match { + case IntegerType => new IntColumnBuilder + case LongType => new LongColumnBuilder + case DoubleType => new DoubleColumnBuilder + case BooleanType => new BooleanColumnBuilder + case ByteType => new ByteColumnBuilder + case ShortType => new ShortColumnBuilder + case StringType => new StringColumnBuilder + case BinaryType => new BinaryColumnBuilder + case DateType => new DateColumnBuilder + case TimestampType => new TimestampColumnBuilder + case DecimalType.Fixed(precision, scale) if precision < 19 => + new FixedDecimalColumnBuilder(precision, scale) + case _ => new GenericColumnBuilder + } builder.initialize(initialSize, columnName, useCompression) builder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index cad0667b46435..04047b9c062be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -181,6 +181,23 @@ private[sql] class FloatColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } +private[sql] class FixedDecimalColumnStats extends ColumnStats { + protected var upper: Decimal = null + protected var lower: Decimal = null + + override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Decimal] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + sizeInBytes += FIXED_DECIMAL.defaultSize + } + } + + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) +} + private[sql] class IntColumnStats extends ColumnStats { protected var upper = Int.MinValue protected var lower = Int.MaxValue diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index db5bc0de363c7..36ea1c77e0470 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -373,6 +373,33 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { } } +private[sql] case class FIXED_DECIMAL(precision: Int, scale: Int) + extends NativeColumnType( + DecimalType(Some(PrecisionInfo(precision, scale))), + 10, + FIXED_DECIMAL.defaultSize) { + + override def extract(buffer: ByteBuffer): Decimal = { + Decimal(buffer.getLong(), precision, scale) + } + + override def append(v: Decimal, buffer: ByteBuffer): Unit = { + buffer.putLong(v.toUnscaledLong) + } + + override def getField(row: Row, ordinal: Int): Decimal = { + row(ordinal).asInstanceOf[Decimal] + } + + override def setField(row: MutableRow, ordinal: Int, value: Decimal): Unit = { + row(ordinal) = value + } +} + +private[sql] object FIXED_DECIMAL { + val defaultSize = 8 +} + private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( typeId: Int, defaultSize: Int) @@ -394,7 +421,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](11, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -405,7 +432,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](12, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -416,18 +443,20 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { private[sql] object ColumnType { def apply(dataType: DataType): ColumnType[_, _] = { dataType match { - case IntegerType => INT - case LongType => LONG - case FloatType => FLOAT - case DoubleType => DOUBLE - case BooleanType => BOOLEAN - case ByteType => BYTE - case ShortType => SHORT - case StringType => STRING - case BinaryType => BINARY - case DateType => DATE + case IntegerType => INT + case LongType => LONG + case FloatType => FLOAT + case DoubleType => DOUBLE + case BooleanType => BOOLEAN + case ByteType => BYTE + case ShortType => SHORT + case StringType => STRING + case BinaryType => BINARY + case DateType => DATE case TimestampType => TIMESTAMP - case _ => GENERIC + case DecimalType.Fixed(precision, scale) if precision < 19 => + FIXED_DECIMAL(precision, scale) + case _ => GENERIC } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 8944a32bc3887..387faee12b3cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -113,7 +113,7 @@ private[sql] case class InMemoryRelation( val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) val initialBufferSize = columnType.defaultSize * batchSize - ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) + ColumnBuilder(attribute.dataType, initialBufferSize, attribute.name, useCompression) }.toArray var rowCount = 0 @@ -274,8 +274,10 @@ private[sql] case class InMemoryColumnarTableScan( def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { val rows = cacheBatches.flatMap { cachedBatch => // Build column accessors - val columnAccessors = requestedColumnIndices.map { batch => - ColumnAccessor(ByteBuffer.wrap(cachedBatch.buffers(batch))) + val columnAccessors = requestedColumnIndices.map { batchColumnIndex => + ColumnAccessor( + relation.output(batchColumnIndex).dataType, + ByteBuffer.wrap(cachedBatch.buffers(batchColumnIndex))) } // Extract rows via column accessors diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 581fccf8ee613..fec487f1d2c82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -29,6 +29,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[LongColumnStats], LONG, Row(Long.MaxValue, Long.MinValue, 0)) testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) + testColumnStats(classOf[FixedDecimalColumnStats], FIXED_DECIMAL(15, 10), Row(null, null, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 9ce845912f1c7..5f08834f73c6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -33,8 +33,9 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( - INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, BOOLEAN -> 1, - STRING -> 8, DATE -> 4, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, + FIXED_DECIMAL(15, 10) -> 8, BOOLEAN -> 1, STRING -> 8, DATE -> 4, TIMESTAMP -> 12, + BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -56,15 +57,16 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - checkActualSize(INT, Int.MaxValue, 4) - checkActualSize(SHORT, Short.MaxValue, 2) - checkActualSize(LONG, Long.MaxValue, 8) - checkActualSize(BYTE, Byte.MaxValue, 1) - checkActualSize(DOUBLE, Double.MaxValue, 8) - checkActualSize(FLOAT, Float.MaxValue, 4) - checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) - checkActualSize(DATE, 0, 4) + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) @@ -93,12 +95,20 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[DoubleType.type](DOUBLE, _.putDouble(_), _.getDouble) + testNativeColumnType[DecimalType]( + FIXED_DECIMAL(15, 10), + (buffer: ByteBuffer, decimal: Decimal) => { + buffer.putLong(decimal.toUnscaledLong) + }, + (buffer: ByteBuffer) => { + Decimal(buffer.getLong(), 15, 10) + }) + testNativeColumnType[FloatType.type](FLOAT, _.putFloat(_), _.getFloat) testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") buffer.putInt(bytes.length) buffer.put(bytes) @@ -206,4 +216,16 @@ class ColumnTypeSuite extends FunSuite with Logging { if (sb.nonEmpty) sb.setLength(sb.length - 1) sb.toString() } + + test("column type for decimal types with different precision") { + (1 to 18).foreach { i => + assertResult(FIXED_DECIMAL(i, 0)) { + ColumnType(DecimalType(i, 0)) + } + } + + assertResult(GENERIC) { + ColumnType(DecimalType(19, 0)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 60ed28cc97bf1..c7a40845db16c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{DataType, NativeType} +import org.apache.spark.sql.types.{Decimal, DataType, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int) = { @@ -41,16 +41,17 @@ object ColumnarTestUtils { } (columnType match { - case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte - case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort - case INT => Random.nextInt() - case LONG => Random.nextLong() - case FLOAT => Random.nextFloat() - case DOUBLE => Random.nextDouble() - case STRING => Random.nextString(Random.nextInt(32)) - case BOOLEAN => Random.nextBoolean() - case BINARY => randomBytes(Random.nextInt(32)) - case DATE => Random.nextInt() + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case DATE => Random.nextInt() case TIMESTAMP => val timestamp = new Timestamp(Random.nextLong()) timestamp.setNanos(Random.nextInt(999999999)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 38b0f666ab90b..27dfabca90217 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar -import org.apache.spark.sql.functions._ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.types.{DecimalType, Decimal} import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -117,4 +117,19 @@ class InMemoryColumnarQuerySuite extends QueryTest { complexData.count() complexData.unpersist() } + + test("decimal type") { + // Casting is required here because ScalaReflection can't capture decimal precision information. + val df = (1 to 10) + .map(i => Tuple1(Decimal(i, 15, 10))) + .toDF("dec") + .select($"dec" cast DecimalType(15, 10)) + + assert(df.schema.head.dataType === DecimalType(15, 10)) + + df.cache().registerTempTable("test_fixed_decimal") + checkAnswer( + sql("SELECT * FROM test_fixed_decimal"), + (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index f95c895587f3f..bb305355276bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -42,7 +42,8 @@ class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ Seq( - INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, FIXED_DECIMAL(15, 10), BINARY, GENERIC, + DATE, TIMESTAMP ).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 80bd5c94570cb..75a47498683f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -38,7 +38,8 @@ class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ Seq( - INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, FIXED_DECIMAL(15, 10), BINARY, GENERIC, + DATE, TIMESTAMP ).foreach { testNullableColumnBuilder(_) } From 127268bc3999201ec1c0a040a29c7fa9ac25476b Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sat, 14 Mar 2015 17:28:13 +0000 Subject: [PATCH 414/817] [SPARK-6329][Docs]: Minor doc changes for Mesos and TOC Updated the configuration docs from the minor items that Reynold had left over from SPARK-1182; specifically I updated the `running-on-mesos` link to point directly to `running-on-mesos#configuration` and upgraded the `yarn`, `mesos`, etc. bullets to `
    ` tags in hopes that they'll get pushed into the TOC. Author: Brennon York Closes #5022 from brennonyork/SPARK-6329 and squashes the following commits: 42a10a9 [Brennon York] minor doc fixes --- docs/configuration.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index a7116fbece9bb..63fc99e7d3e29 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1391,9 +1391,11 @@ Apart from these, the following properties are also available, and may be useful Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: - * [YARN](running-on-yarn.html#configuration) - * [Mesos](running-on-mesos.html) - * [Standalone Mode](spark-standalone.html#cluster-launch-scripts) +##### [YARN](running-on-yarn.html#configuration) + +##### [Mesos](running-on-mesos.html#configuration) + +##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables From c49d156624624a719c0d1262a58933ea3e346963 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sat, 14 Mar 2015 17:38:12 +0000 Subject: [PATCH 415/817] [SPARK-5790][GraphX]: VertexRDD's won't zip properly for `diff` capability (added tests) Added tests that maropu [created](https://github.com/maropu/spark/blob/1f64794b2ce33e64f340e383d4e8a60639a7eb4b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala) for vertices with differing partition counts. Wanted to make sure his work got captured /merged as its not in the master branch and I don't believe there's a PR out already for it. Author: Brennon York Closes #5023 from brennonyork/SPARK-5790 and squashes the following commits: 83bbd29 [Brennon York] added maropu's tests for vertices with differing partition counts --- .../apache/spark/graphx/VertexRDDSuite.scala | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 131959cea3ef7..97533dd3aa6ce 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.spark.{HashPartitioner, SparkContext} import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { @@ -58,6 +58,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("diff vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.diff(vertexB) + assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + } + } + test("leftJoin") { withSpark { sc => val n = 100 @@ -73,6 +83,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("leftJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => + old - newOpt.getOrElse(0) + } + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + } + } + test("innerJoin") { withSpark { sc => val n = 100 @@ -87,6 +110,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } + test("innerJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => + old - newVal + } + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + } + } + test("aggregateUsingIndex") { withSpark { sc => val n = 100 From 62ede5383f64b69570a66d46939638f4bf38d1b1 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Sun, 15 Mar 2015 20:44:45 +0800 Subject: [PATCH 416/817] [SPARK-6285][SQL]Remove ParquetTestData in SparkBuild.scala and in README.md This is a following clean up PR for #5010 This will resolve issues when launching `hive/console` like below: ``` :20: error: object ParquetTestData is not a member of package org.apache.spark.sql.parquet import org.apache.spark.sql.parquet.ParquetTestData ``` Author: OopsOutOfMemory Closes #5032 from OopsOutOfMemory/SPARK-6285 and squashes the following commits: 2996aeb [OopsOutOfMemory] remove ParquetTestData --- project/SparkBuild.scala | 6 ++---- sql/README.md | 1 - 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f4c74c4051014..ac37c605de4b6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -272,8 +272,7 @@ object SQL { |import org.apache.spark.sql.execution |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.test.TestSQLContext._ - |import org.apache.spark.sql.types._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + |import org.apache.spark.sql.types._""".stripMargin, cleanupCommands in console := "sparkContext.stop()" ) } @@ -304,8 +303,7 @@ object Hive { |import org.apache.spark.sql.functions._ |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ - |import org.apache.spark.sql.types._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + |import org.apache.spark.sql.types._""".stripMargin, cleanupCommands in console := "sparkContext.stop()", // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce // in order to generate golden files. This is only required for developers who are adding new diff --git a/sql/README.md b/sql/README.md index 48f83340e37b3..fbb3200a3a4b4 100644 --- a/sql/README.md +++ b/sql/README.md @@ -41,7 +41,6 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.parquet.ParquetTestData Type in expressions to have them evaluated. Type :help for more information. From aa6536fa3c2ed1cac47abc79fc22e273f0814858 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Sun, 15 Mar 2015 15:46:55 +0000 Subject: [PATCH 417/817] [SPARK-3619] Part 2. Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY Author: Jongyoul Lee Closes #4361 from jongyoul/SPARK-3619-1 and squashes the following commits: f1ea91f [Jongyoul Lee] Merge branch 'SPARK-3619-1' of https://github.com/jongyoul/spark into SPARK-3619-1 a6a00c2 [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - Removed 'Known issues' section 2e15a21 [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY 0dace7b [Jongyoul Lee] [SPARK-3619] Upgrade to Mesos 0.21 to work around MESOS-1688 - MESOS_NATIVE_LIBRARY become deprecated - Chagned MESOS_NATIVE_LIBRARY to MESOS_NATIVE_JAVA_LIBRARY --- conf/spark-env.sh.template | 2 +- docs/running-on-mesos.md | 5 +---- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 +- .../src/test/scala/org/apache/spark/repl/ReplSuite.scala | 2 +- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 0886b0276fb90..67f81d33361e1 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -15,7 +15,7 @@ # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_CLASSPATH, default classpath entries to append # - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data -# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos +# - MESOS_NATIVE_JAVA_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index e509e4bf37396..59a3e9d25baf1 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -110,7 +110,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: 1. In `spark-env.sh` set some environment variables: - * `export MESOS_NATIVE_LIBRARY=`. This path is typically + * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. @@ -167,9 +167,6 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). -# Known issues -- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue. - # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 529914a2b6141..249f438459300 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -281,7 +281,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { + if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", """ diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index ed9b207a86a0b..b3bd135548124 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -289,7 +289,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { + if (System.getenv("MESOS_NATIVE_JAVA_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", """ From 45f4c66122c57011e74c694a424756812ab77d99 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 16 Mar 2015 01:06:26 -0700 Subject: [PATCH 418/817] [SPARK-5922][GraphX]: Add diff(other: RDD[VertexId, VD]) in VertexRDD Changed method invocation of 'diff' to match that of 'innerJoin' and 'leftJoin' from VertexRDD[VD] to RDD[(VertexId, VD)]. This change maintains backwards compatibility and better unifies the VertexRDD methods to match each other. Author: Brennon York Closes #4733 from brennonyork/SPARK-5922 and squashes the following commits: e800f08 [Brennon York] fixed merge conflicts b9274af [Brennon York] fixed merge conflicts f86375c [Brennon York] fixed minor include line 398ddb4 [Brennon York] fixed merge conflicts aac1810 [Brennon York] updated to aggregateUsingIndex and added test to ensure that method works properly 2af0b88 [Brennon York] removed deprecation line 753c963 [Brennon York] fixed merge conflicts and set preference to use the diff(other: VertexRDD[VD]) method 2c678c6 [Brennon York] added mima exclude to exclude new public diff method from VertexRDD 93186f3 [Brennon York] added back the original diff method to sustain binary compatibility f18356e [Brennon York] changed method invocation of 'diff' to match that of 'innerJoin' and 'leftJoin' from VertexRDD[VD] to RDD[(VertexId, VD)] --- .../scala/org/apache/spark/graphx/VertexRDD.scala | 9 +++++++++ .../apache/spark/graphx/impl/VertexRDDImpl.scala | 4 ++++ .../org/apache/spark/graphx/VertexRDDSuite.scala | 13 +++++++++++++ project/MimaExcludes.scala | 3 +++ 4 files changed, 29 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 40ecff7107109..ad4bfe077293a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -121,6 +121,15 @@ abstract class VertexRDD[VD]( */ def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] + /** + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other RDD[(VertexId, VD)] with which to diff against. + */ + def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] + /** * For each vertex present in both `this` and `other`, `diff` returns only those vertices with * differing values; for values that are different, keeps the values from `other`. This is diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 904be213147dc..125692ddaad83 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -103,6 +103,10 @@ class VertexRDDImpl[VD] private[graphx] ( override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + diff(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) + } + override def diff(other: VertexRDD[VD]): VertexRDD[VD] = { val otherPartition = other match { case other: VertexRDD[_] if this.partitioner == other.partitioner => diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 97533dd3aa6ce..4f7a442ab503d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite import org.apache.spark.{HashPartitioner, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { @@ -58,6 +59,18 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("diff with RDD[(VertexId, VD)]") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val flipEvens: RDD[(VertexId, Int)] = + sc.parallelize(0L to 100L) + .map(id => if (id % 2 == 0) (id, -id.toInt) else (id, id.toInt)).cache() + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + } + } + test("diff vertices with the non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 627b2cea4d020..a6b07fa7cddec 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -181,6 +181,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.RealClock"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Clock"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.TestClock") + ) ++ Seq( + // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff") ) case v if v.startsWith("1.2") => From 00e730b94cba1202a73af1e2476ff5a44af4b6b2 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Mon, 16 Mar 2015 12:27:15 +0000 Subject: [PATCH 419/817] [SPARK-6300][Spark Core] sc.addFile(path) does not support the relative path. when i run cmd like that sc.addFile("../test.txt"), it did not work and throwed an exception: java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt at org.apache.hadoop.fs.Path.initialize(Path.java:206) at org.apache.hadoop.fs.Path.(Path.java:172) ........ ....... Caused by: java.net.URISyntaxException: Relative path in absolute URI: file:../test.txt at java.net.URI.checkPath(URI.java:1804) at java.net.URI.(URI.java:752) at org.apache.hadoop.fs.Path.initialize(Path.java:203) Author: DoingDone9 <799203320@qq.com> Closes #4993 from DoingDone9/relativePath and squashes the following commits: ee375cd [DoingDone9] Update SparkContextSuite.scala d594e16 [DoingDone9] Update SparkContext.scala 0ff3fa8 [DoingDone9] test for add file dced8eb [DoingDone9] Update SparkContext.scala e4a13fe [DoingDone9] getCanonicalPath 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 51 ++++++++++++++----- 2 files changed, 38 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8121aab3b0b34..4457f40286fda 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1093,7 +1093,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def addFile(path: String, recursive: Boolean): Unit = { val uri = new URI(path) val schemeCorrectedPath = uri.getScheme match { - case null | "local" => "file:" + uri.getPath + case null | "local" => new File(path).getCanonicalFile.toURI.toString case _ => path } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 50f347f1954de..b8e3e83b5a47b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -79,26 +79,49 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { val byteArray2 = converter.convert(bytesWritable) assert(byteArray2.length === 0) } - + test("addFile works") { - val file = File.createTempFile("someprefix", "somesuffix") - val absolutePath = file.getAbsolutePath + val file1 = File.createTempFile("someprefix1", "somesuffix1") + val absolutePath1 = file1.getAbsolutePath + + val pluto = Utils.createTempDir() + val file2 = File.createTempFile("someprefix2", "somesuffix2", pluto) + val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName + val absolutePath2 = file2.getAbsolutePath + try { - Files.write("somewords", file, UTF_8) - val length = file.length() + Files.write("somewords1", file1, UTF_8) + Files.write("somewords2", file2, UTF_8) + val length1 = file1.length() + val length2 = file2.length() + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(file.getAbsolutePath) + sc.addFile(file1.getAbsolutePath) + sc.addFile(relativePath) sc.parallelize(Array(1), 1).map(x => { - val gotten = new File(SparkFiles.get(file.getName)) - if (!gotten.exists()) { - throw new SparkException("file doesn't exist") + val gotten1 = new File(SparkFiles.get(file1.getName)) + val gotten2 = new File(SparkFiles.get(file2.getName)) + if (!gotten1.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath1) + } + if (!gotten2.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath2) } - if (length != gotten.length()) { + + if (length1 != gotten1.length()) { + throw new SparkException( + s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + } + if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length than added file ${gotten.length()}") + s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) } - if (absolutePath == gotten.getAbsolutePath) { - throw new SparkException("file should have been copied") + + if (absolutePath1 == gotten1.getAbsolutePath) { + throw new SparkException("file should have been copied :" + absolutePath1) + } + if (absolutePath2 == gotten2.getAbsolutePath) { + throw new SparkException("file should have been copied : " + absolutePath2) } x }).count() @@ -106,7 +129,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } - + test("addFile recursive works") { val pluto = Utils.createTempDir() val neptune = Utils.createTempDir(pluto.getAbsolutePath) From 12a345adcbaee359199ddfed4f41bf0e19d66d48 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 17 Mar 2015 01:09:27 +0800 Subject: [PATCH 420/817] [SPARK-2087] [SQL] Multiple thriftserver sessions with single HiveContext instance Still, we keep only a single HiveContext within ThriftServer, and we also create a object called `SQLSession` for isolating the different user states. Developers can obtain/release a new user session via `openSession` and `closeSession`, and `SQLContext` and `HiveContext` will also provide a default session if no `openSession` called, for backward-compatibility. Author: Cheng Hao Closes #4885 from chenghao-intel/multisessions_singlecontext and squashes the following commits: 1c47b2a [Cheng Hao] rename the tss => tlSession 815b27a [Cheng Hao] code style issue 57e3fa0 [Cheng Hao] openSession is not compatible between Hive0.12 & 0.13.1 4665b0d [Cheng Hao] thriftservice with single context --- .../org/apache/spark/sql/SQLContext.scala | 43 ++++- .../spark/sql/test/TestSQLContext.scala | 17 +- .../thriftserver/SparkSQLSessionManager.scala | 56 ------ .../HiveThriftServer2Suites.scala | 161 +++++++++++++++++- .../spark/sql/hive/thriftserver/Shim12.scala | 48 +++++- .../spark/sql/hive/thriftserver/Shim13.scala | 49 +++++- .../apache/spark/sql/hive/HiveContext.scala | 70 ++++---- .../apache/spark/sql/hive/test/TestHive.scala | 14 +- 8 files changed, 353 insertions(+), 105 deletions(-) delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9c49e84bf9680..297d0d644a423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -63,8 +63,10 @@ class SQLContext(@transient val sparkContext: SparkContext) def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) - // Note that this is a lazy val so we can override the default value in subclasses. - protected[sql] lazy val conf: SQLConf = new SQLConf + /** + * @return Spark SQL configuration + */ + protected[sql] def conf = tlSession.get().conf /** * Set Spark SQL configuration properties. @@ -103,9 +105,11 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def getAllConfs: immutable.Map[String, String] = conf.getAllConfs + // TODO how to handle the temp table per user session? @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + // TODO how to handle the temp function per user session? @transient protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry(true) @@ -138,6 +142,14 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] def executePlan(plan: LogicalPlan) = new this.QueryExecution(plan) + @transient + protected[sql] val tlSession = new ThreadLocal[SQLSession]() { + override def initialValue = defaultSession + } + + @transient + protected[sql] val defaultSession = createSession() + sparkContext.getConf.getAll.foreach { case (key, value) if key.startsWith("spark.sql") => setConf(key, value) case _ => @@ -194,6 +206,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * }}} * * @group basic + * TODO move to SQLSession? */ @transient val udf: UDFRegistration = new UDFRegistration(this) @@ -1059,6 +1072,32 @@ class SQLContext(@transient val sparkContext: SparkContext) ) } + + protected[sql] def openSession(): SQLSession = { + detachSession() + val session = createSession() + tlSession.set(session) + + session + } + + protected[sql] def currentSession(): SQLSession = { + tlSession.get() + } + + protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[sql] def detachSession(): Unit = { + tlSession.remove() + } + + protected[sql] class SQLSession { + // Note that this is a lazy val so we can override the default value in subclasses. + protected[sql] lazy val conf: SQLConf = new SQLConf + } + /** * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 4e1ec38bd0158..356a6100d2cf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -24,16 +24,22 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** A SQLContext that can be used for local testing. */ -object TestSQLContext +class LocalSQLContext extends SQLContext( new SparkContext( "local[2]", "TestSQLContext", new SparkConf().set("spark.sql.testkey", "true"))) { - /** Fewer partitions to speed up testing. */ - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[sql] class SQLSession extends super.SQLSession { + protected[sql] override lazy val conf: SQLConf = new SQLConf { + /** Fewer partitions to speed up testing. */ + override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + } } /** @@ -45,3 +51,6 @@ object TestSQLContext } } + +object TestSQLContext extends LocalSQLContext + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala deleted file mode 100644 index 89e9ede7261c9..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ /dev/null @@ -1,56 +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.thriftserver - -import java.util.concurrent.Executors - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.session.SessionManager - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager -import org.apache.hive.service.cli.SessionHandle - -private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) - extends SessionManager - with ReflectedCompositeService { - - private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) - setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) - getAncestorField[Log](this, 3, "LOG").info( - s"HiveServer2: Async execution pool size $backgroundPoolSize") - - setSuperField(this, "operationManager", sparkSqlOperationManager) - addService(sparkSqlOperationManager) - - initCompositeService(hiveConf) - } - - override def closeSession(sessionHandle: SessionHandle) { - super.closeSession(sessionHandle) - sparkSqlOperationManager.sessionToActivePool -= sessionHandle - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index d783d487b5c60..aff96e21a5373 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -195,6 +195,146 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } } + + test("test multiple session") { + import org.apache.spark.sql.SQLConf + var defaultV1: String = null + var defaultV2: String = null + + withMultipleConnectionJdbcStatement( + // create table + { statement => + + val queries = Seq( + "DROP TABLE IF EXISTS test_map", + "CREATE TABLE test_map(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map", + "CACHE TABLE test_table AS SELECT key FROM test_map ORDER BY key DESC") + + queries.foreach(statement.execute) + + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + }, + + // first session, we get the default value of the session status + { statement => + + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + defaultV1 = rs1.getString(1) + assert(defaultV1 != "200") + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + + defaultV2 = rs2.getString(1) + assert(defaultV1 != "true") + rs2.close() + }, + + // second session, we update the session status + { statement => + + val queries = Seq( + s"SET ${SQLConf.SHUFFLE_PARTITIONS}=291", + "SET hive.cli.print.header=true" + ) + + queries.map(statement.execute) + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + assert("spark.sql.shuffle.partitions=291" === rs1.getString(1)) + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + assert("hive.cli.print.header=true" === rs2.getString(1)) + rs2.close() + }, + + // third session, we get the latest session status, supposed to be the + // default value + { statement => + + val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}") + rs1.next() + assert(defaultV1 === rs1.getString(1)) + rs1.close() + + val rs2 = statement.executeQuery("SET hive.cli.print.header") + rs2.next() + assert(defaultV2 === rs2.getString(1)) + rs2.close() + }, + + // accessing the cached data in another session + { statement => + + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + statement.executeQuery("UNCACHE TABLE test_table") + + // TODO need to figure out how to determine if the data loaded from cache + val rs3 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf3 = new collection.mutable.ArrayBuffer[Int]() + while (rs3.next()) { + buf3 += rs3.getInt(1) + } + rs3.close() + + assert(buf1 === buf3) + }, + + // accessing the uncached table + { statement => + + // TODO need to figure out how to determine if the data loaded from cache + val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") + val buf1 = new collection.mutable.ArrayBuffer[Int]() + while (rs1.next()) { + buf1 += rs1.getInt(1) + } + rs1.close() + + val rs2 = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") + val buf2 = new collection.mutable.ArrayBuffer[Int]() + while (rs2.next()) { + buf2 += rs2.getInt(1) + } + rs2.close() + + assert(buf1 === buf2) + } + ) + } } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { @@ -245,15 +385,22 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { s"jdbc:hive2://localhost:$serverPort/" } - protected def withJdbcStatement(f: Statement => Unit): Unit = { - val connection = DriverManager.getConnection(jdbcUri, user, "") - val statement = connection.createStatement() - - try f(statement) finally { - statement.close() - connection.close() + def withMultipleConnectionJdbcStatement(fs: (Statement => Unit)*) { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).map { case (s, f) => f(s) } + } finally { + statements.map(_.close()) + connections.map(_.close()) } } + + def withJdbcStatement(f: Statement => Unit) { + withMultipleConnectionJdbcStatement(f) + } } abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll with Logging { diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 13116b40bb259..95a6e86d0546d 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -18,8 +18,15 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} +import java.util.concurrent.Executors import java.util.{ArrayList => JArrayList, Map => JMap} +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.thrift.TProtocolVersion +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} @@ -29,7 +36,7 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation -import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.session.{SessionManager, HiveSession} import org.apache.spark.Logging import org.apache.spark.sql.{DataFrame, SQLConf, Row => SparkRow} @@ -220,3 +227,42 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.FINISHED) } } + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } + + override def openSession( + username: String, + passwd: String, + sessionConf: java.util.Map[String, String], + withImpersonation: Boolean, + delegationToken: String): SessionHandle = { + hiveContext.openSession() + + super.openSession(username, passwd, sessionConf, withImpersonation, delegationToken) + } + + override def closeSession(sessionHandle: SessionHandle) { + super.closeSession(sessionHandle) + sparkSqlOperationManager.sessionToActivePool -= sessionHandle + + hiveContext.detachSession() + } +} diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 9b8faeff94eab..178eb1af7cdcd 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -18,8 +18,15 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} +import java.util.concurrent.Executors import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.thrift.TProtocolVersion +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} @@ -27,7 +34,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation -import org.apache.hive.service.cli.session.HiveSession +import org.apache.hive.service.cli.session.{SessionManager, HiveSession} import org.apache.spark.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf} @@ -191,3 +198,43 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.FINISHED) } } + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + private lazy val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } + + override def openSession( + protocol: TProtocolVersion, + username: String, + passwd: String, + sessionConf: java.util.Map[String, String], + withImpersonation: Boolean, + delegationToken: String): SessionHandle = { + hiveContext.openSession() + + super.openSession(protocol, username, passwd, sessionConf, withImpersonation, delegationToken) + } + + override def closeSession(sessionHandle: SessionHandle) { + super.closeSession(sessionHandle) + sparkSqlOperationManager.sessionToActivePool -= sessionHandle + + hiveContext.detachSession() + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index c439dfe0a71f8..a5c435fdfa778 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -49,10 +49,6 @@ import org.apache.spark.sql.types._ class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - } - /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -214,33 +210,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - /** - * SQLConf and HiveConf contracts: - * - * 1. reuse existing started SessionState if any - * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the - * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be - * set in the SQLConf *as well as* in the HiveConf. - */ - @transient protected[hive] lazy val sessionState: SessionState = { - var state = SessionState.get() - if (state == null) { - state = new SessionState(new HiveConf(classOf[SessionState])) - SessionState.start(state) - } - if (state.out == null) { - state.out = new PrintStream(outputBuffer, true, "UTF-8") - } - if (state.err == null) { - state.err = new PrintStream(outputBuffer, true, "UTF-8") - } - state - } + protected[hive] def sessionState = tlSession.get().asInstanceOf[this.SQLSession].sessionState - @transient protected[hive] lazy val hiveconf: HiveConf = { - setConf(sessionState.getConf.getAllProperties) - sessionState.getConf - } + protected[hive] def hiveconf = tlSession.get().asInstanceOf[this.SQLSession].hiveconf override def setConf(key: String, value: String): Unit = { super.setConf(key, value) @@ -272,6 +244,44 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Nil } + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[hive] class SQLSession extends super.SQLSession { + protected[sql] override lazy val conf: SQLConf = new SQLConf { + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } + + protected[hive] lazy val hiveconf: HiveConf = { + setConf(sessionState.getConf.getAllProperties) + sessionState.getConf + } + + /** + * SQLConf and HiveConf contracts: + * + * 1. reuse existing started SessionState if any + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. + */ + protected[hive] lazy val sessionState: SessionState = { + var state = SessionState.get() + if (state == null) { + state = new SessionState(new HiveConf(classOf[SessionState])) + SessionState.start(state) + } + if (state.out == null) { + state.out = new PrintStream(outputBuffer, true, "UTF-8") + } + if (state.err == null) { + state.err = new PrintStream(outputBuffer, true, "UTF-8") + } + state + } + } + /** * Runs the specified SQL query using Hive. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index a2d99f1f4b28d..4859991e2351a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -102,10 +102,16 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) - /** Fewer partitions to speed up testing. */ - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + override protected[sql] def createSession(): SQLSession = { + new this.SQLSession() + } + + protected[hive] class SQLSession extends super.SQLSession { + /** Fewer partitions to speed up testing. */ + protected[sql] override lazy val conf: SQLConf = new SQLConf { + override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } } /** From d19efeddc0cb710c9496af11e447d39e1ad61b31 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Mon, 16 Mar 2015 12:13:18 -0700 Subject: [PATCH 421/817] [SPARK-6330] Fix filesystem bug in newParquet relation If I'm running this locally and my path points to S3, this would currently error out because of incorrect FS. I tested this in a scenario that previously didn't work, this change seemed to fix the issue. Author: Volodymyr Lyubinets Closes #5020 from vlyubin/parquertbug and squashes the following commits: a645ad5 [Volodymyr Lyubinets] Fix filesystem bug in newParquet relation --- .../main/scala/org/apache/spark/sql/parquet/newParquet.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 234e6bb8443af..c38b6e8c61d8a 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.parquet import java.io.IOException import java.lang.{Double => JDouble, Float => JFloat, Long => JLong} import java.math.{BigDecimal => JBigDecimal} +import java.net.URI import java.text.SimpleDateFormat import java.util.{Date, List => JList} @@ -244,11 +245,10 @@ private[sql] case class ParquetRelation2( * Refreshes `FileStatus`es, footers, partition spec, and table schema. */ def refresh(): Unit = { - val fs = FileSystem.get(sparkContext.hadoopConfiguration) - // Support either reading a collection of raw Parquet part-files, or a collection of folders // containing Parquet files (e.g. partitioned Parquet table). val baseStatuses = paths.distinct.map { p => + val fs = FileSystem.get(URI.create(p), sparkContext.hadoopConfiguration) val qualified = fs.makeQualified(new Path(p)) if (!fs.exists(qualified) && maybeSchema.isDefined) { @@ -262,6 +262,7 @@ private[sql] case class ParquetRelation2( // Lists `FileStatus`es of all leaf nodes (files) under all base directories. val leaves = baseStatuses.flatMap { f => + val fs = FileSystem.get(f.getPath.toUri, sparkContext.hadoopConfiguration) SparkHadoopUtil.get.listLeafStatuses(fs, f.getPath).filter { f => isSummaryFile(f.getPath) || !(f.getPath.getName.startsWith("_") || f.getPath.getName.startsWith(".")) From f149b8b5e542af44650923d0156f037121b45a20 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Mon, 16 Mar 2015 13:10:32 -0700 Subject: [PATCH 422/817] [SPARK-6077] Remove streaming tab while stopping StreamingContext Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect. snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0 How to reproduce: 1) import org.apache.spark.SparkConf import org.apache.spark.streaming. {Seconds, StreamingContext} import org.apache.spark.storage.StorageLevel val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() ..... 2) ssc.stop(false) val ssc = new StreamingContext(sc, Seconds(1)) val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() Author: lisurprise Closes #4828 from zhichao-li/master and squashes the following commits: c329806 [lisurprise] add test for attaching/detaching streaming tab 51e6c7f [lisurprise] move detach method into StreamingTab 31a44fa [lisurprise] add unit test for attaching and detaching new tab db25ed2 [lisurprise] clean code 8281bcb [lisurprise] clean code 193c542 [lisurprise] remove streaming tab while closing streaming context --- .../scala/org/apache/spark/ui/WebUI.scala | 28 +++++- .../org/apache/spark/ui/UISeleniumSuite.scala | 50 +++++++++- .../scala/org/apache/spark/ui/UISuite.scala | 38 +------- streaming/pom.xml | 5 + .../spark/streaming/StreamingContext.scala | 1 + .../spark/streaming/ui/StreamingPage.scala | 4 +- .../spark/streaming/ui/StreamingTab.scala | 4 + .../spark/streaming/UISeleniumSuite.scala | 95 +++++++++++++++++++ .../org/apache/spark/streaming/UISuite.scala | 55 ----------- 9 files changed, 179 insertions(+), 101 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala delete mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ec68837a1516c..ea548f23120d9 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,14 +20,15 @@ package org.apache.spark.ui import javax.servlet.http.HttpServletRequest import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler import org.json4s.JsonAST.{JNothing, JValue} -import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * The top level component of the UI hierarchy that contains the server. @@ -45,6 +46,7 @@ private[spark] abstract class WebUI( protected val tabs = ArrayBuffer[WebUITab]() protected val handlers = ArrayBuffer[ServletContextHandler]() + protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None protected val localHostName = Utils.localHostName() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) @@ -60,14 +62,30 @@ private[spark] abstract class WebUI( tab.pages.foreach(attachPage) tabs += tab } + + def detachTab(tab: WebUITab) { + tab.pages.foreach(detachPage) + tabs -= tab + } + + def detachPage(page: WebUIPage) { + pageToHandlers.remove(page).foreach(_.foreach(detachHandler)) + } /** Attach a page to this UI. */ def attachPage(page: WebUIPage) { val pagePath = "/" + page.prefix - attachHandler(createServletHandler(pagePath, - (request: HttpServletRequest) => page.render(request), securityManager, basePath)) - attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + val renderHandler = createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath) + val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) + attachHandler(renderHandler) + attachHandler(renderJsonHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderJsonHandler) + } /** Attach a handler to this UI. */ 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 6a972381faf14..0d155982a8c54 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,20 +17,24 @@ package org.apache.spark.ui +import javax.servlet.http.HttpServletRequest + import scala.collection.JavaConversions._ +import scala.xml.Node -import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.openqa.selenium.{By, WebDriver} import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ -import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException + /** * Selenium tests for the Spark Web UI. */ @@ -310,4 +314,46 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("attaching and detaching a new tab") { + withSpark(newSparkContext()) { sc => + val sparkUI = sc.ui.get + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should not be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check whether new page exists + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")).get.text should include ("html magic") + } + sparkUI.detachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check new page not exist + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")) should be(None) + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 92a21f82f3c21..77a038dc1720d 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.ui import java.net.ServerSocket -import javax.servlet.http.HttpServletRequest import scala.io.Source import scala.util.{Failure, Success, Try} @@ -28,9 +27,8 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.LocalSparkContext._ -import scala.xml.Node +import org.apache.spark.{SparkConf, SparkContext} class UISuite extends FunSuite { @@ -72,40 +70,6 @@ class UISuite extends FunSuite { } } - ignore("attaching a new tab") { - withSpark(newSparkContext()) { sc => - val sparkUI = sc.ui.get - - val newTab = new WebUITab(sparkUI, "foo") { - attachPage(new WebUIPage("") { - def render(request: HttpServletRequest): Seq[Node] = { - "html magic" - } - }) - } - sparkUI.attachTab(newTab) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress).mkString - assert(!html.contains("random data that should not be present")) - - // check whether new page exists - assert(html.toLowerCase.contains("foo")) - - // check whether other pages still exist - assert(html.toLowerCase.contains("stages")) - assert(html.toLowerCase.contains("storage")) - assert(html.toLowerCase.contains("environment")) - assert(html.toLowerCase.contains("executors")) - } - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString - // check whether new page exists - assert(html.contains("magic")) - } - } - } - test("jetty selects different port under contention") { val server = new ServerSocket(0) val startPort = server.getLocalPort diff --git a/streaming/pom.xml b/streaming/pom.xml index 0370b0e9e1aa3..96508d83f4049 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -82,6 +82,11 @@ junit test + + org.seleniumhq.selenium + selenium-java + test + com.novocode junit-interface 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 ba3f23434f24c..b5b6770a8a150 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -578,6 +578,7 @@ class StreamingContext private[streaming] ( // Even if we have already stopped, we still need to attempt to stop the SparkContext because // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true). if (stopSparkContext) sc.stop() + uiTab.foreach(_.detach()) // The state should always be Stopped after calling `stop()`, even if we haven't started yet: state = Stopped } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 98e9a2e639e25..bfe8086fcf8fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -32,7 +32,7 @@ private[ui] class StreamingPage(parent: StreamingTab) extends WebUIPage("") with Logging { private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + private val startTime = System.currentTimeMillis() private val emptyCell = "-" /** Render the page */ @@ -47,7 +47,7 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { - val timeSinceStart = System.currentTimeMillis() - startTime.getTime + val timeSinceStart = System.currentTimeMillis() - startTime
    • Started at: {startTime.toString} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index d9d04cd706a04..9a860ea4a6c68 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -36,6 +36,10 @@ private[spark] class StreamingTab(ssc: StreamingContext) ssc.addStreamingListener(listener) attachPage(new StreamingPage(this)) parent.attachTab(this) + + def detach() { + getSparkUI(ssc).detachTab(this) + } } private object StreamingTab { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala new file mode 100644 index 0000000000000..87a0395efbf2a --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.scalatest._ +import org.scalatest.concurrent.Eventually._ +import org.scalatest.selenium.WebBrowser +import org.scalatest.time.SpanSugar._ + +import org.apache.spark._ + + + + +/** + * Selenium tests for the Spark Web UI. + */ +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { + + implicit var webDriver: WebDriver = _ + + override def beforeAll(): Unit = { + webDriver = new HtmlUnitDriver + } + + override def afterAll(): Unit = { + if (webDriver != null) { + webDriver.quit() + } + } + + /** + * Create a test SparkStreamingContext with the SparkUI enabled. + */ + private def newSparkStreamingContext(): StreamingContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val ssc = new StreamingContext(conf, Seconds(1)) + assert(ssc.sc.ui.isDefined, "Spark UI is not started!") + ssc + } + + test("attaching and detaching a Streaming tab") { + withStreamingContext(newSparkStreamingContext()) { ssc => + val sparkUI = ssc.sparkContext.ui.get + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/")) + find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check whether streaming page exists + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq + statisticText should contain("Network receivers:") + statisticText should contain("Batch interval:") + } + + ssc.stop(false) + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/")) + find(cssSelector( """ul li a[href*="streaming"]""")) should be(None) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq + statisticText should not contain ("Network receivers:") + statisticText should not contain ("Batch interval:") + } + } + } +} + diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala deleted file mode 100644 index 8e30118266855..0000000000000 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming - -import scala.io.Source - -import org.scalatest.FunSuite -import org.scalatest.concurrent.Eventually._ -import org.scalatest.time.SpanSugar._ - -import org.apache.spark.SparkConf - -class UISuite extends FunSuite { - - // Ignored: See SPARK-1530 - ignore("streaming tab in spark UI") { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - val ssc = new StreamingContext(conf, Seconds(1)) - assert(ssc.sc.ui.isDefined, "Spark UI is not started!") - val ui = ssc.sc.ui.get - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ui.appUIAddress).mkString - assert(!html.contains("random data that should not be present")) - // test if streaming tab exist - assert(html.toLowerCase.contains("streaming")) - // test if other Spark tabs still exist - assert(html.toLowerCase.contains("stages")) - } - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ui.appUIAddress.stripSuffix("/") + "/streaming").mkString - assert(html.toLowerCase.contains("batch")) - assert(html.toLowerCase.contains("network")) - } - } -} From e3f315ac358dfe4f5b9705c3eac76e8b1e24f82a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 16 Mar 2015 16:26:55 -0700 Subject: [PATCH 423/817] [SPARK-6327] [PySpark] fix launch spark-submit from python SparkSubmit should be launched without setting PYSPARK_SUBMIT_ARGS cc JoshRosen , this mode is actually used by python unit test, so I will not add more test for it. Author: Davies Liu Closes #5019 from davies/fix_submit and squashes the following commits: 2c20b0c [Davies Liu] fix launch spark-submit from python --- bin/pyspark | 1 - python/pyspark/java_gateway.py | 6 ++---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index e7f6a1a072c2a..776b28dc41099 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,7 +89,6 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR - export PYSPARK_SUBMIT_ARGS=pyspark-shell if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 43d2cf5171880..0a16cbd8bff62 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -38,10 +38,8 @@ def launch_gateway(): # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" script = "./bin/spark-submit.cmd" if on_windows else "./bin/spark-submit" - submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") - submit_args = submit_args if submit_args is not None else "" - submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script)] + submit_args + submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") + command = [os.path.join(SPARK_HOME, script)] + shlex.split(submit_args) # Start a socket that will be used by PythonGatewayServer to communicate its port to us callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) From 9667b9f9c3239f814a0b1120355d9e7bd7a89158 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 17 Mar 2015 12:29:15 +0800 Subject: [PATCH 424/817] [SPARK-5712] [SQL] fix comment with semicolon at end ---- comment; Author: Daoyuan Wang Closes #4500 from adrian-wang/semicolon and squashes the following commits: 70b8abb [Daoyuan Wang] use mkstring instead of reduce 2d49738 [Daoyuan Wang] remove outdated golden file 317346e [Daoyuan Wang] only skip comment with semicolon at end of line, to avoid golden file outdated d3ae01e [Daoyuan Wang] fix error a11602d [Daoyuan Wang] fix comment with semicolon at end --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 25 ++++++++++--------- .../execution/HiveCompatibilitySuite.scala | 1 + ...micolon-0-f104632770dc96b81f00ccdac51fe5a8 | 1 + .../hive/execution/HiveComparisonTest.scala | 5 +++- 4 files changed, 19 insertions(+), 13 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 401e97b162dea..895688ab2ec2e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -202,20 +202,21 @@ private[hive] object SparkSQLCLIDriver { var line = reader.readLine(currentPrompt + "> ") while (line != null) { - if (prefix.nonEmpty) { - prefix += '\n' - } + if (!line.startsWith("--")) { + if (prefix.nonEmpty) { + prefix += '\n' + } - if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { - line = prefix + line - ret = cli.processLine(line, true) - prefix = "" - currentPrompt = promptWithCurrentDB - } else { - prefix = prefix + line - currentPrompt = continuedPromptWithDBSpaces + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } } - line = reader.readLine(currentPrompt + "> ") } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 6126ce7130426..68cb34d698ef3 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -726,6 +726,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "select_unquote_and", "select_unquote_not", "select_unquote_or", + "semicolon", "semijoin", "serde_regex", "serde_reported_schema", diff --git a/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 b/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/semicolon-0-f104632770dc96b81f00ccdac51fe5a8 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a90bd1e257ade..8f3285242091c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -241,7 +241,10 @@ abstract class HiveComparisonTest // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) - val allQueries = sql.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq + val sqlWithoutComment = + sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") + val allQueries = + sqlWithoutComment.split("(?<=[^\\\\]);").map(_.trim).filterNot(q => q == "").toSeq // TODO: DOCUMENT UNSUPPORTED val queryList = From f0edeae7f9ab7eae02c227be9162ec69d22c92bd Mon Sep 17 00:00:00 2001 From: "Kevin (Sangwoo) Kim" Date: Mon, 16 Mar 2015 23:49:23 -0700 Subject: [PATCH 425/817] [SPARK-6299][CORE] ClassNotFoundException in standalone mode when running groupByKey with class defined in REPL ``` case class ClassA(value: String) val rdd = sc.parallelize(List(("k1", ClassA("v1")), ("k1", ClassA("v2")) )) rdd.groupByKey.collect ``` This code used to be throw exception in spark-shell, because while shuffling ```JavaSerializer```uses ```defaultClassLoader``` which was defined like ```env.serializer.setDefaultClassLoader(urlClassLoader)```. It should be ```env.serializer.setDefaultClassLoader(replClassLoader)```, like ``` override def run() { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) ``` in TaskRunner. When ```replClassLoader``` cannot be defined, it's identical with ```urlClassLoader``` Author: Kevin (Sangwoo) Kim Closes #5046 from swkimme/master and squashes the following commits: fa2b9ee [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) 6e9620b [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) d23e4e2 [Kevin (Sangwoo) Kim] stylish test codes ( collect -> collect() ) a4a3c8a [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite bd00da5 [Kevin (Sangwoo) Kim] add 'class defined in repl - shuffle' test to ReplSuite c1b1fc7 [Kevin (Sangwoo) Kim] use REPL class loader for executor's serializer --- .../org/apache/spark/executor/Executor.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 50 ++++++++++++------- .../org/apache/spark/repl/ReplSuite.scala | 50 ++++++++++++------- 3 files changed, 63 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a897e532184ac..6196f7b165049 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -103,7 +103,7 @@ private[spark] class Executor( private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) // Set the classloader for serializer - env.serializer.setDefaultClassLoader(urlClassLoader) + env.serializer.setDefaultClassLoader(replClassLoader) // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. diff --git a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 249f438459300..934daaeaafca1 100644 --- a/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -121,9 +121,9 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -137,7 +137,7 @@ class ReplSuite extends FunSuite { |class C { |def foo = 5 |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => (new C).foo).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -148,7 +148,7 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => double(x)).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -160,9 +160,9 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -178,9 +178,9 @@ class ReplSuite extends FunSuite { """ |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -216,14 +216,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -262,7 +262,7 @@ class ReplSuite extends FunSuite { |val sqlContext = new org.apache.spark.sql.SQLContext(sc) |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect() + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF().collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -275,7 +275,7 @@ class ReplSuite extends FunSuite { |val t = new TestClass |import t.testMethod |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -287,14 +287,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -309,10 +309,22 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local[2]", """ |case class Foo(i: Int) - |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("ret: Array[Foo] = Array(Foo(1),", output) } + + test("collecting objects of class defined in repl - shuffling") { + val output = runInterpreter("local-cluster[1,1,512]", + """ + |case class Foo(i: Int) + |val list = List((1, Foo(1)), (1, Foo(2))) + |val ret = sc.parallelize(list).groupByKey().collect() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output) + } } diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index b3bd135548124..fbef5b25ba688 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -128,9 +128,9 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => v).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -144,7 +144,7 @@ class ReplSuite extends FunSuite { |class C { |def foo = 5 |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => (new C).foo).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -155,7 +155,7 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local", """ |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => double(x)).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -167,9 +167,9 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -185,9 +185,9 @@ class ReplSuite extends FunSuite { """ |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -224,14 +224,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -270,7 +270,7 @@ class ReplSuite extends FunSuite { |val sqlContext = new org.apache.spark.sql.SQLContext(sc) |import sqlContext.implicits._ |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF().collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -283,7 +283,7 @@ class ReplSuite extends FunSuite { |val t = new TestClass |import t.testMethod |case class TestCaseClass(value: Int) - |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -295,14 +295,14 @@ class ReplSuite extends FunSuite { """ |var v = 7 |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |sc.parallelize(1 to 10).map(x => getV()).collect().reduceLeft(_+_) |var array = new Array[Int](5) |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) @@ -317,10 +317,22 @@ class ReplSuite extends FunSuite { val output = runInterpreter("local[2]", """ |case class Foo(i: Int) - |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect() """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("ret: Array[Foo] = Array(Foo(1),", output) } + + test("collecting objects of class defined in repl - shuffling") { + val output = runInterpreter("local-cluster[1,1,512]", + """ + |case class Foo(i: Int) + |val list = List((1, Foo(1)), (1, Foo(2))) + |val ret = sc.parallelize(list).groupByKey().collect() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[(Int, Iterable[Foo])] = Array((1,", output) + } } From 68707225f1a4081aadbf0fd7e6221293a190529b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lomig=20Me=CC=81gard?= Date: Mon, 16 Mar 2015 23:52:42 -0700 Subject: [PATCH 426/817] [SQL][docs][minor] Fixed sample code in SQLContext scaladoc MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Error in the code sample of the `implicits` object in `SQLContext`. Author: Lomig Mégard Closes #5051 from tarfaa/simple and squashes the following commits: 5a88acc [Lomig Mégard] [docs][minor] Fixed sample code in SQLContext scaladoc --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 297d0d644a423..6de46a50db20e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -242,8 +242,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * common Scala objects into [[DataFrame]]s. * * {{{ - * val sqlContext = new SQLContext - * import sqlContext._ + * val sqlContext = new SQLContext(sc) + * import sqlContext.implicits._ * }}} * * @group basic From b3e6eca81f79ba3c9205211797fa825b199bac83 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Mon, 16 Mar 2015 23:54:54 -0700 Subject: [PATCH 427/817] [SPARK-6357][GraphX] Add unapply in EdgeContext This extractor is mainly used for Graph#aggregateMessages*. Author: Takeshi YAMAMURO Closes #5047 from maropu/AddUnapplyInEdgeContext and squashes the following commits: 87e04df [Takeshi YAMAMURO] Add unapply in EdgeContext --- .../org/apache/spark/graphx/EdgeContext.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index f70715fca6eea..d8be02e2023d5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -49,3 +49,20 @@ abstract class EdgeContext[VD, ED, A] { et } } + +object EdgeContext { + + /** + * Extractor mainly used for Graph#aggregateMessages*. + * Example: + * {{{ + * val messages = graph.aggregateMessages( + * case ctx @ EdgeContext(_, _, _, _, attr) => + * ctx.sendToDst(attr) + * , _ + _) + * }}} + */ + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]) = + Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) +} + From b2d8c02224892192b1aa314b4265fe50845932f9 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 16 Mar 2015 23:58:52 -0700 Subject: [PATCH 428/817] SPARK-6044 [CORE] RDD.aggregate() should not use the closure serializer on the zero value Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer. Compare with https://github.com/apache/spark/blob/e60ad2f4c47b011be7a3198689ac2b82ee317d96/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L127 Author: Sean Owen Closes #5028 from srowen/SPARK-6044 and squashes the following commits: a4040a7 [Sean Owen] Use configured serializer in RDD.aggregate, to match PairRDDFunctions.aggregateByKey, instead of closure serializer --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index cf0433010aa03..a139780d967e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -960,7 +960,7 @@ abstract class RDD[T: ClassTag]( */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) val cleanCombOp = sc.clean(combOp) val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) From 25f35806e307c9635e63b8b12698446a14bdd29d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 17 Mar 2015 11:18:27 +0000 Subject: [PATCH 429/817] [SPARK-4011] tighten the visibility of the members in Master/Worker class https://issues.apache.org/jira/browse/SPARK-4011 Currently, most of the members in Master/Worker are with public accessibility. We might wish to tighten the accessibility of them a bit more discussion is here: https://github.com/apache/spark/pull/2828 Author: CodingCat Closes #4844 from CodingCat/SPARK-4011 and squashes the following commits: 1a64175 [CodingCat] fix compilation issue e7fd375 [CodingCat] Sean is right.... f5034a4 [CodingCat] fix rebase mistake 8d5b0c0 [CodingCat] loose more fields 0072f96 [CodingCat] lose some restrictions based on the possible design intention de77286 [CodingCat] tighten accessibility of deploy package 12b4fd3 [CodingCat] tighten accessibility of deploy.worker 1243bc7 [CodingCat] tighten accessibility of deploy.rest c5f622c [CodingCat] tighten the accessibility of deploy.history d441e20 [CodingCat] tighten accessibility of deploy.client 4e0ce4a [CodingCat] tighten the accessibility of the members of classes in master 23cddbb [CodingCat] stylistic fix 9a3a340 [CodingCat] tighten the access of worker class 67a0559 [CodingCat] tighten the access permission in Master --- .../apache/spark/deploy/ClientArguments.scala | 12 +- .../spark/deploy/DriverDescription.scala | 2 +- .../spark/deploy/ExecutorDescription.scala | 2 +- .../apache/spark/deploy/ExecutorState.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 60 +++++----- .../apache/spark/deploy/JsonProtocol.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 38 +++--- .../spark/deploy/SparkSubmitArguments.scala | 2 +- .../spark/deploy/client/AppClient.scala | 18 +-- .../spark/deploy/client/TestClient.scala | 2 +- .../history/ApplicationHistoryProvider.scala | 4 +- .../spark/deploy/history/HistoryPage.scala | 2 +- .../history/HistoryServerArguments.scala | 3 +- .../spark/deploy/master/ApplicationInfo.scala | 19 +-- .../deploy/master/ApplicationSource.scala | 2 +- .../deploy/master/ApplicationState.scala | 2 +- .../spark/deploy/master/DriverInfo.scala | 2 +- .../spark/deploy/master/DriverState.scala | 2 +- .../spark/deploy/master/ExecutorDesc.scala | 2 +- .../master/FileSystemPersistenceEngine.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 110 +++++++++--------- .../spark/deploy/master/MasterArguments.scala | 6 +- .../deploy/master/PersistenceEngine.scala | 2 +- .../deploy/master/RecoveryModeFactory.scala | 4 +- .../spark/deploy/master/RecoveryState.scala | 2 +- .../deploy/master/SparkCuratorUtil.scala | 10 +- .../spark/deploy/master/WorkerState.scala | 2 +- .../master/ZooKeeperLeaderElectionAgent.scala | 6 +- .../master/ZooKeeperPersistenceEngine.scala | 12 +- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../master/ui/HistoryNotFoundPage.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 2 +- .../spark/deploy/master/ui/MasterWebUI.scala | 6 +- .../deploy/rest/StandaloneRestClient.scala | 14 +-- .../deploy/rest/StandaloneRestServer.scala | 2 +- .../rest/SubmitRestProtocolException.scala | 6 +- .../rest/SubmitRestProtocolMessage.scala | 2 +- .../rest/SubmitRestProtocolRequest.scala | 4 +- .../rest/SubmitRestProtocolResponse.scala | 10 +- .../spark/deploy/worker/CommandUtils.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 24 ++-- .../spark/deploy/worker/ExecutorRunner.scala | 24 ++-- .../apache/spark/deploy/worker/Worker.scala | 85 +++++++------- .../spark/deploy/worker/WorkerArguments.scala | 4 +- .../spark/deploy/worker/WorkerSource.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../spark/deploy/worker/ui/LogPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerPage.scala | 7 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 6 +- 49 files changed, 277 insertions(+), 265 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 415bd50591692..53bc62aff7395 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.{IntParam, MemoryParam} /** * Command-line parser for the driver client. */ -private[spark] class ClientArguments(args: Array[String]) { +private[deploy] class ClientArguments(args: Array[String]) { import ClientArguments._ var cmd: String = "" // 'launch' or 'kill' @@ -96,7 +96,7 @@ private[spark] class ClientArguments(args: Array[String]) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. val usage = @@ -116,10 +116,10 @@ private[spark] class ClientArguments(args: Array[String]) { } } -object ClientArguments { - private[spark] val DEFAULT_CORES = 1 - private[spark] val DEFAULT_MEMORY = 512 // MB - private[spark] val DEFAULT_SUPERVISE = false +private[deploy] object ClientArguments { + val DEFAULT_CORES = 1 + val DEFAULT_MEMORY = 512 // MB + val DEFAULT_SUPERVISE = false def isValidJarUrl(s: String): Boolean = { try { diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index b056a19ce6598..659fb434a80f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] class DriverDescription( +private[deploy] class DriverDescription( val jarUrl: String, val mem: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 2abf0b69dddb3..ec23371b52f93 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -22,7 +22,7 @@ package org.apache.spark.deploy * This state is sufficient for the Master to reconstruct its internal data structures during * failover. */ -private[spark] class ExecutorDescription( +private[deploy] class ExecutorDescription( val appId: String, val execId: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 9f34d01e6db48..efa88c62e1f5d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] object ExecutorState extends Enumeration { +private[deploy] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST, EXITED = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 47dbcd87c35b5..4e58aa0ed4c7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -55,29 +55,29 @@ import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} * - The docker images tagged spark-test-master and spark-test-worker are built from the * docker/ directory. Run 'docker/spark-test/build' to generate these. */ -private[spark] object FaultToleranceTest extends App with Logging { +private object FaultToleranceTest extends App with Logging { - val conf = new SparkConf() - val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val conf = new SparkConf() + private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") - val masters = ListBuffer[TestMasterInfo]() - val workers = ListBuffer[TestWorkerInfo]() - var sc: SparkContext = _ + private val masters = ListBuffer[TestMasterInfo]() + private val workers = ListBuffer[TestWorkerInfo]() + private var sc: SparkContext = _ - val zk = SparkCuratorUtil.newClient(conf) + private val zk = SparkCuratorUtil.newClient(conf) - var numPassed = 0 - var numFailed = 0 + private var numPassed = 0 + private var numFailed = 0 - val sparkHome = System.getenv("SPARK_HOME") + private val sparkHome = System.getenv("SPARK_HOME") assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") - val containerSparkHome = "/opt/spark" - val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) + private val containerSparkHome = "/opt/spark" + private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip - def afterEach() { + private def afterEach() { if (sc != null) { sc.stop() sc = null @@ -179,7 +179,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def test(name: String)(fn: => Unit) { + private def test(name: String)(fn: => Unit) { try { fn numPassed += 1 @@ -197,19 +197,19 @@ private[spark] object FaultToleranceTest extends App with Logging { afterEach() } - def addMasters(num: Int) { + private def addMasters(num: Int) { logInfo(s">>>>> ADD MASTERS $num <<<<<") (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } } - def addWorkers(num: Int) { + private def addWorkers(num: Int) { logInfo(s">>>>> ADD WORKERS $num <<<<<") val masterUrls = getMasterUrls(masters) (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ - def createClient() = { + private def createClient() = { logInfo(">>>>> CREATE CLIENT <<<<<") if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this @@ -218,17 +218,17 @@ private[spark] object FaultToleranceTest extends App with Logging { sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } - def getMasterUrls(masters: Seq[TestMasterInfo]): String = { + private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { "spark://" + masters.map(master => master.ip + ":7077").mkString(",") } - def getLeader: TestMasterInfo = { + private def getLeader: TestMasterInfo = { val leaders = masters.filter(_.state == RecoveryState.ALIVE) assertTrue(leaders.size == 1) leaders(0) } - def killLeader(): Unit = { + private def killLeader(): Unit = { logInfo(">>>>> KILL LEADER <<<<<") masters.foreach(_.readState()) val leader = getLeader @@ -236,9 +236,9 @@ private[spark] object FaultToleranceTest extends App with Logging { leader.kill() } - def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) + private def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) - def terminateCluster() { + private def terminateCluster() { logInfo(">>>>> TERMINATE CLUSTER <<<<<") masters.foreach(_.kill()) workers.foreach(_.kill()) @@ -247,7 +247,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } /** This includes Client retry logic, so it may take a while if the cluster is recovering. */ - def assertUsable() = { + private def assertUsable() = { val f = future { try { val res = sc.parallelize(0 until 10).collect() @@ -269,7 +269,7 @@ private[spark] object FaultToleranceTest extends App with Logging { * Asserts that the cluster is usable and that the expected masters and workers * are all alive in a proper configuration (e.g., only one leader). */ - def assertValidClusterState() = { + private def assertValidClusterState() = { logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") assertUsable() var numAlive = 0 @@ -325,7 +325,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def assertTrue(bool: Boolean, message: String = "") { + private def assertTrue(bool: Boolean, message: String = "") { if (!bool) { throw new IllegalStateException("Assertion failed: " + message) } @@ -335,7 +335,7 @@ private[spark] object FaultToleranceTest extends App with Logging { numFailed)) } -private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -377,7 +377,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val format(ip, dockerId.id, logFile.getAbsolutePath, state) } -private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -390,7 +390,7 @@ private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath) } -private[spark] object SparkDocker { +private object SparkDocker { def startMaster(mountDir: String): TestMasterInfo = { val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir) val (ip, id, outFile) = startNode(cmd) @@ -425,11 +425,11 @@ private[spark] object SparkDocker { } } -private[spark] class DockerId(val id: String) { +private class DockerId(val id: String) { override def toString = id } -private[spark] object Docker extends Logging { +private object Docker extends Logging { def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 696f32a6f5730..458a7c3a455de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -23,7 +23,7 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateR import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner -private[spark] object JsonProtocol { +private[deploy] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4a74641f4e1fa..4f506be63fe59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, U * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone cluster mode. */ -private[spark] object SparkSubmitAction extends Enumeration { +private[deploy] object SparkSubmitAction extends Enumeration { type SparkSubmitAction = Value val SUBMIT, KILL, REQUEST_STATUS = Value } @@ -137,7 +137,7 @@ object SparkSubmit { * Second, we use this launch environment to invoke the main method of the child * main class. */ - private[spark] def submit(args: SparkSubmitArguments): Unit = { + private def submit(args: SparkSubmitArguments): Unit = { val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args) def doRunMain(): Unit = { @@ -199,7 +199,7 @@ object SparkSubmit { * (4) the main class for the child * Exposed for testing. */ - private[spark] def prepareSubmitEnvironment(args: SparkSubmitArguments) + private[deploy] def prepareSubmitEnvironment(args: SparkSubmitArguments) : (Seq[String], Seq[String], Map[String, String], String) = { // Return values val childArgs = new ArrayBuffer[String]() @@ -598,32 +598,32 @@ object SparkSubmit { /** * Return whether the given primary resource represents a shell. */ - private[spark] def isShell(primaryResource: String): Boolean = { + private[deploy] def isShell(primaryResource: String): Boolean = { primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL } /** * Return whether the given main class represents a sql shell. */ - private[spark] def isSqlShell(mainClass: String): Boolean = { + private def isSqlShell(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" } /** * Return whether the given main class represents a thrift server. */ - private[spark] def isThriftServer(mainClass: String): Boolean = { + private def isThriftServer(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" } /** * Return whether the given primary resource requires running python. */ - private[spark] def isPython(primaryResource: String): Boolean = { + private[deploy] def isPython(primaryResource: String): Boolean = { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { + private[deploy] def isInternal(primaryResource: String): Boolean = { primaryResource == SPARK_INTERNAL } @@ -631,7 +631,7 @@ object SparkSubmit { * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. */ - private[spark] def mergeFileLists(lists: String*): String = { + private def mergeFileLists(lists: String*): String = { val merged = lists.filter(_ != null) .flatMap(_.split(",")) .mkString(",") @@ -640,10 +640,10 @@ object SparkSubmit { } /** Provides utility functions to be used inside SparkSubmit. */ -private[spark] object SparkSubmitUtils { +private[deploy] object SparkSubmitUtils { // Exposed for testing - private[spark] var printStream = SparkSubmit.printStream + var printStream = SparkSubmit.printStream /** * Represents a Maven Coordinate @@ -651,7 +651,7 @@ private[spark] object SparkSubmitUtils { * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate */ - private[spark] case class MavenCoordinate(groupId: String, artifactId: String, version: String) + private[deploy] case class MavenCoordinate(groupId: String, artifactId: String, version: String) /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided @@ -659,7 +659,7 @@ private[spark] object SparkSubmitUtils { * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ - private[spark] def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { + def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { coordinates.split(",").map { p => val splits = p.replace("/", ":").split(":") require(splits.length == 3, s"Provided Maven Coordinates must be in the form " + @@ -679,7 +679,7 @@ private[spark] object SparkSubmitUtils { * @param remoteRepos Comma-delimited string of remote repositories * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ - private[spark] def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { + def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { // We need a chain resolver if we want to check multiple repositories val cr = new ChainResolver cr.setName("list") @@ -722,7 +722,7 @@ private[spark] object SparkSubmitUtils { * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies */ - private[spark] def resolveDependencyPaths( + def resolveDependencyPaths( artifacts: Array[AnyRef], cacheDirectory: File): String = { artifacts.map { artifactInfo => @@ -734,7 +734,7 @@ private[spark] object SparkSubmitUtils { } /** Adds the given maven coordinates to Ivy's module descriptor. */ - private[spark] def addDependenciesToIvy( + def addDependenciesToIvy( md: DefaultModuleDescriptor, artifacts: Seq[MavenCoordinate], ivyConfName: String): Unit = { @@ -748,7 +748,7 @@ private[spark] object SparkSubmitUtils { } /** Add exclusion rules for dependencies already included in the spark-assembly */ - private[spark] def addExclusionRules( + def addExclusionRules( ivySettings: IvySettings, ivyConfName: String, md: DefaultModuleDescriptor): Unit = { @@ -777,7 +777,7 @@ private[spark] object SparkSubmitUtils { } /** A nice function to use in tests as well. Values are dummy strings. */ - private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) /** @@ -788,7 +788,7 @@ private[spark] object SparkSubmitUtils { * @return The comma-delimited path to the jars of the given maven artifacts including their * transitive dependencies */ - private[spark] def resolveMavenCoordinates( + def resolveMavenCoordinates( coordinates: String, remoteRepos: Option[String], ivyPath: Option[String], diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 94e4bdbfb7d7b..2250d5a28e4ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Parses and encapsulates arguments from the spark-submit script. * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) +private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index ffe940fbda2fb..2d24083a77b73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -47,18 +47,18 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) + private val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) - val REGISTRATION_TIMEOUT = 20.seconds - val REGISTRATION_RETRIES = 3 + private val REGISTRATION_TIMEOUT = 20.seconds + private val REGISTRATION_RETRIES = 3 - var masterAddress: Address = null - var actor: ActorRef = null - var appId: String = null - var registered = false - var activeMasterUrl: String = null + private var masterAddress: Address = null + private var actor: ActorRef = null + private var appId: String = null + private var registered = false + private var activeMasterUrl: String = null - class ClientActor extends Actor with ActorLogReceive with Logging { + private class ClientActor extends Actor with ActorLogReceive with Logging { var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 88a0862b96afe..c1c4812f17fbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { - class TestListener extends AppClientListener with Logging { + private class TestListener extends AppClientListener with Logging { def connected(id: String) { logInfo("Connected to master, got app ID " + id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 553bf3cb945ab..ea6c85ee511d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[spark] case class ApplicationHistoryInfo( +private[history] case class ApplicationHistoryInfo( id: String, name: String, startTime: Long, @@ -28,7 +28,7 @@ private[spark] case class ApplicationHistoryInfo( sparkUser: String, completed: Boolean = false) -private[spark] abstract class ApplicationHistoryProvider { +private[history] abstract class ApplicationHistoryProvider { /** * Returns a list of applications available for the history server to show. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 26ebc75971c66..6e432d63c6b5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -23,7 +23,7 @@ import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { +private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 private val plusOrMinus = 2 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index b1270ade9f750..a2a97a7877ce7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -23,7 +23,8 @@ import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { +private[history] class HistoryServerArguments(conf: SparkConf, args: Array[String]) + extends Logging { private var propertiesFile: String = null parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index a962dc4af2f6c..536aedb6f9fe9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[spark] class ApplicationInfo( +private[deploy] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, @@ -75,14 +75,15 @@ private[spark] class ApplicationInfo( } } - def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = { + private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): + ExecutorDesc = { val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) executors(exec.id) = exec coresGranted += cores exec } - def removeExecutor(exec: ExecutorDesc) { + private[master] def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.id)) { removedExecutors += executors(exec.id) executors -= exec.id @@ -90,22 +91,22 @@ private[spark] class ApplicationInfo( } } - val requestedCores = desc.maxCores.getOrElse(defaultCores) + private[master] val requestedCores = desc.maxCores.getOrElse(defaultCores) - def coresLeft: Int = requestedCores - coresGranted + private[master] def coresLeft: Int = requestedCores - coresGranted private var _retryCount = 0 - def retryCount = _retryCount + private[master] def retryCount = _retryCount - def incrementRetryCount() = { + private[master] def incrementRetryCount() = { _retryCount += 1 _retryCount } - def resetRetryCount() = _retryCount = 0 + private[master] def resetRetryCount() = _retryCount = 0 - def markFinished(endState: ApplicationState.Value) { + private[master] def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index 38db02cd2421b..017e8b55cbe7f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -class ApplicationSource(val application: ApplicationInfo) extends Source { +private[master] class ApplicationSource(val application: ApplicationInfo) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "%s.%s.%s".format("application", application.desc.name, System.currentTimeMillis()) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index f5b946329ae9b..37bfcdfdf4777 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object ApplicationState extends Enumeration { +private[master] object ApplicationState extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 9d3d7938c6ccb..b197dbcbfe294 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.DriverDescription import org.apache.spark.util.Utils -private[spark] class DriverInfo( +private[deploy] class DriverInfo( val startTime: Long, val id: String, val desc: DriverDescription, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 26a68bade3c60..35ff33a61653c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object DriverState extends Enumeration { +private[deploy] object DriverState extends Enumeration { type DriverState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index 5d620dfcabad5..fc62b094def67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} -private[spark] class ExecutorDesc( +private[master] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 36a2e2c6a6349..d2d30bfd7fcba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -33,7 +33,7 @@ import org.apache.spark.Logging * @param dir Directory to store files. Created if non-existent (but not recursively). * @param serialization Used to serialize our objects. */ -private[spark] class FileSystemPersistenceEngine( +private[master] class FileSystemPersistenceEngine( val dir: String, val serialization: Serialization) extends PersistenceEngine with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 22935c9b1d394..1b42121c8db05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -49,7 +49,7 @@ import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} -private[spark] class Master( +private[master] class Master( host: String, port: Int, webUiPort: Int, @@ -59,65 +59,68 @@ private[spark] class Master( import context.dispatcher // to use Akka's scheduler.schedule() - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 - val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + + private val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 + private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) + private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) + private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") val workers = new HashSet[WorkerInfo] - val idToWorker = new HashMap[String, WorkerInfo] - val addressToWorker = new HashMap[Address, WorkerInfo] - - val apps = new HashSet[ApplicationInfo] val idToApp = new HashMap[String, ApplicationInfo] - val actorToApp = new HashMap[ActorRef, ApplicationInfo] - val addressToApp = new HashMap[Address, ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo] - val completedApps = new ArrayBuffer[ApplicationInfo] - var nextAppNumber = 0 - val appIdToUI = new HashMap[String, SparkUI] + val apps = new HashSet[ApplicationInfo] + + private val idToWorker = new HashMap[String, WorkerInfo] + private val addressToWorker = new HashMap[Address, WorkerInfo] + + private val actorToApp = new HashMap[ActorRef, ApplicationInfo] + private val addressToApp = new HashMap[Address, ApplicationInfo] + private val completedApps = new ArrayBuffer[ApplicationInfo] + private var nextAppNumber = 0 + private val appIdToUI = new HashMap[String, SparkUI] - val drivers = new HashSet[DriverInfo] - val completedDrivers = new ArrayBuffer[DriverInfo] - val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling - var nextDriverNumber = 0 + private val drivers = new HashSet[DriverInfo] + private val completedDrivers = new ArrayBuffer[DriverInfo] + // Drivers currently spooled for scheduling + private val waitingDrivers = new ArrayBuffer[DriverInfo] + private var nextDriverNumber = 0 Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, + private val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) + private val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, securityMgr) - val masterSource = new MasterSource(this) + private val masterSource = new MasterSource(this) - val webUi = new MasterWebUI(this, webUiPort) + private val webUi = new MasterWebUI(this, webUiPort) - val masterPublicAddress = { + private val masterPublicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val masterUrl = "spark://" + host + ":" + port - var masterWebUiUrl: String = _ + private val masterUrl = "spark://" + host + ":" + port + private var masterWebUiUrl: String = _ - var state = RecoveryState.STANDBY + private var state = RecoveryState.STANDBY - var persistenceEngine: PersistenceEngine = _ + private var persistenceEngine: PersistenceEngine = _ - var leaderElectionAgent: LeaderElectionAgent = _ + private var leaderElectionAgent: LeaderElectionAgent = _ private var recoveryCompletionTask: Cancellable = _ // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) if (defaultCores < 1) { throw new SparkException("spark.deploy.defaultCores must be positive") } @@ -449,11 +452,11 @@ private[spark] class Master( } } - def canCompleteRecovery = + private def canCompleteRecovery = workers.count(_.state == WorkerState.UNKNOWN) == 0 && apps.count(_.state == ApplicationState.UNKNOWN) == 0 - def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], + private def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], storedWorkers: Seq[WorkerInfo]) { for (app <- storedApps) { logInfo("Trying to recover app: " + app.id) @@ -484,7 +487,7 @@ private[spark] class Master( } } - def completeRecovery() { + private def completeRecovery() { // Ensure "only-once" recovery semantics using a short synchronization period. synchronized { if (state != RecoveryState.RECOVERING) { return } @@ -517,7 +520,7 @@ private[spark] class Master( * launched an executor for the app on it (right now the standalone backend doesn't like having * two executors on the same worker). */ - def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { + private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) } @@ -596,7 +599,7 @@ private[spark] class Master( } } - def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, @@ -605,7 +608,7 @@ private[spark] class Master( exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } - def registerWorker(worker: WorkerInfo): Boolean = { + private def registerWorker(worker: WorkerInfo): Boolean = { // There may be one or more refs to dead workers on this same node (w/ different ID's), // remove them. workers.filter { w => @@ -633,7 +636,7 @@ private[spark] class Master( true } - def removeWorker(worker: WorkerInfo) { + private def removeWorker(worker: WorkerInfo) { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) idToWorker -= worker.id @@ -656,20 +659,20 @@ private[spark] class Master( persistenceEngine.removeWorker(worker) } - def relaunchDriver(driver: DriverInfo) { + private def relaunchDriver(driver: DriverInfo) { driver.worker = None driver.state = DriverState.RELAUNCHING waitingDrivers += driver schedule() } - def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { + private def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } - def registerApplication(app: ApplicationInfo): Unit = { + private def registerApplication(app: ApplicationInfo): Unit = { val appAddress = app.driver.path.address if (addressToApp.contains(appAddress)) { logInfo("Attempted to re-register application at same address: " + appAddress) @@ -684,7 +687,7 @@ private[spark] class Master( waitingApps += app } - def finishApplication(app: ApplicationInfo) { + private def finishApplication(app: ApplicationInfo) { removeApplication(app, ApplicationState.FINISHED) } @@ -732,7 +735,7 @@ private[spark] class Master( * Rebuild a new SparkUI from the given application's event logs. * Return whether this is successful. */ - def rebuildSparkUI(app: ApplicationInfo): Boolean = { + private def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -798,14 +801,14 @@ private[spark] class Master( } /** Generate a new app ID given a app's submission date */ - def newApplicationId(submitDate: Date): String = { + private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId } /** Check for, and remove, any timed-out workers */ - def timeOutDeadWorkers() { + private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT).toArray @@ -822,19 +825,19 @@ private[spark] class Master( } } - def newDriverId(submitDate: Date): String = { + private def newDriverId(submitDate: Date): String = { val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) nextDriverNumber += 1 appId } - def createDriver(desc: DriverDescription): DriverInfo = { + private def createDriver(desc: DriverDescription): DriverInfo = { val now = System.currentTimeMillis() val date = new Date(now) new DriverInfo(now, newDriverId(date), desc, date) } - def launchDriver(worker: WorkerInfo, driver: DriverInfo) { + private def launchDriver(worker: WorkerInfo, driver: DriverInfo) { logInfo("Launching driver " + driver.id + " on worker " + worker.id) worker.addDriver(driver) driver.worker = Some(worker) @@ -842,7 +845,10 @@ private[spark] class Master( driver.state = DriverState.RUNNING } - def removeDriver(driverId: String, finalState: DriverState, exception: Option[Exception]) { + private def removeDriver( + driverId: String, + finalState: DriverState, + exception: Option[Exception]) { drivers.find(d => d.id == driverId) match { case Some(driver) => logInfo(s"Removing driver: $driverId") @@ -863,7 +869,7 @@ private[spark] class Master( } } -private[spark] object Master extends Logging { +private[deploy] object Master extends Logging { val systemName = "sparkMaster" private val actorName = "Master" diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index e34bee7854292..435b9b12f83b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { +private[master] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -49,7 +49,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { webUiPort = conf.get("spark.master.ui.port").toInt } - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value @@ -84,7 +84,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { System.err.println( "Usage: Master [options]\n" + "\n" + diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index 2e0e1e7036ac8..da5060778edeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -87,7 +87,7 @@ trait PersistenceEngine { def close() {} } -private[spark] class BlackHolePersistenceEngine extends PersistenceEngine { +private[master] class BlackHolePersistenceEngine extends PersistenceEngine { override def persist(name: String, obj: Object): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 1096eb0368357..1583bf1f60032 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -49,7 +49,7 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") @@ -61,7 +61,7 @@ private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) { def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index 256a5a7c28e47..aa0f02fa625cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object RecoveryState extends Enumeration { +private[deploy] object RecoveryState extends Enumeration { type MasterState = Value val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala index 4781a80d470e1..5b22481ea8c5f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -25,12 +25,12 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.{Logging, SparkConf} -object SparkCuratorUtil extends Logging { +private[deploy] object SparkCuratorUtil extends Logging { - val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 - val ZK_SESSION_TIMEOUT_MILLIS = 60000 - val RETRY_WAIT_MILLIS = 5000 - val MAX_RECONNECT_ATTEMPTS = 3 + private val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 + private val ZK_SESSION_TIMEOUT_MILLIS = 60000 + private val RETRY_WAIT_MILLIS = 5000 + private val MAX_RECONNECT_ATTEMPTS = 3 def newClient(conf: SparkConf): CuratorFramework = { val ZK_URL = conf.get("spark.deploy.zookeeper.url") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index 0b36ef60051fc..b60baaadfb4bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration { +private[master] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 8eaa0ad948519..4823fd7cac0cb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -24,7 +24,7 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, +private[master] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" @@ -35,7 +35,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab start() - def start() { + private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) leaderLatch = new LeaderLatch(zk, WORKING_DIR) @@ -72,7 +72,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab } } - def updateLeadershipStatus(isLeader: Boolean) { + private def updateLeadershipStatus(isLeader: Boolean) { if (isLeader && status == LeadershipStatus.NOT_LEADER) { status = LeadershipStatus.LEADER masterActor.electedLeader() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index e11ac031fb9c6..1ac6677ad2b6d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -28,12 +28,12 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) +private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine - with Logging -{ - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) + with Logging { + + private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -61,7 +61,7 @@ private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializati zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { + private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 76fc40e17d9a8..761aa8f7b1ef6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { +private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala index d8daff3e7fb9c..e021f1eef794f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUIPage} -private[spark] class HistoryNotFoundPage(parent: MasterWebUI) +private[ui] class HistoryNotFoundPage(parent: MasterWebUI) extends WebUIPage("history/not-found") { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index c086cadca2c7d..dee2e4a447c6e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.master._ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { +private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout 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 170f90a00ad2a..1b670418ab1ff 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 @@ -26,7 +26,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone master. */ -private[spark] +private[master] class MasterWebUI(val master: Master, requestedPort: Int) extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { @@ -62,6 +62,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) } } -private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR +private[master] object MasterWebUI { + private val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index c4be1f19e8e9f..420442f7564cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -52,7 +52,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} * implementation of this client can use that information to retry using the version specified * by the server. */ -private[spark] class StandaloneRestClient extends Logging { +private[deploy] class StandaloneRestClient extends Logging { import StandaloneRestClient._ /** @@ -61,7 +61,7 @@ private[spark] class StandaloneRestClient extends Logging { * If the submission was successful, poll the status of the submission and report * it to the user. Otherwise, report the error message provided by the server. */ - def createSubmission( + private[rest] def createSubmission( master: String, request: CreateSubmissionRequest): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to launch an application in $master.") @@ -106,7 +106,7 @@ private[spark] class StandaloneRestClient extends Logging { } /** Construct a message that captures the specified parameters for submitting an application. */ - def constructSubmitRequest( + private[rest] def constructSubmitRequest( appResource: String, mainClass: String, appArgs: Array[String], @@ -291,16 +291,16 @@ private[spark] class StandaloneRestClient extends Logging { } } -private[spark] object StandaloneRestClient { - val REPORT_DRIVER_STATUS_INTERVAL = 1000 - val REPORT_DRIVER_STATUS_MAX_TRIES = 10 +private[rest] object StandaloneRestClient { + private val REPORT_DRIVER_STATUS_INTERVAL = 1000 + private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" /** * Submit an application, assuming Spark parameters are specified through the given config. * This is abstracted to its own method for testing purposes. */ - private[rest] def run( + def run( appResource: String, mainClass: String, appArgs: Array[String], diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index f9e0478e4f874..4f19af59f409f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -58,7 +58,7 @@ import org.apache.spark.deploy.ClientArguments._ * @param masterUrl the URL of the Master new drivers will attempt to connect to * @param masterConf the conf used by the Master */ -private[spark] class StandaloneRestServer( +private[deploy] class StandaloneRestServer( host: String, requestedPort: Int, masterActor: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala index d7a0bdbe10778..b97921ec934a0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala @@ -20,17 +20,17 @@ package org.apache.spark.deploy.rest /** * An exception thrown in the REST application submission protocol. */ -private[spark] class SubmitRestProtocolException(message: String, cause: Throwable = null) +private[rest] class SubmitRestProtocolException(message: String, cause: Throwable = null) extends Exception(message, cause) /** * An exception thrown if a field is missing from a [[SubmitRestProtocolMessage]]. */ -private[spark] class SubmitRestMissingFieldException(message: String) +private[rest] class SubmitRestMissingFieldException(message: String) extends SubmitRestProtocolException(message) /** * An exception thrown if the REST client cannot reach the REST server. */ -private[spark] class SubmitRestConnectionException(message: String, cause: Throwable) +private[deploy] class SubmitRestConnectionException(message: String, cause: Throwable) extends SubmitRestProtocolException(message, cause) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala index 8f36635674a28..e6615a3174ce1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -39,7 +39,7 @@ import org.apache.spark.util.Utils @JsonInclude(Include.NON_NULL) @JsonAutoDetect(getterVisibility = Visibility.ANY, setterVisibility = Visibility.ANY) @JsonPropertyOrder(alphabetic = true) -private[spark] abstract class SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolMessage { @JsonIgnore val messageType = Utils.getFormattedClassName(this) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 9e1fd8c40cabd..d80abdf15fb34 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils /** * An abstract request sent from the client in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { var clientSparkVersion: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtoc /** * A request to launch a new application in the REST application submission protocol. */ -private[spark] class CreateSubmissionRequest extends SubmitRestProtocolRequest { +private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { var appResource: String = null var mainClass: String = null var appArgs: Array[String] = null diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala index 16dfe041d4bea..8fde8c142a4c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala @@ -22,7 +22,7 @@ import java.lang.Boolean /** * An abstract response sent from the server in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { var serverSparkVersion: String = null var success: Boolean = null var unknownFields: Array[String] = null @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProto /** * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. */ -private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -46,7 +46,7 @@ private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse /** * A response to a kill request in the REST application submission protocol. */ -private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -58,7 +58,7 @@ private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { /** * A response to a status request in the REST application submission protocol. */ -private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { +private[rest] class SubmissionStatusResponse extends SubmitRestProtocolResponse { var submissionId: String = null var driverState: String = null var workerId: String = null @@ -74,7 +74,7 @@ private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse /** * An error response message used in the REST application submission protocol. */ -private[spark] class ErrorResponse extends SubmitRestProtocolResponse { +private[rest] class ErrorResponse extends SubmitRestProtocolResponse { // The highest protocol version that the server knows about // This is set when the client specifies an unknown version var highestProtocolVersion: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 83f78cf47306c..0a1d60f58bc58 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils /** ** Utilities for running commands with the spark classpath. */ -private[spark] +private[deploy] object CommandUtils extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e16bccb24d2c4..27a9eabb1ede7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -37,8 +37,8 @@ import org.apache.spark.util.{Clock, SystemClock} * Manages the execution of one driver, including automatically restarting the driver on failure. * This is currently only used in standalone cluster deploy mode. */ -private[spark] class DriverRunner( - val conf: SparkConf, +private[deploy] class DriverRunner( + conf: SparkConf, val driverId: String, val workDir: File, val sparkHome: File, @@ -47,24 +47,24 @@ private[spark] class DriverRunner( val workerUrl: String) extends Logging { - @volatile var process: Option[Process] = None - @volatile var killed = false + @volatile private var process: Option[Process] = None + @volatile private var killed = false // Populated once finished - var finalState: Option[DriverState] = None - var finalException: Option[Exception] = None - var finalExitCode: Option[Int] = None + private[worker] var finalState: Option[DriverState] = None + private[worker] var finalException: Option[Exception] = None + private var finalExitCode: Option[Int] = None // Decoupled for testing - private[deploy] def setClock(_clock: Clock) = clock = _clock - private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + def setClock(_clock: Clock) = clock = _clock + def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) } /** Starts a thread to run and manage the driver. */ - def start() = { + private[worker] def start() = { new Thread("DriverRunner for " + driverId) { override def run() { try { @@ -106,7 +106,7 @@ private[spark] class DriverRunner( } /** Terminate this driver (or prevent it from ever starting if not yet started) */ - def kill() { + private[worker] def kill() { synchronized { process.foreach(p => p.destroy()) killed = true @@ -169,7 +169,7 @@ private[spark] class DriverRunner( runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, + def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean) { // Time to wait between submission retries. var waitSeconds = 1 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 023f3c6269062..83e24a7a1f80c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.logging.FileAppender * Manages the execution of one executor process. * This is currently only used in standalone mode. */ -private[spark] class ExecutorRunner( +private[deploy] class ExecutorRunner( val appId: String, val execId: Int, val appDesc: ApplicationDescription, @@ -48,22 +48,22 @@ private[spark] class ExecutorRunner( val sparkHome: File, val executorDir: File, val workerUrl: String, - val conf: SparkConf, + conf: SparkConf, val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { - val fullId = appId + "/" + execId - var workerThread: Thread = null - var process: Process = null - var stdoutAppender: FileAppender = null - var stderrAppender: FileAppender = null + private val fullId = appId + "/" + execId + private var workerThread: Thread = null + private var process: Process = null + private var stdoutAppender: FileAppender = null + private var stderrAppender: FileAppender = null // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. - var shutdownHook: Thread = null + private var shutdownHook: Thread = null - def start() { + private[worker] def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } @@ -99,7 +99,7 @@ private[spark] class ExecutorRunner( } /** Stop this executor runner, including killing the process it launched */ - def kill() { + private[worker] def kill() { if (workerThread != null) { // the workerThread will kill the child process when interrupted workerThread.interrupt() @@ -114,7 +114,7 @@ private[spark] class ExecutorRunner( } /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ - def substituteVariables(argument: String): String = argument match { + private[worker] def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => host @@ -126,7 +126,7 @@ private[spark] class ExecutorRunner( /** * Download and run the executor described in our ApplicationDescription */ - def fetchAndRunExecutor() { + private def fetchAndRunExecutor() { try { // Launch the process val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f2e7418f4bf15..c1b0a295f9f74 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** * @param masterAkkaUrls Each url should be a valid akka url. */ -private[spark] class Worker( +private[worker] class Worker( host: String, port: Int, webUiPort: Int, @@ -60,85 +60,90 @@ private[spark] class Worker( Utils.checkHost(host, "Expected hostname") assert (port > 0) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs + // For worker and executor IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) // Afterwards, the next 10 attempts are between 30 and 90 seconds. // A bit of randomness is introduced so that not all of the workers attempt to reconnect at // the same time. - val INITIAL_REGISTRATION_RETRIES = 6 - val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 - val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 - val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { + private val INITIAL_REGISTRATION_RETRIES = 6 + private val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 + private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 + private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits) randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND } - val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * + private val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 + private val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = + conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - - val testing: Boolean = sys.props.contains("spark.testing") - var master: ActorSelection = null - var masterAddress: Address = null - var activeMasterUrl: String = "" - var activeMasterWebUiUrl : String = "" - val akkaUrl = AkkaUtils.address( + private val APP_DATA_RETENTION_SECS = + conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + + private val testing: Boolean = sys.props.contains("spark.testing") + private var master: ActorSelection = null + private var masterAddress: Address = null + private var activeMasterUrl: String = "" + private[worker] var activeMasterWebUiUrl : String = "" + private val akkaUrl = AkkaUtils.address( AkkaUtils.protocol(context.system), actorSystemName, host, port, actorName) - @volatile var registered = false - @volatile var connected = false - val workerId = generateWorkerId() - val sparkHome = + @volatile private var registered = false + @volatile private var connected = false + private val workerId = generateWorkerId() + private val sparkHome = if (testing) { assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") new File(sys.props("spark.test.home")) } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } + var workDir: File = null - val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] + val executors = new HashMap[String, ExecutorRunner] val finishedDrivers = new HashMap[String, DriverRunner] val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. - val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) + private val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) - val publicAddress = { + private val publicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - var webUi: WorkerWebUI = null + private var webUi: WorkerWebUI = null - var coresUsed = 0 - var memoryUsed = 0 - var connectionAttemptCount = 0 + private var connectionAttemptCount = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) - val workerSource = new WorkerSource(this) + private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) + private val workerSource = new WorkerSource(this) + + private var registrationRetryTimer: Option[Cancellable] = None - var registrationRetryTimer: Option[Cancellable] = None + var coresUsed = 0 + var memoryUsed = 0 def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed - def createWorkDir() { + private def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() @@ -175,7 +180,7 @@ private[spark] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } - def changeMaster(url: String, uiUrl: String) { + private def changeMaster(url: String, uiUrl: String) { // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl @@ -252,7 +257,7 @@ private[spark] class Worker( } } - def registerWithMaster() { + private def registerWithMaster() { // DisassociatedEvent may be triggered multiple times, so don't attempt registration // if there are outstanding registration attempts scheduled. registrationRetryTimer match { @@ -506,7 +511,7 @@ private[spark] class Worker( } } - def generateWorkerId(): String = { + private def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } @@ -521,7 +526,7 @@ private[spark] class Worker( } } -private[spark] object Worker extends Logging { +private[deploy] object Worker extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) val conf = new SparkConf @@ -554,7 +559,7 @@ private[spark] object Worker extends Logging { (actorSystem, boundPort) } - private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { + def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r val result = cmd.javaOpts.collectFirst { case pattern(_result) => _result.toBoolean @@ -562,7 +567,7 @@ private[spark] object Worker extends Logging { result.getOrElse(false) } - private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { + def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { val prefix = "spark.ssl." val useNLC = "spark.ssl.useNodeLocalConf" if (isUseLocalNodeSSLConfig(cmd)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 019cd70f2a229..88f9d880ac209 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -25,7 +25,7 @@ import org.apache.spark.SparkConf /** * Command-line parser for the worker. */ -private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { +private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -63,7 +63,7 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { checkWorkerMemory() - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index df1e01b23b932..b36023bc40c3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -private[spark] class WorkerSource(val worker: Worker) extends Source { +private[worker] class WorkerSource(val worker: Worker) extends Source { override val sourceName = "worker" override val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 63a8ac817b618..09d866fb0cd90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -48,7 +48,7 @@ private[spark] class WorkerWatcher(workerUrl: String) private val expectedHostPort = AddressFromURIString(workerUrl).hostPort private def isWorker(address: Address) = address.hostPort == expectedHostPort - def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) + private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) override def receiveWithLogging = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index ecb358c399819..88170d4df3053 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils import org.apache.spark.Logging import org.apache.spark.util.logging.RollingFileAppender -private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { +private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker private val workDir = parent.workDir diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 720f13bfa829b..9f9f27d71e1ae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -31,10 +31,9 @@ import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { - val workerActor = parent.worker.self - val worker = parent.worker - val timeout = parent.timeout +private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { + private val workerActor = parent.worker.self + private val timeout = parent.timeout override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 7ac81a2d87efd..de6423beb543e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ -private[spark] +private[worker] class WorkerWebUI( val worker: Worker, val workDir: File, @@ -38,7 +38,7 @@ class WorkerWebUI( extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { - val timeout = AkkaUtils.askTimeout(worker.conf) + private[ui] val timeout = AkkaUtils.askTimeout(worker.conf) initialize() @@ -53,6 +53,6 @@ class WorkerWebUI( } } -private[spark] object WorkerWebUI { +private[ui] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR } From 005d1c5f290decc606a0be59fb191136dafc0c9d Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 17 Mar 2015 11:20:20 +0000 Subject: [PATCH 430/817] [SPARK-6269] [CORE] Use ScalaRunTime's array methods instead of java.lang.reflect.Array in size estimation This patch switches the usage of java.lang.reflect.Array in Size estimation to using scala's RunTime array-getter methods. The notes on https://bugs.openjdk.java.net/browse/JDK-8051447 tipped me off to the fact that using java.lang.reflect.Array was not ideal. At first, I used the code from that ticket, but it turns out that ScalaRunTime's array-related methods avoid the bottleneck of invoking native code anyways, so that was sufficient to boost performance in size estimation. The idea is to use pure Java code in implementing the methods there, as opposed to relying on native C code which ends up being ill-performing. This improves the performance of estimating the size of arrays when we are checking for spilling in Spark. Here's the benchmark discussion from the ticket: I did two tests. The first, less convincing, take-with-a-block-of-salt test I did was do a simple groupByKey operation to collect objects in a 4.0 GB text file RDD into 30,000 buckets. I ran 1 Master and 4 Spark Worker JVMs on my mac, fetching the RDD from a text file simply stored on disk, and saving it out to another file located on local disk. The wall clock times I got back before and after the change were: Before: 352.195s, 343.871s, 359.080s After (using code directly from the JDK ticket, not the scala code in this PR): 342.929583s, 329.456623s, 326.151481s So, there is a bit of an improvement after the change. I also did some YourKit profiling of the executors to get an idea of how much time was spent in size estimation before and after the change. I roughly saw that size estimation took up less of the time after my change, but YourKit's profiling can be inconsistent and who knows if I was profiling the executors that had the same data between runs? The more convincing test I did was to run the size-estimation logic itself in an isolated unit test. I ran the following code: ``` val bigArray = Array.fill(1000)(Array.fill(1000)(java.util.UUID.randomUUID().toString())) test("String arrays only perf testing") { val startTime = System.currentTimeMillis() for (i <- 1 to 50000) { SizeEstimator.estimate(bigArray) } println("Runtime: " + (System.currentTimeMillis() - startTime) / 1000.0000) } ``` I wanted to use a 2D array specifically because I wanted to measure the performance of repeatedly calling Array.getLength. I used UUID-Strings to ensure that the strings were randomized (so String object re-use doesn't happen), but that they would all be the same size. The results were as follows: Before PR: 222.681 s, 218.34 s, 211.739s After latest change: 170.715 s, 176.775 s, 180.298 s . Author: mcheah Author: Justin Uang Closes #4972 from mccheah/feature/spark-6269-reflect-array and squashes the following commits: 8527852 [mcheah] Respect CamelCase for numElementsDrawn 18d4b50 [mcheah] Addressing style comments - while loops instead of for loops 16ce534 [mcheah] Organizing imports properly db890ea [mcheah] Removing CastedArray and just using ScalaRunTime. cb67ce2 [mcheah] Fixing a scalastyle error - line too long 5d53c4c [mcheah] Removing unused parameter in visitArray. 6467759 [mcheah] Including primitive size information inside CastedArray. 93f4b05 [mcheah] Using Scala instead of Java for the array-reflection implementation. a557ab8 [mcheah] Using a wrapper around arrays to do casting only once ca063fc [mcheah] Fixing a compiler error made while refactoring style 1fe09de [Justin Uang] [SPARK-6269] Use a different implementation of java.lang.reflect.Array --- .../org/apache/spark/util/SizeEstimator.scala | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index bce3b3afe9aba..26ffbf9350388 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -18,18 +18,16 @@ package org.apache.spark.util import java.lang.management.ManagementFactory -import java.lang.reflect.{Array => JArray} -import java.lang.reflect.Field -import java.lang.reflect.Modifier -import java.util.IdentityHashMap -import java.util.Random +import java.lang.reflect.{Field, Modifier} +import java.util.{IdentityHashMap, Random} import java.util.concurrent.ConcurrentHashMap - import scala.collection.mutable.ArrayBuffer +import scala.runtime.ScalaRunTime import org.apache.spark.Logging import org.apache.spark.util.collection.OpenHashSet + /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in * memory-aware caches. @@ -184,9 +182,9 @@ private[spark] object SizeEstimator extends Logging { private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING - private def visitArray(array: AnyRef, cls: Class[_], state: SearchState) { - val length = JArray.getLength(array) - val elementClass = cls.getComponentType + private def visitArray(array: AnyRef, arrayClass: Class[_], state: SearchState) { + val length = ScalaRunTime.array_length(array) + val elementClass = arrayClass.getComponentType() // Arrays have object header and length field which is an integer var arrSize: Long = alignSize(objectSize + INT_SIZE) @@ -199,22 +197,26 @@ private[spark] object SizeEstimator extends Logging { state.size += arrSize if (length <= ARRAY_SIZE_FOR_SAMPLING) { - for (i <- 0 until length) { - state.enqueue(JArray.get(array, i)) + var arrayIndex = 0 + while (arrayIndex < length) { + state.enqueue(ScalaRunTime.array_apply(array, arrayIndex).asInstanceOf[AnyRef]) + arrayIndex += 1 } } else { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) - for (i <- 0 until ARRAY_SAMPLE_SIZE) { + var numElementsDrawn = 0 + while (numElementsDrawn < ARRAY_SAMPLE_SIZE) { var index = 0 do { index = rand.nextInt(length) } while (drawn.contains(index)) drawn.add(index) - val elem = JArray.get(array, index) + val elem = ScalaRunTime.array_apply(array, index).asInstanceOf[AnyRef] size += SizeEstimator.estimate(elem, state.visited) + numElementsDrawn += 1 } state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong } From e26db9be47b295c4ec9e651f4cce321ba1fddfcd Mon Sep 17 00:00:00 2001 From: Theodore Vasiloudis Date: Tue, 17 Mar 2015 11:25:01 +0000 Subject: [PATCH 431/817] [docs] [SPARK-4820] Spark build encounters "File name too long" on some encrypted filesystems Added a note instructing users how to build Spark in an encrypted file system. Author: Theodore Vasiloudis Closes #5041 from thvasilo/patch-2 and squashes the following commits: 09d890b [Theodore Vasiloudis] Workaroung for buiding in an encrypted filesystem --- docs/building-spark.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/building-spark.md b/docs/building-spark.md index 57d0ca834f460..ea79c5bc276d3 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -23,6 +23,18 @@ build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package Other build examples can be found below. +**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: + + -Xmax-classfile-name + 128 + +and in `project/SparkBuild.scala` add: + + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), + +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. + + # Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: From c928796ade54f68e26bc55734a9867a046d2e3fe Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 17 Mar 2015 05:31:27 -0700 Subject: [PATCH 432/817] [SPARK-6331] Load new master URL if present when recovering streaming context from checkpoint In streaming driver recovery, when the SparkConf is reconstructed based on the checkpointed configuration, it recovers the old master URL. This okay if the cluster on which the streaming application is relaunched is the same cluster as it was running before. But if that cluster changes, there is no way to inject the new master URL of the new cluster. As a result, the restarted app tries to connect to the non-existent old cluster and fails. The solution is to check whether a master URL is set in the System properties (by Spark submit) before recreating the SparkConf. If a new master url is set in the properties, then use it as that is obviously the most relevant one. Otherwise load the old one (to maintain existing behavior). Author: Tathagata Das Closes #5024 from tdas/SPARK-6331 and squashes the following commits: 392fd44 [Tathagata Das] Fixed naming issue. c7c0b99 [Tathagata Das] Addressed comments. 6a0857c [Tathagata Das] Updated testsuites. 222485d [Tathagata Das] Load new master URL if present when recovering streaming context from checkpoint --- .../apache/spark/streaming/Checkpoint.scala | 7 +++++-- .../spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 21 ++++++++++++++++--- .../streaming/StreamingContextSuite.scala | 2 +- 4 files changed, 25 insertions(+), 7 deletions(-) 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 f88a8a0151550..cb4c94fb9d5a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -43,10 +43,13 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConfPairs = ssc.conf.getAll - def sparkConf = { - new SparkConf(false).setAll(sparkConfPairs) + def createSparkConf(): SparkConf = { + val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) .remove("spark.driver.host") .remove("spark.driver.port") + val newMasterOption = new SparkConf(loadDefaults = true).getOption("spark.master") + newMasterOption.foreach { newMaster => newSparkConf.setMaster(newMaster) } + newSparkConf } def validate() { 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 b5b6770a8a150..543224d4b07bc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -116,7 +116,7 @@ class StreamingContext private[streaming] ( private[streaming] val sc: SparkContext = { if (isCheckpointPresent) { - new SparkContext(cp_.sparkConf) + new SparkContext(cp_.createSparkConf()) } else { sc_ } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 03c448f1df5f1..8ea91eca683cf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -146,7 +146,7 @@ class CheckpointSuite extends TestSuiteBase { // This tests whether spark conf persists through checkpoints, and certain // configs gets scrubbed - test("persistence of conf through checkpoints") { + test("recovery of conf through checkpoints") { val key = "spark.mykey" val value = "myvalue" System.setProperty(key, value) @@ -154,7 +154,7 @@ class CheckpointSuite extends TestSuiteBase { val originalConf = ssc.conf val cp = new Checkpoint(ssc, Time(1000)) - val cpConf = cp.sparkConf + val cpConf = cp.createSparkConf() assert(cpConf.get("spark.master") === originalConf.get("spark.master")) assert(cpConf.get("spark.app.name") === originalConf.get("spark.app.name")) assert(cpConf.get(key) === value) @@ -163,7 +163,8 @@ class CheckpointSuite extends TestSuiteBase { // Serialize/deserialize to simulate write to storage and reading it back val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - val newCpConf = newCp.sparkConf + // Verify new SparkConf has all the previous properties + val newCpConf = newCp.createSparkConf() assert(newCpConf.get("spark.master") === originalConf.get("spark.master")) assert(newCpConf.get("spark.app.name") === originalConf.get("spark.app.name")) assert(newCpConf.get(key) === value) @@ -174,6 +175,20 @@ class CheckpointSuite extends TestSuiteBase { ssc = new StreamingContext(null, newCp, null) val restoredConf = ssc.conf assert(restoredConf.get(key) === value) + ssc.stop() + + // Verify new SparkConf picks up new master url if it is set in the properties. See SPARK-6331. + try { + val newMaster = "local[100]" + System.setProperty("spark.master", newMaster) + val newCpConf = newCp.createSparkConf() + assert(newCpConf.get("spark.master") === newMaster) + assert(newCpConf.get("spark.app.name") === originalConf.get("spark.app.name")) + ssc = new StreamingContext(null, newCp, null) + assert(ssc.sparkContext.master === newMaster) + } finally { + System.clearProperty("spark.master") + } } 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 6a7cd97aa3222..2e5005ef6ff14 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -100,7 +100,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.sparkConf.getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) ssc = new StreamingContext(null, newCp, null) assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } From e9f22c61290348c58af54c0ae3c6226105126a8d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 17 Mar 2015 09:41:06 -0500 Subject: [PATCH 433/817] [SPARK-6365] jetty-security also needed for SPARK_PREPEND_CLASSES to work https://issues.apache.org/jira/browse/SPARK-6365 thanks vanzin for helping me figure this out Author: Imran Rashid Closes #5052 from squito/fix_prepend_classes and squashes the following commits: 09d334c [Imran Rashid] jetty-security also needed for SPARK_PREPEND_CLASSES to work --- core/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 4164a3a7208d4..81f8cba711df6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -414,7 +414,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server + guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true From 0f673c21f68ee3d5df3c01ae405709d3c1f4909b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 17 Mar 2015 09:18:57 -0700 Subject: [PATCH 434/817] [SPARK-3266] Use intermediate abstract classes to fix type erasure issues in Java APIs This PR addresses a Scala compiler bug ([SI-8905](https://issues.scala-lang.org/browse/SI-8905)) that was breaking some of the Spark Java APIs. In a nutshell, it seems that methods whose implementations are inherited from generic traits sometimes have their type parameters erased to Object. This was causing methods like `DoubleRDD.min()` to throw confusing NoSuchMethodErrors at runtime. The fix implemented here is to introduce an intermediate layer of abstract classes and inherit from those instead of directly extends the `Java*Like` traits. This should not break binary compatibility. I also improved the test coverage of the Java API, adding several new tests for methods that failed at runtime due to this bug. Author: Josh Rosen Closes #5050 from JoshRosen/javardd-si-8905-fix and squashes the following commits: 2feb068 [Josh Rosen] Use intermediate abstract classes to work around SPARK-3266 d5f3e5d [Josh Rosen] Add failing regression tests for SPARK-3266 --- .../apache/spark/api/java/JavaDoubleRDD.scala | 3 +- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../apache/spark/api/java/JavaRDDLike.scala | 8 ++ .../java/org/apache/spark/JavaAPISuite.java | 129 ++++++++++++++++++ .../streaming/api/java/JavaDStream.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 9 ++ .../streaming/api/java/JavaPairDStream.scala | 2 +- 8 files changed, 152 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 8e8f7f6c4fda2..79e4ebf2db578 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -32,7 +32,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter import org.apache.spark.util.Utils -class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { +class JavaDoubleRDD(val srdd: RDD[scala.Double]) + extends AbstractJavaRDDLike[JDouble, JavaDoubleRDD] { override val classTag: ClassTag[JDouble] = implicitly[ClassTag[JDouble]] diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 7af3538262fd6..4eadc9a85613e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) - extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { + extends AbstractJavaRDDLike[(K, V), JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 86fb374bef1e3..645dc3bfb6b06 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) - extends JavaRDDLike[T, JavaRDD[T]] { + extends AbstractJavaRDDLike[T, JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 0f91c942ecd50..8da42934a7d96 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -38,6 +38,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils +/** + * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations + * of JavaRDDLike should extend this dummy abstract class instead of directly inheriting + * from the trait. See SPARK-3266 for additional details. + */ +private[spark] abstract class AbstractJavaRDDLike[T, This <: JavaRDDLike[T, This]] + extends JavaRDDLike[T, This] + /** * Defines operations common to several Java RDD implementations. * Note that this trait is not intended to be implemented by user code. diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 74e88c767ee07..8ec54360ca42a 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -267,6 +267,22 @@ public void call(String s) throws IOException { Assert.assertEquals(2, accum.value().intValue()); } + @Test + public void foreachPartition() { + final Accumulator accum = sc.accumulator(0); + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreachPartition(new VoidFunction>() { + @Override + public void call(Iterator iter) throws IOException { + while (iter.hasNext()) { + iter.next(); + accum.add(1); + } + } + }); + Assert.assertEquals(2, accum.value().intValue()); + } + @Test public void toLocalIterator() { List correct = Arrays.asList(1, 2, 3, 4); @@ -657,6 +673,13 @@ public Boolean call(Integer i) { }).isEmpty()); } + @Test + public void toArray() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3)); + List list = rdd.toArray(); + Assert.assertEquals(Arrays.asList(1, 2, 3), list); + } + @Test public void cartesian() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -714,6 +737,80 @@ public void javaDoubleRDDHistoGram() { sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); } + private static class DoubleComparator implements Comparator, Serializable { + public int compare(Double o1, Double o2) { + return o1.compareTo(o2); + } + } + + @Test + public void max() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(new DoubleComparator()); + Assert.assertEquals(4.0, max, 0.001); + } + + @Test + public void min() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(new DoubleComparator()); + Assert.assertEquals(1.0, max, 0.001); + } + + @Test + public void takeOrdered() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); + } + + @Test + public void top() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + List top2 = rdd.top(2); + Assert.assertEquals(Arrays.asList(4, 3), top2); + } + + private static class AddInts implements Function2 { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + } + + @Test + public void reduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.reduce(new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void reduceOnJavaDoubleRDD() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double sum = rdd.reduce(new Function2() { + @Override + public Double call(Double v1, Double v2) throws Exception { + return v1 + v2; + } + }); + Assert.assertEquals(10.0, sum, 0.001); + } + + @Test + public void fold() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.fold(0, new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void aggregate() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.aggregate(0, new AddInts(), new AddInts()); + Assert.assertEquals(10, sum); + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); @@ -830,6 +927,25 @@ public Iterable call(Iterator iter) { Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); } + + @Test + public void mapPartitionsWithIndex() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitionsWithIndex( + new Function2, Iterator>() { + @Override + public Iterator call(Integer index, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + } + }, false); + Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test public void repartition() { // Shrinking number of partitions @@ -1516,6 +1632,19 @@ public void collectAsync() throws Exception { Assert.assertEquals(1, future.jobIds().size()); } + @Test + public void takeAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.takeAsync(1); + List result = future.get(); + Assert.assertEquals(1, result.size()); + Assert.assertEquals((Integer) 1, result.get(0)); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + @Test public void foreachAsync() throws Exception { List data = Arrays.asList(1, 2, 3, 4, 5); diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 505e4431e4350..01cdcb0574040 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -36,7 +36,7 @@ import org.apache.spark.streaming.dstream.DStream * [[org.apache.spark.streaming.api.java.JavaPairDStream]]. */ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T]) - extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { + extends AbstractJavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index c382a12f4d099..2eabdd9387913 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -34,6 +34,15 @@ import org.apache.spark.streaming._ import org.apache.spark.streaming.api.java.JavaDStream._ import org.apache.spark.streaming.dstream.DStream +/** + * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations + * of JavaDStreamLike should extend this dummy abstract class instead of directly inheriting + * from the trait. See SPARK-3266 for additional details. + */ +private[streaming] +abstract class AbstractJavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], + R <: JavaRDDLike[T, R]] extends JavaDStreamLike[T, This, R] + trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] extends Serializable { implicit val classTag: ClassTag[T] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index bd01789b611a4..f94f2d0e8bd31 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -45,7 +45,7 @@ import org.apache.spark.streaming.dstream.DStream class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifest: ClassTag[K], implicit val vManifest: ClassTag[V]) - extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { + extends AbstractJavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) From 4cca3917dc30ee907e6cbd6a569b6ac58af963f7 Mon Sep 17 00:00:00 2001 From: nemccarthy Date: Tue, 17 Mar 2015 09:33:11 -0700 Subject: [PATCH 435/817] [SPARK-6313] Add config option to disable file locks/fetchFile cache to ... ...support NFS mounts. This is a work around for now with the goal to find a more permanent solution. https://issues.apache.org/jira/browse/SPARK-6313 Author: nemccarthy Closes #5036 from nemccarthy/master and squashes the following commits: 2eaaf42 [nemccarthy] [SPARK-6313] Update config wording doc for spark.files.useFetchCache 5de7eb4 [nemccarthy] [SPARK-6313] Add config option to disable file locks/fetchFile cache to support NFS mounts --- .../src/main/scala/org/apache/spark/util/Utils.scala | 3 ++- docs/configuration.md | 12 ++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d3dc1d09cb7b4..af8a24553a461 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -403,7 +403,8 @@ private[spark] object Utils extends Logging { useCache: Boolean) { val fileName = url.split("/").last val targetFile = new File(targetDir, fileName) - if (useCache) { + val fetchCacheEnabled = conf.getBoolean("spark.files.useFetchCache", defaultValue = true) + if (useCache && fetchCacheEnabled) { val cachedFileName = s"${url.hashCode}${timestamp}_cache" val lockFileName = s"${url.hashCode}${timestamp}_lock" val localDir = new File(getLocalDir(conf)) diff --git a/docs/configuration.md b/docs/configuration.md index 63fc99e7d3e29..7fe11475212b3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -745,6 +745,18 @@ Apart from these, the following properties are also available, and may be useful the driver, in seconds. +
    + + + + From d9f3e01688ad0a8d5fc2419a948a682ad7d957c9 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Tue, 17 Mar 2015 12:11:57 -0700 Subject: [PATCH 436/817] [SPARK-6336] LBFGS should document what convergenceTol means LBFGS uses convergence tolerance. This value should be written in document as an argument. Author: lewuathe Closes #5033 from Lewuathe/SPARK-6336 and squashes the following commits: e738b33 [lewuathe] Modify text to be more natural ac03c3a [lewuathe] Modify documentations 6ccb304 [lewuathe] [SPARK-6336] LBFGS should document what convergenceTol means --- docs/mllib-optimization.md | 4 ++++ .../scala/org/apache/spark/mllib/optimization/LBFGS.scala | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 4d101afca2c97..6cabc1610a151 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -203,6 +203,10 @@ regularization, as well as L2 regularizer. recommended. * `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. * `regParam` is the regularization parameter when using regularization. +* `convergenceTol` controls how much relative change is still allowed when L-BFGS +is considered to converge. This must be nonnegative. Lower values are less tolerant and +therefore generally cause more iterations to be run. This value looks at both average +improvement and the norm of gradient inside [Breeze LBFGS](https://github.com/scalanlp/breeze/blob/master/math/src/main/scala/breeze/optimize/LBFGS.scala). The `return` is a tuple containing two elements. The first element is a column matrix containing weights for every feature, and the second element is an array containing diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index d5e4f4ccbff10..ef6eccd90711a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -60,6 +60,8 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. * Smaller value will lead to higher accuracy with the cost of more iterations. + * This value must be nonnegative. Lower convergence values are less tolerant + * and therefore generally cause more iterations to be run. */ def setConvergenceTol(tolerance: Double): this.type = { this.convergenceTol = tolerance @@ -142,7 +144,9 @@ object LBFGS extends Logging { * one single data example) * @param updater - Updater function to actually perform a gradient step in a given direction. * @param numCorrections - The number of corrections used in the L-BFGS update. - * @param convergenceTol - The convergence tolerance of iterations for L-BFGS + * @param convergenceTol - The convergence tolerance of iterations for L-BFGS which is must be + * nonnegative. Lower values are less tolerant and therefore generally + * cause more iterations to be run. * @param maxNumIterations - Maximal number of iterations that L-BFGS can be run. * @param regParam - Regularization parameter * From c94d0626471e209ab7ebfc588f9a2992946b7ed5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 17 Mar 2015 12:14:40 -0700 Subject: [PATCH 437/817] [SPARK-6226][MLLIB] add save/load in PySpark's KMeansModel Use `_py2java` and `_java2py` to convert Python model to/from Java model. yinxusen Author: Xiangrui Meng Closes #5049 from mengxr/SPARK-6226-mengxr and squashes the following commits: 570ba81 [Xiangrui Meng] fix python style b10b911 [Xiangrui Meng] add save/load in PySpark's KMeansModel --- .../spark/mllib/clustering/KMeansModel.scala | 5 ++++ python/pyspark/mllib/clustering.py | 28 +++++++++++++++++-- python/pyspark/mllib/common.py | 4 +-- 3 files changed, 32 insertions(+), 5 deletions(-) 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 707da537d238f..e4e411a3c8b42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.clustering +import scala.collection.JavaConverters._ + import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -34,6 +36,9 @@ import org.apache.spark.sql.Row */ class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { + /** A Java-friendly constructor that takes an Iterable of Vectors. */ + def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) + /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 949db5705abd7..464f49aeee3cd 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -19,14 +19,16 @@ from pyspark import RDD from pyspark import SparkContext -from pyspark.mllib.common import callMLlibFunc, callJavaFunc -from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector +from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _py2java, _java2py +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.stat.distribution import MultivariateGaussian +from pyspark.mllib.util import Saveable, Loader, inherit_doc __all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture'] -class KMeansModel(object): +@inherit_doc +class KMeansModel(Saveable, Loader): """A clustering model derived from the k-means method. @@ -55,6 +57,16 @@ class KMeansModel(object): True >>> type(model.clusterCenters) + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> model.save(sc, path) + >>> sameModel = KMeansModel.load(sc, path) + >>> sameModel.predict(sparse_data[0]) == model.predict(sparse_data[0]) + True + >>> try: + ... os.removedirs(path) + ... except OSError: + ... pass """ def __init__(self, centers): @@ -77,6 +89,16 @@ def predict(self, x): best_distance = distance return best + def save(self, sc, path): + java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel.load(sc._jsc.sc(), path) + return KMeansModel(_java2py(sc, java_model.clusterCenters())) + class KMeans(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 621591c26b77f..a539d2f2846f9 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -70,8 +70,8 @@ def _py2java(sc, obj): obj = _to_java_object_rdd(obj) elif isinstance(obj, SparkContext): obj = obj._jsc - elif isinstance(obj, list) and (obj or isinstance(obj[0], JavaObject)): - obj = ListConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, list): + obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass elif isinstance(obj, (int, long, float, bool, basestring)): From 4633a87b86a6ef01fa724d31763dcb97cb5bc746 Mon Sep 17 00:00:00 2001 From: Pei-Lun Lee Date: Wed, 18 Mar 2015 08:34:46 +0800 Subject: [PATCH 438/817] [SPARK-6330] [SQL] Add a test case for SPARK-6330 When getting file statuses, create file system from each path instead of a single one from hadoop configuration. Author: Pei-Lun Lee Closes #5039 from ypcat/spark-6351 and squashes the following commits: a19a3fe [Pei-Lun Lee] [SPARK-6330] [SQL] fix test 506f5a0 [Pei-Lun Lee] [SPARK-6351] [SQL] fix test fa2290e [Pei-Lun Lee] [SPARK-6330] [SQL] Rename test case and add comment 606c967 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6351 896e80a [Pei-Lun Lee] [SPARK-6351] [SQL] Add test case 2ae0916 [Pei-Lun Lee] [SPARK-6351] [SQL] ParquetRelation2 supporting multiple file systems --- .../apache/spark/sql/parquet/ParquetIOSuite.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 36f3406a7825f..a70b3c7ce48d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -329,6 +329,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { checkAnswer(parquetFile(file), (data ++ newData).map(Row.fromTuple)) } } + } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { @@ -341,6 +342,18 @@ class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterA override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6330 regression test") { + // In 1.3.0, save to fs other than file: without configuring core-site.xml would get: + // IllegalArgumentException: Wrong FS: hdfs://..., expected: file:/// + intercept[java.io.FileNotFoundException] { + sqlContext.parquetFile("file:///nonexistent") + } + val errorMessage = intercept[Throwable] { + sqlContext.parquetFile("hdfs://nonexistent") + }.toString + assert(errorMessage.contains("UnknownHostException")) + } } class ParquetDataSourceOffIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From dc9c9196d63aa465e86ac52f0e86e10c12472100 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 18 Mar 2015 09:41:06 +0800 Subject: [PATCH 439/817] [SPARK-6366][SQL] In Python API, the default save mode for save and saveAsTable should be "error" instead of "append". https://issues.apache.org/jira/browse/SPARK-6366 Author: Yin Huai Closes #5053 from yhuai/SPARK-6366 and squashes the following commits: fc81897 [Yin Huai] Use error as the default save mode for save/saveAsTable. --- python/pyspark/sql/dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 94001aec3774b..5cb89da7a8ed5 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -162,7 +162,7 @@ def _java_save_mode(self, mode): "Only 'append', 'overwrite', 'ignore', and 'error' are acceptable save mode.") return jmode - def saveAsTable(self, tableName, source=None, mode="append", **options): + def saveAsTable(self, tableName, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source as a table. The data source is specified by the `source` and a set of `options`. @@ -188,7 +188,7 @@ def saveAsTable(self, tableName, source=None, mode="append", **options): self.sql_ctx._sc._gateway._gateway_client) self._jdf.saveAsTable(tableName, source, jmode, joptions) - def save(self, path=None, source=None, mode="append", **options): + def save(self, path=None, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source. The data source is specified by the `source` and a set of `options`. From a012e08635dc2d643715e11680fd6a3fb3afe44a Mon Sep 17 00:00:00 2001 From: Tijo Thomas Date: Tue, 17 Mar 2015 18:50:19 -0700 Subject: [PATCH 440/817] [SPARK-6383][SQL]Fixed compiler and errors in Dataframe examples Author: Tijo Thomas Closes #5068 from tijoparacka/fix_sql_dataframe_example and squashes the following commits: 6953ac1 [Tijo Thomas] Handled Java and Python example sections 0751a74 [Tijo Thomas] Fixed compiler and errors in Dataframe examples --- docs/sql-programming-guide.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 11c29e20632ae..2cbb4c967eb81 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -170,14 +170,14 @@ df.select("name").show() // Justin // Select everybody, but increment the age by 1 -df.select("name", df("age") + 1).show() +df.select(df("name"), df("age") + 1).show() // name (age + 1) // Michael null // Andy 31 // Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show() +df.filter(df("age") > 21).show() // age name // 30 Andy @@ -220,14 +220,14 @@ df.select("name").show(); // Justin // Select everybody, but increment the age by 1 -df.select("name", df.col("age").plus(1)).show(); +df.select(df.col("name"), df.col("age").plus(1)).show(); // name (age + 1) // Michael null // Andy 31 // Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show(); +df.filter(df.col("age").gt(21)).show(); // age name // 30 Andy @@ -270,14 +270,14 @@ df.select("name").show() ## Justin # Select everybody, but increment the age by 1 -df.select("name", df.age + 1).show() +df.select(df.name, df.age + 1).show() ## name (age + 1) ## Michael null ## Andy 31 ## Justin 20 # Select people older than 21 -df.filter(df.name > 21).show() +df.filter(df.age > 21).show() ## age name ## 30 Andy From 5c80643d137008ce8a0ac7467b31d8d52383c105 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Mar 2015 18:58:52 -0700 Subject: [PATCH 441/817] [SPARK-5908][SQL] Resolve UdtfsAlias when only single Alias is used `ResolveUdtfsAlias` in `hiveUdfs` only considers the `HiveGenericUdtf` with multiple alias. When only single alias is used with `HiveGenericUdtf`, the alias is not working. Author: Liang-Chi Hsieh Closes #4692 from viirya/udft_alias and squashes the following commits: 8a3bae4 [Liang-Chi Hsieh] No need to test selected column from DataFrame since DataFrame API is updated. 160a379 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into udft_alias e6531cc [Liang-Chi Hsieh] Selected column from DataFrame should not re-analyze logical plan. a45cc2a [Liang-Chi Hsieh] Resolve UdtfsAlias when only single Alias is used. --- .../main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 2 ++ .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 34c21c11761ae..4a702d96563d5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -333,6 +333,8 @@ private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 1 => throw new TreeNodeException(p, "only single Generator supported for SELECT clause") + case Project(Seq(Alias(udtf @ HiveGenericUdtf(_, _, _), name)), child) => + Generate(udtf.copy(aliasNames = Seq(name)), join = false, outer = false, None, child) case Project(Seq(MultiAlias(udtf @ HiveGenericUdtf(_, _, _), names)), child) => Generate(udtf.copy(aliasNames = names), join = false, outer = false, None, child) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 22ea19bd82f86..1187228f4c3db 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -397,6 +397,13 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") } + test("resolve udtf with single alias") { + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + jsonRDD(rdd).registerTempTable("data") + val df = sql("SELECT explode(a) AS val FROM data") + val col = df("val") + } + test("logical.Project should not be resolved if it contains aggregates or generators") { // This test is used to test the fix of SPARK-5875. // The original issue was that Project's resolved will be true when it contains From 78cb08a5db7b3e1b61ffb28bc95d0b23e8db5c40 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 17 Mar 2015 19:32:38 -0700 Subject: [PATCH 442/817] [SPARK-5404] [SQL] Update the default statistic number By default, the statistic for logical plan with multiple children is quite aggressive, and those statistic are quite critical for the join optimization, hence we need to estimate the statistics as accurate as possible. For `Union`, which has 2 children, and overwrite the default implementation by `adding` its children `byteInSize` instead of `multiplying`. For `Expand`, which only has a single child, but it will grows the size, and we need to multiply its inflating factor. Author: Cheng Hao Closes #4914 from chenghao-intel/statistic and squashes the following commits: d466bbc [Cheng Hao] Update the default statistic --- .../sql/catalyst/plans/logical/basicOperators.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 20cc8e90a71a3..624912dab4652 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -81,6 +81,11 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } + + override def statistics: Statistics = { + val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes + Statistics(sizeInBytes = sizeInBytes) + } } case class Join( @@ -174,7 +179,12 @@ case class Aggregate( case class Expand( projections: Seq[GroupExpression], output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode + child: LogicalPlan) extends UnaryNode { + override def statistics: Statistics = { + val sizeInBytes = child.statistics.sizeInBytes * projections.length + Statistics(sizeInBytes = sizeInBytes) + } +} trait GroupingAnalytics extends UnaryNode { self: Product => From a6ee2f7940b9a64a81667615586ae597da837974 Mon Sep 17 00:00:00 2001 From: watermen Date: Tue, 17 Mar 2015 19:35:18 -0700 Subject: [PATCH 443/817] [SPARK-5651][SQL] Add input64 in blacklist and add test suit for create table within backticks Now spark version is only support ```create table table_in_database_creation.test1 as select * from src limit 1;``` in HiveContext. This patch is used to support ```create table `table_in_database_creation.test2` as select * from src limit 1;``` in HiveContext. Author: watermen Author: q00251598 Closes #4427 from watermen/SPARK-5651 and squashes the following commits: c5c8ed1 [watermen] add the generated golden files 1f0e42e [q00251598] add input64 in blacklist and add test suit --- .../execution/HiveCompatibilitySuite.scala | 6 +- ...ckticks-0-a253b1ed35dbf503d1b8902dacbe23ac | 0 ...ckticks-1-61dc640dfeaff471f3d2b730f9cbf959 | 0 ...ckticks-2-ce780d068b8d24786e639e361101a0c7 | 500 ++++++++++++++++++ ...ckticks-3-afd6e46b6a289c3c24a8eec75a94043c | 0 .../sql/hive/execution/HiveQuerySuite.scala | 8 + 6 files changed, 513 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 68cb34d698ef3..5180a7f09d80f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -236,7 +236,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // timestamp in array, the output format of Hive contains double quotes, while // Spark SQL doesn't - "udf_sort_array" + "udf_sort_array", + + // It has a bug and it has been fixed by + // https://issues.apache.org/jira/browse/HIVE-7673 (in Hive 0.14 and trunk). + "input46" ) ++ HiveShim.compatibilityBlackList /** diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac b/sql/hive/src/test/resources/golden/create table as with db name within backticks-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 b/sql/hive/src/test/resources/golden/create table as with db name within backticks-1-61dc640dfeaff471f3d2b730f9cbf959 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 b/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/create table as with db name within backticks-2-ce780d068b8d24786e639e361101a0c7 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c b/sql/hive/src/test/resources/golden/create table as with db name within backticks-3-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c0d21bc9a89da..de140fc72a2c3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -334,6 +334,14 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |DROP DATABASE IF EXISTS testdb CASCADE """.stripMargin) + createQueryTest("create table as with db name within backticks", + """ + |CREATE DATABASE IF NOT EXISTS testdb; + |CREATE TABLE `testdb`.`createdtable` AS SELECT * FROM default.src; + |SELECT * FROM testdb.createdtable; + |DROP DATABASE IF EXISTS testdb CASCADE + """.stripMargin) + createQueryTest("insert table with db name", """ |CREATE DATABASE IF NOT EXISTS testdb; From 3579003115fa3217cff6aa400729d96b0c7b257b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 17 Mar 2015 19:47:51 -0700 Subject: [PATCH 444/817] [SPARK-6247][SQL] Fix resolution of ambiguous joins caused by new aliases We need to handle ambiguous `exprId`s that are produced by new aliases as well as those caused by leaf nodes (`MultiInstanceRelation`). Attempting to fix this revealed a bug in `equals` for `Alias` as these objects were comparing equal even when the expression ids did not match. Additionally, `LocalRelation` did not correctly provide statistics, and some tests in `catalyst` and `hive` were not using the helper functions for comparing plans. Based on #4991 by chenghao-intel Author: Michael Armbrust Closes #5062 from marmbrus/selfJoins and squashes the following commits: 8e9b84b [Michael Armbrust] check qualifier too 8038a36 [Michael Armbrust] handle aggs too 0b9c687 [Michael Armbrust] fix more tests c3c574b [Michael Armbrust] revert change. 725f1ab [Michael Armbrust] add statistics a925d08 [Michael Armbrust] check for conflicting attributes in join resolution b022ef7 [Michael Armbrust] Handle project aliases. d8caa40 [Michael Armbrust] test case: SPARK-6247 f9c67c2 [Michael Armbrust] Check for duplicate attributes in join resolution. 898af73 [Michael Armbrust] Fix Alias equality. --- .../sql/catalyst/analysis/Analyzer.scala | 30 +++++++++++++++--- .../expressions/namedExpressions.scala | 6 ++++ .../plans/logical/LocalRelation.scala | 3 ++ .../plans/logical/basicOperators.scala | 7 +++++ .../analysis/HiveTypeCoercionSuite.scala | 10 +++--- .../spark/sql/catalyst/plans/PlanTest.scala | 11 +++++-- .../spark/sql/ColumnExpressionSuite.scala | 6 +++- .../org/apache/spark/sql/SQLQuerySuite.scala | 31 +++++++++++++++++++ .../spark/sql/catalyst/plans/PlanTest.scala | 4 ++- 9 files changed, 96 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7753331748d7b..92d3db077c5e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -237,22 +237,33 @@ class Analyzer(catalog: Catalog, // Special handling for cases when self-join introduce duplicate expression ids. case j @ Join(left, right, _, _) if left.outputSet.intersect(right.outputSet).nonEmpty => val conflictingAttributes = left.outputSet.intersect(right.outputSet) + logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} in $j") - val (oldRelation, newRelation, attributeRewrites) = right.collect { + val (oldRelation, newRelation) = right.collect { + // Handle base relations that might appear more than once. case oldVersion: MultiInstanceRelation if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => val newVersion = oldVersion.newInstance() - val newAttributes = AttributeMap(oldVersion.output.zip(newVersion.output)) - (oldVersion, newVersion, newAttributes) + (oldVersion, newVersion) + + // Handle projects that create conflicting aliases. + case oldVersion @ Project(projectList, _) + if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(projectList = newAliases(projectList))) + + case oldVersion @ Aggregate(_, aggregateExpressions, _) + if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => + (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) }.head // Only handle first case found, others will be fixed on the next pass. + val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) val newRight = right transformUp { case r if r == oldRelation => newRelation + } transformUp { case other => other transformExpressions { case a: Attribute => attributeRewrites.get(a).getOrElse(a) } } - j.copy(right = newRight) case q: LogicalPlan => @@ -272,6 +283,17 @@ class Analyzer(catalog: Catalog, } } + def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { + expressions.map { + case a: Alias => Alias(a.child, a.name)() + case other => other + } + } + + def findAliases(projectList: Seq[NamedExpression]): AttributeSet = { + AttributeSet(projectList.collect { case a: Alias => a.toAttribute }) + } + /** * Returns true if `exprs` contains a [[Star]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 62c062be6d820..17f7f9fe51376 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -124,6 +124,12 @@ case class Alias(child: Expression, name: String) override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + + override def equals(other: Any): Boolean = other match { + case a: Alias => + name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers + case _ => false + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index 92bd057c6f4b6..bb79dc340553b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -54,4 +54,7 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Row] = Nil) otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data case _ => false } + + override lazy val statistics = + Statistics(sizeInBytes = output.map(_.dataType.defaultSize).sum * data.length) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 624912dab4652..1e7b449d75b80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -108,6 +108,13 @@ case class Join( left.output ++ right.output } } + + def selfJoinResolved = left.outputSet.intersect(right.outputSet).isEmpty + + // Joins are only resolved if they don't introduce ambiguious expression ids. + override lazy val resolved: Boolean = { + childrenResolved && !expressions.exists(!_.resolved) && selfJoinResolved + } } case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 85798d0871fda..ecbb54218d457 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.catalyst.analysis -import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.types._ -class HiveTypeCoercionSuite extends FunSuite { +class HiveTypeCoercionSuite extends PlanTest { test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { @@ -106,7 +106,8 @@ class HiveTypeCoercionSuite extends FunSuite { val booleanCasts = new HiveTypeCoercion { }.BooleanCasts def ruleTest(initial: Expression, transformed: Expression) { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) - assert(booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)) == + comparePlans( + booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)), Project(Seq(Alias(transformed, "a")()), testRelation)) } // Remove superflous boolean -> boolean casts. @@ -119,7 +120,8 @@ class HiveTypeCoercionSuite extends FunSuite { val fac = new HiveTypeCoercion { }.FunctionArgumentConversion def ruleTest(initial: Expression, transformed: Expression) { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) - assert(fac(Project(Seq(Alias(initial, "a")()), testRelation)) == + comparePlans( + fac(Project(Seq(Alias(initial, "a")()), testRelation)), Project(Seq(Alias(transformed, "a")()), testRelation)) } ruleTest( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7d609b91389c6..48884040bfce7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.plans import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{NoRelation, Filter, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** @@ -36,6 +36,8 @@ class PlanTest extends FunSuite { plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + case a: Alias => + Alias(a.child, a.name)(exprId = ExprId(0)) } } @@ -50,4 +52,9 @@ class PlanTest extends FunSuite { |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} """.stripMargin) } + + /** Fails the test if the two expressions do not match */ + protected def compareExpressions(e1: Expression, e2: Expression): Unit = { + comparePlans(Filter(e1, NoRelation), Filter(e2, NoRelation)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 3036fbc05d021..a53ae97d6243a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.catalyst.plans.logical.{Project, NoRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ @@ -311,7 +313,9 @@ class ColumnExpressionSuite extends QueryTest { } test("lift alias out of cast") { - assert(col("1234").as("name").cast("int").expr === col("1234").cast("int").as("name").expr) + compareExpressions( + col("1234").as("name").cast("int").expr, + col("1234").cast("int").as("name").expr) } test("columns can be compared") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4dedcd365f6cc..a3c0076e16d6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -36,6 +36,37 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { import org.apache.spark.sql.test.TestSQLContext.implicits._ val sqlCtx = TestSQLContext + test("self join with aliases") { + Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + + checkAnswer( + sql( + """ + |SELECT x.str, COUNT(*) + |FROM df x JOIN df y ON x.str = y.str + |GROUP BY x.str + """.stripMargin), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + + test("self join with alias in agg") { + Seq(1,2,3) + .map(i => (i, i.toString)) + .toDF("int", "str") + .groupBy("str") + .agg($"str", count("str").as("strCount")) + .registerTempTable("df") + + checkAnswer( + sql( + """ + |SELECT x.str, SUM(x.strCount) + |FROM df x JOIN df y ON x.str = y.str + |GROUP BY x.str + """.stripMargin), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( sql("SELECT a FROM testData2 SORT BY a"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 44ee5ab5975fb..98f1c0e69e29d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.scalatest.FunSuite @@ -38,6 +38,8 @@ class PlanTest extends FunSuite { plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + case a: Alias => + Alias(a.child, a.name)(exprId = ExprId(0)) } } From 6205a255aea0652dddadf953771f5405065b5bec Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Mar 2015 09:06:57 -0400 Subject: [PATCH 445/817] [SPARK-6372] [core] Propagate --conf to child processes. And add unit test. Author: Marcelo Vanzin Closes #5057 from vanzin/SPARK-6372 and squashes the following commits: b33728b [Marcelo Vanzin] [SPARK-6372] [core] Propagate --conf to child processes. --- .../spark/launcher/SparkSubmitCommandBuilder.java | 10 +--------- .../spark/launcher/SparkSubmitCommandBuilderSuite.java | 4 ++++ 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 6ffdff63d3c78..91dcf70f105db 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -253,12 +253,6 @@ private boolean isClientMode(Properties userProps) { private class OptionParser extends SparkSubmitOptionParser { - private final List driverJvmKeys = Arrays.asList( - SparkLauncher.DRIVER_EXTRA_CLASSPATH, - SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, - SparkLauncher.DRIVER_MEMORY); - @Override protected boolean handle(String opt, String value) { if (opt.equals(MASTER)) { @@ -278,9 +272,7 @@ protected boolean handle(String opt, String value) { } else if (opt.equals(CONF)) { String[] setConf = value.split("=", 2); checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); - if (driverJvmKeys.contains(setConf[0])) { - conf.put(setConf[0], setConf[1]); - } + conf.put(setConf[0], setConf[1]); } else if (opt.equals(CLASS)) { // The special classes require some special command line handling, since they allow // mixing spark-submit arguments with arguments that should be propagated to the shell diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 815edc4e4971f..626116a9e7477 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -68,6 +68,8 @@ public void testCliParser() throws Exception { parser.DRIVER_JAVA_OPTIONS, "extraJavaOpt", parser.CONF, + "spark.randomOption=foo", + parser.CONF, SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); Map env = new HashMap(); List cmd = buildCommand(sparkSubmitArgs, env); @@ -77,6 +79,8 @@ public void testCliParser() throws Exception { assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); + assertTrue("Command should contain user-defined conf.", + Collections.indexOfSubList(cmd, Arrays.asList(parser.CONF, "spark.randomOption=foo")) > 0); } @Test From e09c852d6b83b9b112685d113f2792daec8785a3 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 18 Mar 2015 09:09:32 -0400 Subject: [PATCH 446/817] SPARK-6389 YARN app diagnostics report doesn't report NPEs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Trivial patch to implicitly call `Exception.toString()` over `Exception.getMessage()` —this defaults to including the exception class & any non-null message; some subclasses include more. No test. Author: Steve Loughran Closes #5070 from steveloughran/stevel/patches/SPARK-6389-NPE-reporting and squashes the following commits: 8239d85 [Steve Loughran] SPARK-6389 cull use of getMessage over toString in the container launcher 6fbaf6a [Steve Loughran] SPARK-6389 YARN app diagnostics report doesn't report NPEs --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 e966bfba7bb7d..056b8c0257cfe 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 @@ -151,7 +151,7 @@ private[spark] class ApplicationMaster( logError("Uncaught exception: ", e) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, - "Uncaught exception: " + e.getMessage()) + "Uncaught exception: " + e) } exitCode } @@ -486,10 +486,10 @@ private[spark] class ApplicationMaster( case _: InterruptedException => // Reporter thread can interrupt to stop user class case cause: Throwable => - logError("User class threw exception: " + cause.getMessage, cause) + logError("User class threw exception: " + cause, cause) finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, - "User class threw exception: " + cause.getMessage) + "User class threw exception: " + cause) } } } From 9d112a958ee2facad179344dd367a6d1ccbc9614 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Wed, 18 Mar 2015 09:15:33 -0400 Subject: [PATCH 447/817] [SPARK-6286][minor] Handle missing Mesos case TASK_ERROR. Author: Iulian Dragos Closes #5000 from dragos/issue/task-error-case and squashes the following commits: e063627 [Iulian Dragos] Handle TASK_ERROR in Mesos scheduler backends. ac17cf0 [Iulian Dragos] Handle missing Mesos case TASK_ERROR. --- core/src/main/scala/org/apache/spark/TaskState.scala | 1 + .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 12 ++---------- .../cluster/mesos/MesosSchedulerBackend.scala | 10 +--------- 3 files changed, 4 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index 0bf1e4a5e2ccd..d85a6d683427d 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -46,5 +46,6 @@ private[spark] object TaskState extends Enumeration { case MesosTaskState.TASK_FAILED => FAILED case MesosTaskState.TASK_KILLED => KILLED case MesosTaskState.TASK_LOST => LOST + case MesosTaskState.TASK_ERROR => LOST } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 90dfe14352a8e..fc92b9c35c3a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.{Utils, AkkaUtils} @@ -262,20 +262,12 @@ private[spark] class CoarseMesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - private def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) synchronized { - if (isFinished(state)) { + if (TaskState.isFinished(TaskState.fromMesos(state))) { val slaveId = taskIdToSlaveId(taskId) slaveIdsWithExecutors -= slaveId taskIdToSlaveId -= taskId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index cfb6592e14aa8..df8f4306b88a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -313,14 +313,6 @@ private[spark] class MesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { inClassLoader() { val tid = status.getTaskId.getValue.toLong @@ -330,7 +322,7 @@ private[spark] class MesosSchedulerBackend( // We lost the executor on this slave, so remember that it's gone removeExecutor(taskIdToSlaveId(tid), "Lost executor") } - if (isFinished(status.getState)) { + if (TaskState.isFinished(state)) { taskIdToSlaveId.remove(tid) } } From 981fbafa2a878e86abeefe1d77cca01fd848f9f6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Mar 2015 09:18:28 -0400 Subject: [PATCH 448/817] [SPARK-6325] [core,yarn] Do not change target executor count when killing executors. The dynamic execution code has two ways to reduce the number of executors: one where it reduces the total number of executors it wants, by asking for an absolute number of executors that is lower than the previous one. The second is by explicitly killing idle executors. YarnAllocator was mixing those up and lowering the target number of executors when a kill was issued. Instead, trust the frontend knows what it's doing, and kill executors without messing with other accounting. That means that if the frontend kills an executor without lowering the target, it will get a new executor shortly. The one situation where both actions (lower the target and kill executor) need to happen together is when user code explicitly calls `SparkContext.killExecutors`. In that case, issue two calls to the backend to achieve the goal. I also did some minor cleanup in related code: - avoid sending a request for executors when target is unchanged, to avoid log spam in the AM - avoid printing misleading log messages in the AM when there are no requests to cancel - fix a slow memory leak plus misleading error message on the driver caused by failing to completely unregister the executor. Author: Marcelo Vanzin Closes #5018 from vanzin/SPARK-6325 and squashes the following commits: 2e782a3 [Marcelo Vanzin] Avoid redundant logging on the AM side. a3567cd [Marcelo Vanzin] Add parentheses. a363926 [Marcelo Vanzin] Update logic. a158101 [Marcelo Vanzin] [SPARK-6325] [core,yarn] Disallow reducing executor count past running count. --- .../CoarseGrainedSchedulerBackend.scala | 7 ++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 1 - .../spark/deploy/yarn/YarnAllocator.scala | 13 ++++++----- .../deploy/yarn/YarnAllocatorSuite.scala | 22 +++++++++++++++++++ 4 files changed, 37 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6f77fa32ce37b..87ebf31139ce9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -211,6 +211,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { + addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingToRemove -= executorId } @@ -371,6 +372,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste logWarning(s"Executor to kill $id does not exist!") } } + // Killing executors means effectively that we want less executors than before, so also update + // the target number of executors to avoid having the backend allocate new ones. + val newTotal = (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size + - filteredExecutorIds.size) + doRequestTotalExecutors(newTotal) + executorsPendingToRemove ++= filteredExecutorIds doKillExecutors(filteredExecutorIds) } 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 056b8c0257cfe..3d18690cd9cbf 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 @@ -534,7 +534,6 @@ private[spark] class ApplicationMaster( driver ! x case RequestExecutors(requestedTotal) => - logInfo(s"Driver requested a total number of $requestedTotal executor(s).") Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 55bfbcd9cb84b..c98763e15b58f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -86,7 +86,8 @@ private[yarn] class YarnAllocator( @volatile private var targetNumExecutors = args.numExecutors // Keep track of which container is running which executor to remove the executors later - private val executorIdToContainer = new HashMap[String, Container] + // Visible for testing. + private[yarn] val executorIdToContainer = new HashMap[String, Container] // Executor memory in MB. protected val executorMemory = args.executorMemory @@ -137,7 +138,10 @@ private[yarn] class YarnAllocator( * be killed. */ def requestTotalExecutors(requestedTotal: Int): Unit = synchronized { - targetNumExecutors = requestedTotal + if (requestedTotal != targetNumExecutors) { + logInfo(s"Driver requested a total number of $requestedTotal executor(s).") + targetNumExecutors = requestedTotal + } } /** @@ -148,8 +152,6 @@ private[yarn] class YarnAllocator( val container = executorIdToContainer.remove(executorId).get internalReleaseContainer(container) numExecutorsRunning -= 1 - targetNumExecutors -= 1 - assert(targetNumExecutors >= 0, "Allocator killed more executors than are allocated!") } else { logWarning(s"Attempted to kill unknown executor $executorId!") } @@ -351,7 +353,8 @@ private[yarn] class YarnAllocator( } } - private def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { + // Visible for testing. + private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { for (completedContainer <- completedContainers) { val containerId = completedContainer.getContainerId diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 3c224f148802e..c09b01bafce37 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -206,6 +206,28 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.getNumExecutorsRunning should be (2) } + test("kill executors") { + val handler = createAllocator(4) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (4) + + val container1 = createContainer("host1") + val container2 = createContainer("host2") + handler.handleAllocatedContainers(Array(container1, container2)) + + handler.requestTotalExecutors(1) + handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } + + val statuses = Seq(container1, container2).map { c => + ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0) + } + handler.updateResourceRequests() + handler.processCompletedContainers(statuses.toSeq) + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (1) + } + test("memory exceeded diagnostic regexes") { val diagnostics = "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + From a95ee242b0a9644c912fc54ed68b4301c9558bc9 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 18 Mar 2015 13:44:37 -0400 Subject: [PATCH 449/817] [SPARK-6374] [MLlib] add get for GeneralizedLinearAlgo I find it's better to have getter for NumFeatures and addIntercept within GeneralizedLinearAlgorithm during actual usage, otherwise I 'll have to get the value through debug. Author: Yuhao Yang Closes #5058 from hhbyyh/addGetLinear and squashes the following commits: 9dc90e8 [Yuhao Yang] add get for GeneralizedLinearAlgo --- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 7c66e8cdebdbe..b262bec904525 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -123,6 +123,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ private var useFeatureScaling = false + /** + * The dimension of training features. + */ + def getNumFeatures: Int = this.numFeatures + /** * The dimension of training features. */ @@ -141,6 +146,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ protected def createModel(weights: Vector, intercept: Double): M + /** + * Get if the algorithm uses addIntercept + */ + def isAddIntercept: Boolean = this.addIntercept + /** * Set if the algorithm should add an intercept. Default false. * We set the default to false because adding the intercept will cause memory allocation. From 3db13874250ded267d7455898e4048a70a47fdcd Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Wed, 18 Mar 2015 20:54:22 -0400 Subject: [PATCH 450/817] SPARK-6085 Part. 2 Increase default value for memory overhead - fixed a description of spark.mesos.executor.memoryOverhead from 7% to 10% - This is a second part of SPARK-6085 Author: Jongyoul Lee Closes #5065 from jongyoul/SPARK-6085-1 and squashes the following commits: c5af84c [Jongyoul Lee] SPARK-6085 Part. 2 Increase default value for memory overhead - Changed "MiB" to "MB" dbac1c0 [Jongyoul Lee] SPARK-6085 Part. 2 Increase default value for memory overhead - fixed a description of spark.mesos.executor.memoryOverhead from 7% to 10% --- docs/running-on-mesos.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 59a3e9d25baf1..6a9d304501dc0 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -224,11 +224,11 @@ See the [configuration page](configuration.html) for information on Spark config
    spark.files.useFetchCachetrue + If set to true (default), file fetching will use a local cache that is shared by executors + that belong to the same application, which can improve task launching performance when + running many executors on the same host. If set to false, these caching optimizations will + be disabled and all executors will fetch their own copies of files. This optimization may be + disabled in order to use Spark local directories that reside on NFS filesystems (see + SPARK-6313 for more details). +
    spark.files.overwrite falsespark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - This value is an additive for spark.executor.memory, specified in MiB, + This value is an additive for spark.executor.memory, specified in MB, which is used to calculate the total Mesos task memory. A value of 384 - implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + implies a 384MB overhead. Additionally, there is a hard-coded 10% minimum overhead. The final overhead will be the larger of either - `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. + `spark.mesos.executor.memoryOverhead` or 10% of `spark.executor.memory`.
    From 540b2a4eabe0bad2455f5140c4ad6a315e37cc3f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 18 Mar 2015 19:43:04 -0700 Subject: [PATCH 451/817] [SPARK-6394][Core] cleanup BlockManager companion object and improve the getCacheLocs method in DAGScheduler The current implementation include searching a HashMap many times, we can avoid this. Actually if you look into `BlockManager.blockIdsToBlockManagers`, the core function call is [this](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L1258), so we can call `blockManagerMaster.getLocations` directly and avoid building a HashMap. Author: Wenchen Fan Closes #5043 from cloud-fan/small and squashes the following commits: e959d12 [Wenchen Fan] fix style 203c493 [Wenchen Fan] some cleanup in BlockManager companion object d409099 [Wenchen Fan] address rxin's comment faec999 [Wenchen Fan] add regression test 2fb57aa [Wenchen Fan] imporve the getCacheLocs method --- .../apache/spark/scheduler/DAGScheduler.scala | 11 +++++----- .../apache/spark/storage/BlockManager.scala | 22 ++++--------------- .../spark/scheduler/DAGSchedulerSuite.scala | 12 ++++++++++ 3 files changed, 22 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index e4170a55b7981..1021172e6afb4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -104,7 +104,7 @@ class DAGScheduler( * * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454). */ - private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] + private val cacheLocs = new HashMap[Int, Seq[Seq[TaskLocation]]] // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with // every task. When we detect a node failing, we note the current epoch number and failed @@ -188,14 +188,15 @@ class DAGScheduler( eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } - private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized { + private[scheduler] + def getCacheLocs(rdd: RDD[_]): Seq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] - val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) - cacheLocs(rdd.id) = blockIds.map { id => - locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) + val locs: Seq[Seq[TaskLocation]] = blockManagerMaster.getLocations(blockIds).map { bms => + bms.map(bm => TaskLocation(bm.host, bm.executorId)) } + cacheLocs(rdd.id) = locs } cacheLocs(rdd.id) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c8b7763f03fb7..80d66e59132da 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1245,10 +1245,10 @@ private[spark] object BlockManager extends Logging { } } - def blockIdsToBlockManagers( + def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { // blockManagerMaster != null is used in tests assert(env != null || blockManagerMaster != null) @@ -1258,24 +1258,10 @@ private[spark] object BlockManager extends Logging { blockManagerMaster.getLocations(blockIds) } - val blockManagers = new HashMap[BlockId, Seq[BlockManagerId]] + val blockManagers = new HashMap[BlockId, Seq[String]] for (i <- 0 until blockIds.length) { - blockManagers(blockIds(i)) = blockLocations(i) + blockManagers(blockIds(i)) = blockLocations(i).map(_.host) } blockManagers.toMap } - - def blockIdsToExecutorIds( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) - } - - def blockIdsToHosts( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) - } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 30119ce5d4eec..63360a0f189a3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -322,6 +322,18 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertDataStructuresEmpty } + test("regression test for getCacheLocs") { + val rdd = new MyRDD(sc, 3, Nil) + cacheLocations(rdd.id -> 0) = + Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) + cacheLocations(rdd.id -> 1) = + Seq(makeBlockManagerId("hostB"), makeBlockManagerId("hostC")) + cacheLocations(rdd.id -> 2) = + Seq(makeBlockManagerId("hostC"), makeBlockManagerId("hostD")) + val locs = scheduler.getCacheLocs(rdd).map(_.map(_.host)) + assert(locs === Seq(Seq("hostA", "hostB"), Seq("hostB", "hostC"), Seq("hostC", "hostD"))) + } + test("avoid exponential blowup when getting preferred locs list") { // Build up a complex dependency graph with repeated zip operations, without preferred locations. var rdd: RDD[_] = new MyRDD(sc, 1, Nil) From 645cf3fcc21987417b2946bdeeeb60af3edf667e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 19 Mar 2015 02:15:50 -0400 Subject: [PATCH 452/817] [SPARK-6222][Streaming] Dont delete checkpoint data when doing pre-batch-start checkpoint This is another alternative approach to https://github.com/apache/spark/pull/4964/ I think this is a simpler fix that can be backported easily to other branches (1.2 and 1.3). All it does it introduce a flag so that the pre-batch-start checkpoint does not call clear checkpoint. There is not unit test yet. I will add it when this approach is commented upon. Not sure if this is testable easily. Author: Tathagata Das Closes #5008 from tdas/SPARK-6222 and squashes the following commits: 7315bc2 [Tathagata Das] Removed empty line. c438de4 [Tathagata Das] Revert unnecessary change. 5e98374 [Tathagata Das] Added unit test 50cb60b [Tathagata Das] Fixed style issue 295ca5c [Tathagata Das] Fixing SPARK-6222 --- .../apache/spark/streaming/Checkpoint.scala | 12 +- .../streaming/scheduler/JobGenerator.scala | 20 +-- .../scheduler/JobGeneratorSuite.scala | 133 ++++++++++++++++++ 3 files changed, 153 insertions(+), 12 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala 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 cb4c94fb9d5a6..db64e11e16304 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -119,7 +119,10 @@ class CheckpointWriter( private var stopped = false private var fs_ : FileSystem = _ - class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable { + class CheckpointWriteHandler( + checkpointTime: Time, + bytes: Array[Byte], + clearCheckpointDataLater: Boolean) extends Runnable { def run() { var attempts = 0 val startTime = System.currentTimeMillis() @@ -166,7 +169,7 @@ class CheckpointWriter( val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") - jobGenerator.onCheckpointCompletion(checkpointTime) + jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => @@ -180,7 +183,7 @@ class CheckpointWriter( } } - def write(checkpoint: Checkpoint) { + def write(checkpoint: Checkpoint, clearCheckpointDataLater: Boolean) { val bos = new ByteArrayOutputStream() val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) @@ -188,7 +191,8 @@ class CheckpointWriter( oos.close() bos.close() try { - executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + executor.execute(new CheckpointWriteHandler( + checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) logDebug("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") } catch { case rej: RejectedExecutionException => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index ac92774a38273..59488dfb0f8c6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -30,7 +30,8 @@ import org.apache.spark.util.{Clock, ManualClock} private[scheduler] sealed trait JobGeneratorEvent private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent private[scheduler] case class ClearMetadata(time: Time) extends JobGeneratorEvent -private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent +private[scheduler] case class DoCheckpoint( + time: Time, clearCheckpointDataLater: Boolean) extends JobGeneratorEvent private[scheduler] case class ClearCheckpointData(time: Time) extends JobGeneratorEvent /** @@ -163,8 +164,10 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** * Callback called when the checkpoint of a batch has been written. */ - def onCheckpointCompletion(time: Time) { - eventActor ! ClearCheckpointData(time) + def onCheckpointCompletion(time: Time, clearCheckpointDataLater: Boolean) { + if (clearCheckpointDataLater) { + eventActor ! ClearCheckpointData(time) + } } /** Processes all events */ @@ -173,7 +176,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { event match { case GenerateJobs(time) => generateJobs(time) case ClearMetadata(time) => clearMetadata(time) - case DoCheckpoint(time) => doCheckpoint(time) + case DoCheckpoint(time, clearCheckpointDataLater) => + doCheckpoint(time, clearCheckpointDataLater) case ClearCheckpointData(time) => clearCheckpointData(time) } } @@ -245,7 +249,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } - eventActor ! DoCheckpoint(time) + eventActor ! DoCheckpoint(time, clearCheckpointDataLater = false) } /** Clear DStream metadata for the given `time`. */ @@ -255,7 +259,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // If checkpointing is enabled, then checkpoint, // else mark batch to be fully processed if (shouldCheckpoint) { - eventActor ! DoCheckpoint(time) + eventActor ! DoCheckpoint(time, clearCheckpointDataLater = true) } else { // If checkpointing is not enabled, then delete metadata information about // received blocks (block data not saved in any case). Otherwise, wait for @@ -278,11 +282,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } /** Perform checkpoint for the give `time`. */ - private def doCheckpoint(time: Time) { + private def doCheckpoint(time: Time, clearCheckpointDataLater: Boolean) { if (shouldCheckpoint && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) ssc.graph.updateCheckpointData(time) - checkpointWriter.write(new Checkpoint(ssc, time)) + checkpointWriter.write(new Checkpoint(ssc, time), clearCheckpointDataLater) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala new file mode 100644 index 0000000000000..4150b60635ed6 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import java.util.concurrent.CountDownLatch + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming._ +import org.apache.spark.util.{ManualClock, Utils} + +class JobGeneratorSuite extends TestSuiteBase { + + // SPARK-6222 is a tricky regression bug which causes received block metadata + // to be deleted before the corresponding batch has completed. This occurs when + // the following conditions are met. + // 1. streaming checkpointing is enabled by setting streamingContext.checkpoint(dir) + // 2. input data is received through a receiver as blocks + // 3. a batch processing a set of blocks takes a long time, such that a few subsequent + // batches have been generated and submitted for processing. + // + // The JobGenerator (as of Mar 16, 2015) checkpoints twice per batch, once after generation + // of a batch, and another time after the completion of a batch. The cleanup of + // checkpoint data (including block metadata, etc.) from DStream must be done only after the + // 2nd checkpoint has completed, that is, after the batch has been completely processed. + // However, the issue is that the checkpoint data and along with it received block data is + // cleaned even in the case of the 1st checkpoint, causing pre-mature deletion of received block + // data. For example, if the 3rd batch is still being process, the 7th batch may get generated, + // and the corresponding "1st checkpoint" will delete received block metadata of batch older + // than 6th batch. That, is 3rd batch's block metadata gets deleted even before 3rd batch has + // been completely processed. + // + // This test tries to create that scenario by the following. + // 1. enable checkpointing + // 2. generate batches with received blocks + // 3. make the 3rd batch never complete + // 4. allow subsequent batches to be generated (to allow premature deletion of 3rd batch metadata) + // 5. verify whether 3rd batch's block metadata still exists + // + test("SPARK-6222: Do not clear received block data too soon") { + import JobGeneratorSuite._ + val checkpointDir = Utils.createTempDir() + val testConf = conf + testConf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + testConf.set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + + withStreamingContext(new StreamingContext(testConf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val numBatches = 10 + val longBatchNumber = 3 // 3rd batch will take a long time + val longBatchTime = longBatchNumber * batchDuration.milliseconds + + val testTimeout = timeout(10 seconds) + val inputStream = ssc.receiverStream(new TestReceiver) + + inputStream.foreachRDD((rdd: RDD[Int], time: Time) => { + if (time.milliseconds == longBatchTime) { + while (waitLatch.getCount() > 0) { + waitLatch.await() + println("Await over") + } + } + }) + + val batchCounter = new BatchCounter(ssc) + ssc.checkpoint(checkpointDir.getAbsolutePath) + ssc.start() + + // Make sure the only 1 batch of information is to be remembered + assert(inputStream.rememberDuration === batchDuration) + val receiverTracker = ssc.scheduler.receiverTracker + + // Get the blocks belonging to a batch + def getBlocksOfBatch(batchTime: Long) = { + receiverTracker.getBlocksOfBatchAndStream(Time(batchTime), inputStream.id) + } + + // Wait for new blocks to be received + def waitForNewReceivedBlocks() { + eventually(testTimeout) { + assert(receiverTracker.hasUnallocatedBlocks) + } + } + + // Wait for received blocks to be allocated to a batch + def waitForBlocksToBeAllocatedToBatch(batchTime: Long) { + eventually(testTimeout) { + assert(getBlocksOfBatch(batchTime).nonEmpty) + } + } + + // Generate a large number of batches with blocks in them + for (batchNum <- 1 to numBatches) { + waitForNewReceivedBlocks() + clock.advance(batchDuration.milliseconds) + waitForBlocksToBeAllocatedToBatch(clock.getTimeMillis()) + } + + // Wait for 3rd batch to start + eventually(testTimeout) { + ssc.scheduler.getPendingTimes().contains(Time(numBatches * batchDuration.milliseconds)) + } + + // Verify that the 3rd batch's block data is still present while the 3rd batch is incomplete + assert(getBlocksOfBatch(longBatchTime).nonEmpty, "blocks of incomplete batch already deleted") + assert(batchCounter.getNumCompletedBatches < longBatchNumber) + waitLatch.countDown() + } + } +} + +object JobGeneratorSuite { + val waitLatch = new CountDownLatch(1) +} From 2c3f83c34bb8d2c1bf13b33633d8c5a8089545d1 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 18 Mar 2015 23:48:45 -0700 Subject: [PATCH 453/817] [SPARK-4012] stop SparkContext when the exception is thrown from an infinite loop https://issues.apache.org/jira/browse/SPARK-4012 This patch is a resubmission for https://github.com/apache/spark/pull/2864 What I am proposing in this patch is that ***when the exception is thrown from an infinite loop, we should stop the SparkContext, instead of let JVM throws exception forever*** So, in the infinite loops where we originally wrapped with a ` logUncaughtExceptions`, I changed to `tryOrStopSparkContext`, so that the Spark component is stopped Early stopped JVM process is helpful for HA scheme design, for example, The user has a script checking the existence of the pid of the Spark Streaming driver for monitoring the availability; with the code before this patch, the JVM process is still available but not functional when the exceptions are thrown andrewor14, srowen , mind taking further consideration about the change? Author: CodingCat Closes #5004 from CodingCat/SPARK-4012-1 and squashes the following commits: 589276a [CodingCat] throw fatal error again 3c72cd8 [CodingCat] address the comments 6087864 [CodingCat] revise comments 6ad3eb0 [CodingCat] stop SparkContext instead of quit the JVM process 6322959 [CodingCat] exit JVM process when the exception is thrown from an infinite loop --- .../org/apache/spark/ContextCleaner.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/util/AsynchronousListenerBus.scala | 10 +++++-- .../scala/org/apache/spark/util/Utils.scala | 28 +++++++++++++++++++ .../scheduler/EventLoggingListenerSuite.scala | 9 +++--- .../spark/scheduler/SparkListenerSuite.scala | 10 +++---- .../streaming/scheduler/JobScheduler.scala | 2 +- 9 files changed, 51 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 0c59a61e81393..9b05c9623b704 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -145,7 +145,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning(): Unit = Utils.logUncaughtExceptions { + private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4457f40286fda..228ff715fe7cb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1736,7 +1736,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - listenerBus.start() + listenerBus.start(this) } /** Post the application start event */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 16d88c17d1a76..7fde02040927d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -93,7 +93,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def getRunner(operateFun: () => Unit): Runnable = { new Runnable() { - override def run() = Utils.logUncaughtExceptions { + override def run() = Utils.tryOrExit { operateFun() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7a9cf1c2e7f30..f33fd4450b2a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -145,7 +145,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - Utils.tryOrExit { checkSpeculatableTasks() } + Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } } diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index 18c627e8c7a15..ce7887b76ff96 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -21,6 +21,7 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean import com.google.common.annotations.VisibleForTesting +import org.apache.spark.SparkContext /** * Asynchronously passes events to registered listeners. @@ -38,6 +39,8 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri self => + private var sparkContext: SparkContext = null + /* Cap the capacity of the event queue so we get an explicit error (rather than * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ private val EVENT_QUEUE_CAPACITY = 10000 @@ -57,7 +60,7 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri private val listenerThread = new Thread(name) { setDaemon(true) - override def run(): Unit = Utils.logUncaughtExceptions { + override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) { while (true) { eventLock.acquire() self.synchronized { @@ -89,9 +92,12 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri * This first sends out all buffered events posted before this listener bus has started, then * listens for any additional events asynchronously while the listener bus is still running. * This should only be called once. + * + * @param sc Used to stop the SparkContext in case the listener thread dies. */ - def start() { + def start(sc: SparkContext) { if (started.compareAndSet(false, true)) { + sparkContext = sc listenerThread.start() } else { throw new IllegalStateException(s"$name already started!") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index af8a24553a461..91aa70870ab20 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1146,6 +1146,8 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler + * + * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { try { @@ -1156,6 +1158,32 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * exception + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * spark-started JVM process . + */ + def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { + try { + block + } catch { + case e: ControlThrowable => throw e + case t: Throwable => + val currentThreadName = Thread.currentThread().getName + if (sc != null) { + logError(s"uncaught error in thread $currentThreadName, stopping SparkContext", t) + sc.stop() + } + if (!NonFatal(t)) { + logError(s"throw uncaught fatal error in thread $currentThreadName", t) + throw t + } + } + } + /** * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught * exceptions as IOException. This is used when implementing Externalizable and Serializable's diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 992dde66f982f..448258a754153 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -25,9 +25,9 @@ import scala.io.Source import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.{FunSuiteLike, BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION} +import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -39,7 +39,8 @@ import org.apache.spark.util.{JsonProtocol, Utils} * logging events, whether the parsing of the file names is correct, and whether the logged events * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { +class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter + with Logging { import EventLoggingListenerSuite._ private val fileSystem = Utils.getHadoopFileSystem("/", @@ -144,7 +145,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite eventLogger.start() - listenerBus.start() + listenerBus.start(sc) listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 3a41ee8d4ae0c..627c9a4ddfffc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -46,7 +46,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(counter.count === 0) // Starting listener bus should flush all buffered events - bus.start() + bus.start(sc) assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) @@ -58,8 +58,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers // Listener bus must not be started twice intercept[IllegalStateException] { val bus = new LiveListenerBus - bus.start() - bus.start() + bus.start(sc) + bus.start(sc) } // ... or stopped before starting @@ -96,7 +96,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val blockingListener = new BlockingListener bus.addListener(blockingListener) - bus.start() + bus.start(sc) bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() @@ -347,7 +347,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(badListener) bus.addListener(jobCounter1) bus.addListener(jobCounter2) - bus.start() + bus.start(sc) // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index b3ffc71904c76..60bc099b27a4c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -61,7 +61,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } }), "JobScheduler") - listenerBus.start() + listenerBus.start(ssc.sparkContext) receiverTracker = new ReceiverTracker(ssc) receiverTracker.start() jobGenerator.start() From 797f8a000773d848fa52c7fe2eb1b5e5e7f6c55a Mon Sep 17 00:00:00 2001 From: Pierre Borckmans Date: Thu, 19 Mar 2015 08:02:06 -0400 Subject: [PATCH 454/817] [SPARK-6402][DOC] - Remove some refererences to shark in docs and ec2 EC2 script and job scheduling documentation still refered to Shark. I removed these references. I also removed a remaining `SHARK_VERSION` variable from `ec2-variables.sh`. Author: Pierre Borckmans Closes #5083 from pierre-borckmans/remove_refererences_to_shark_in_docs and squashes the following commits: 4e90ffc [Pierre Borckmans] Removed deprecated SHARK_VERSION caea407 [Pierre Borckmans] Remove shark reference from ec2 script doc 196c744 [Pierre Borckmans] Removed references to Shark --- docs/ec2-scripts.md | 2 +- docs/job-scheduling.md | 6 ++---- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 1 - 3 files changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 8c9a1e1262d8f..7f60f82b966fe 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -5,7 +5,7 @@ title: Running Spark on EC2 The `spark-ec2` script, located in Spark's `ec2` directory, allows you to launch, manage and shut down Spark clusters on Amazon EC2. It automatically -sets up Spark, Shark and HDFS on the cluster for you. This guide describes +sets up Spark and HDFS on the cluster for you. This guide describes how to use `spark-ec2` to launch clusters, how to run jobs on them, and how to shut them down. It assumes you've already signed up for an EC2 account on the [Amazon Web Services site](http://aws.amazon.com/). diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 5295e351dd711..963e88a3e1d8f 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -14,8 +14,7 @@ runs an independent set of executor processes. The cluster managers that Spark r facilities for [scheduling across applications](#scheduling-across-applications). Second, _within_ each Spark application, multiple "jobs" (Spark actions) may be running concurrently if they were submitted by different threads. This is common if your application is serving requests -over the network; for example, the [Shark](http://shark.cs.berkeley.edu) server works this way. Spark -includes a [fair scheduler](#scheduling-within-an-application) to schedule resources within each SparkContext. +over the network. Spark includes a [fair scheduler](#scheduling-within-an-application) to schedule resources within each SparkContext. # Scheduling Across Applications @@ -52,8 +51,7 @@ an application to gain back cores on one node when it has work to do. To use thi Note that none of the modes currently provide memory sharing across applications. If you would like to share data this way, we recommend running a single server application that can serve multiple requests by querying -the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC server works this way for SQL -queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will +the same RDDs. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will provide another approach to share RDDs. ## Dynamic Resource Allocation diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 0857657152ec7..4f3e8da809f7f 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -25,7 +25,6 @@ export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" export MODULES="{{modules}}" export SPARK_VERSION="{{spark_version}}" -export SHARK_VERSION="{{shark_version}}" export TACHYON_VERSION="{{tachyon_version}}" export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" From 3c4e486b9c8d3f256e801db7c176ab650c976135 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 19 Mar 2015 08:51:49 -0400 Subject: [PATCH 455/817] [SPARK-5843] [API] Allowing map-side combine to be specified in Java. Specifically, when calling JavaPairRDD.combineByKey(), there is a new six-parameter method that exposes the map-side-combine boolean as the fifth parameter and the serializer as the sixth parameter. Author: mcheah Closes #4634 from mccheah/pair-rdd-map-side-combine and squashes the following commits: 5c58319 [mcheah] Fixing compiler errors. 3ce7deb [mcheah] Addressing style and documentation comments. 7455c7a [mcheah] Allowing Java combineByKey to specify Serializer as well. 6ddd729 [mcheah] [SPARK-5843] Allowing map-side combine to be specified in Java. --- .../apache/spark/api/java/JavaPairRDD.scala | 46 ++++++++++++---- .../java/org/apache/spark/JavaAPISuite.java | 53 +++++++++++++++++-- 2 files changed, 87 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 4eadc9a85613e..a023712be1166 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.rdd.RDD.rddToPairRDDFunctions +import org.apache.spark.serializer.Serializer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -227,24 +228,51 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). + * In addition, users can control the partitioning of the output RDD, the serializer that is use + * for the shuffle, and whether to perform map-side aggregation (if a mapper can produce multiple + * items with the same key). */ def combineByKey[C](createCombiner: JFunction[V, C], - mergeValue: JFunction2[C, V, C], - mergeCombiners: JFunction2[C, C, C], - partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val ctag: ClassTag[C] = fakeClassTag + mergeValue: JFunction2[C, V, C], + mergeCombiners: JFunction2[C, C, C], + partitioner: Partitioner, + mapSideCombine: Boolean, + serializer: Serializer): JavaPairRDD[K, C] = { + implicit val ctag: ClassTag[C] = fakeClassTag fromRDD(rdd.combineByKey( createCombiner, mergeValue, mergeCombiners, - partitioner + partitioner, + mapSideCombine, + serializer )) } /** - * Simplified version of combineByKey that hash-partitions the output RDD. + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a + * "combined type" C * Note that V and C can be different -- for example, one might group an + * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three + * functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD. This method automatically + * uses map-side aggregation in shuffling the RDD. + */ + def combineByKey[C](createCombiner: JFunction[V, C], + mergeValue: JFunction2[C, V, C], + mergeCombiners: JFunction2[C, C, C], + partitioner: Partitioner): JavaPairRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, true, null) + } + + /** + * Simplified version of combineByKey that hash-partitions the output RDD and uses map-side + * aggregation. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], @@ -488,7 +516,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing - * partitioner/parallelism level. + * partitioner/parallelism level and using map-side aggregation. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 8ec54360ca42a..d4b5bb519157c 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -24,11 +24,12 @@ import java.util.*; import java.util.concurrent.*; -import org.apache.spark.input.PortableDataStream; +import scala.collection.JavaConversions; import scala.Tuple2; import scala.Tuple3; import scala.Tuple4; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -51,8 +52,11 @@ import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.*; import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.input.PortableDataStream; import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; +import org.apache.spark.rdd.RDD; +import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; @@ -726,8 +730,8 @@ public void javaDoubleRDDHistoGram() { Tuple2 results = rdd.histogram(2); double[] expected_buckets = {1.0, 2.5, 4.0}; long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1, 0.1); - Assert.assertArrayEquals(expected_counts, results._2); + Assert.assertArrayEquals(expected_buckets, results._1(), 0.1); + Assert.assertArrayEquals(expected_counts, results._2()); // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); Assert.assertArrayEquals(expected_counts, histogram); @@ -1424,6 +1428,49 @@ public void checkpointAndRestore() { Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } + @Test + public void combineByKey() { + JavaRDD originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6)); + Function keyFunction = new Function() { + @Override + public Integer call(Integer v1) throws Exception { + return v1 % 3; + } + }; + Function createCombinerFunction = new Function() { + @Override + public Integer call(Integer v1) throws Exception { + return v1; + } + }; + + Function2 mergeValueFunction = new Function2() { + @Override + public Integer call(Integer v1, Integer v2) throws Exception { + return v1 + v2; + } + }; + + JavaPairRDD combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); + Map results = combinedRDD.collectAsMap(); + ImmutableMap expected = ImmutableMap.of(0, 9, 1, 5, 2, 7); + Assert.assertEquals(expected, results); + + Partitioner defaultPartitioner = Partitioner.defaultPartitioner( + combinedRDD.rdd(), JavaConversions.asScalaBuffer(Lists.>newArrayList())); + combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey( + createCombinerFunction, + mergeValueFunction, + mergeValueFunction, + defaultPartitioner, + false, + new KryoSerializer(new SparkConf())); + results = combinedRDD.collectAsMap(); + Assert.assertEquals(expected, results); + } + @SuppressWarnings("unchecked") @Test public void mapOnPairRDD() { From dda4dedca0459fc7c00eb1d9cb07e14af1621e0f Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 19 Mar 2015 11:10:20 -0400 Subject: [PATCH 456/817] [SPARK-6291] [MLLIB] GLM toString & toDebugString GLM toString prints out intercept, numFeatures. For LogisticRegression and SVM model, toString also prints out numClasses, threshold. GLM toDebugString prints out the whole weights, intercept. Author: Yanbo Liang Closes #5038 from yanboliang/spark-6291 and squashes the following commits: 2f578b0 [Yanbo Liang] code format 78b33f2 [Yanbo Liang] fix typos 1e8a023 [Yanbo Liang] GLM toString & toDebugString --- .../spark/mllib/classification/LogisticRegression.scala | 4 ++++ .../scala/org/apache/spark/mllib/classification/SVM.scala | 4 ++++ .../mllib/regression/GeneralizedLinearAlgorithm.scala | 7 ++++++- 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index b787667b018e6..e7c3599ff619c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -163,6 +163,10 @@ class LogisticRegressionModel ( } override protected def formatVersion: String = "1.0" + + override def toString: String = { + s"${super.toString}, numClasses = ${numClasses}, threshold = ${threshold.get}" + } } object LogisticRegressionModel extends Loader[LogisticRegressionModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index cfc7f868a02f0..52fb62dcff1b4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -86,6 +86,10 @@ class SVMModel ( } override protected def formatVersion: String = "1.0" + + override def toString: String = { + s"${super.toString}, numClasses = 2, threshold = ${threshold.get}" + } } object SVMModel extends Loader[SVMModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index b262bec904525..45b9ebb4cc0d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -76,7 +76,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double predictPoint(testData, weights, intercept) } - override def toString() = "(weights=%s, intercept=%s)".format(weights, intercept) + /** + * Print a summary of the model. + */ + override def toString: String = { + s"${this.getClass.getName}: intercept = ${intercept}, numFeatures = ${weights.size}" + } } /** From 8cb23a1f9a3ed08e57865bcb6cc1cc7902881073 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 19 Mar 2015 11:18:24 -0400 Subject: [PATCH 457/817] [SPARK-5313][Project Infra]: Create simple framework for highlighting changes introduced in a PR Built a simple framework with a `dev/tests` directory to house all pull request related tests. I've moved the two original tests (`pr_merge_ability` and `pr_public_classes`) into the new `dev/tests` directory and tested to the best of my ability. At this point I need to test against Jenkins actually running the new `run-tests-jenkins` script to ensure things aren't broken down the path. Author: Brennon York Closes #5072 from brennonyork/SPARK-5313 and squashes the following commits: 8ae990c [Brennon York] added dev/run-tests back, removed echo 5db4ed4 [Brennon York] removed the git checkout 1b50050 [Brennon York] adding echos to see what jenkins is seeing b823959 [Brennon York] removed run-tests to further test the public_classes pr test 2b9ce12 [Brennon York] added the dev/run-tests call back in ffd49c0 [Brennon York] remove -c from bash as that was removing the trailing args 735d615 [Brennon York] removed the actual dev/run-tests command to further test jenkins d579662 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-5313 aa48029 [Brennon York] removed echo lines for testing jenkins 24cd965 [Brennon York] added test output to check within jenkins to verify 3a38e73 [Brennon York] removed the temporary read 9c881ff [Brennon York] updated test suite 183b7ee [Brennon York] added documentation on how to create tests 0bc2efe [Brennon York] ensure each test starts on the current pr branch 1743378 [Brennon York] added tests in test suite abd7430 [Brennon York] updated to include test suite --- dev/run-tests-jenkins | 75 ++++++++++++++-------------------- dev/tests/pr_merge_ability.sh | 39 ++++++++++++++++++ dev/tests/pr_public_classes.sh | 65 +++++++++++++++++++++++++++++ 3 files changed, 135 insertions(+), 44 deletions(-) create mode 100755 dev/tests/pr_merge_ability.sh create mode 100755 dev/tests/pr_public_classes.sh diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 6a849e4f77207..5f4000e83925c 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -49,6 +49,21 @@ SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +# Array to capture all tests to run on the pull request. These tests are held under the +#+ dev/tests/ directory. +# +# To write a PR test: +#+ * the file must reside within the dev/tests directory +#+ * be an executable bash script +#+ * accept two arguments on the command line, the first being the Github PR long commit +#+ hash and the second the Github SHA1 hash +#+ * and, lastly, return string output to be included in the pr message output that will +#+ be posted to Github +PR_TESTS=( + "pr_merge_ability" + "pr_public_classes" +) + function post_message () { local message=$1 local data="{\"body\": \"$message\"}" @@ -131,48 +146,22 @@ function send_archived_logs () { fi } - -# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR -#+ and not anything else added to master since the PR was branched. - -# check PR merge-ability and check for new public classes -{ - if [ "$sha1" == "$ghprbActualCommit" ]; then - merge_note=" * This patch **does not merge cleanly**." - else - merge_note=" * This patch merges cleanly." +# Environment variable to capture PR test output +pr_message="" + +# Run pull request tests +for t in "${PR_TESTS[@]}"; do + this_test="${FWDIR}/dev/tests/${t}.sh" + # Ensure the test is a file and is executable + if [ -x "$this_test" ]; then + echo "ghprb: $ghprbActualCommit sha1: $sha1" + this_mssg="`bash \"${this_test}\" \"${ghprbActualCommit}\" \"${sha1}\" 2>/dev/null`" + # Check if this is the merge test as we submit that note *before* and *after* + # the tests run + [ "$t" == "pr_merge_ability" ] && merge_note="${this_mssg}" + pr_message="${pr_message}\n${this_mssg}" fi - - source_files=$( - git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ - | grep -v -e "\/test" `# ignore files in test directories` \ - | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ - | tr "\n" " " - ) - new_public_classes=$( - git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ - | grep "^\+" `# filter in only added lines` \ - | sed -r -e "s/^\+//g" `# remove the leading +` \ - | grep -e "trait " -e "class " `# filter in lines with these key words` \ - | grep -e "{" -e "(" `# filter in lines with these key words, too` \ - | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ - | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ - | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ - | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ - | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ - | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ - | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ - | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ - | tr -d "\n" `# remove actual LF characters` - ) - - if [ -z "$new_public_classes" ]; then - public_classes_note=" * This patch adds no public classes." - else - public_classes_note=" * This patch adds the following public classes _(experimental)_:" - public_classes_note="${public_classes_note}\n${new_public_classes}" - fi -} +done # post start message { @@ -181,7 +170,6 @@ function send_archived_logs () { PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." start_message="${start_message}\n${merge_note}" - # start_message="${start_message}\n${public_classes_note}" post_message "$start_message" } @@ -234,8 +222,7 @@ function send_archived_logs () { PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" - result_message="${result_message}\n${merge_note}" - result_message="${result_message}\n${public_classes_note}" + result_message="${result_message}\n${pr_message}" post_message "$result_message" } diff --git a/dev/tests/pr_merge_ability.sh b/dev/tests/pr_merge_ability.sh new file mode 100755 index 0000000000000..d9a347fe24a8c --- /dev/null +++ b/dev/tests/pr_merge_ability.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# + +ghprbActualCommit="$1" +sha1="$2" + +# check PR merge-ability +if [ "${sha1}" == "${ghprbActualCommit}" ]; then + echo " * This patch **does not merge cleanly**." +else + echo " * This patch merges cleanly." +fi diff --git a/dev/tests/pr_public_classes.sh b/dev/tests/pr_public_classes.sh new file mode 100755 index 0000000000000..927295b88c963 --- /dev/null +++ b/dev/tests/pr_public_classes.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# + +# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR +#+ and not anything else added to master since the PR was branched. + +ghprbActualCommit="$1" +sha1="$2" + +source_files=$( + git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ + | grep -v -e "\/test" `# ignore files in test directories` \ + | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ + | tr "\n" " " +) +new_public_classes=$( + git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` +) + +if [ -z "$new_public_classes" ]; then + echo " * This patch adds no public classes." +else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + echo "${public_classes_note}\n${new_public_classes}" +fi From 3b5aaa6a5fe0d838a8570c9d500ebca5f63764f8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 19 Mar 2015 15:25:32 -0400 Subject: [PATCH 458/817] [Core][minor] remove unused `visitedStages` in `DAGScheduler.stageDependsOn` We define and update `visitedStages` in `DAGScheduler.stageDependsOn`, but never read it. So we can safely remove it. Author: Wenchen Fan Closes #5086 from cloud-fan/minor and squashes the following commits: 24663ea [Wenchen Fan] remove un-used variable --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 1021172e6afb4..8feac6cb6b7a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1262,7 +1262,6 @@ class DAGScheduler( return true } val visitedRdds = new HashSet[RDD[_]] - val visitedStages = new HashSet[Stage] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting val waitingForVisit = new Stack[RDD[_]] @@ -1274,7 +1273,6 @@ class DAGScheduler( case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { - visitedStages += mapStage waitingForVisit.push(mapStage.rdd) } // Otherwise there's no need to follow the dependency back case narrowDep: NarrowDependency[_] => From f17d43b033d928dbc46aef8e367aa08902e698ad Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 19 Mar 2015 12:46:10 -0700 Subject: [PATCH 459/817] [SPARK-6219] [Build] Check that Python code compiles This PR expands the Python lint checks so that they check for obvious compilation errors in our Python code. For example: ``` $ ./dev/lint-python Python lint checks failed. Compiling ./ec2/spark_ec2.py ... File "./ec2/spark_ec2.py", line 618 return (master_nodes,, slave_nodes) ^ SyntaxError: invalid syntax ./ec2/spark_ec2.py:618:25: E231 missing whitespace after ',' ./ec2/spark_ec2.py:1117:101: E501 line too long (102 > 100 characters) ``` This PR also bumps up the version of `pep8`. It ignores new types of checks introduced by that version bump while fixing problems missed by the older version of `pep8` we were using. Author: Nicholas Chammas Closes #4941 from nchammas/compile-spark-ec2 and squashes the following commits: 75e31d8 [Nicholas Chammas] upgrade pep8 + check compile b33651c [Nicholas Chammas] PEP8 line length --- dev/lint-python | 44 +++++++++++++++++++++++++++----------------- ec2/spark_ec2.py | 4 ++-- 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index 772f856154ae0..fded654893a7c 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -19,43 +19,53 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" -PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" +PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" +PYTHON_LINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/python-lint-report.txt" cd "$SPARK_ROOT_DIR" +# compileall: https://docs.python.org/2/library/compileall.html +python -B -m compileall -q -l $PATHS_TO_CHECK > "$PYTHON_LINT_REPORT_PATH" +compile_status="${PIPESTATUS[0]}" + # Get pep8 at runtime so that we don't rely on it being installed on the build server. #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 #+ TODOs: -#+ - Dynamically determine latest release version of pep8 and use that. -#+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) +#+ - Download pep8 from PyPI. It's more "official". PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" -PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" -PEP8_PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.6.2/pep8.py" +# if [ ! -e "$PEP8_SCRIPT_PATH" ]; then curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" -curl_status=$? +curl_status="$?" -if [ $curl_status -ne 0 ]; then +if [ "$curl_status" -ne 0 ]; then echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." - exit $curl_status + exit "$curl_status" fi - +# fi # There is no need to write this output to a file #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python "$PEP8_SCRIPT_PATH" $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" -pep8_status=${PIPESTATUS[0]} #$? +python "$PEP8_SCRIPT_PATH" --ignore=E402,E731,E241,W503,E226 $PATHS_TO_CHECK >> "$PYTHON_LINT_REPORT_PATH" +pep8_status="${PIPESTATUS[0]}" + +if [ "$compile_status" -eq 0 -a "$pep8_status" -eq 0 ]; then + lint_status=0 +else + lint_status=1 +fi -if [ $pep8_status -ne 0 ]; then - echo "PEP 8 checks failed." - cat "$PEP8_REPORT_PATH" +if [ "$lint_status" -ne 0 ]; then + echo "Python lint checks failed." + cat "$PYTHON_LINT_REPORT_PATH" else - echo "PEP 8 checks passed." + echo "Python lint checks passed." fi -rm "$PEP8_REPORT_PATH" rm "$PEP8_SCRIPT_PATH" +rm "$PYTHON_LINT_REPORT_PATH" -exit $pep8_status +exit "$lint_status" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index f848874b0c775..c467cd08ed742 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1159,8 +1159,8 @@ def real_main(): if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves with " + \ - "different AMI virtualization types." + "Error: spark-ec2 currently does not support having a master and slaves " + \ + "with different AMI virtualization types." print >> stderr, "master instance virtualization type: {t}".format( t=EC2_INSTANCE_TYPES[opts.master_instance_type]) print >> stderr, "slave instance virtualization type: {t}".format( From 0745a305fac622a6eeb8aa4a7401205a14252939 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 19 Mar 2015 22:12:01 -0400 Subject: [PATCH 460/817] Tighten up field/method visibility in Executor and made some code more clear to read. I was reading Executor just now and found that some latest changes introduced some weird code path with too much monadic chaining and unnecessary fields. I cleaned it up a bit, and also tightened up the visibility of various fields/methods. Also added some inline documentation to help understand this code better. Author: Reynold Xin Closes #4850 from rxin/executor and squashes the following commits: 866fc60 [Reynold Xin] Code review feedback. 020efbb [Reynold Xin] Tighten up field/method visibility in Executor and made some code more clear to read. --- .../org/apache/spark/TaskEndReason.scala | 6 +- .../executor/CommitDeniedException.scala | 6 +- .../org/apache/spark/executor/Executor.scala | 196 ++++++++++-------- .../spark/executor/ExecutorSource.scala | 16 +- .../org/apache/spark/scheduler/Task.scala | 2 +- 5 files changed, 120 insertions(+), 106 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 29a5cd5fdac76..48fd3e7e23d52 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -151,11 +151,7 @@ case object TaskKilled extends TaskFailedReason { * Task requested the driver to commit, but was denied. */ @DeveloperApi -case class TaskCommitDenied( - jobID: Int, - partitionID: Int, - attemptID: Int) - extends TaskFailedReason { +case class TaskCommitDenied(jobID: Int, partitionID: Int, attemptID: Int) extends TaskFailedReason { override def toErrorString: String = s"TaskCommitDenied (Driver denied task commit)" + s" for job: $jobID, partition: $partitionID, attempt: $attemptID" } diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala index f7604a321f007..f47d7ef511da1 100644 --- a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala @@ -22,14 +22,12 @@ import org.apache.spark.{TaskCommitDenied, TaskEndReason} /** * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver. */ -class CommitDeniedException( +private[spark] class CommitDeniedException( msg: String, jobID: Int, splitID: Int, attemptID: Int) extends Exception(msg) { - def toTaskEndReason: TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID) - + def toTaskEndReason: TaskEndReason = TaskCommitDenied(jobID, splitID, attemptID) } - diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6196f7b165049..bf3135ef081c1 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent._ +import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -31,15 +31,17 @@ import akka.actor.Props import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, - SparkUncaughtExceptionHandler, AkkaUtils, Utils} +import org.apache.spark.util._ /** - * Spark executor used with Mesos, YARN, and the standalone scheduler. - * In coarse-grained mode, an existing actor system is provided. + * Spark executor, backed by a threadpool to run tasks. + * + * This can be used with Mesos, YARN, and the standalone scheduler. + * An internal RPC interface (at the moment Akka) is used for communication with the driver, + * except in the case of Mesos fine-grained mode. */ private[spark] class Executor( executorId: String, @@ -47,8 +49,8 @@ private[spark] class Executor( env: SparkEnv, userClassPath: Seq[URL] = Nil, isLocal: Boolean = false) - extends Logging -{ + extends Logging { + logInfo(s"Starting executor ID $executorId on host $executorHostname") // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -78,9 +80,8 @@ private[spark] class Executor( } // Start worker thread pool - val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") - - val executorSource = new ExecutorSource(this, executorId) + private val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") + private val executorSource = new ExecutorSource(threadPool, executorId) if (!isLocal) { env.metricsSystem.registerSource(executorSource) @@ -122,21 +123,21 @@ private[spark] class Executor( taskId: Long, attemptNumber: Int, taskName: String, - serializedTask: ByteBuffer) { + serializedTask: ByteBuffer): Unit = { val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName, serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } - def killTask(taskId: Long, interruptThread: Boolean) { + def killTask(taskId: Long, interruptThread: Boolean): Unit = { val tr = runningTasks.get(taskId) if (tr != null) { tr.kill(interruptThread) } } - def stop() { + def stop(): Unit = { env.metricsSystem.report() env.actorSystem.stop(executorActor) isStopped = true @@ -146,7 +147,10 @@ private[spark] class Executor( } } - private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + /** Returns the total amount of time this JVM process has spent in garbage collection. */ + private def computeTotalGcTime(): Long = { + ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + } class TaskRunner( execBackend: ExecutorBackend, @@ -156,12 +160,19 @@ private[spark] class Executor( serializedTask: ByteBuffer) extends Runnable { + /** Whether this task has been killed. */ @volatile private var killed = false - @volatile var task: Task[Any] = _ - @volatile var attemptedTask: Option[Task[Any]] = None + + /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ - def kill(interruptThread: Boolean) { + /** + * The task to run. This will be set in run() by deserializing the task binary coming + * from the driver. Once it is set, it will never be changed. + */ + @volatile var task: Task[Any] = _ + + def kill(interruptThread: Boolean): Unit = { logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { @@ -169,14 +180,14 @@ private[spark] class Executor( } } - override def run() { + override def run(): Unit = { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 - startGCTime = gcTime + startGCTime = computeTotalGcTime() try { val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) @@ -193,7 +204,6 @@ private[spark] class Executor( throw new TaskKilledException } - attemptedTask = Some(task) logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) @@ -215,18 +225,17 @@ private[spark] class Executor( for (m <- task.metrics) { m.setExecutorDeserializeTime(taskStart - deserializeStartTime) m.setExecutorRunTime(taskFinish - taskStart) - m.setJvmGCTime(gcTime - startGCTime) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) m.setResultSerializationTime(afterSerialization - beforeSerialization) } val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit // directSend = sending directly back to the driver - val serializedResult = { + val serializedResult: ByteBuffer = { if (maxResultSize > 0 && resultSize > maxResultSize) { logWarning(s"Finished $taskName (TID $taskId). Result is larger than maxResultSize " + s"(${Utils.bytesToString(resultSize)} > ${Utils.bytesToString(maxResultSize)}), " + @@ -248,42 +257,40 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { - case ffe: FetchFailedException => { + case ffe: FetchFailedException => val reason = ffe.toTaskEndReason execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - } - case _: TaskKilledException | _: InterruptedException if task.killed => { + case _: TaskKilledException | _: InterruptedException if task.killed => logInfo(s"Executor killed $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - } - case cDE: CommitDeniedException => { + case cDE: CommitDeniedException => val reason = cDE.toTaskEndReason execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - } - case t: Throwable => { + case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to // the default uncaught exception handler, which will terminate the Executor. logError(s"Exception in $taskName (TID $taskId)", t) - val serviceTime = System.currentTimeMillis() - taskStart - val metrics = attemptedTask.flatMap(t => t.metrics) - for (m <- metrics) { - m.setExecutorRunTime(serviceTime) - m.setJvmGCTime(gcTime - startGCTime) + val metrics: Option[TaskMetrics] = Option(task).flatMap { task => + task.metrics.map { m => + m.setExecutorRunTime(System.currentTimeMillis() - taskStart) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) + m + } } - val reason = new ExceptionFailure(t, metrics) - execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + val taskEndReason = new ExceptionFailure(t, metrics) + execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(taskEndReason)) // Don't forcibly exit unless the exception was inherently fatal, to avoid // stopping other tasks unnecessarily. if (Utils.isFatalError(t)) { SparkUncaughtExceptionHandler.uncaughtException(t) } - } + } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() @@ -358,7 +365,7 @@ private[spark] class Executor( for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) // Fetch file with useCache mode, close cache for local mode. - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentFiles(name) = timestamp } @@ -370,12 +377,12 @@ private[spark] class Executor( if (currentTimeStamp < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) // Fetch file with useCache mode, close cache for local mode. - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentJars(name) = timestamp // Add it to our class loader - val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL - if (!urlClassLoader.getURLs.contains(url)) { + val url = new File(SparkFiles.getRootDirectory(), localName).toURI.toURL + if (!urlClassLoader.getURLs().contains(url)) { logInfo("Adding " + url + " to class loader") urlClassLoader.addURL(url) } @@ -384,61 +391,70 @@ private[spark] class Executor( } } - def startDriverHeartbeater() { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) - val timeout = AkkaUtils.lookupTimeout(conf) - val retryAttempts = AkkaUtils.numRetries(conf) - val retryIntervalMs = AkkaUtils.retryWaitMs(conf) - val heartbeatReceiverRef = AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + private val timeout = AkkaUtils.lookupTimeout(conf) + private val retryAttempts = AkkaUtils.numRetries(conf) + private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) + private val heartbeatReceiverRef = + AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + + /** Reports heartbeat and metrics for active tasks to the driver. */ + private def reportHeartBeat(): Unit = { + // list of (task id, metrics) to send back to the driver + val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + val curGCTime = computeTotalGcTime() + + for (taskRunner <- runningTasks.values()) { + if (taskRunner.task != null) { + taskRunner.task.metrics.foreach { metrics => + metrics.updateShuffleReadMetrics() + metrics.updateInputMetrics() + metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) + + if (isLocal) { + // JobProgressListener will hold an reference of it during + // onExecutorMetricsUpdate(), then JobProgressListener can not see + // the changes of metrics any more, so make a deep copy of it + val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) + tasksMetrics += ((taskRunner.taskId, copiedMetrics)) + } else { + // It will be copied by serialization + tasksMetrics += ((taskRunner.taskId, metrics)) + } + } + } + } - val t = new Thread() { + val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + try { + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + } catch { + case NonFatal(e) => logWarning("Issue communicating with driver in heartbeater", e) + } + } + + /** + * Starts a thread to report heartbeat and partial metrics for active tasks to driver. + * This thread stops running when the executor is stopped. + */ + private def startDriverHeartbeater(): Unit = { + val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) - while (!isStopped) { - val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() - val curGCTime = gcTime - - for (taskRunner <- runningTasks.values()) { - if (taskRunner.attemptedTask.nonEmpty) { - Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => - metrics.updateShuffleReadMetrics() - metrics.updateInputMetrics() - metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - - if (isLocal) { - // JobProgressListener will hold an reference of it during - // onExecutorMetricsUpdate(), then JobProgressListener can not see - // the changes of metrics any more, so make a deep copy of it - val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) - tasksMetrics += ((taskRunner.taskId, copiedMetrics)) - } else { - // It will be copied by serialization - tasksMetrics += ((taskRunner.taskId, metrics)) - } - } - } - } - - val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) - try { - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) - if (response.reregisterBlockManager) { - logWarning("Told to re-register on heartbeat") - env.blockManager.reregister() - } - } catch { - case NonFatal(t) => logWarning("Issue communicating with driver in heartbeater", t) - } - + reportHeartBeat() Thread.sleep(interval) } } } - t.setDaemon(true) - t.setName("Driver Heartbeater") - t.start() + thread.setDaemon(true) + thread.setName("driver-heartbeater") + thread.start() } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index c4d73622c4727..293c512f8b70c 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import java.util.concurrent.ThreadPoolExecutor + import scala.collection.JavaConversions._ import com.codahale.metrics.{Gauge, MetricRegistry} @@ -24,9 +26,11 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] +class ExecutorSource(threadPool: ThreadPoolExecutor, executorId: String) extends Source { + private def fileStats(scheme: String) : Option[FileSystem.Statistics] = - FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption + FileSystem.getAllStatistics().find(s => s.getScheme.equals(scheme)) private def registerFileSystemStat[T]( scheme: String, name: String, f: FileSystem.Statistics => T, defaultValue: T) = { @@ -41,23 +45,23 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() + override def getValue: Int = threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "completeTasks"), new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() + override def getValue: Long = threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "currentPool_size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() + override def getValue: Int = threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously // been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "maxPool_size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() + override def getValue: Int = threadPool.getMaximumPoolSize() }) // Gauge for file system stats of this executor diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 847a4912eec13..4d9f940813b8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { /** - * Called by Executor to run this task. + * Called by [[Executor]] to run this task. * * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. * @param attemptNumber how many times this task has been attempted (0 for the first attempt) From 116c553fd6f6d2adcbbf000cd80b5c46d4516e87 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Fri, 20 Mar 2015 12:24:34 +0000 Subject: [PATCH 461/817] [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR - Made TaskState.isFailed for handling TASK_LOST and TASK_ERROR and synchronizing CoarseMesosSchedulerBackend and MesosSchedulerBackend - This is related #5000 Author: Jongyoul Lee Closes #5088 from jongyoul/SPARK-6286-1 and squashes the following commits: 4f2362f [Jongyoul Lee] [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR - Fixed scalastyle ac4336a [Jongyoul Lee] [SPARK-6286][Mesos][minor] Handle missing Mesos case TASK_ERROR - Made TaskState.isFailed for handling TASK_LOST and TASK_ERROR and synchronizing CoarseMesosSchedulerBackend and MesosSchedulerBackend --- core/src/main/scala/org/apache/spark/TaskState.scala | 2 ++ .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 3 ++- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index d85a6d683427d..c415fe99b105e 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -27,6 +27,8 @@ private[spark] object TaskState extends Enumeration { type TaskState = Value + def isFailed(state: TaskState) = (LOST == state) || (FAILED == state) + def isFinished(state: TaskState) = FINISHED_STATES.contains(state) def toMesos(state: TaskState): MesosTaskState = state match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index fc92b9c35c3a3..e13de0f46ef89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -277,7 +277,7 @@ private[spark] class CoarseMesosSchedulerBackend( coresByTaskId -= taskId } // If it was a failure, mark the slave as failed for blacklisting purposes - if (state == MesosTaskState.TASK_FAILED || state == MesosTaskState.TASK_LOST) { + if (TaskState.isFailed(TaskState.fromMesos(state))) { failuresBySlaveId(slaveId) = failuresBySlaveId.getOrElse(slaveId, 0) + 1 if (failuresBySlaveId(slaveId) >= MAX_SLAVE_FAILURES) { logInfo("Blacklisting Mesos slave " + slaveId + " due to too many failures; " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index df8f4306b88a8..06bb527522141 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -318,7 +318,8 @@ private[spark] class MesosSchedulerBackend( val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) synchronized { - if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { + if (TaskState.isFailed(TaskState.fromMesos(status.getState)) + && taskIdToSlaveId.contains(tid)) { // We lost the executor on this slave, so remember that it's gone removeExecutor(taskIdToSlaveId(tid), "Lost executor") } From d08e3eb3dc455970b685a7b8b7e00c537c89a8e4 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Mar 2015 14:14:53 +0000 Subject: [PATCH 462/817] SPARK-5134 [BUILD] Bump default Hadoop version to 2+ Bump default Hadoop version to 2.2.0. (This is already the dependency version reported by published Maven artifacts.) See JIRA for further discussion. Author: Sean Owen Closes #5027 from srowen/SPARK-5134 and squashes the following commits: acbee14 [Sean Owen] Bump default Hadoop version to 2.2.0. (This is already the dependency version reported by published Maven artifacts.) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6fc56a86d44ac..efb9f172f4751 100644 --- a/pom.xml +++ b/pom.xml @@ -120,7 +120,7 @@ shaded-protobuf 1.7.10 1.2.17 - 1.0.4 + 2.2.0 2.4.1 ${hadoop.version} 0.98.7-hadoop1 From 6f80c3e8880340597f161f87e64697bec86cc586 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Mar 2015 14:16:21 +0000 Subject: [PATCH 463/817] SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify Author: Sean Owen Closes #5029 from srowen/SPARK-6338 and squashes the following commits: 27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir 4a212fa [Sean Owen] Standardize a bit more temp dir management 9004081 [Sean Owen] Revert some added recursive-delete calls 57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify --- .../spark/deploy/FaultToleranceTest.scala | 4 ++-- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../org/apache/spark/CheckpointSuite.scala | 3 +-- .../apache/spark/SecurityManagerSuite.scala | 5 +++-- .../org/apache/spark/SparkContextSuite.scala | 11 +++++----- .../spark/deploy/SparkSubmitSuite.scala | 8 ++++--- .../org/apache/spark/rdd/PipedRDDSuite.scala | 6 +++-- .../storage/BlockObjectWriterSuite.scala | 16 +++++++------- .../apache/spark/util/FileAppenderSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 7 +++--- .../kafka/ReliableKafkaStreamSuite.scala | 10 +++------ .../org/apache/spark/graphx/GraphSuite.scala | 6 ++--- .../org/apache/spark/repl/ReplSuite.scala | 5 +---- .../expressions/codegen/package.scala | 4 ++-- .../spark/sql/catalyst/util/package.scala | 15 ++----------- .../spark/sql/parquet/ParquetTest.scala | 6 ++--- .../spark/sql/UserDefinedTypeSuite.scala | 6 +++-- .../org/apache/spark/sql/json/JsonSuite.scala | 22 +++++++++++-------- .../sources/CreateTableAsSelectSuite.scala | 5 ++--- .../spark/sql/sources/InsertSuite.scala | 5 ++--- .../spark/sql/sources/SaveLoadSuite.scala | 6 ++--- .../sql/hive/thriftserver/CliSuite.scala | 8 ++++--- .../HiveThriftServer2Suites.scala | 7 +++--- .../apache/spark/sql/hive/test/TestHive.scala | 16 +++++--------- .../sql/hive/InsertIntoHiveTableSuite.scala | 5 ++--- .../sql/hive/MetastoreDataSourcesSuite.scala | 14 ++++++------ .../apache/spark/sql/hive/parquetSuites.scala | 22 +++++-------------- .../spark/streaming/CheckpointSuite.scala | 6 ++--- .../apache/spark/streaming/FailureSuite.scala | 10 ++++----- .../streaming/ReceivedBlockHandlerSuite.scala | 11 +++------- .../streaming/ReceivedBlockTrackerSuite.scala | 9 ++------ .../spark/streaming/ReceiverSuite.scala | 5 ++--- .../yarn/YarnSparkHadoopUtilSuite.scala | 2 +- 33 files changed, 116 insertions(+), 153 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 4e58aa0ed4c7e..5668b53fc6f4f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -33,6 +33,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} +import org.apache.spark.util.Utils /** * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. @@ -405,8 +406,7 @@ private object SparkDocker { private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = { val ipPromise = promise[String]() - val outFile = File.createTempFile("fault-tolerance-test", "") - outFile.deleteOnExit() + val outFile = File.createTempFile("fault-tolerance-test", "", Utils.createTempDir()) val outStream: FileWriter = new FileWriter(outFile) def findIpAndLog(line: String): Unit = { if (line.startsWith("CONTAINER_IP=")) { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 91aa70870ab20..fa56bb09e4e5c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -288,7 +288,7 @@ private[spark] object Utils extends Logging { } catch { case e: SecurityException => dir = null; } } - dir + dir.getCanonicalFile } /** diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 3b10b3a042317..32abc65385267 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -33,8 +33,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def beforeEach() { super.beforeEach() - checkpointDir = File.createTempFile("temp", "") - checkpointDir.deleteOnExit() + checkpointDir = File.createTempFile("temp", "", Utils.createTempDir()) checkpointDir.delete() sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 43fbd3ff3f756..62cb7649c0284 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -21,6 +21,8 @@ import java.io.File import org.scalatest.FunSuite +import org.apache.spark.util.Utils + class SecurityManagerSuite extends FunSuite { test("set security with conf") { @@ -160,8 +162,7 @@ class SecurityManagerSuite extends FunSuite { } test("ssl off setup") { - val file = File.createTempFile("SSLOptionsSuite", "conf") - file.deleteOnExit() + val file = File.createTempFile("SSLOptionsSuite", "conf", Utils.createTempDir()) System.setProperty("spark.ssl.configFile", file.getAbsolutePath) val conf = new SparkConf() diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index b8e3e83b5a47b..b07c4d93db4e6 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -79,13 +79,14 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { val byteArray2 = converter.convert(bytesWritable) assert(byteArray2.length === 0) } - + test("addFile works") { - val file1 = File.createTempFile("someprefix1", "somesuffix1") + val dir = Utils.createTempDir() + + val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) val absolutePath1 = file1.getAbsolutePath - val pluto = Utils.createTempDir() - val file2 = File.createTempFile("someprefix2", "somesuffix2", pluto) + val file2 = File.createTempFile("someprefix2", "somesuffix2", dir) val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName val absolutePath2 = file2.getAbsolutePath @@ -129,7 +130,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } - + test("addFile recursive works") { val pluto = Utils.createTempDir() val neptune = Utils.createTempDir(pluto.getAbsolutePath) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 46d745c4ecbfa..4561e5b8e9663 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -402,8 +402,10 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val archives = "file:/archive1,archive2" // spark.yarn.dist.archives val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles + val tmpDir = Utils.createTempDir() + // Test jars and files - val f1 = File.createTempFile("test-submit-jars-files", "") + val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) val writer1 = new PrintWriter(f1) writer1.println("spark.jars " + jars) writer1.println("spark.files " + files) @@ -420,7 +422,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties sysProps("spark.files") should be(Utils.resolveURIs(files)) // Test files and archives (Yarn) - val f2 = File.createTempFile("test-submit-files-archives", "") + val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) val writer2 = new PrintWriter(f2) writer2.println("spark.yarn.dist.files " + files) writer2.println("spark.yarn.dist.archives " + archives) @@ -437,7 +439,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) // Test python files - val f3 = File.createTempFile("test-submit-python-files", "") + val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) val writer3 = new PrintWriter(f3) writer3.println("spark.submit.pyFiles " + pyFiles) writer3.close() diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 1a9a0e857e546..aea76c1adcc09 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -22,7 +22,6 @@ import java.io.File import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} -import org.apache.spark._ import org.scalatest.FunSuite import scala.collection.Map @@ -30,6 +29,9 @@ import scala.language.postfixOps import scala.sys.process._ import scala.util.Try +import org.apache.spark._ +import org.apache.spark.util.Utils + class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { @@ -141,7 +143,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { // make sure symlinks were created assert(pipedLs.length > 0) // clean up top level tasks directory - new File("tasks").delete() + Utils.deleteRecursively(new File("tasks")) } else { assert(true) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala index c21c92b63ad13..78bbc4ec2c620 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -16,16 +16,18 @@ */ package org.apache.spark.storage -import org.scalatest.FunSuite import java.io.File + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.SparkConf +import org.apache.spark.util.Utils class BlockObjectWriterSuite extends FunSuite { test("verify write metrics") { - val file = new File("somefile") - file.deleteOnExit() + val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) @@ -47,8 +49,7 @@ class BlockObjectWriterSuite extends FunSuite { } test("verify write metrics on revert") { - val file = new File("somefile") - file.deleteOnExit() + val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) @@ -71,8 +72,7 @@ class BlockObjectWriterSuite extends FunSuite { } test("Reopening a closed block writer") { - val file = new File("somefile") - file.deleteOnExit() + val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 4dc5b6103db74..43b6a405cb68c 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.logging.{RollingFileAppender, SizeBasedRollingPolic class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { - val testFile = new File("FileAppenderSuite-test-" + System.currentTimeMillis).getAbsoluteFile + val testFile = new File(Utils.createTempDir(), "FileAppenderSuite-test").getAbsoluteFile before { cleanup() diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index b91428efadfd0..5d93086082189 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -122,7 +122,6 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("reading offset bytes of a file") { val tmpDir2 = Utils.createTempDir() - tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(UTF_8)) @@ -151,7 +150,6 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("reading offset bytes across multiple files") { val tmpDir = Utils.createTempDir() - tmpDir.deleteOnExit() val files = (1 to 3).map(i => new File(tmpDir, i.toString)) Files.write("0123456789", files(0), UTF_8) Files.write("abcdefghij", files(1), UTF_8) @@ -357,7 +355,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { } test("loading properties from file") { - val outFile = File.createTempFile("test-load-spark-properties", "test") + val tmpDir = Utils.createTempDir() + val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) try { System.setProperty("spark.test.fileNameLoadB", "2") Files.write("spark.test.fileNameLoadA true\n" + @@ -370,7 +369,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assert(sparkConf.getBoolean("spark.test.fileNameLoadA", false) === true) assert(sparkConf.getInt("spark.test.fileNameLoadB", 1) === 2) } finally { - outFile.delete() + Utils.deleteRecursively(tmpDir) } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index fc53c23abda85..3cd960d1fd1d4 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -25,16 +25,15 @@ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import com.google.common.io.Files import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} -import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.util.Utils class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { @@ -60,7 +59,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter ) ssc = new StreamingContext(sparkConf, Milliseconds(500)) - tempDirectory = Files.createTempDir() + tempDirectory = Utils.createTempDir() ssc.checkpoint(tempDirectory.getAbsolutePath) } @@ -68,10 +67,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter if (ssc != null) { ssc.stop() } - if (tempDirectory != null && tempDirectory.exists()) { - FileUtils.deleteDirectory(tempDirectory) - tempDirectory = null - } + Utils.deleteRecursively(tempDirectory) tearDownKafka() } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index b61d9f0fbe5e4..8d15150458d26 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,13 +19,12 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import com.google.common.io.Files - import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class GraphSuite extends FunSuite with LocalSparkContext { @@ -369,8 +368,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } test("checkpoint") { - val checkpointDir = Files.createTempDir() - checkpointDir.deleteOnExit() + val checkpointDir = Utils.createTempDir() withSpark { sc => sc.setCheckpointDir(checkpointDir.getAbsolutePath) val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)} diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index fbef5b25ba688..14f5e9ed4f25e 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -21,11 +21,9 @@ import java.io._ import java.net.URLClassLoader import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await import scala.concurrent.duration._ import scala.tools.nsc.interpreter.SparkILoop -import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.SparkContext @@ -196,8 +194,7 @@ class ReplSuite extends FunSuite { } test("interacting with files") { - val tempDir = Files.createTempDir() - tempDir.deleteOnExit() + val tempDir = Utils.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala index 80c7dfd376c96..528e38a50a740 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.catalyst.util +import org.apache.spark.util.Utils /** * A collection of generators that build custom bytecode at runtime for performing the evaluation @@ -52,7 +52,7 @@ package object codegen { @DeveloperApi object DumpByteCode { import scala.sys.process._ - val dumpDirectory = util.getTempFilePath("sparkSqlByteCode") + val dumpDirectory = Utils.createTempDir() dumpDirectory.mkdir() def apply(obj: Any): Unit = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index d8da45ae70c4b..feed50f9a2a2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -19,20 +19,9 @@ package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} -import org.apache.spark.util.{Utils => SparkUtils} +import org.apache.spark.util.Utils package object util { - /** - * Returns a path to a temporary file that probably does not exist. - * Note, there is always the race condition that someone created this - * file since the last time we checked. Thus, this shouldn't be used - * for anything security conscious. - */ - def getTempFilePath(prefix: String, suffix: String = ""): File = { - val tempFile = File.createTempFile(prefix, suffix) - tempFile.delete() - tempFile - } def fileToString(file: File, encoding: String = "UTF-8") = { val inStream = new FileInputStream(file) @@ -56,7 +45,7 @@ package object util { def resourceToString( resource:String, encoding: String = "UTF-8", - classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = { + classLoader: ClassLoader = Utils.getSparkClassLoader) = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index d6ea6679c5966..9d17516e0ef7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -23,7 +23,6 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Try -import org.apache.spark.sql.catalyst.util import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} import org.apache.spark.util.Utils @@ -67,8 +66,9 @@ private[sql] trait ParquetTest { * @todo Probably this method should be moved to a more general place */ protected def withTempPath(f: File => Unit): Unit = { - val file = util.getTempFilePath("parquetTest").getCanonicalFile - try f(file) finally if (file.exists()) Utils.deleteRecursively(file) + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 23f424c0bfc7c..fe618e0e8e767 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.io.File +import org.apache.spark.util.Utils + import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD @@ -98,13 +100,13 @@ class UserDefinedTypeSuite extends QueryTest { test("UDTs with Parquet") { - val tempDir = File.createTempFile("parquet", "test") + val tempDir = Utils.createTempDir() tempDir.delete() pointsRDD.saveAsParquetFile(tempDir.getCanonicalPath) } test("Repartition UDTs with Parquet") { - val tempDir = File.createTempFile("parquet", "test") + val tempDir = Utils.createTempDir() tempDir.delete() pointsRDD.repartition(1).saveAsParquetFile(tempDir.getCanonicalPath) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 320b80d80e997..706c966ee05f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -22,7 +22,6 @@ import java.sql.{Date, Timestamp} import org.scalactic.Tolerance._ import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} import org.apache.spark.sql.sources.LogicalRelation @@ -31,6 +30,7 @@ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, Row, SQLConf} +import org.apache.spark.util.Utils class JsonSuite extends QueryTest { import org.apache.spark.sql.json.TestJsonData._ @@ -554,8 +554,9 @@ class JsonSuite extends QueryTest { } test("jsonFile should be based on JSONRelation") { - val file = getTempFilePath("json") - val path = file.toString + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath sparkContext.parallelize(1 to 100).map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path) val jsonDF = jsonFile(path, 0.49) @@ -580,8 +581,9 @@ class JsonSuite extends QueryTest { } test("Loading a JSON dataset from a text file") { - val file = getTempFilePath("json") - val path = file.toString + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) val jsonDF = jsonFile(path) @@ -611,8 +613,9 @@ class JsonSuite extends QueryTest { } test("Loading a JSON dataset from a text file with SQL") { - val file = getTempFilePath("json") - val path = file.toString + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) sql( @@ -637,8 +640,9 @@ class JsonSuite extends QueryTest { } test("Applying schemas") { - val file = getTempFilePath("json") - val path = file.toString + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) val schema = StructType( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 60355414a40fa..2975a7fee4c96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -22,7 +22,6 @@ import java.io.File import org.apache.spark.sql.AnalysisException import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.catalyst.util import org.apache.spark.util.Utils class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { @@ -32,7 +31,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { var path: File = null override def beforeAll(): Unit = { - path = util.getTempFilePath("jsonCTAS").getCanonicalFile + path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) jsonRDD(rdd).registerTempTable("jt") } @@ -42,7 +41,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { } after { - if (path.exists()) Utils.deleteRecursively(path) + Utils.deleteRecursively(path) } test("CREATE TEMPORARY TABLE AS SELECT") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index b5b16f9546691..80efe9728fbc2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -22,7 +22,6 @@ import java.io.File import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.{AnalysisException, Row} -import org.apache.spark.sql.catalyst.util import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with BeforeAndAfterAll { @@ -32,7 +31,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll { var path: File = null override def beforeAll: Unit = { - path = util.getTempFilePath("jsonCTAS").getCanonicalFile + path = Utils.createTempDir() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) jsonRDD(rdd).registerTempTable("jt") sql( @@ -48,7 +47,7 @@ class InsertSuite extends DataSourceTest with BeforeAndAfterAll { override def afterAll: Unit = { dropTempTable("jsonTable") dropTempTable("jt") - if (path.exists()) Utils.deleteRecursively(path) + Utils.deleteRecursively(path) } test("Simple INSERT OVERWRITE a JSONRelation") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 607488ccfdd6a..43bc8eb2d11a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -21,7 +21,6 @@ import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.catalyst.util import org.apache.spark.sql.{SaveMode, SQLConf, DataFrame} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -39,7 +38,8 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { override def beforeAll(): Unit = { originalDefaultSource = conf.defaultDataSourceName - path = util.getTempFilePath("datasource").getCanonicalFile + path = Utils.createTempDir() + path.delete() val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) df = jsonRDD(rdd) @@ -52,7 +52,7 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { after { conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) - if (path.exists()) Utils.deleteRecursively(path) + Utils.deleteRecursively(path) } def checkLoad(): Unit = { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 8bca4b33b3ad1..75738fa22b572 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.util.Utils class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { def runCliWithin( @@ -38,8 +38,10 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { queriesAndExpectedAnswers: (String, String)*) { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip - val warehousePath = getTempFilePath("warehouse") - val metastorePath = getTempFilePath("metastore") + val warehousePath = Utils.createTempDir() + warehousePath.delete() + val metastorePath = Utils.createTempDir() + metastorePath.delete() val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val command = { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index aff96e21a5373..bf20acecb1f32 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -37,7 +37,6 @@ import org.apache.thrift.transport.TSocket import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.util import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils @@ -447,8 +446,10 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit } private def startThriftServer(port: Int, attempt: Int) = { - warehousePath = util.getTempFilePath("warehouse") - metastorePath = util.getTempFilePath("metastore") + warehousePath = Utils.createTempDir() + warehousePath.delete() + metastorePath = Utils.createTempDir() + metastorePath.delete() logPath = null logTailingProcess = null diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 4859991e2351a..b4aee78046383 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -30,7 +30,6 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.execution.HiveNativeCommand @@ -69,22 +68,19 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveconf.set("hive.plan.serialization.format", "javaXML") - lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath - lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + lazy val warehousePath = Utils.createTempDir() + lazy val metastorePath = Utils.createTempDir() /** Sets up the system initially or after a RESET command */ protected def configure(): Unit = { + warehousePath.delete() + metastorePath.delete() setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") - setConf("hive.metastore.warehouse.dir", warehousePath) - Utils.registerShutdownDeleteDir(new File(warehousePath)) - Utils.registerShutdownDeleteDir(new File(metastorePath)) + setConf("hive.metastore.warehouse.dir", warehousePath.toString) } - val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp") - testTempDir.delete() - testTempDir.mkdir() - Utils.registerShutdownDeleteDir(testTempDir) + val testTempDir = Utils.createTempDir() // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index d4b175fa443a4..381cd2a29123e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -21,12 +21,11 @@ import java.io.File import org.scalatest.BeforeAndAfter -import com.google.common.io.Files - import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ @@ -112,7 +111,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { test("SPARK-4203:random partition directory order") { sql("CREATE TABLE tmp_table (key int, value string)") - val tmpDir = Files.createTempDir() + val tmpDir = Utils.createTempDir() sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 5d6a6f3b64f03..ff2e6ea9ea51d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql.hive import java.io.File +import scala.collection.mutable.ArrayBuffer + import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.InvalidInputException -import org.apache.spark.sql.catalyst.util import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ @@ -34,8 +35,6 @@ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.ParquetRelation2 import org.apache.spark.sql.sources.LogicalRelation -import scala.collection.mutable.ArrayBuffer - /** * Tests for persisting tables created though the data sources API into the metastore. */ @@ -43,11 +42,12 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { override def afterEach(): Unit = { reset() - if (tempPath.exists()) Utils.deleteRecursively(tempPath) + Utils.deleteRecursively(tempPath) } val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile - var tempPath: File = util.getTempFilePath("jsonCTAS").getCanonicalFile + var tempPath: File = Utils.createTempDir() + tempPath.delete() test ("persistent JSON table") { sql( @@ -154,7 +154,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { } test("check change without refresh") { - val tempDir = File.createTempFile("sparksql", "json") + val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir()) tempDir.delete() sparkContext.parallelize(("a", "b") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) @@ -192,7 +192,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { } test("drop, change, recreate") { - val tempDir = File.createTempFile("sparksql", "json") + val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir()) tempDir.delete() sparkContext.parallelize(("a", "b") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 1904f5faef3a0..d891c4e8903d9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.SaveMode import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils // The data where the partitioning key exists only in the directory structure. case class ParquetData(intField: Int, stringField: String) @@ -579,13 +580,8 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll var partitionedTableDirWithKeyAndComplexTypes: File = null override def beforeAll(): Unit = { - partitionedTableDir = File.createTempFile("parquettests", "sparksql") - partitionedTableDir.delete() - partitionedTableDir.mkdir() - - normalTableDir = File.createTempFile("parquettests", "sparksql") - normalTableDir.delete() - normalTableDir.mkdir() + partitionedTableDir = Utils.createTempDir() + normalTableDir = Utils.createTempDir() (1 to 10).foreach { p => val partDir = new File(partitionedTableDir, s"p=$p") @@ -601,9 +597,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll .toDF() .saveAsParquetFile(new File(normalTableDir, "normal").getCanonicalPath) - partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") - partitionedTableDirWithKey.delete() - partitionedTableDirWithKey.mkdir() + partitionedTableDirWithKey = Utils.createTempDir() (1 to 10).foreach { p => val partDir = new File(partitionedTableDirWithKey, s"p=$p") @@ -613,9 +607,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll .saveAsParquetFile(partDir.getCanonicalPath) } - partitionedTableDirWithKeyAndComplexTypes = File.createTempFile("parquettests", "sparksql") - partitionedTableDirWithKeyAndComplexTypes.delete() - partitionedTableDirWithKeyAndComplexTypes.mkdir() + partitionedTableDirWithKeyAndComplexTypes = Utils.createTempDir() (1 to 10).foreach { p => val partDir = new File(partitionedTableDirWithKeyAndComplexTypes, s"p=$p") @@ -625,9 +617,7 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll }.toDF().saveAsParquetFile(partDir.getCanonicalPath) } - partitionedTableDirWithComplexTypes = File.createTempFile("parquettests", "sparksql") - partitionedTableDirWithComplexTypes.delete() - partitionedTableDirWithComplexTypes.mkdir() + partitionedTableDirWithComplexTypes = Utils.createTempDir() (1 to 10).foreach { p => val partDir = new File(partitionedTableDirWithComplexTypes, s"p=$p") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 8ea91eca683cf..91a2b2bba461d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -222,7 +222,7 @@ class CheckpointSuite extends TestSuiteBase { } test("recovery with saveAsHadoopFiles operation") { - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() try { testCheckpointedOperation( Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), @@ -245,7 +245,7 @@ class CheckpointSuite extends TestSuiteBase { } test("recovery with saveAsNewAPIHadoopFiles operation") { - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() try { testCheckpointedOperation( Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), @@ -283,7 +283,7 @@ class CheckpointSuite extends TestSuiteBase { // // After SPARK-5079 is addressed, should be able to remove this test since a strengthened // version of the other saveAsHadoopFile* tests would prevent regressions for this issue. - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() try { testCheckpointedOperation( Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 6500608bba87c..26435d8515815 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -20,15 +20,13 @@ package org.apache.spark.streaming import org.apache.spark.Logging import org.apache.spark.util.Utils -import java.io.File - /** * This testsuite tests master failures at random times while the stream is running using * the real clock. */ class FailureSuite extends TestSuiteBase with Logging { - val directory = Utils.createTempDir().getAbsolutePath + val directory = Utils.createTempDir() val numBatches = 30 override def batchDuration = Milliseconds(1000) @@ -36,16 +34,16 @@ class FailureSuite extends TestSuiteBase with Logging { override def useManualClock = false override def afterFunction() { - Utils.deleteRecursively(new File(directory)) + Utils.deleteRecursively(directory) super.afterFunction() } test("multiple failures with map") { - MasterFailureTest.testMap(directory, numBatches, batchDuration) + MasterFailureTest.testMap(directory.getAbsolutePath, numBatches, batchDuration) } test("multiple failures with updateStateByKey") { - MasterFailureTest.testUpdateStateByKey(directory, numBatches, batchDuration) + MasterFailureTest.testUpdateStateByKey(directory.getAbsolutePath, numBatches, batchDuration) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 818f551dbe996..18a477f92094d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -25,8 +25,6 @@ import scala.concurrent.duration._ import scala.language.postfixOps import akka.actor.{ActorSystem, Props} -import com.google.common.io.Files -import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ @@ -39,7 +37,7 @@ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util._ -import org.apache.spark.util.{AkkaUtils, ManualClock} +import org.apache.spark.util.{AkkaUtils, ManualClock, Utils} import WriteAheadLogBasedBlockHandler._ import WriteAheadLogSuite._ @@ -76,7 +74,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche new NioBlockTransferService(conf, securityMgr), securityMgr, 0) blockManager.initialize("app-id") - tempDirectory = Files.createTempDir() + tempDirectory = Utils.createTempDir() manualClock.setTime(0) } @@ -93,10 +91,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche actorSystem.awaitTermination() actorSystem = null - if (tempDirectory != null && tempDirectory.exists()) { - FileUtils.deleteDirectory(tempDirectory) - tempDirectory = null - } + Utils.deleteRecursively(tempDirectory) } test("BlockManagerBasedBlockHandler - store blocks") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index a3a0fd5187403..42fad769f0c1a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -24,8 +24,6 @@ import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} import scala.util.Random -import com.google.common.io.Files -import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ @@ -51,15 +49,12 @@ class ReceivedBlockTrackerSuite before { conf = new SparkConf().setMaster("local[2]").setAppName("ReceivedBlockTrackerSuite") - checkpointDirectory = Files.createTempDir() + checkpointDirectory = Utils.createTempDir() } after { allReceivedBlockTrackers.foreach { _.stop() } - if (checkpointDirectory != null && checkpointDirectory.exists()) { - FileUtils.deleteDirectory(checkpointDirectory) - checkpointDirectory = null - } + Utils.deleteRecursively(checkpointDirectory) } test("block addition, and block to batch allocation") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index e8c34a9ee40b9..aa20ad0b5374e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -24,7 +24,6 @@ import java.util.concurrent.Semaphore import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import com.google.common.io.Files import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -34,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler._ +import org.apache.spark.util.Utils /** Testsuite for testing the network receiver behavior */ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { @@ -222,7 +222,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { .set("spark.streaming.receiver.writeAheadLog.enable", "true") .set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") val batchDuration = Milliseconds(500) - val tempDirectory = Files.createTempDir() + val tempDirectory = Utils.createTempDir() val logDirectory1 = new File(checkpointDirToLogDir(tempDirectory.getAbsolutePath, 0)) val logDirectory2 = new File(checkpointDirToLogDir(tempDirectory.getAbsolutePath, 1)) val allLogFiles1 = new mutable.HashSet[String]() @@ -251,7 +251,6 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { } withStreamingContext(new StreamingContext(sparkConf, batchDuration)) { ssc => - tempDirectory.deleteOnExit() val receiver1 = ssc.sparkContext.clean(new FakeReceiver(sendData = true)) val receiver2 = ssc.sparkContext.clean(new FakeReceiver(sendData = true)) val receiverStream1 = ssc.receiverStream(receiver1) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index b5a2db8f6225c..4194f36499e66 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -50,7 +50,7 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { if (hasBash) test(name)(fn) else ignore(name)(fn) bashTest("shell script escaping") { - val scriptFile = File.createTempFile("script.", ".sh") + val scriptFile = File.createTempFile("script.", ".sh", Utils.createTempDir()) val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6") try { val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ") From db4d317ccfdd9bd1dc7e8beac54ebcc35966b7d5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Mar 2015 14:13:02 -0400 Subject: [PATCH 464/817] [SPARK-6428][MLlib] Added explicit type for public methods and implemented hashCode when equals is defined. I want to add a checker to turn public type checking on, since future pull requests can accidentally expose a non-public type. This is the first cleanup task. Author: Reynold Xin Closes #5102 from rxin/mllib-hashcode-publicmethodtypes and squashes the following commits: 617f19e [Reynold Xin] Fixed Scala compilation error. 52bc2d5 [Reynold Xin] [MLlib] Added explicit type for public methods and implemented hashCode when equals is defined. --- .../spark/examples/mllib/MovieLensALS.scala | 3 +- .../PowerIterationClusteringExample.scala | 4 +-- .../apache/spark/ml/feature/HashingTF.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 18 ++++++---- .../mllib/classification/NaiveBayes.scala | 6 ++-- .../impl/GLMClassificationModel.scala | 2 +- .../spark/mllib/clustering/KMeans.scala | 2 +- .../mllib/evaluation/MultilabelMetrics.scala | 18 +++++----- .../apache/spark/mllib/linalg/Matrices.scala | 12 +++++-- .../apache/spark/mllib/linalg/Vectors.scala | 4 ++- .../linalg/distributed/BlockMatrix.scala | 10 +++++- .../mllib/random/RandomDataGenerator.scala | 4 +-- .../regression/impl/GLMRegressionModel.scala | 2 +- .../mllib/tree/configuration/Strategy.scala | 9 +++-- .../spark/mllib/tree/impurity/Entropy.scala | 2 +- .../spark/mllib/tree/impurity/Gini.scala | 2 +- .../spark/mllib/tree/impurity/Variance.scala | 2 +- .../mllib/tree/model/DecisionTreeModel.scala | 4 +-- .../tree/model/InformationGainStats.scala | 35 ++++++++++++------- .../apache/spark/mllib/tree/model/Node.scala | 6 ++-- .../spark/mllib/tree/model/Predict.scala | 6 +++- .../apache/spark/mllib/tree/model/Split.scala | 3 +- .../mllib/tree/model/treeEnsembleModels.scala | 6 ++-- 23 files changed, 101 insertions(+), 61 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 91a0a860d6c71..1f4ca4fbe7778 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -175,7 +175,8 @@ object MovieLensALS { } /** Compute RMSE (Root Mean Squared Error). */ - def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], implicitPrefs: Boolean) = { + def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], implicitPrefs: Boolean) + : Double = { def mapPredictedRating(r: Double) = if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 91c9772744f18..9f22d40c15f3f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -116,7 +116,7 @@ object PowerIterationClusteringExample { sc.stop() } - def generateCircle(radius: Double, n: Int) = { + def generateCircle(radius: Double, n: Int): Seq[(Double, Double)] = { Seq.tabulate(n) { i => val theta = 2.0 * math.Pi * i / n (radius * math.cos(theta), radius * math.sin(theta)) @@ -147,7 +147,7 @@ object PowerIterationClusteringExample { /** * Gaussian Similarity: http://en.wikipedia.org/wiki/Radial_basis_function_kernel */ - def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double), sigma: Double) = { + def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double), sigma: Double): Double = { val coeff = 1.0 / (math.sqrt(2.0 * math.Pi) * sigma) val expCoeff = -1.0 / 2.0 * math.pow(sigma, 2.0) val ssquares = (p1._1 - p2._1) * (p1._1 - p2._1) + (p1._2 - p2._2) * (p1._2 - p2._2) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index 6131ba8832691..fc4e12773c46d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -41,7 +41,7 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { def getNumFeatures: Int = get(numFeatures) /** @group setParam */ - def setNumFeatures(value: Int) = set(numFeatures, value) + def setNumFeatures(value: Int): this.type = set(numFeatures, value) override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index cbd87ea8aeb37..15ca2547d56a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -345,9 +345,13 @@ private[python] class PythonMLLibAPI extends Serializable { def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = predict(SerDe.asTupleRDD(userAndProducts.rdd)) - def getUserFeatures = SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + def getUserFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + } - def getProductFeatures = SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + def getProductFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + } } @@ -909,7 +913,7 @@ private[spark] object SerDe extends Serializable { // Pickler for DenseVector private[python] class DenseVectorPickler extends BasePickler[DenseVector] { - def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val vector: DenseVector = obj.asInstanceOf[DenseVector] val bytes = new Array[Byte](8 * vector.size) val bb = ByteBuffer.wrap(bytes) @@ -941,7 +945,7 @@ private[spark] object SerDe extends Serializable { // Pickler for DenseMatrix private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { - def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] val bytes = new Array[Byte](8 * m.values.size) val order = ByteOrder.nativeOrder() @@ -973,7 +977,7 @@ private[spark] object SerDe extends Serializable { // Pickler for SparseVector private[python] class SparseVectorPickler extends BasePickler[SparseVector] { - def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val v: SparseVector = obj.asInstanceOf[SparseVector] val n = v.indices.size val indiceBytes = new Array[Byte](4 * n) @@ -1015,7 +1019,7 @@ private[spark] object SerDe extends Serializable { // Pickler for LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { - def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val point: LabeledPoint = obj.asInstanceOf[LabeledPoint] saveObjects(out, pickler, point.label, point.features) } @@ -1031,7 +1035,7 @@ private[spark] object SerDe extends Serializable { // Pickler for Rating private[python] class RatingPickler extends BasePickler[Rating] { - def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val rating: Rating = obj.asInstanceOf[Rating] saveObjects(out, pickler, rating.user, rating.product, rating.rating) } 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 2ebc7fa5d4234..068449aa1d346 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 @@ -83,10 +83,10 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { private object SaveLoadV1_0 { - def thisFormatVersion = "1.0" + def thisFormatVersion: String = "1.0" /** Hard-code class name string in case it changes in the future */ - def thisClassName = "org.apache.spark.mllib.classification.NaiveBayesModel" + def thisClassName: String = "org.apache.spark.mllib.classification.NaiveBayesModel" /** Model data for model import/export */ case class Data(labels: Array[Double], pi: Array[Double], theta: Array[Array[Double]]) @@ -174,7 +174,7 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with * * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ - def run(data: RDD[LabeledPoint]) = { + def run(data: RDD[LabeledPoint]): NaiveBayesModel = { val requireNonnegativeValues: Vector => Unit = (v: Vector) => { val values = v match { case SparseVector(size, indices, values) => 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 8956189ff1158..3b6790cce47c6 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 @@ -32,7 +32,7 @@ private[classification] object GLMClassificationModel { object SaveLoadV1_0 { - def thisFormatVersion = "1.0" + def thisFormatVersion: String = "1.0" /** Model data for import/export */ case class Data(weights: Vector, intercept: Double, threshold: Option[Double]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index e41f941fd2c2c..0f8d6a399682d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -536,5 +536,5 @@ class VectorWithNorm(val vector: Vector, val norm: Double) extends Serializable def this(array: Array[Double]) = this(Vectors.dense(array)) /** Converts the vector to a dense vector. */ - def toDense = new VectorWithNorm(Vectors.dense(vector.toArray), norm) + def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm) } 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 ea10bde5fa252..a8378a76d20ae 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 @@ -96,30 +96,30 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns precision for a given label (category) * @param label the label. */ - def precision(label: Double) = { + def precision(label: Double): Double = { val tp = tpPerClass(label) val fp = fpPerClass.getOrElse(label, 0L) - if (tp + fp == 0) 0 else tp.toDouble / (tp + fp) + if (tp + fp == 0) 0.0 else tp.toDouble / (tp + fp) } /** * Returns recall for a given label (category) * @param label the label. */ - def recall(label: Double) = { + def recall(label: Double): Double = { val tp = tpPerClass(label) val fn = fnPerClass.getOrElse(label, 0L) - if (tp + fn == 0) 0 else tp.toDouble / (tp + fn) + if (tp + fn == 0) 0.0 else tp.toDouble / (tp + fn) } /** * Returns f1-measure for a given label (category) * @param label the label. */ - def f1Measure(label: Double) = { + def f1Measure(label: Double): Double = { val p = precision(label) val r = recall(label) - if((p + r) == 0) 0 else 2 * p * r / (p + r) + if((p + r) == 0) 0.0 else 2 * p * r / (p + r) } private lazy val sumTp = tpPerClass.foldLeft(0L) { case (sum, (_, tp)) => sum + tp } @@ -130,7 +130,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based precision * (equals to micro-averaged document-based precision) */ - lazy val microPrecision = { + lazy val microPrecision: Double = { val sumFp = fpPerClass.foldLeft(0L){ case(cum, (_, fp)) => cum + fp} sumTp.toDouble / (sumTp + sumFp) } @@ -139,7 +139,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based recall * (equals to micro-averaged document-based recall) */ - lazy val microRecall = { + lazy val microRecall: Double = { val sumFn = fnPerClass.foldLeft(0.0){ case(cum, (_, fn)) => cum + fn} sumTp.toDouble / (sumTp + sumFn) } @@ -148,7 +148,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based f1-measure * (equals to micro-averaged document-based f1-measure) */ - lazy val microF1Measure = 2.0 * sumTp / (2 * sumTp + sumFnClass + sumFpClass) + lazy val microF1Measure: Double = 2.0 * sumTp / (2 * sumTp + sumFnClass + sumFpClass) /** * Returns the sequence of labels in ascending order diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 0e4a4d0085895..fdd8848189f19 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -146,12 +146,16 @@ class DenseMatrix( def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) - override def equals(o: Any) = o match { + override def equals(o: Any): Boolean = o match { case m: DenseMatrix => m.numRows == numRows && m.numCols == numCols && Arrays.equals(toArray, m.toArray) case _ => false } + override def hashCode: Int = { + com.google.common.base.Objects.hashCode(numRows : Integer, numCols: Integer, toArray) + } + private[mllib] def toBreeze: BM[Double] = { if (!isTransposed) { new BDM[Double](numRows, numCols, values) @@ -173,7 +177,7 @@ class DenseMatrix( values(index(i, j)) = v } - override def copy = new DenseMatrix(numRows, numCols, values.clone()) + override def copy: DenseMatrix = new DenseMatrix(numRows, numCols, values.clone()) private[mllib] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f)) @@ -431,7 +435,9 @@ class SparseMatrix( } } - override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) + override def copy: SparseMatrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) + } private[mllib] def map(f: Double => Double) = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index e9d25dcb7e778..2cda9b252ee06 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -183,6 +183,8 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { } } + override def hashCode: Int = 7919 + private[spark] override def asNullable: VectorUDT = this } @@ -478,7 +480,7 @@ class DenseVector(val values: Array[Double]) extends Vector { private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) - override def apply(i: Int) = values(i) + override def apply(i: Int): Double = values(i) override def copy: DenseVector = { new DenseVector(values.clone()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 1d253963130f1..3323ae7b1fba0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -49,7 +49,7 @@ private[mllib] class GridPartitioner( private val rowPartitions = math.ceil(rows * 1.0 / rowsPerPart).toInt private val colPartitions = math.ceil(cols * 1.0 / colsPerPart).toInt - override val numPartitions = rowPartitions * colPartitions + override val numPartitions: Int = rowPartitions * colPartitions /** * Returns the index of the partition the input coordinate belongs to. @@ -85,6 +85,14 @@ private[mllib] class GridPartitioner( false } } + + override def hashCode: Int = { + com.google.common.base.Objects.hashCode( + rows: java.lang.Integer, + cols: java.lang.Integer, + rowsPerPart: java.lang.Integer, + colsPerPart: java.lang.Integer) + } } private[mllib] object GridPartitioner { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 405bae62ee8b6..9349ecaa13f56 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -56,7 +56,7 @@ class UniformGenerator extends RandomDataGenerator[Double] { random.nextDouble() } - override def setSeed(seed: Long) = random.setSeed(seed) + override def setSeed(seed: Long): Unit = random.setSeed(seed) override def copy(): UniformGenerator = new UniformGenerator() } @@ -75,7 +75,7 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { random.nextGaussian() } - override def setSeed(seed: Long) = random.setSeed(seed) + override def setSeed(seed: Long): Unit = random.setSeed(seed) override def copy(): StandardNormalGenerator = new StandardNormalGenerator() } 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 bd7e340ca2d8e..b55944f74f623 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 @@ -32,7 +32,7 @@ private[regression] object GLMRegressionModel { object SaveLoadV1_0 { - def thisFormatVersion = "1.0" + def thisFormatVersion: String = "1.0" /** Model data for model import/export */ case class Data(weights: Vector, intercept: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 8d5c36da32bdb..ada227c200a79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -83,10 +83,13 @@ class Strategy ( @BeanProperty var useNodeIdCache: Boolean = false, @BeanProperty var checkpointInterval: Int = 10) extends Serializable { - def isMulticlassClassification = + def isMulticlassClassification: Boolean = { algo == Classification && numClasses > 2 - def isMulticlassWithCategoricalFeatures - = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) + } + + def isMulticlassWithCategoricalFeatures: Boolean = { + isMulticlassClassification && (categoricalFeaturesInfo.size > 0) + } /** * Java-friendly constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index b7950e00786ab..5ac10f3fd32dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -71,7 +71,7 @@ object Entropy extends Impurity { * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. */ - def instance = this + def instance: this.type = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c946db9c0d1c8..19d318203c344 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -67,7 +67,7 @@ object Gini extends Impurity { * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. */ - def instance = this + def instance: this.type = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index df9eafa5da16a..7104a7fa4dd4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -58,7 +58,7 @@ object Variance extends Impurity { * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. */ - def instance = this + def instance: this.type = this } 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 8a57ebc387d01..c9bafd60fba4d 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 @@ -120,10 +120,10 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { private[tree] object SaveLoadV1_0 { - def thisFormatVersion = "1.0" + def thisFormatVersion: String = "1.0" // Hard-code class name string in case it changes in the future - def thisClassName = "org.apache.spark.mllib.tree.DecisionTreeModel" + def thisClassName: String = "org.apache.spark.mllib.tree.DecisionTreeModel" case class PredictData(predict: Double, prob: Double) { def toPredict: Predict = new Predict(predict, prob) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 80990aa9a603f..f209fdafd3653 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -38,23 +38,32 @@ class InformationGainStats( val leftPredict: Predict, val rightPredict: Predict) extends Serializable { - override def toString = { + override def toString: String = { "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" .format(gain, impurity, leftImpurity, rightImpurity) } - override def equals(o: Any) = - o match { - case other: InformationGainStats => { - gain == other.gain && - impurity == other.impurity && - leftImpurity == other.leftImpurity && - rightImpurity == other.rightImpurity && - leftPredict == other.leftPredict && - rightPredict == other.rightPredict - } - case _ => false - } + override def equals(o: Any): Boolean = o match { + case other: InformationGainStats => + gain == other.gain && + impurity == other.impurity && + leftImpurity == other.leftImpurity && + rightImpurity == other.rightImpurity && + leftPredict == other.leftPredict && + rightPredict == other.rightPredict + + case _ => false + } + + override def hashCode: Int = { + com.google.common.base.Objects.hashCode( + gain: java.lang.Double, + impurity: java.lang.Double, + leftImpurity: java.lang.Double, + rightImpurity: java.lang.Double, + leftPredict, + rightPredict) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index d961081d185e9..4f72bb8014cc0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -50,8 +50,10 @@ class Node ( var rightNode: Option[Node], var stats: Option[InformationGainStats]) extends Serializable with Logging { - override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + - "impurity = " + impurity + "split = " + split + ", stats = " + stats + override def toString: String = { + "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + + "impurity = " + impurity + "split = " + split + ", stats = " + stats + } /** * build the left node and right nodes if not leaf diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index ad4c0dbbfb3e5..25990af7c6cf7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -29,7 +29,7 @@ class Predict( val predict: Double, val prob: Double = 0.0) extends Serializable { - override def toString = { + override def toString: String = { "predict = %f, prob = %f".format(predict, prob) } @@ -39,4 +39,8 @@ class Predict( case _ => false } } + + override def hashCode: Int = { + com.google.common.base.Objects.hashCode(predict: java.lang.Double, prob: java.lang.Double) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index b7a85f58544a3..fb35e70a8d077 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -38,9 +38,10 @@ case class Split( featureType: FeatureType, categories: List[Double]) { - override def toString = + override def toString: String = { "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + ", categories = " + categories + } } /** 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 30a8f7ca301af..f160852c69c77 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 @@ -79,7 +79,7 @@ object RandomForestModel extends Loader[RandomForestModel] { private object SaveLoadV1_0 { // Hard-code class name string in case it changes in the future - def thisClassName = "org.apache.spark.mllib.tree.model.RandomForestModel" + def thisClassName: String = "org.apache.spark.mllib.tree.model.RandomForestModel" } } @@ -130,7 +130,7 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { private object SaveLoadV1_0 { // Hard-code class name string in case it changes in the future - def thisClassName = "org.apache.spark.mllib.tree.model.GradientBoostedTreesModel" + def thisClassName: String = "org.apache.spark.mllib.tree.model.GradientBoostedTreesModel" } } @@ -257,7 +257,7 @@ private[tree] object TreeEnsembleModel extends Logging { import org.apache.spark.mllib.tree.model.DecisionTreeModel.SaveLoadV1_0.{NodeData, constructTrees} - def thisFormatVersion = "1.0" + def thisFormatVersion: String = "1.0" case class Metadata( algo: String, From 28bcb9e9e86a4b643fbf96b2b7e03928ebcfc060 Mon Sep 17 00:00:00 2001 From: mbonaci Date: Fri, 20 Mar 2015 18:30:45 +0000 Subject: [PATCH 465/817] [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample The docs for the `sample` method were insufficient, now less so. Author: mbonaci Closes #5097 from mbonaci/master and squashes the following commits: a6a9d97 [mbonaci] [SPARK-6370][core] Documentation: Improve all 3 docs for RDD.sample method --- .../scala/org/apache/spark/api/java/JavaRDD.scala | 11 +++++++++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 ++++++ python/pyspark/rdd.py | 6 ++++++ 3 files changed, 23 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 645dc3bfb6b06..3e9beb670f7ad 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -101,12 +101,23 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) /** * Return a sampled subset of this RDD. + * + * @param withReplacement can elements be sampled multiple times (replaced when sampled out) + * @param fraction expected size of the sample as a fraction of this RDD's size + * without replacement: probability that each element is chosen; fraction must be [0, 1] + * with replacement: expected number of times each element is chosen; fraction must be >= 0 */ def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = sample(withReplacement, fraction, Utils.random.nextLong) /** * Return a sampled subset of this RDD. + * + * @param withReplacement can elements be sampled multiple times (replaced when sampled out) + * @param fraction expected size of the sample as a fraction of this RDD's size + * without replacement: probability that each element is chosen; fraction must be [0, 1] + * with replacement: expected number of times each element is chosen; fraction must be >= 0 + * @param seed seed for the random number generator */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a139780d967e9..a4c74ed03e330 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -377,6 +377,12 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. + * + * @param withReplacement can elements be sampled multiple times (replaced when sampled out) + * @param fraction expected size of the sample as a fraction of this RDD's size + * without replacement: probability that each element is chosen; fraction must be [0, 1] + * with replacement: expected number of times each element is chosen; fraction must be >= 0 + * @param seed seed for the random number generator */ def sample(withReplacement: Boolean, fraction: Double, diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index bf17f513c0bc3..c337a43c8a7fc 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -346,6 +346,12 @@ def sample(self, withReplacement, fraction, seed=None): """ Return a sampled subset of this RDD. + :param withReplacement: can elements be sampled multiple times (replaced when sampled out) + :param fraction: expected size of the sample as a fraction of this RDD's size + without replacement: probability that each element is chosen; fraction must be [0, 1] + with replacement: expected number of times each element is chosen; fraction must be >= 0 + :param seed: seed for the random number generator + >>> rdd = sc.parallelize(range(100), 4) >>> rdd.sample(False, 0.1, 81).count() 10 From 385b2ff10d9ef5083df49233f77c8e873561dc16 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Fri, 20 Mar 2015 18:42:18 +0000 Subject: [PATCH 466/817] [SPARK-6426][Doc]User could also point the yarn cluster config directory via YARN_CONF_DI... ...R https://issues.apache.org/jira/browse/SPARK-6426 Author: WangTaoTheTonic Closes #5103 from WangTaoTheTonic/SPARK-6426 and squashes the following commits: e6dd78d [WangTaoTheTonic] User could also point the yarn cluster config directory via YARN_CONF_DIR --- docs/submitting-applications.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 57b074778f2b0..3ecbf2308cd44 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -133,10 +133,10 @@ The master URL passed to Spark can be in one of the following formats: Or, for a Mesos cluster using ZooKeeper, use mesos://zk://.... yarn-client Connect to a YARN cluster in -client mode. The cluster location will be found based on the HADOOP_CONF_DIR variable. +client mode. The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. yarn-cluster Connect to a YARN cluster in -cluster mode. The cluster location will be found based on HADOOP_CONF_DIR. +cluster mode. The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. From a74564591f1c824f9eed516ae79e079b355fd32b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 20 Mar 2015 18:43:57 +0000 Subject: [PATCH 467/817] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT. Author: Marcelo Vanzin Closes #5056 from vanzin/SPARK-6371 and squashes the following commits: 63220df [Marcelo Vanzin] Merge branch 'master' into SPARK-6371 6506f75 [Marcelo Vanzin] Use more fine-grained exclusion. 178ba71 [Marcelo Vanzin] Oops. 75b2375 [Marcelo Vanzin] Exclude VertexRDD in MiMA. a45a62c [Marcelo Vanzin] Work around MIMA warning. 1d8a670 [Marcelo Vanzin] Re-group jetty exclusion. 0e8e909 [Marcelo Vanzin] Ignore ml, don't ignore graphx. cef4603 [Marcelo Vanzin] Indentation. 296cf82 [Marcelo Vanzin] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- core/src/main/scala/org/apache/spark/package.scala | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-assembly/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/java8-tests/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib/pom.xml | 2 +- network/common/pom.xml | 2 +- network/shuffle/pom.xml | 2 +- network/yarn/pom.xml | 2 +- pom.xml | 2 +- project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 14 ++++++++++++++ repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 33 files changed, 47 insertions(+), 33 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index d3bb4bde0c412..f1f8b0d3682e2 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 1fe61062b4606..1f3dec91314f2 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 81f8cba711df6..6cd1965ec37c2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index b6249b492150a..2ab41ba488ff6 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -43,5 +43,5 @@ package org.apache package object spark { // For package docs only - val SPARK_VERSION = "1.3.0-SNAPSHOT" + val SPARK_VERSION = "1.4.0-SNAPSHOT" } diff --git a/docs/_config.yml b/docs/_config.yml index 0652927a8ce9b..b22b627f09007 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 1.3.0-SNAPSHOT -SPARK_VERSION_SHORT: 1.3.0 +SPARK_VERSION: 1.4.0-SNAPSHOT +SPARK_VERSION_SHORT: 1.4.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.21.0 diff --git a/examples/pom.xml b/examples/pom.xml index 994071d94d0ad..7e93f0eec0b91 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 96c2787e35cd0..67907bbfb6d1b 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 172d447b77cda..8df7edbdcad33 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 5109b8ed87524..0b79f47647f6b 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 369856187a244..f695cff410a18 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index a344f000c5002..98f95a9a64fa0 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index e95853f005ce2..8b6a8959ac4cf 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 9b3475d7c3dc2..a50d378b34335 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index bc2f8be10c9ce..4351a8a12fe21 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 7e49a71907336..25847a1b33d9c 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 6eb29af03f833..e14bbae4a9b6e 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c0d534e185d7f..d38a3aa8256b7 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index ccbd9d0419a98..0fe2814135d88 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index a76704a8c2c59..4c183543e3fa8 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/network/common/pom.xml b/network/common/pom.xml index 74437f37c47e4..7b51845206f4a 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index a2bcca26d8344..7dc7c65825e34 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index cea7a20c223e2..1e2e9c80af6cc 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index efb9f172f4751..23bb16130b504 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index f0cbf4e57b8c5..dde92949fa175 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -91,7 +91,7 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "1.2.0" + val previousSparkVersion = "1.3.0" val fullId = "spark-" + projectRef.project + "_2.10" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a6b07fa7cddec..328d59485a731 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -16,6 +16,7 @@ */ import com.typesafe.tools.mima.core._ +import com.typesafe.tools.mima.core.ProblemFilters._ /** * Additional excludes for checking of Spark's binary compatibility. @@ -33,6 +34,19 @@ import com.typesafe.tools.mima.core._ object MimaExcludes { def excludes(version: String) = version match { + case v if v.startsWith("1.4") => + Seq( + MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("ml"), + // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff"), + // These are needed if checking against the sbt build, since they are part of + // the maven-generated artifacts in 1.3. + excludePackage("org.spark-project.jetty"), + MimaBuild.excludeSparkPackage("unused"), + ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional") + ) + case v if v.startsWith("1.3") => Seq( MimaBuild.excludeSparkPackage("deploy"), diff --git a/repl/pom.xml b/repl/pom.xml index 295f88ea3ecf9..edfa1c7f2c29c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 8ad026dbdf8ff..3dea2ee76542f 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3640104e497d4..e3a6b1fe72435 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index f466a3c0b5dc2..a96b1ffc26966 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 0e3f4eb98cbf7..a9816f6c38cd2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 96508d83f4049..23a8358d45c2a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 181236d1bcbf6..1c6f3e83a1819 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index c13534f0410a1..7c8c3613e7a05 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.10 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml From 48866f789712b0cdbaf76054d1014c6df032fff1 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Fri, 20 Mar 2015 14:44:21 -0400 Subject: [PATCH 468/817] [SPARK-6095] [MLLIB] Support model save/load in Python's linear models For Python's linear models, weights and intercept are stored in Python. This PR implements Python's linear models sava/load functions which do the same thing as scala. It can also make model import/export cross languages. Author: Yanbo Liang Closes #5016 from yanboliang/spark-6095 and squashes the following commits: d9bb824 [Yanbo Liang] fix python style b3813ca [Yanbo Liang] linear model save/load for Python reuse the Scala implementation --- python/pyspark/mllib/classification.py | 58 +++++++++++++++++- python/pyspark/mllib/regression.py | 84 +++++++++++++++++++++++++- python/pyspark/mllib/util.py | 6 +- 3 files changed, 145 insertions(+), 3 deletions(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index e4765173709e8..b66159c5bfb66 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -21,7 +21,7 @@ from numpy import array from pyspark import RDD -from pyspark.mllib.common import callMLlibFunc +from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -99,6 +99,18 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): 1 >>> lrm.predict(SparseVector(2, {0: 1.0})) 0 + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> lrm.save(sc, path) + >>> sameModel = LogisticRegressionModel.load(sc, path) + >>> sameModel.predict(array([0.0, 1.0])) + 1 + >>> sameModel.predict(SparseVector(2, {0: 1.0})) + 0 + >>> try: + ... os.removedirs(path) + ... except: + ... pass """ def __init__(self, weights, intercept): super(LogisticRegressionModel, self).__init__(weights, intercept) @@ -124,6 +136,22 @@ def predict(self, x): else: return 1 if prob > self._threshold else 0 + def save(self, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.classification.LogisticRegressionModel( + _py2java(sc, self._coeff), self.intercept) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.classification.LogisticRegressionModel.load( + sc._jsc.sc(), path) + weights = _java2py(sc, java_model.weights()) + intercept = java_model.intercept() + threshold = java_model.getThreshold().get() + model = LogisticRegressionModel(weights, intercept) + model.setThreshold(threshold) + return model + class LogisticRegressionWithSGD(object): @@ -243,6 +271,18 @@ class SVMModel(LinearBinaryClassificationModel): 1 >>> svm.predict(SparseVector(2, {0: -1.0})) 0 + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> svm.save(sc, path) + >>> sameModel = SVMModel.load(sc, path) + >>> sameModel.predict(SparseVector(2, {1: 1.0})) + 1 + >>> sameModel.predict(SparseVector(2, {0: -1.0})) + 0 + >>> try: + ... os.removedirs(path) + ... except: + ... pass """ def __init__(self, weights, intercept): super(SVMModel, self).__init__(weights, intercept) @@ -263,6 +303,22 @@ def predict(self, x): else: return 1 if margin > self._threshold else 0 + def save(self, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.classification.SVMModel( + _py2java(sc, self._coeff), self.intercept) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.classification.SVMModel.load( + sc._jsc.sc(), path) + weights = _java2py(sc, java_model.weights()) + intercept = java_model.intercept() + threshold = java_model.getThreshold().get() + model = SVMModel(weights, intercept) + model.setThreshold(threshold) + return model + class SVMWithSGD(object): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 0c21ad578793f..015a7860116c9 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,8 +18,9 @@ import numpy as np from numpy import array -from pyspark.mllib.common import callMLlibFunc, inherit_doc +from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.util import Saveable, Loader __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'LinearRegressionWithSGD', @@ -114,6 +115,20 @@ class LinearRegressionModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> lrm.save(sc, path) + >>> sameModel = LinearRegressionModel.load(sc, path) + >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + True + >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> try: + ... os.removedirs(path) + ... except: + ... pass >>> data = [ ... LabeledPoint(0.0, SparseVector(1, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(1, {0: 1.0})), @@ -126,6 +141,19 @@ class LinearRegressionModel(LinearRegressionModelBase): >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True """ + def save(self, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.LinearRegressionModel( + _py2java(sc, self._coeff), self.intercept) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.LinearRegressionModel.load( + sc._jsc.sc(), path) + weights = _java2py(sc, java_model.weights()) + intercept = java_model.intercept() + model = LinearRegressionModel(weights, intercept) + return model # train_func should take two parameters, namely data and initial_weights, and @@ -199,6 +227,20 @@ class LassoModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> lrm.save(sc, path) + >>> sameModel = LassoModel.load(sc, path) + >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + True + >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> try: + ... os.removedirs(path) + ... except: + ... pass >>> data = [ ... LabeledPoint(0.0, SparseVector(1, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(1, {0: 1.0})), @@ -211,6 +253,19 @@ class LassoModel(LinearRegressionModelBase): >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True """ + def save(self, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.LassoModel( + _py2java(sc, self._coeff), self.intercept) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.LassoModel.load( + sc._jsc.sc(), path) + weights = _java2py(sc, java_model.weights()) + intercept = java_model.intercept() + model = LassoModel(weights, intercept) + return model class LassoWithSGD(object): @@ -246,6 +301,20 @@ class RidgeRegressionModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> lrm.save(sc, path) + >>> sameModel = RidgeRegressionModel.load(sc, path) + >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + True + >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> try: + ... os.removedirs(path) + ... except: + ... pass >>> data = [ ... LabeledPoint(0.0, SparseVector(1, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(1, {0: 1.0})), @@ -258,6 +327,19 @@ class RidgeRegressionModel(LinearRegressionModelBase): >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True """ + def save(self, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.RidgeRegressionModel( + _py2java(sc, self._coeff), self.intercept) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.regression.RidgeRegressionModel.load( + sc._jsc.sc(), path) + weights = _java2py(sc, java_model.weights()) + intercept = java_model.intercept() + model = RidgeRegressionModel(weights, intercept) + return model class RidgeRegressionWithSGD(object): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index e877c720ac77a..c5c3468eb95e9 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -20,7 +20,6 @@ from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector -from pyspark.mllib.regression import LabeledPoint class MLUtils(object): @@ -50,6 +49,7 @@ def _parse_libsvm_line(line, multiclass=None): @staticmethod def _convert_labeled_point_to_libsvm(p): """Converts a LabeledPoint to a string in LIBSVM format.""" + from pyspark.mllib.regression import LabeledPoint assert isinstance(p, LabeledPoint) items = [str(p.label)] v = _convert_to_vector(p.features) @@ -92,6 +92,7 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> from tempfile import NamedTemporaryFile >>> from pyspark.mllib.util import MLUtils + >>> from pyspark.mllib.regression import LabeledPoint >>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() @@ -110,6 +111,7 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> print examples[2] (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ + from pyspark.mllib.regression import LabeledPoint if multiclass is not None: warnings.warn("deprecated", DeprecationWarning) @@ -130,6 +132,7 @@ def saveAsLibSVMFile(data, dir): >>> from tempfile import NamedTemporaryFile >>> from fileinput import input + >>> from pyspark.mllib.regression import LabeledPoint >>> from glob import glob >>> from pyspark.mllib.util import MLUtils >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \ @@ -156,6 +159,7 @@ def loadLabeledPoints(sc, path, minPartitions=None): >>> from tempfile import NamedTemporaryFile >>> from pyspark.mllib.util import MLUtils + >>> from pyspark.mllib.regression import LabeledPoint >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \ LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))] >>> tempFile = NamedTemporaryFile(delete=True) From 5e6ad24ff645a9b0f63d9c0f17193550963aa0a7 Mon Sep 17 00:00:00 2001 From: Shuo Xiang Date: Fri, 20 Mar 2015 14:45:44 -0400 Subject: [PATCH 469/817] [MLlib] SPARK-5954: Top by key This PR implements two functions - `topByKey(num: Int): RDD[(K, Array[V])]` finds the top-k values for each key in a pair RDD. This can be used, e.g., in computing top recommendations. - `takeOrderedByKey(num: Int): RDD[(K, Array[V])] ` does the opposite of `topByKey` The `sorted` is used here as the `toArray` method of the PriorityQueue does not return a necessarily sorted array. Author: Shuo Xiang Closes #5075 from coderxiang/topByKey and squashes the following commits: 1611c37 [Shuo Xiang] code clean up 6f565c0 [Shuo Xiang] naming a80e0ec [Shuo Xiang] typo and warning 82dded9 [Shuo Xiang] Merge remote-tracking branch 'upstream/master' into topByKey d202745 [Shuo Xiang] move to MLPairRDDFunctions 901b0af [Shuo Xiang] style check 70c6e35 [Shuo Xiang] remove takeOrderedByKey, update doc and test 0895c17 [Shuo Xiang] Merge remote-tracking branch 'upstream/master' into topByKey b10e325 [Shuo Xiang] Merge remote-tracking branch 'upstream/master' into topByKey debccad [Shuo Xiang] topByKey --- .../spark/mllib/rdd/MLPairRDDFunctions.scala | 60 +++++++++++++++++++ .../mllib/rdd/MLPairRDDFunctionsSuite.scala | 36 +++++++++++ 2 files changed, 96 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala new file mode 100644 index 0000000000000..9213fd3f595c3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.util.BoundedPriorityQueue + +/** + * Machine learning specific Pair RDD functions. + */ +@DeveloperApi +class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Serializable { + /** + * Returns the top k (largest) elements for each key from this RDD as defined by the specified + * implicit Ordering[T]. + * If the number of elements for a certain key is less than k, all of them will be returned. + * + * @param num k, the number of top elements to return + * @param ord the implicit ordering for T + * @return an RDD that contains the top k values for each key + */ + def topByKey(num: Int)(implicit ord: Ordering[V]): RDD[(K, Array[V])] = { + self.aggregateByKey(new BoundedPriorityQueue[V](num)(ord))( + seqOp = (queue, item) => { + queue += item + queue + }, + combOp = (queue1, queue2) => { + queue1 ++= queue2 + queue1 + } + ).mapValues(_.toArray.sorted(ord.reverse)) + } +} + +@DeveloperApi +object MLPairRDDFunctions { + /** Implicit conversion from a pair RDD to MLPairRDDFunctions. */ + implicit def fromPairRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): MLPairRDDFunctions[K, V] = + new MLPairRDDFunctions[K, V](rdd) +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala new file mode 100644 index 0000000000000..1ac7c12c4e8e6 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ + +class MLPairRDDFunctionsSuite extends FunSuite with MLlibTestSparkContext { + test("topByKey") { + val topMap = sc.parallelize(Array((1, 1), (1, 2), (3, 2), (3, 7), (3, 5), (5, 1), (5, 3)), 2) + .topByKey(2) + .collectAsMap() + + assert(topMap.size === 3) + assert(topMap(1) === Array(2, 1)) + assert(topMap(3) === Array(7, 5)) + assert(topMap(5) === Array(3, 1)) + } +} From 25636d9867c6bc901463b6b227cb444d701cfdd1 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 20 Mar 2015 14:53:59 -0400 Subject: [PATCH 470/817] [Spark 6096][MLlib] Add Naive Bayes load save methods in Python See [SPARK-6096](https://issues.apache.org/jira/browse/SPARK-6096). Author: Xusen Yin Closes #5090 from yinxusen/SPARK-6096 and squashes the following commits: bd0fea5 [Xusen Yin] fix style problem, etc. 3fd41f2 [Xusen Yin] use hanging indent in Python style e83803d [Xusen Yin] fix Python style d6dbde5 [Xusen Yin] fix python call java error a054bb3 [Xusen Yin] add save load for NaiveBayes python --- .../mllib/classification/NaiveBayes.scala | 11 +++++++ python/pyspark/mllib/classification.py | 31 ++++++++++++++++++- 2 files changed, 41 insertions(+), 1 deletion(-) 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 068449aa1d346..d60e82c410979 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 @@ -17,6 +17,10 @@ package org.apache.spark.mllib.classification +import java.lang.{Iterable => JIterable} + +import scala.collection.JavaConverters._ + import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -41,6 +45,13 @@ class NaiveBayesModel private[mllib] ( val pi: Array[Double], val theta: Array[Array[Double]]) extends ClassificationModel with Serializable with Saveable { + /** A Java-friendly constructor that takes three Iterable parameters. */ + private[mllib] def this( + labels: JIterable[Double], + pi: JIterable[Double], + theta: JIterable[JIterable[Double]]) = + this(labels.asScala.toArray, pi.asScala.toArray, theta.asScala.toArray.map(_.asScala.toArray)) + private val brzPi = new BDV[Double](pi) private val brzTheta = new BDM[Double](theta.length, theta(0).length) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index b66159c5bfb66..6766f3ebb8894 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -24,6 +24,7 @@ from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper +from pyspark.mllib.util import Saveable, Loader, inherit_doc __all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'LogisticRegressionWithLBFGS', @@ -359,7 +360,8 @@ def train(rdd, i): return _regression_train_wrapper(train, SVMModel, data, initialWeights) -class NaiveBayesModel(object): +@inherit_doc +class NaiveBayesModel(Saveable, Loader): """ Model for Naive Bayes classifiers. @@ -390,6 +392,16 @@ class NaiveBayesModel(object): 0.0 >>> model.predict(SparseVector(2, {0: 1.0})) 1.0 + >>> import os, tempfile + >>> path = tempfile.mkdtemp() + >>> model.save(sc, path) + >>> sameModel = NaiveBayesModel.load(sc, path) + >>> sameModel.predict(SparseVector(2, {0: 1.0})) == model.predict(SparseVector(2, {0: 1.0})) + True + >>> try: + ... os.removedirs(path) + ... except OSError: + ... pass """ def __init__(self, labels, pi, theta): @@ -404,6 +416,23 @@ def predict(self, x): x = _convert_to_vector(x) return self.labels[numpy.argmax(self.pi + x.dot(self.theta.transpose()))] + def save(self, sc, path): + java_labels = _py2java(sc, self.labels.tolist()) + java_pi = _py2java(sc, self.pi.tolist()) + java_theta = _py2java(sc, self.theta.tolist()) + java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel( + java_labels, java_pi, java_theta) + java_model.save(sc._jsc.sc(), path) + + @classmethod + def load(cls, sc, path): + java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel.load( + sc._jsc.sc(), path) + py_labels = _java2py(sc, java_model.labels()) + py_pi = _java2py(sc, java_model.pi()) + py_theta = _java2py(sc, java_model.theta()) + return NaiveBayesModel(py_labels, py_pi, numpy.array(py_theta)) + class NaiveBayes(object): From 6b36470c66bd6140c45e45d3f1d51b0082c3fd97 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 20 Mar 2015 15:02:57 -0400 Subject: [PATCH 471/817] [SPARK-5955][MLLIB] add checkpointInterval to ALS Add checkpiontInterval to ALS to prevent: 1. StackOverflow exceptions caused by long lineage, 2. large shuffle files generated during iterations, 3. slow recovery when some node fail. srowen coderxiang Author: Xiangrui Meng Closes #5076 from mengxr/SPARK-5955 and squashes the following commits: df56791 [Xiangrui Meng] update impl to reuse code 29affcb [Xiangrui Meng] do not materialize factors in implicit 20d3f7f [Xiangrui Meng] add checkpointInterval to ALS --- .../apache/spark/ml/param/sharedParams.scala | 11 +++++ .../apache/spark/ml/recommendation/ALS.scala | 42 ++++++++++++++++--- .../spark/mllib/recommendation/ALS.scala | 17 ++++++++ .../spark/ml/recommendation/ALSSuite.scala | 17 ++++++++ 4 files changed, 82 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 1a70322b4cace..5d660d1e151a7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -138,3 +138,14 @@ private[ml] trait HasOutputCol extends Params { /** @group getParam */ def getOutputCol: String = get(outputCol) } + +private[ml] trait HasCheckpointInterval extends Params { + /** + * param for checkpoint interval + * @group param + */ + val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") + + /** @group getParam */ + def getCheckpointInterval: Int = get(checkpointInterval) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index e3515ee81af3d..514b4ef98dc5b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.recommendation import java.{util => ju} +import java.io.IOException import scala.collection.mutable import scala.reflect.ClassTag @@ -26,6 +27,7 @@ import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} +import org.apache.hadoop.fs.{FileSystem, Path} import org.netlib.util.intW import org.apache.spark.{Logging, Partitioner} @@ -46,7 +48,7 @@ import org.apache.spark.util.random.XORShiftRandom * Common params for ALS. */ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasRegParam - with HasPredictionCol { + with HasPredictionCol with HasCheckpointInterval { /** * Param for rank of the matrix factorization. @@ -164,6 +166,7 @@ class ALSModel private[ml] ( itemFactors: RDD[(Int, Array[Float])]) extends Model[ALSModel] with ALSParams { + /** @group setParam */ def setPredictionCol(value: String): this.type = set(predictionCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { @@ -262,6 +265,9 @@ class ALS extends Estimator[ALSModel] with ALSParams { /** @group setParam */ def setNonnegative(value: Boolean): this.type = set(nonnegative, value) + /** @group setParam */ + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + /** * Sets both numUserBlocks and numItemBlocks to the specific value. * @group setParam @@ -274,6 +280,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { setMaxIter(20) setRegParam(1.0) + setCheckpointInterval(10) override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = { val map = this.paramMap ++ paramMap @@ -285,7 +292,8 @@ class ALS extends Estimator[ALSModel] with ALSParams { val (userFactors, itemFactors) = ALS.train(ratings, rank = map(rank), numUserBlocks = map(numUserBlocks), numItemBlocks = map(numItemBlocks), maxIter = map(maxIter), regParam = map(regParam), implicitPrefs = map(implicitPrefs), - alpha = map(alpha), nonnegative = map(nonnegative)) + alpha = map(alpha), nonnegative = map(nonnegative), + checkpointInterval = map(checkpointInterval)) val model = new ALSModel(this, map, map(rank), userFactors, itemFactors) Params.inheritValues(map, this, model) model @@ -494,6 +502,7 @@ object ALS extends Logging { nonnegative: Boolean = false, intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, + checkpointInterval: Int = 10, seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { require(intermediateRDDStorageLevel != StorageLevel.NONE, @@ -521,6 +530,18 @@ object ALS extends Logging { val seedGen = new XORShiftRandom(seed) var userFactors = initialize(userInBlocks, rank, seedGen.nextLong()) var itemFactors = initialize(itemInBlocks, rank, seedGen.nextLong()) + var previousCheckpointFile: Option[String] = None + val shouldCheckpoint: Int => Boolean = (iter) => + sc.checkpointDir.isDefined && (iter % checkpointInterval == 0) + val deletePreviousCheckpointFile: () => Unit = () => + previousCheckpointFile.foreach { file => + try { + FileSystem.get(sc.hadoopConfiguration).delete(new Path(file), true) + } catch { + case e: IOException => + logWarning(s"Cannot delete checkpoint file $file:", e) + } + } if (implicitPrefs) { for (iter <- 1 to maxIter) { userFactors.setName(s"userFactors-$iter").persist(intermediateRDDStorageLevel) @@ -528,19 +549,30 @@ object ALS extends Logging { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, implicitPrefs, alpha, solver) previousItemFactors.unpersist() - if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { - itemFactors.checkpoint() - } itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) + // TODO: Generalize PeriodicGraphCheckpointer and use it here. + if (shouldCheckpoint(iter)) { + itemFactors.checkpoint() // itemFactors gets materialized in computeFactors. + } val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) + if (shouldCheckpoint(iter)) { + deletePreviousCheckpointFile() + previousCheckpointFile = itemFactors.getCheckpointFile + } previousUserFactors.unpersist() } } else { for (iter <- 0 until maxIter) { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, solver = solver) + if (shouldCheckpoint(iter)) { + itemFactors.checkpoint() + itemFactors.count() // checkpoint item factors and cut lineage + deletePreviousCheckpointFile() + previousCheckpointFile = itemFactors.getCheckpointFile + } userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, solver = solver) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index caacab943030b..dddefe1944e9d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -82,6 +82,9 @@ class ALS private ( private var intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK private var finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK + /** checkpoint interval */ + private var checkpointInterval: Int = 10 + /** * Set the number of blocks for both user blocks and product blocks to parallelize the computation * into; pass -1 for an auto-configured number of blocks. Default: -1. @@ -182,6 +185,19 @@ class ALS private ( this } + /** + * Set period (in iterations) between checkpoints (default = 10). Checkpointing helps with + * recovery (when nodes fail) and StackOverflow exceptions caused by long lineage. It also helps + * with eliminating temporary shuffle files on disk, which can be important when there are many + * ALS iterations. If the checkpoint directory is not set in [[org.apache.spark.SparkContext]], + * this setting is ignored. + */ + @DeveloperApi + def setCheckpointInterval(checkpointInterval: Int): this.type = { + this.checkpointInterval = checkpointInterval + this + } + /** * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. @@ -212,6 +228,7 @@ class ALS private ( nonnegative = nonnegative, intermediateRDDStorageLevel = intermediateRDDStorageLevel, finalRDDStorageLevel = StorageLevel.NONE, + checkpointInterval = checkpointInterval, seed = seed) val userFactors = floatUserFactors diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index bb86bafc0eb0a..0bb06e9e8ac9c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.recommendation +import java.io.File import java.util.Random import scala.collection.mutable @@ -32,16 +33,25 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.util.Utils class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { private var sqlContext: SQLContext = _ + private var tempDir: File = _ override def beforeAll(): Unit = { super.beforeAll() + tempDir = Utils.createTempDir() + sc.setCheckpointDir(tempDir.getAbsolutePath) sqlContext = new SQLContext(sc) } + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDir) + super.afterAll() + } + test("LocalIndexEncoder") { val random = new Random for (numBlocks <- Seq(1, 2, 5, 10, 20, 50, 100)) { @@ -485,4 +495,11 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { }.count() } } + + test("als with large number of iterations") { + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + ALS.train(ratings, rank = 1, maxIter = 50, numUserBlocks = 2, numItemBlocks = 2) + ALS.train( + ratings, rank = 1, maxIter = 50, numUserBlocks = 2, numItemBlocks = 2, implicitPrefs = true) + } } From 49a01c7ea2c48feee7ab4551c4fa03fd1cdb1a32 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Fri, 20 Mar 2015 19:14:35 +0000 Subject: [PATCH 472/817] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Fixed calculateTotalMemory to use spark.mesos.executor.memoryOverhead - Added testCase Author: Jongyoul Lee Closes #5099 from jongyoul/SPARK-6423 and squashes the following commits: 6747fce [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Changed a description of spark.mesos.executor.memoryOverhead 475a7c8 [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Fit the import rules 453c5a2 [Jongyoul Lee] [SPARK-6423][Mesos] MemoryUtils should use memoryOverhead if it's set - Fixed calculateTotalMemory to use spark.mesos.executor.memoryOverhead - Added testCase --- .../scheduler/cluster/mesos/MemoryUtils.scala | 10 ++-- .../cluster/mesos/MemoryUtilsSuite.scala | 47 +++++++++++++++++++ docs/running-on-mesos.md | 8 ++-- 3 files changed, 53 insertions(+), 12 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala index 705116cb13f54..aa3ec0f8cfb9c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -21,15 +21,11 @@ import org.apache.spark.SparkContext private[spark] object MemoryUtils { // These defaults copied from YARN - val OVERHEAD_FRACTION = 1.10 + val OVERHEAD_FRACTION = 0.10 val OVERHEAD_MINIMUM = 384 def calculateTotalMemory(sc: SparkContext) = { - math.max( - sc.conf.getOption("spark.mesos.executor.memoryOverhead") - .getOrElse(OVERHEAD_MINIMUM.toString) - .toInt + sc.executorMemory, - OVERHEAD_FRACTION * sc.executorMemory - ) + sc.conf.getInt("spark.mesos.executor.memoryOverhead", + math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala new file mode 100644 index 0000000000000..3fa0115e68259 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import org.mockito.Mockito._ +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.{SparkConf, SparkContext} + +class MemoryUtilsSuite extends FunSuite with MockitoSugar { + test("MesosMemoryUtils should always override memoryOverhead when it's set") { + val sparkConf = new SparkConf + + val sc = mock[SparkContext] + when(sc.conf).thenReturn(sparkConf) + + // 384 > sc.executorMemory * 0.1 => 512 + 384 = 896 + when(sc.executorMemory).thenReturn(512) + assert(MemoryUtils.calculateTotalMemory(sc) === 896) + + // 384 < sc.executorMemory * 0.1 => 4096 + (4096 * 0.1) = 4505.6 + when(sc.executorMemory).thenReturn(4096) + assert(MemoryUtils.calculateTotalMemory(sc) === 4505) + + // set memoryOverhead + sparkConf.set("spark.mesos.executor.memoryOverhead", "100") + assert(MemoryUtils.calculateTotalMemory(sc) === 4196) + sparkConf.set("spark.mesos.executor.memoryOverhead", "400") + assert(MemoryUtils.calculateTotalMemory(sc) === 4496) + } +} diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 6a9d304501dc0..c984639bd34cf 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -224,11 +224,9 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - This value is an additive for spark.executor.memory, specified in MB, - which is used to calculate the total Mesos task memory. A value of 384 - implies a 384MB overhead. Additionally, there is a hard-coded 10% minimum - overhead. The final overhead will be the larger of either - `spark.mesos.executor.memoryOverhead` or 10% of `spark.executor.memory`. + The amount of additional memory, specified in MB, to be allocated per executor. By default, + the overhead will be larger of either 384 or 10% of `spark.executor.memory`. If it's set, + the final overhead will be this value. From 11e025956be3818c00effef0d650734f8feeb436 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Fri, 20 Mar 2015 17:13:18 -0400 Subject: [PATCH 473/817] [SPARK-6309] [SQL] [MLlib] Implement MatrixUDT Utilities to serialize and deserialize Matrices in MLlib Author: MechCoder Closes #5048 from MechCoder/spark-6309 and squashes the following commits: 05dc6f2 [MechCoder] Hashcode and organize imports 16d5d47 [MechCoder] Test some more 6e67020 [MechCoder] TST: Test using Array conversion instead of equals 7fa7a2c [MechCoder] [SPARK-6309] [SQL] [MLlib] Implement MatrixUDT --- .../apache/spark/mllib/linalg/Matrices.scala | 90 +++++++++++++++++++ .../spark/mllib/linalg/MatricesSuite.scala | 13 +++ 2 files changed, 103 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index fdd8848189f19..849f44295f089 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -23,9 +23,15 @@ import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHash import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + /** * Trait for a local matrix. */ +@SQLUserDefinedType(udt = classOf[MatrixUDT]) sealed trait Matrix extends Serializable { /** Number of rows. */ @@ -102,6 +108,88 @@ sealed trait Matrix extends Serializable { private[spark] def foreachActive(f: (Int, Int, Double) => Unit) } +@DeveloperApi +private[spark] class MatrixUDT extends UserDefinedType[Matrix] { + + override def sqlType: StructType = { + // type: 0 = sparse, 1 = dense + // the dense matrix is built by numRows, numCols, values and isTransposed, all of which are + // set as not nullable, except values since in the future, support for binary matrices might + // be added for which values are not needed. + // the sparse matrix needs colPtrs and rowIndices, which are set as + // null, while building the dense matrix. + StructType(Seq( + StructField("type", ByteType, nullable = false), + StructField("numRows", IntegerType, nullable = false), + StructField("numCols", IntegerType, nullable = false), + StructField("colPtrs", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("rowIndices", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true), + StructField("isTransposed", BooleanType, nullable = false) + )) + } + + override def serialize(obj: Any): Row = { + val row = new GenericMutableRow(7) + obj match { + case sm: SparseMatrix => + row.setByte(0, 0) + row.setInt(1, sm.numRows) + row.setInt(2, sm.numCols) + row.update(3, sm.colPtrs.toSeq) + row.update(4, sm.rowIndices.toSeq) + row.update(5, sm.values.toSeq) + row.setBoolean(6, sm.isTransposed) + + case dm: DenseMatrix => + row.setByte(0, 1) + row.setInt(1, dm.numRows) + row.setInt(2, dm.numCols) + row.setNullAt(3) + row.setNullAt(4) + row.update(5, dm.values.toSeq) + row.setBoolean(6, dm.isTransposed) + } + row + } + + override def deserialize(datum: Any): Matrix = { + datum match { + // TODO: something wrong with UDT serialization, should never happen. + case m: Matrix => m + case row: Row => + require(row.length == 7, + s"MatrixUDT.deserialize given row with length ${row.length} but requires length == 7") + val tpe = row.getByte(0) + val numRows = row.getInt(1) + val numCols = row.getInt(2) + val values = row.getAs[Iterable[Double]](5).toArray + val isTransposed = row.getBoolean(6) + tpe match { + case 0 => + val colPtrs = row.getAs[Iterable[Int]](3).toArray + val rowIndices = row.getAs[Iterable[Int]](4).toArray + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) + case 1 => + new DenseMatrix(numRows, numCols, values, isTransposed) + } + } + } + + override def userClass: Class[Matrix] = classOf[Matrix] + + override def equals(o: Any): Boolean = { + o match { + case v: MatrixUDT => true + case _ => false + } + } + + override def hashCode(): Int = 1994 + + private[spark] override def asNullable: MatrixUDT = this +} + /** * Column-major dense matrix. * The entry values are stored in a single array of doubles with columns listed in sequence. @@ -119,6 +207,7 @@ sealed trait Matrix extends Serializable { * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. */ +@SQLUserDefinedType(udt = classOf[MatrixUDT]) class DenseMatrix( val numRows: Int, val numCols: Int, @@ -360,6 +449,7 @@ object DenseMatrix { * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. */ +@SQLUserDefinedType(udt = classOf[MatrixUDT]) class SparseMatrix( val numRows: Int, val numCols: Int, diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index c098b5458fe6b..96f677db3f377 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -424,4 +424,17 @@ class MatricesSuite extends FunSuite { assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) } + + test("MatrixUDT") { + val dm1 = new DenseMatrix(2, 2, Array(0.9, 1.2, 2.3, 9.8)) + val dm2 = new DenseMatrix(3, 2, Array(0.0, 1.21, 2.3, 9.8, 9.0, 0.0)) + val dm3 = new DenseMatrix(0, 0, Array()) + val sm1 = dm1.toSparse + val sm2 = dm2.toSparse + val sm3 = dm3.toSparse + val mUDT = new MatrixUDT() + Seq(dm1, dm2, dm3, sm1, sm2, sm3).foreach { + mat => assert(mat.toArray === mUDT.deserialize(mUDT.serialize(mat)).toArray) + } + } } From 257cde7c363efb3317bfb5c13975cca9154894e2 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Fri, 20 Mar 2015 17:18:18 -0400 Subject: [PATCH 474/817] [SPARK-6421][MLLIB] _regression_train_wrapper does not test initialWeights correctly Weight parameters must be initialized correctly even when numpy array is passed as initial weights. Author: lewuathe Closes #5101 from Lewuathe/SPARK-6421 and squashes the following commits: 7795201 [lewuathe] Fix lint-python errors 21d4fe3 [lewuathe] Fix init logic of weights --- python/pyspark/mllib/regression.py | 3 ++- python/pyspark/mllib/tests.py | 7 +++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 015a7860116c9..414a0ada80787 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -163,7 +163,8 @@ def _regression_train_wrapper(train_func, modelClass, data, initial_weights): first = data.first() if not isinstance(first, LabeledPoint): raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) - initial_weights = initial_weights or [0.0] * len(data.first().features) + if initial_weights is None: + initial_weights = [0.0] * len(data.first().features) weights, intercept = train_func(data, _convert_to_vector(initial_weights)) return modelClass(weights, intercept) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 5328d99b69684..155019638f806 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -323,6 +323,13 @@ def test_regression(self): self.assertTrue(gbt_model.predict(features[2]) <= 0) self.assertTrue(gbt_model.predict(features[3]) > 0) + try: + LinearRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) + LassoWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) + RidgeRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) + except ValueError: + self.fail() + class StatTests(PySparkTestCase): # SPARK-4023 From a95043b1780bfde556db2dcc01511e40a12498dd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Mar 2015 15:47:07 -0700 Subject: [PATCH 475/817] [SPARK-6428][SQL] Added explicit type for all public methods in sql/core Also implemented equals/hashCode when they are missing. This is done in order to enable automatic public method type checking. Author: Reynold Xin Closes #5104 from rxin/sql-hashcode-explicittype and squashes the following commits: ffce6f3 [Reynold Xin] Code review feedback. 8b36733 [Reynold Xin] [SPARK-6428][SQL] Added explicit type for all public methods. --- .../catalyst/expressions/AttributeSet.scala | 3 +- .../spark/sql/catalyst/expressions/rows.scala | 21 +++++ .../org/apache/spark/sql/types/Decimal.scala | 2 +- .../apache/spark/sql/types/dataTypes.scala | 20 ++--- .../scala/org/apache/spark/sql/Column.scala | 2 +- .../org/apache/spark/sql/DataFrame.scala | 6 +- .../org/apache/spark/sql/SQLContext.scala | 8 +- .../apache/spark/sql/UDFRegistration.scala | 2 +- .../spark/sql/columnar/ColumnAccessor.scala | 4 +- .../spark/sql/columnar/ColumnBuilder.scala | 4 +- .../spark/sql/columnar/ColumnStats.scala | 24 +++--- .../spark/sql/columnar/ColumnType.scala | 56 +++++++------ .../columnar/InMemoryColumnarTableScan.scala | 56 +++++++------ .../sql/columnar/NullableColumnAccessor.scala | 2 +- .../CompressibleColumnAccessor.scala | 4 +- .../CompressibleColumnBuilder.scala | 2 +- .../compression/compressionSchemes.scala | 80 ++++++++++--------- .../spark/sql/execution/Aggregate.scala | 8 +- .../apache/spark/sql/execution/Exchange.scala | 19 ++--- .../spark/sql/execution/ExistingRDD.scala | 20 ++--- .../apache/spark/sql/execution/Expand.scala | 5 +- .../apache/spark/sql/execution/Generate.scala | 3 +- .../sql/execution/GeneratedAggregate.scala | 11 +-- .../spark/sql/execution/LocalTableScan.scala | 7 +- .../spark/sql/execution/SparkPlan.scala | 1 + .../sql/execution/SparkSqlSerializer.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 6 +- .../spark/sql/execution/basicOperators.scala | 73 +++++++++-------- .../apache/spark/sql/execution/commands.scala | 33 ++++---- .../spark/sql/execution/debug/package.scala | 30 +++---- .../execution/joins/BroadcastHashJoin.scala | 10 ++- .../joins/BroadcastLeftSemiJoinHash.scala | 10 +-- .../joins/BroadcastNestedLoopJoin.scala | 5 +- .../execution/joins/CartesianProduct.scala | 8 +- .../spark/sql/execution/joins/HashJoin.scala | 4 +- .../sql/execution/joins/HashOuterJoin.scala | 53 ++++++------ .../sql/execution/joins/HashedRelation.scala | 4 +- .../sql/execution/joins/LeftSemiJoinBNL.scala | 10 ++- .../execution/joins/LeftSemiJoinHash.scala | 11 +-- .../execution/joins/ShuffledHashJoin.scala | 6 +- .../spark/sql/execution/pythonUdfs.scala | 21 +++-- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 3 +- .../apache/spark/sql/jdbc/JDBCRelation.scala | 8 +- .../apache/spark/sql/json/JSONRelation.scala | 6 +- .../spark/sql/parquet/ParquetConverter.scala | 2 +- .../spark/sql/parquet/ParquetRelation.scala | 10 ++- .../sql/parquet/ParquetTableOperations.scala | 12 +-- .../apache/spark/sql/parquet/newParquet.scala | 37 ++++++--- .../sql/parquet/timestamp/NanoTime.scala | 6 +- .../spark/sql/sources/LogicalRelation.scala | 16 ++-- .../apache/spark/sql/sources/commands.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 6 +- .../org/apache/spark/sql/sources/rules.scala | 4 +- 53 files changed, 438 insertions(+), 330 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index a9ba0be596349..adaeab0b5c027 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.analysis.Star protected class AttributeEquals(val a: Attribute) { override def hashCode() = a match { @@ -115,7 +114,7 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) // sorts of things in its closure. override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq - override def toString = "{" + baseSet.map(_.a).mkString(", ") + "}" + override def toString: String = "{" + baseSet.map(_.a).mkString(", ") + "}" override def isEmpty: Boolean = baseSet.isEmpty } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index faa366771824b..f03d6f71a9fae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -146,6 +146,27 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { result } + override def equals(o: Any): Boolean = o match { + case other: Row => + if (values.length != other.length) { + return false + } + + var i = 0 + while (i < values.length) { + if (isNullAt(i) != other.isNullAt(i)) { + return false + } + if (apply(i) != other.apply(i)) { + return false + } + i += 1 + } + true + + case _ => false + } + def copy() = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 21cc6cea4bf54..994c5202c15dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -246,7 +246,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case d: Decimal => compare(d) == 0 case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index bf39603d13bd5..d973144de3468 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -307,7 +307,7 @@ protected[sql] object NativeType { protected[sql] trait PrimitiveType extends DataType { - override def isPrimitive = true + override def isPrimitive: Boolean = true } @@ -442,7 +442,7 @@ class TimestampType private() extends NativeType { @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = new Ordering[JvmType] { - def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) + def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) } /** @@ -542,7 +542,7 @@ class LongType private() extends IntegralType { */ override def defaultSize: Int = 8 - override def simpleString = "bigint" + override def simpleString: String = "bigint" private[spark] override def asNullable: LongType = this } @@ -572,7 +572,7 @@ class IntegerType private() extends IntegralType { */ override def defaultSize: Int = 4 - override def simpleString = "int" + override def simpleString: String = "int" private[spark] override def asNullable: IntegerType = this } @@ -602,7 +602,7 @@ class ShortType private() extends IntegralType { */ override def defaultSize: Int = 2 - override def simpleString = "smallint" + override def simpleString: String = "smallint" private[spark] override def asNullable: ShortType = this } @@ -632,7 +632,7 @@ class ByteType private() extends IntegralType { */ override def defaultSize: Int = 1 - override def simpleString = "tinyint" + override def simpleString: String = "tinyint" private[spark] override def asNullable: ByteType = this } @@ -696,7 +696,7 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT */ override def defaultSize: Int = 4096 - override def simpleString = precisionInfo match { + override def simpleString: String = precisionInfo match { case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" case None => "decimal(10,0)" } @@ -836,7 +836,7 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT */ override def defaultSize: Int = 100 * elementType.defaultSize - override def simpleString = s"array<${elementType.simpleString}>" + override def simpleString: String = s"array<${elementType.simpleString}>" private[spark] override def asNullable: ArrayType = ArrayType(elementType.asNullable, containsNull = true) @@ -1065,7 +1065,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru */ override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum - override def simpleString = { + override def simpleString: String = { val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") s"struct<${fieldTypes.mkString(",")}>" } @@ -1142,7 +1142,7 @@ case class MapType( */ override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) - override def simpleString = s"map<${keyType.simpleString},${valueType.simpleString}>" + override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" private[spark] override def asNullable: MapType = MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 908c78a4d3f10..b7a13a1b26802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -59,7 +59,7 @@ class Column(protected[sql] val expr: Expression) { override def toString: String = expr.prettyString - override def equals(that: Any) = that match { + override def equals(that: Any): Boolean = that match { case that: Column => that.expr.equals(this.expr) case _ => false } 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 46f50708a9184..8b8f86c4127e0 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 @@ -33,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.{expressions, ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} @@ -722,7 +722,7 @@ class DataFrame private[sql]( : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil - def rowFunction(row: Row) = { + def rowFunction(row: Row): TraversableOnce[Row] = { f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) } val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) @@ -1155,7 +1155,7 @@ class DataFrame private[sql]( val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null) new Iterator[String] { - override def hasNext = iter.hasNext + override def hasNext: Boolean = iter.hasNext override def next(): String = { JsonRDD.rowToJSON(rowSchema, gen)(iter.next()) gen.flush() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 6de46a50db20e..dc9912b52dcab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -144,7 +144,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val tlSession = new ThreadLocal[SQLSession]() { - override def initialValue = defaultSession + override def initialValue: SQLSession = defaultSession } @transient @@ -988,9 +988,9 @@ class SQLContext(@transient val sparkContext: SparkContext) val sqlContext: SQLContext = self - def codegenEnabled = self.conf.codegenEnabled + def codegenEnabled: Boolean = self.conf.codegenEnabled - def numPartitions = self.conf.numShufflePartitions + def numPartitions: Int = self.conf.numShufflePartitions def strategies: Seq[Strategy] = experimental.extraStrategies ++ ( @@ -1109,7 +1109,7 @@ class SQLContext(@transient val sparkContext: SparkContext) lazy val analyzed: LogicalPlan = analyzer(logical) lazy val withCachedData: LogicalPlan = { - assertAnalyzed + assertAnalyzed() cacheManager.useCachedData(analyzed) } lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 8051df299252c..b97aaf73529a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -61,7 +61,7 @@ class UDFRegistration private[sql] (sqlContext: SQLContext) extends Logging { val dataType = sqlContext.parseDataType(stringDataType) - def builder(e: Seq[Expression]) = + def builder(e: Seq[Expression]): PythonUDF = PythonUDF( name, command, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index b615eaa0dca0d..f615fb33a7c35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -48,9 +48,9 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( protected def initialize() {} - def hasNext = buffer.hasRemaining + override def hasNext: Boolean = buffer.hasRemaining - def extractTo(row: MutableRow, ordinal: Int): Unit = { + override def extractTo(row: MutableRow, ordinal: Int): Unit = { extractSingle(row, ordinal) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index d8d24a577347c..c881747751520 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -58,7 +58,7 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( override def initialize( initialSize: Int, columnName: String = "", - useCompression: Boolean = false) = { + useCompression: Boolean = false): Unit = { val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize this.columnName = columnName @@ -73,7 +73,7 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( columnType.append(row, ordinal, buffer) } - override def build() = { + override def build(): ByteBuffer = { buffer.flip().asInstanceOf[ByteBuffer] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 04047b9c062be..87a6631da8300 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -76,7 +76,7 @@ private[sql] sealed trait ColumnStats extends Serializable { private[sql] class NoopColumnStats extends ColumnStats { override def gatherStats(row: Row, ordinal: Int): Unit = super.gatherStats(row, ordinal) - def collectedStatistics = Row(null, null, nullCount, count, 0L) + override def collectedStatistics: Row = Row(null, null, nullCount, count, 0L) } private[sql] class BooleanColumnStats extends ColumnStats { @@ -93,7 +93,7 @@ private[sql] class BooleanColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class ByteColumnStats extends ColumnStats { @@ -110,7 +110,7 @@ private[sql] class ByteColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class ShortColumnStats extends ColumnStats { @@ -127,7 +127,7 @@ private[sql] class ShortColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class LongColumnStats extends ColumnStats { @@ -144,7 +144,7 @@ private[sql] class LongColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DoubleColumnStats extends ColumnStats { @@ -161,7 +161,7 @@ private[sql] class DoubleColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class FloatColumnStats extends ColumnStats { @@ -178,7 +178,7 @@ private[sql] class FloatColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class FixedDecimalColumnStats extends ColumnStats { @@ -212,7 +212,7 @@ private[sql] class IntColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class StringColumnStats extends ColumnStats { @@ -229,7 +229,7 @@ private[sql] class StringColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DateColumnStats extends IntColumnStats @@ -248,7 +248,7 @@ private[sql] class TimestampColumnStats extends ColumnStats { } } - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class BinaryColumnStats extends ColumnStats { @@ -259,7 +259,7 @@ private[sql] class BinaryColumnStats extends ColumnStats { } } - def collectedStatistics = Row(null, null, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(null, null, nullCount, count, sizeInBytes) } private[sql] class GenericColumnStats extends ColumnStats { @@ -270,5 +270,5 @@ private[sql] class GenericColumnStats extends ColumnStats { } } - def collectedStatistics = Row(null, null, nullCount, count, sizeInBytes) + override def collectedStatistics: Row = Row(null, null, nullCount, count, sizeInBytes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 36ea1c77e0470..c47497e0662d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -98,7 +98,7 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( */ def clone(v: JvmType): JvmType = v - override def toString = getClass.getSimpleName.stripSuffix("$") + override def toString: String = getClass.getSimpleName.stripSuffix("$") } private[sql] abstract class NativeColumnType[T <: NativeType]( @@ -114,7 +114,7 @@ private[sql] abstract class NativeColumnType[T <: NativeType]( } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { - def append(v: Int, buffer: ByteBuffer): Unit = { + override def append(v: Int, buffer: ByteBuffer): Unit = { buffer.putInt(v) } @@ -122,7 +122,7 @@ private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { buffer.putInt(row.getInt(ordinal)) } - def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Int = { buffer.getInt() } @@ -134,7 +134,7 @@ private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { row.setInt(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getInt(ordinal) + override def getField(row: Row, ordinal: Int): Int = row.getInt(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setInt(toOrdinal, from.getInt(fromOrdinal)) @@ -150,7 +150,7 @@ private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { buffer.putLong(row.getLong(ordinal)) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Long = { buffer.getLong() } @@ -162,7 +162,7 @@ private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { row.setLong(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getLong(ordinal) + override def getField(row: Row, ordinal: Int): Long = row.getLong(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setLong(toOrdinal, from.getLong(fromOrdinal)) @@ -178,7 +178,7 @@ private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { buffer.putFloat(row.getFloat(ordinal)) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Float = { buffer.getFloat() } @@ -190,7 +190,7 @@ private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { row.setFloat(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal) + override def getField(row: Row, ordinal: Int): Float = row.getFloat(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setFloat(toOrdinal, from.getFloat(fromOrdinal)) @@ -206,7 +206,7 @@ private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { buffer.putDouble(row.getDouble(ordinal)) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Double = { buffer.getDouble() } @@ -218,7 +218,7 @@ private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { row.setDouble(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal) + override def getField(row: Row, ordinal: Int): Double = row.getDouble(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setDouble(toOrdinal, from.getDouble(fromOrdinal)) @@ -234,7 +234,7 @@ private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { buffer.put(if (row.getBoolean(ordinal)) 1: Byte else 0: Byte) } - override def extract(buffer: ByteBuffer) = buffer.get() == 1 + override def extract(buffer: ByteBuffer): Boolean = buffer.get() == 1 override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { row.setBoolean(ordinal, buffer.get() == 1) @@ -244,7 +244,7 @@ private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { row.setBoolean(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal) + override def getField(row: Row, ordinal: Int): Boolean = row.getBoolean(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setBoolean(toOrdinal, from.getBoolean(fromOrdinal)) @@ -260,7 +260,7 @@ private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { buffer.put(row.getByte(ordinal)) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Byte = { buffer.get() } @@ -272,7 +272,7 @@ private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { row.setByte(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getByte(ordinal) + override def getField(row: Row, ordinal: Int): Byte = row.getByte(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setByte(toOrdinal, from.getByte(fromOrdinal)) @@ -288,7 +288,7 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { buffer.putShort(row.getShort(ordinal)) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Short = { buffer.getShort() } @@ -300,7 +300,7 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { row.setShort(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getShort(ordinal) + override def getField(row: Row, ordinal: Int): Short = row.getShort(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setShort(toOrdinal, from.getShort(fromOrdinal)) @@ -317,7 +317,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): String = { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) @@ -328,7 +328,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { row.setString(ordinal, value) } - override def getField(row: Row, ordinal: Int) = row.getString(ordinal) + override def getField(row: Row, ordinal: Int): String = row.getString(ordinal) override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { to.setString(toOrdinal, from.getString(fromOrdinal)) @@ -336,7 +336,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Int = { buffer.getInt } @@ -344,7 +344,7 @@ private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { buffer.putInt(v) } - override def getField(row: Row, ordinal: Int) = { + override def getField(row: Row, ordinal: Int): Int = { row(ordinal).asInstanceOf[Int] } @@ -354,7 +354,7 @@ private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { } private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Timestamp = { val timestamp = new Timestamp(buffer.getLong()) timestamp.setNanos(buffer.getInt()) timestamp @@ -364,7 +364,7 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { buffer.putLong(v.getTime).putInt(v.getNanos) } - override def getField(row: Row, ordinal: Int) = { + override def getField(row: Row, ordinal: Int): Timestamp = { row(ordinal).asInstanceOf[Timestamp] } @@ -405,7 +405,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( defaultSize: Int) extends ColumnType[T, Array[Byte]](typeId, defaultSize) { - override def actualSize(row: Row, ordinal: Int) = { + override def actualSize(row: Row, ordinal: Int): Int = { getField(row, ordinal).length + 4 } @@ -413,7 +413,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( buffer.putInt(v.length).put(v, 0, v.length) } - override def extract(buffer: ByteBuffer) = { + override def extract(buffer: ByteBuffer): Array[Byte] = { val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) @@ -426,7 +426,9 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](11, 16) row(ordinal) = value } - override def getField(row: Row, ordinal: Int) = row(ordinal).asInstanceOf[Array[Byte]] + override def getField(row: Row, ordinal: Int): Array[Byte] = { + row(ordinal).asInstanceOf[Array[Byte]] + } } // Used to process generic objects (all types other than those listed above). Objects should be @@ -437,7 +439,9 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](12, 16) { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } - override def getField(row: Row, ordinal: Int) = SparkSqlSerializer.serialize(row(ordinal)) + override def getField(row: Row, ordinal: Int): Array[Byte] = { + SparkSqlSerializer.serialize(row(ordinal)) + } } private[sql] object ColumnType { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 387faee12b3cd..6eee0c86d6a1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import org.apache.spark.Accumulator +import org.apache.spark.sql.catalyst.expressions + import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD @@ -77,20 +80,23 @@ private[sql] case class InMemoryRelation( _statistics } - override def statistics = if (_statistics == null) { - if (batchStats.value.isEmpty) { - // Underlying columnar RDD hasn't been materialized, no useful statistics information - // available, return the default statistics. - Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) + override def statistics: Statistics = { + if (_statistics == null) { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) + } else { + // Underlying columnar RDD has been materialized, required information has also been + // collected via the `batchStats` accumulator, compute the final statistics, + // and update `_statistics`. + _statistics = Statistics(sizeInBytes = computeSizeInBytes) + _statistics + } } else { - // Underlying columnar RDD has been materialized, required information has also been collected - // via the `batchStats` accumulator, compute the final statistics, and update `_statistics`. - _statistics = Statistics(sizeInBytes = computeSizeInBytes) + // Pre-computed statistics _statistics } - } else { - // Pre-computed statistics - _statistics } // If the cached column buffers were not passed in, we calculate them in the constructor. @@ -99,7 +105,7 @@ private[sql] case class InMemoryRelation( buildBuffers() } - def recache() = { + def recache(): Unit = { _cachedColumnBuffers.unpersist() _cachedColumnBuffers = null buildBuffers() @@ -109,7 +115,7 @@ private[sql] case class InMemoryRelation( val output = child.output val cached = child.execute().mapPartitions { rowIterator => new Iterator[CachedBatch] { - def next() = { + def next(): CachedBatch = { val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) val initialBufferSize = columnType.defaultSize * batchSize @@ -144,7 +150,7 @@ private[sql] case class InMemoryRelation( CachedBatch(columnBuilders.map(_.build().array()), stats) } - def hasNext = rowIterator.hasNext + def hasNext: Boolean = rowIterator.hasNext } }.persist(storageLevel) @@ -158,9 +164,9 @@ private[sql] case class InMemoryRelation( _cachedColumnBuffers, statisticsToBePropagated) } - override def children = Seq.empty + override def children: Seq[LogicalPlan] = Seq.empty - override def newInstance() = { + override def newInstance(): this.type = { new InMemoryRelation( output.map(_.newInstance()), useCompression, @@ -172,7 +178,7 @@ private[sql] case class InMemoryRelation( statisticsToBePropagated).asInstanceOf[this.type] } - def cachedColumnBuffers = _cachedColumnBuffers + def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, statisticsToBePropagated) @@ -220,7 +226,7 @@ private[sql] case class InMemoryColumnarTableScan( case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 } - val partitionFilters = { + val partitionFilters: Seq[Expression] = { predicates.flatMap { p => val filter = buildFilter.lift(p) val boundFilter = @@ -239,12 +245,12 @@ private[sql] case class InMemoryColumnarTableScan( } // Accumulators used for testing purposes - val readPartitions = sparkContext.accumulator(0) - val readBatches = sparkContext.accumulator(0) + val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) + val readBatches: Accumulator[Int] = sparkContext.accumulator(0) private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - override def execute() = { + override def execute(): RDD[Row] = { readPartitions.setValue(0) readBatches.setValue(0) @@ -271,7 +277,7 @@ private[sql] case class InMemoryColumnarTableScan( val nextRow = new SpecificMutableRow(requestedColumnDataTypes) - def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { + def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]): Iterator[Row] = { val rows = cacheBatches.flatMap { cachedBatch => // Build column accessors val columnAccessors = requestedColumnIndices.map { batchColumnIndex => @@ -283,7 +289,7 @@ private[sql] case class InMemoryColumnarTableScan( // Extract rows via column accessors new Iterator[Row] { private[this] val rowLen = nextRow.length - override def next() = { + override def next(): Row = { var i = 0 while (i < rowLen) { columnAccessors(i).extractTo(nextRow, i) @@ -292,7 +298,7 @@ private[sql] case class InMemoryColumnarTableScan( nextRow } - override def hasNext = columnAccessors(0).hasNext + override def hasNext: Boolean = columnAccessors(0).hasNext } } @@ -308,7 +314,7 @@ private[sql] case class InMemoryColumnarTableScan( if (inMemoryPartitionPruningEnabled) { cachedBatchIterator.filter { cachedBatch => if (!partitionFilter(cachedBatch.stats)) { - def statsString = relation.partitionStatistics.schema + def statsString: String = relation.partitionStatistics.schema .zip(cachedBatch.stats.toSeq) .map { case (a, s) => s"${a.name}: $s" } .mkString(", ") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 965782a40031b..4d35650d4b1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -55,5 +55,5 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { pos += 1 } - abstract override def hasNext = seenNulls < nullCount || super.hasNext + abstract override def hasNext: Boolean = seenNulls < nullCount || super.hasNext } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index 7dff9deac8dc0..d0b602a834dfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -26,12 +26,12 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc private var decoder: Decoder[T] = _ - abstract override protected def initialize() = { + abstract override protected def initialize(): Unit = { super.initialize() decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType) } - abstract override def hasNext = super.hasNext || decoder.hasNext + abstract override def hasNext: Boolean = super.hasNext || decoder.hasNext override def extractSingle(row: MutableRow, ordinal: Int): Unit = { decoder.next(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index aead768ecdf0a..b9cfc5df550d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -81,7 +81,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] } } - override def build() = { + override def build(): ByteBuffer = { val nonNullBuffer = buildNonNulls() val typeId = nonNullBuffer.getInt() val encoder: Encoder[T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 68a5b1de7691b..8727d71c48bb7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -33,22 +33,23 @@ import org.apache.spark.util.Utils private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 - override def supports(columnType: ColumnType[_, _]) = true + override def supports(columnType: ColumnType[_, _]): Boolean = true - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType]( + buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { - override def uncompressedSize = 0 + override def uncompressedSize: Int = 0 - override def compressedSize = 0 + override def compressedSize: Int = 0 - override def compress(from: ByteBuffer, to: ByteBuffer) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { // Writes compression type ID and copies raw contents to.putInt(PassThrough.typeId).put(from).rewind() to @@ -62,22 +63,23 @@ private[sql] case object PassThrough extends CompressionScheme { columnType.extract(buffer, row, ordinal) } - override def hasNext = buffer.hasRemaining + override def hasNext: Boolean = buffer.hasRemaining } } private[sql] case object RunLengthEncoding extends CompressionScheme { override val typeId = 1 - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType]( + buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } - override def supports(columnType: ColumnType[_, _]) = columnType match { + override def supports(columnType: ColumnType[_, _]): Boolean = columnType match { case INT | LONG | SHORT | BYTE | STRING | BOOLEAN => true case _ => false } @@ -90,9 +92,9 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { private val lastValue = new SpecificMutableRow(Seq(columnType.dataType)) private var lastRun = 0 - override def uncompressedSize = _uncompressedSize + override def uncompressedSize: Int = _uncompressedSize - override def compressedSize = _compressedSize + override def compressedSize: Int = _compressedSize override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { val value = columnType.getField(row, ordinal) @@ -114,7 +116,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } } - override def compress(from: ByteBuffer, to: ByteBuffer) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { @@ -169,7 +171,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { columnType.setField(row, ordinal, currentValue) } - override def hasNext = valueCount < run || buffer.hasRemaining + override def hasNext: Boolean = valueCount < run || buffer.hasRemaining } } @@ -179,15 +181,16 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // 32K unique values allowed val MAX_DICT_SIZE = Short.MaxValue - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + : Decoder[T] = { new this.Decoder(buffer, columnType) } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def supports(columnType: ColumnType[_, _]) = columnType match { + override def supports(columnType: ColumnType[_, _]): Boolean = columnType match { case INT | LONG | STRING => true case _ => false } @@ -237,7 +240,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def compress(from: ByteBuffer, to: ByteBuffer) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { if (overflow) { throw new IllegalStateException( "Dictionary encoding should not be used because of dictionary overflow.") @@ -260,9 +263,9 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { to } - override def uncompressedSize = _uncompressedSize + override def uncompressedSize: Int = _uncompressedSize - override def compressedSize = if (overflow) Int.MaxValue else dictionarySize + count * 2 + override def compressedSize: Int = if (overflow) Int.MaxValue else dictionarySize + count * 2 } class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) @@ -284,7 +287,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { columnType.setField(row, ordinal, dictionary(buffer.getShort())) } - override def hasNext = buffer.hasRemaining + override def hasNext: Boolean = buffer.hasRemaining } } @@ -293,15 +296,16 @@ private[sql] case object BooleanBitSet extends CompressionScheme { val BITS_PER_LONG = 64 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + : compression.Decoder[T] = { new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new this.Encoder).asInstanceOf[compression.Encoder[T]] } - override def supports(columnType: ColumnType[_, _]) = columnType == BOOLEAN + override def supports(columnType: ColumnType[_, _]): Boolean = columnType == BOOLEAN class Encoder extends compression.Encoder[BooleanType.type] { private var _uncompressedSize = 0 @@ -310,7 +314,7 @@ private[sql] case object BooleanBitSet extends CompressionScheme { _uncompressedSize += BOOLEAN.defaultSize } - override def compress(from: ByteBuffer, to: ByteBuffer) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { to.putInt(BooleanBitSet.typeId) // Total element count (1 byte per Boolean value) .putInt(from.remaining) @@ -347,9 +351,9 @@ private[sql] case object BooleanBitSet extends CompressionScheme { to } - override def uncompressedSize = _uncompressedSize + override def uncompressedSize: Int = _uncompressedSize - override def compressedSize = { + override def compressedSize: Int = { val extra = if (_uncompressedSize % BITS_PER_LONG == 0) 0 else 1 (_uncompressedSize / BITS_PER_LONG + extra) * 8 + 4 } @@ -380,22 +384,23 @@ private[sql] case object BooleanBitSet extends CompressionScheme { private[sql] case object IntDelta extends CompressionScheme { override def typeId: Int = 4 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + : compression.Decoder[T] = { new Decoder(buffer, INT).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } - override def supports(columnType: ColumnType[_, _]) = columnType == INT + override def supports(columnType: ColumnType[_, _]): Boolean = columnType == INT class Encoder extends compression.Encoder[IntegerType.type] { protected var _compressedSize: Int = 0 protected var _uncompressedSize: Int = 0 - override def compressedSize = _compressedSize - override def uncompressedSize = _uncompressedSize + override def compressedSize: Int = _compressedSize + override def uncompressedSize: Int = _uncompressedSize private var prevValue: Int = _ @@ -459,22 +464,23 @@ private[sql] case object IntDelta extends CompressionScheme { private[sql] case object LongDelta extends CompressionScheme { override def typeId: Int = 5 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + : compression.Decoder[T] = { new Decoder(buffer, LONG).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } - override def supports(columnType: ColumnType[_, _]) = columnType == LONG + override def supports(columnType: ColumnType[_, _]): Boolean = columnType == LONG class Encoder extends compression.Encoder[LongType.type] { protected var _compressedSize: Int = 0 protected var _uncompressedSize: Int = 0 - override def compressedSize = _compressedSize - override def uncompressedSize = _uncompressedSize + override def compressedSize: Int = _compressedSize + override def uncompressedSize: Int = _uncompressedSize private var prevValue: Long = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index ad44a01d0e164..18b1ba4c5c4b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -21,6 +21,7 @@ import java.util.HashMap import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -45,7 +46,7 @@ case class Aggregate( child: SparkPlan) extends UnaryNode { - override def requiredChildDistribution = + override def requiredChildDistribution: List[Distribution] = { if (partial) { UnspecifiedDistribution :: Nil } else { @@ -55,8 +56,9 @@ case class Aggregate( ClusteredDistribution(groupingExpressions) :: Nil } } + } - override def output = aggregateExpressions.map(_.toAttribute) + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) /** * An aggregate that needs to be computed for each row in a group. @@ -119,7 +121,7 @@ case class Aggregate( } } - override def execute() = attachTree(this, "execute") { + override def execute(): RDD[Row] = attachTree(this, "execute") { if (groupingExpressions.isEmpty) { child.execute().mapPartitions { iter => val buffer = newAggregateBuffer() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 7c0b72aab448e..437408d30bfd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.catalyst.expressions import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} -import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.RowOrdering +import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair @@ -34,9 +35,9 @@ import org.apache.spark.util.MutablePair @DeveloperApi case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { - override def outputPartitioning = newPartitioning + override def outputPartitioning: Partitioning = newPartitioning - override def output = child.output + override def output: Seq[Attribute] = child.output /** We must copy rows when sort based shuffle is on */ protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] @@ -44,7 +45,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una private val bypassMergeThreshold = child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) - override def execute() = attachTree(this , "execute") { + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. @@ -123,13 +124,13 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una */ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. - def numPartitions = sqlContext.conf.numShufflePartitions + def numPartitions: Int = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => // Check if every child's outputPartitioning satisfies the corresponding // required data distribution. - def meetsRequirements = + def meetsRequirements: Boolean = !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => val valid = child.outputPartitioning.satisfies(required) @@ -147,7 +148,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl // datasets are both clustered by "a", but these two outputPartitionings are not // compatible. // TODO: ASSUMES TRANSITIVITY? - def compatible = + def compatible: Boolean = !operator.children .map(_.outputPartitioning) .sliding(2) @@ -158,7 +159,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl // Check if the partitioning we want to ensure is the same as the child's output // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan) = + def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan): SparkPlan = if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child if (meetsRequirements && compatible) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 248dc1512b4d3..d8955725e59b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -26,6 +26,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType +import scala.collection.immutable + /** * :: DeveloperApi :: */ @@ -58,17 +60,17 @@ object RDDConversions { case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) extends LogicalPlan with MultiInstanceRelation { - override def children = Nil + override def children: Seq[LogicalPlan] = Nil - override def newInstance() = + override def newInstance(): LogicalRDD.this.type = LogicalRDD(output.map(_.newInstance()), rdd)(sqlContext).asInstanceOf[this.type] - override def sameResult(plan: LogicalPlan) = plan match { + override def sameResult(plan: LogicalPlan): Boolean = plan match { case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id case _ => false } - @transient override lazy val statistics = Statistics( + @transient override lazy val statistics: Statistics = Statistics( // TODO: Instead of returning a default value here, find a way to return a meaningful size // estimate for RDDs. See PR 1238 for more discussions. sizeInBytes = BigInt(sqlContext.conf.defaultSizeInBytes) @@ -77,24 +79,24 @@ case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLCont /** Physical plan node for scanning data from an RDD. */ case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { - override def execute() = rdd + override def execute(): RDD[Row] = rdd } /** Logical plan node for scanning data from a local collection. */ case class LogicalLocalTable(output: Seq[Attribute], rows: Seq[Row])(sqlContext: SQLContext) extends LogicalPlan with MultiInstanceRelation { - override def children = Nil + override def children: Seq[LogicalPlan] = Nil - override def newInstance() = + override def newInstance(): this.type = LogicalLocalTable(output.map(_.newInstance()), rows)(sqlContext).asInstanceOf[this.type] - override def sameResult(plan: LogicalPlan) = plan match { + override def sameResult(plan: LogicalPlan): Boolean = plan match { case LogicalRDD(_, otherRDD) => rows == rows case _ => false } - @transient override lazy val statistics = Statistics( + @transient override lazy val statistics: Statistics = Statistics( // TODO: Improve the statistics estimation. // This is made small enough so it can be broadcasted. sizeInBytes = sqlContext.conf.autoBroadcastJoinThreshold - 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala index 95172420608f9..575849481faad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ @@ -42,7 +43,7 @@ case class Expand( // as UNKNOWN partitioning override def outputPartitioning: Partitioning = UnknownPartitioning(0) - override def execute() = attachTree(this, "execute") { + override def execute(): RDD[Row] = attachTree(this, "execute") { child.execute().mapPartitions { iter => // TODO Move out projection objects creation and transfer to // workers via closure. However we can't assume the Projection @@ -55,7 +56,7 @@ case class Expand( private[this] var idx = -1 // -1 means the initial state private[this] var input: Row = _ - override final def hasNext = (-1 < idx && idx < groups.length) || iter.hasNext + override final def hasNext: Boolean = (-1 < idx && idx < groups.length) || iter.hasNext override final def next(): Row = { if (idx <= 0) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 38877c28de3a8..12271048bb39c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ /** @@ -54,7 +55,7 @@ case class Generate( val boundGenerator = BindReferences.bindReference(generator, child.output) - override def execute() = { + override def execute(): RDD[Row] = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 4abe26fe4afc6..89682d25ca7dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -49,7 +50,7 @@ case class GeneratedAggregate( child: SparkPlan) extends UnaryNode { - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[Distribution] = if (partial) { UnspecifiedDistribution :: Nil } else { @@ -60,9 +61,9 @@ case class GeneratedAggregate( } } - override def output = aggregateExpressions.map(_.toAttribute) + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) - override def execute() = { + override def execute(): RDD[Row] = { val aggregatesToCompute = aggregateExpressions.flatMap { a => a.collect { case agg: AggregateExpression => agg} } @@ -271,9 +272,9 @@ case class GeneratedAggregate( private[this] val resultIterator = buffers.entrySet.iterator() private[this] val resultProjection = resultProjectionBuilder() - def hasNext = resultIterator.hasNext + def hasNext: Boolean = resultIterator.hasNext - def next() = { + def next(): Row = { val currentGroup = resultIterator.next() resultProjection(joinedRow(currentGroup.getKey, currentGroup.getValue)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala index d3a18b37d52b9..5bd699a2fa949 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Attribute @@ -29,11 +30,11 @@ case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNo private lazy val rdd = sqlContext.sparkContext.parallelize(rows) - override def execute() = rdd + override def execute(): RDD[Row] = rdd - override def executeCollect() = + override def executeCollect(): Array[Row] = rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray - override def executeTake(limit: Int) = + override def executeTake(limit: Int): Array[Row] = rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 052766c20abc2..d239637cd4b4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -67,6 +67,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! + /** Specifies any partition requirements on the input data for this operator. */ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 30564e14fa896..c4534fd5f67e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -74,7 +74,7 @@ private[execution] class KryoResourcePool(size: Int) new KryoSerializer(sparkConf) } - def newInstance() = ser.newInstance() + def newInstance(): SerializerInstance = ser.newInstance() } private[sql] object SparkSqlSerializer { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5281c7502556a..2b581152e5f77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -154,7 +154,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => Nil } - def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { + def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && @@ -162,7 +162,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => true } - def allAggregates(exprs: Seq[Expression]) = + def allAggregates(exprs: Seq[Expression]): Seq[AggregateExpression] = exprs.flatMap(_.collect { case a: AggregateExpression => a }) } @@ -257,7 +257,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { - def numPartitions = self.numPartitions + def numPartitions: Int = self.numPartitions def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case r: RunnableCommand => ExecutedCommand(r) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 710268584cff1..20c9bc3e75542 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -24,7 +24,7 @@ import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.ExternalSorter @@ -33,11 +33,11 @@ import org.apache.spark.util.collection.ExternalSorter */ @DeveloperApi case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { - override def output = projectList.map(_.toAttribute) + override def output: Seq[Attribute] = projectList.map(_.toAttribute) @transient lazy val buildProjection = newMutableProjection(projectList, child.output) - override def execute() = child.execute().mapPartitions { iter => + override def execute(): RDD[Row] = child.execute().mapPartitions { iter => val resuableProjection = buildProjection() iter.map(resuableProjection) } @@ -48,11 +48,11 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends */ @DeveloperApi case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output - @transient lazy val conditionEvaluator = newPredicate(condition, child.output) + @transient lazy val conditionEvaluator: (Row) => Boolean = newPredicate(condition, child.output) - override def execute() = child.execute().mapPartitions { iter => + override def execute(): RDD[Row] = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } } @@ -64,10 +64,12 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: SparkPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output // TODO: How to pick seed? - override def execute() = child.execute().map(_.copy()).sample(withReplacement, fraction, seed) + override def execute(): RDD[Row] = { + child.execute().map(_.copy()).sample(withReplacement, fraction, seed) + } } /** @@ -76,8 +78,8 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: @DeveloperApi case class Union(children: Seq[SparkPlan]) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes - override def output = children.head.output - override def execute() = sparkContext.union(children.map(_.execute())) + override def output: Seq[Attribute] = children.head.output + override def execute(): RDD[Row] = sparkContext.union(children.map(_.execute())) } /** @@ -97,12 +99,12 @@ case class Limit(limit: Int, child: SparkPlan) /** We must copy rows when sort based shuffle is on */ private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - override def output = child.output - override def outputPartitioning = SinglePartition + override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[Row] = child.executeTake(limit) - override def execute() = { + override def execute(): RDD[Row] = { val rdd: RDD[_ <: Product2[Boolean, Row]] = if (sortBasedShuffleOn) { child.execute().mapPartitions { iter => iter.take(limit).map(row => (false, row.copy())) @@ -129,20 +131,21 @@ case class Limit(limit: Int, child: SparkPlan) @DeveloperApi case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { - override def output = child.output - override def outputPartitioning = SinglePartition + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = SinglePartition - val ord = new RowOrdering(sortOrder, child.output) + private val ord: RowOrdering = new RowOrdering(sortOrder, child.output) - private def collectData() = child.execute().map(_.copy()).takeOrdered(limit)(ord) + private def collectData(): Array[Row] = child.execute().map(_.copy()).takeOrdered(limit)(ord) // TODO: Is this copying for no reason? - override def executeCollect() = + override def executeCollect(): Array[Row] = collectData().map(ScalaReflection.convertRowToScala(_, this.schema)) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - override def execute() = sparkContext.makeRDD(collectData(), 1) + override def execute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1) } /** @@ -157,17 +160,17 @@ case class Sort( global: Boolean, child: SparkPlan) extends UnaryNode { - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil - override def execute() = attachTree(this, "sort") { + override def execute(): RDD[Row] = attachTree(this, "sort") { child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) iterator.map(_.copy()).toArray.sorted(ordering).iterator }, preservesPartitioning = true) } - override def output = child.output + override def output: Seq[Attribute] = child.output } /** @@ -182,10 +185,11 @@ case class ExternalSort( global: Boolean, child: SparkPlan) extends UnaryNode { - override def requiredChildDistribution = + + override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil - override def execute() = attachTree(this, "sort") { + override def execute(): RDD[Row] = attachTree(this, "sort") { child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) @@ -194,7 +198,7 @@ case class ExternalSort( }, preservesPartitioning = true) } - override def output = child.output + override def output: Seq[Attribute] = child.output } /** @@ -206,12 +210,12 @@ case class ExternalSort( */ @DeveloperApi case class Distinct(partial: Boolean, child: SparkPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[Distribution] = if (partial) UnspecifiedDistribution :: Nil else ClusteredDistribution(child.output) :: Nil - override def execute() = { + override def execute(): RDD[Row] = { child.execute().mapPartitions { iter => val hashSet = new scala.collection.mutable.HashSet[Row]() @@ -236,9 +240,9 @@ case class Distinct(partial: Boolean, child: SparkPlan) extends UnaryNode { */ @DeveloperApi case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { - override def output = left.output + override def output: Seq[Attribute] = left.output - override def execute() = { + override def execute(): RDD[Row] = { left.execute().map(_.copy()).subtract(right.execute().map(_.copy())) } } @@ -250,9 +254,9 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { */ @DeveloperApi case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { - override def output = children.head.output + override def output: Seq[Attribute] = children.head.output - override def execute() = { + override def execute(): RDD[Row] = { left.execute().map(_.copy()).intersection(right.execute().map(_.copy())) } } @@ -265,6 +269,7 @@ case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { */ @DeveloperApi case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { - def children = child :: Nil - def execute() = child.execute() + def children: Seq[SparkPlan] = child :: Nil + + def execute(): RDD[Row] = child.execute() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index a11232142d0fb..fad7a281dc1e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import scala.collection.mutable.ArrayBuffer /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -54,9 +53,9 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { */ protected[sql] lazy val sideEffectResult: Seq[Row] = cmd.run(sqlContext) - override def output = cmd.output + override def output: Seq[Attribute] = cmd.output - override def children = Nil + override def children: Seq[SparkPlan] = Nil override def executeCollect(): Array[Row] = sideEffectResult.toArray @@ -71,9 +70,10 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { @DeveloperApi case class SetCommand( kv: Option[(String, Option[String])], - override val output: Seq[Attribute]) extends RunnableCommand with Logging { + override val output: Seq[Attribute]) + extends RunnableCommand with Logging { - override def run(sqlContext: SQLContext) = kv match { + override def run(sqlContext: SQLContext): Seq[Row] = kv match { // Configures the deprecated "mapred.reduce.tasks" property. case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => logWarning( @@ -119,10 +119,11 @@ case class ExplainCommand( logicalPlan: LogicalPlan, override val output: Seq[Attribute] = Seq(AttributeReference("plan", StringType, nullable = false)()), - extended: Boolean = false) extends RunnableCommand { + extended: Boolean = false) + extends RunnableCommand { // Run through the optimizer to generate the physical plan. - override def run(sqlContext: SQLContext) = try { + override def run(sqlContext: SQLContext): Seq[Row] = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. val queryExecution = sqlContext.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString @@ -140,9 +141,10 @@ case class ExplainCommand( case class CacheTableCommand( tableName: String, plan: Option[LogicalPlan], - isLazy: Boolean) extends RunnableCommand { + isLazy: Boolean) + extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { plan.foreach { logicalPlan => sqlContext.registerDataFrameAsTable(DataFrame(sqlContext, logicalPlan), tableName) } @@ -166,7 +168,7 @@ case class CacheTableCommand( @DeveloperApi case class UncacheTableCommand(tableName: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.table(tableName).unpersist(blocking = false) Seq.empty[Row] } @@ -181,7 +183,7 @@ case class UncacheTableCommand(tableName: String) extends RunnableCommand { @DeveloperApi case object ClearCacheCommand extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.clearCache() Seq.empty[Row] } @@ -196,9 +198,10 @@ case object ClearCacheCommand extends RunnableCommand { case class DescribeCommand( child: SparkPlan, override val output: Seq[Attribute], - isExtended: Boolean) extends RunnableCommand { + isExtended: Boolean) + extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { child.schema.fields.map { field => val cmtKey = "comment" val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else "" @@ -220,7 +223,7 @@ case class DescribeCommand( case class ShowTablesCommand(databaseName: Option[String]) extends RunnableCommand { // The result of SHOW TABLES has two columns, tableName and isTemporary. - override val output = { + override val output: Seq[Attribute] = { val schema = StructType( StructField("tableName", StringType, false) :: StructField("isTemporary", BooleanType, false) :: Nil) @@ -228,7 +231,7 @@ case class ShowTablesCommand(databaseName: Option[String]) extends RunnableComma schema.toAttributes } - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { // Since we need to return a Seq of rows, we will call getTables directly // instead of calling tables in sqlContext. val rows = sqlContext.catalog.getTables(databaseName).map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index ffe388cfa9532..e916e68e58b5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.execution +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute + import scala.collection.mutable.HashSet -import org.apache.spark.{AccumulatorParam, Accumulator, SparkContext} +import org.apache.spark.{AccumulatorParam, Accumulator} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.SparkContext._ import org.apache.spark.sql.{SQLConf, SQLContext, DataFrame, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.types._ @@ -43,7 +45,7 @@ package object debug { * Augments [[SQLContext]] with debug methods. */ implicit class DebugSQLContext(sqlContext: SQLContext) { - def debug() = { + def debug(): Unit = { sqlContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "false") } } @@ -88,7 +90,7 @@ package object debug { } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { - def output = child.output + def output: Seq[Attribute] = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { def zero(initialValue: HashSet[String]): HashSet[String] = { @@ -109,10 +111,10 @@ package object debug { */ case class ColumnMetrics( elementTypes: Accumulator[HashSet[String]] = sparkContext.accumulator(HashSet.empty)) - val tupleCount = sparkContext.accumulator[Int](0) + val tupleCount: Accumulator[Int] = sparkContext.accumulator[Int](0) - val numColumns = child.output.size - val columnStats = Array.fill(child.output.size)(new ColumnMetrics()) + val numColumns: Int = child.output.size + val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { println(s"== ${child.simpleString} ==") @@ -123,11 +125,11 @@ package object debug { } } - def execute() = { + def execute(): RDD[Row] = { child.execute().mapPartitions { iter => new Iterator[Row] { - def hasNext = iter.hasNext - def next() = { + def hasNext: Boolean = iter.hasNext + def next(): Row = { val currentRow = iter.next() tupleCount += 1 var i = 0 @@ -180,18 +182,18 @@ package object debug { private[sql] case class TypeCheck(child: SparkPlan) extends SparkPlan { import TypeCheck._ - override def nodeName = "" + override def nodeName: String = "" /* Only required when defining this class in a REPL. override def makeCopy(args: Array[Object]): this.type = TypeCheck(args(0).asInstanceOf[SparkPlan]).asInstanceOf[this.type] */ - def output = child.output + def output: Seq[Attribute] = child.output - def children = child :: Nil + def children: List[SparkPlan] = child :: Nil - def execute() = { + def execute(): RDD[Row] = { child.execute().map { row => try typeCheck(row, child.schema) catch { case e: Exception => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 2dd22c020ef12..926f5e6c137ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution.joins +import org.apache.spark.rdd.RDD + import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Row, Expression} -import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} /** @@ -42,7 +44,7 @@ case class BroadcastHashJoin( right: SparkPlan) extends BinaryNode with HashJoin { - val timeout = { + val timeout: Duration = { val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { Duration.Inf @@ -53,7 +55,7 @@ case class BroadcastHashJoin( override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[Distribution] = UnspecifiedDistribution :: UnspecifiedDistribution :: Nil @transient @@ -64,7 +66,7 @@ case class BroadcastHashJoin( sparkContext.broadcast(hashed) } - override def execute() = { + override def execute(): RDD[Row] = { val broadcastRelation = Await.result(broadcastFuture, timeout) streamedPlan.execute().mapPartitions { streamedIter => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala index 2ab064fd0151e..3ef1e0d7fbdd4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.{Expression, Row} -import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} /** @@ -34,11 +34,11 @@ case class BroadcastLeftSemiJoinHash( left: SparkPlan, right: SparkPlan) extends BinaryNode with HashJoin { - override val buildSide = BuildRight + override val buildSide: BuildSide = BuildRight - override def output = left.output + override def output: Seq[Attribute] = left.output - override def execute() = { + override def execute(): RDD[Row] = { val buildIter= buildPlan.execute().map(_.copy()).collect().toIterator val hashSet = new java.util.HashSet[Row]() var currentRow: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 36aad13778bd2..83b1a83765153 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} @@ -44,7 +45,7 @@ case class BroadcastNestedLoopJoin( override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def output = { + override def output: Seq[Attribute] = { joinType match { case LeftOuter => left.output ++ right.output.map(_.withNullability(true)) @@ -63,7 +64,7 @@ case class BroadcastNestedLoopJoin( .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) - override def execute() = { + override def execute(): RDD[Row] = { val broadcastedRelation = sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala index 76c14c02aab34..1cbc98354d673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} /** @@ -26,9 +28,9 @@ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} */ @DeveloperApi case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { - override def output = left.output ++ right.output + override def output: Seq[Attribute] = left.output ++ right.output - override def execute() = { + override def execute(): RDD[Row] = { val leftResults = left.execute().map(_.copy()) val rightResults = right.execute().map(_.copy()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 4012d757d5f9a..851de1685509a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -41,7 +41,7 @@ trait HashJoin { case BuildRight => (rightKeys, leftKeys) } - override def output = left.output ++ right.output + override def output: Seq[Attribute] = left.output ++ right.output @transient protected lazy val buildSideKeyGenerator: Projection = newProjection(buildKeys, buildPlan.output) @@ -65,7 +65,7 @@ trait HashJoin { (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || (streamIter.hasNext && fetchNext()) - override final def next() = { + override final def next(): Row = { val ret = buildSide match { case BuildRight => joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) case BuildLeft => joinRow(currentHashMatches(currentMatchPosition), currentStreamedRow) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala index 59ef904272545..a396c0f5d56ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.joins import java.util.{HashMap => JavaHashMap} +import org.apache.spark.rdd.RDD + import scala.collection.JavaConversions._ import org.apache.spark.annotation.DeveloperApi @@ -49,10 +51,10 @@ case class HashOuterJoin( case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") } - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[ClusteredDistribution] = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - override def output = { + override def output: Seq[Attribute] = { joinType match { case LeftOuter => left.output ++ right.output.map(_.withNullability(true)) @@ -78,12 +80,12 @@ case class HashOuterJoin( private[this] def leftOuterIterator( key: Row, joinedRow: JoinedRow, rightIter: Iterable[Row]): Iterator[Row] = { - val ret: Iterable[Row] = ( + val ret: Iterable[Row] = { if (!key.anyNull) { val temp = rightIter.collect { - case r if (boundCondition(joinedRow.withRight(r))) => joinedRow.copy + case r if boundCondition(joinedRow.withRight(r)) => joinedRow.copy() } - if (temp.size == 0) { + if (temp.size == 0) { joinedRow.withRight(rightNullRow).copy :: Nil } else { temp @@ -91,19 +93,19 @@ case class HashOuterJoin( } else { joinedRow.withRight(rightNullRow).copy :: Nil } - ) + } ret.iterator } private[this] def rightOuterIterator( key: Row, leftIter: Iterable[Row], joinedRow: JoinedRow): Iterator[Row] = { - val ret: Iterable[Row] = ( + val ret: Iterable[Row] = { if (!key.anyNull) { val temp = leftIter.collect { - case l if (boundCondition(joinedRow.withLeft(l))) => joinedRow.copy + case l if boundCondition(joinedRow.withLeft(l)) => joinedRow.copy } - if (temp.size == 0) { + if (temp.size == 0) { joinedRow.withLeft(leftNullRow).copy :: Nil } else { temp @@ -111,7 +113,7 @@ case class HashOuterJoin( } else { joinedRow.withLeft(leftNullRow).copy :: Nil } - ) + } ret.iterator } @@ -130,12 +132,12 @@ case class HashOuterJoin( // 1. For those matched (satisfy the join condition) records with both sides filled, // append them directly - case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { + case (r, idx) if boundCondition(joinedRow.withRight(r)) => matched = true // if the row satisfy the join condition, add its index into the matched set rightMatchedSet.add(idx) - joinedRow.copy - } + joinedRow.copy() + } ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // 2. For those unmatched records in left, append additional records with empty right. @@ -143,22 +145,21 @@ case class HashOuterJoin( // as we don't know whether we need to append it until finish iterating all // of the records in right side. // If we didn't get any proper row, then append a single row with empty right. - joinedRow.withRight(rightNullRow).copy + joinedRow.withRight(rightNullRow).copy() }) } ++ rightIter.zipWithIndex.collect { // 3. For those unmatched records in right, append additional records with empty left. // Re-visiting the records in right, and append additional row with empty left, if its not // in the matched set. - case (r, idx) if (!rightMatchedSet.contains(idx)) => { - joinedRow(leftNullRow, r).copy - } + case (r, idx) if !rightMatchedSet.contains(idx) => + joinedRow(leftNullRow, r).copy() } } else { leftIter.iterator.map[Row] { l => - joinedRow(l, rightNullRow).copy + joinedRow(l, rightNullRow).copy() } ++ rightIter.iterator.map[Row] { r => - joinedRow(leftNullRow, r).copy + joinedRow(leftNullRow, r).copy() } } } @@ -182,13 +183,13 @@ case class HashOuterJoin( hashTable } - override def execute() = { + override def execute(): RDD[Row] = { val joinedRow = new JoinedRow() left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => // TODO this probably can be replaced by external sort (sort merged join?) joinType match { - case LeftOuter => { + case LeftOuter => val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) val keyGenerator = newProjection(leftKeys, left.output) leftIter.flatMap( currentRow => { @@ -196,8 +197,8 @@ case class HashOuterJoin( joinedRow.withLeft(currentRow) leftOuterIterator(rowKey, joinedRow, rightHashTable.getOrElse(rowKey, EMPTY_LIST)) }) - } - case RightOuter => { + + case RightOuter => val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) val keyGenerator = newProjection(rightKeys, right.output) rightIter.flatMap ( currentRow => { @@ -205,8 +206,8 @@ case class HashOuterJoin( joinedRow.withRight(currentRow) rightOuterIterator(rowKey, leftHashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) }) - } - case FullOuter => { + + case FullOuter => val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => @@ -214,7 +215,7 @@ case class HashOuterJoin( leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST), joinedRow) } - } + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 38b8993b03f82..2fa1cf5add3b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -38,7 +38,7 @@ private[joins] sealed trait HashedRelation { private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) extends HashedRelation with Serializable { - override def get(key: Row) = hashTable.get(key) + override def get(key: Row): CompactBuffer[Row] = hashTable.get(key) } @@ -49,7 +49,7 @@ private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, Com private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) extends HashedRelation with Serializable { - override def get(key: Row) = { + override def get(key: Row): CompactBuffer[Row] = { val v = hashTable.get(key) if (v eq null) null else CompactBuffer(v) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 60003d1900d85..1fa7e7bd0406c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} @@ -35,12 +36,13 @@ case class LeftSemiJoinBNL( override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def output = left.output + override def output: Seq[Attribute] = left.output /** The Streamed Relation */ - override def left = streamed + override def left: SparkPlan = streamed + /** The Broadcast relation */ - override def right = broadcast + override def right: SparkPlan = broadcast @transient private lazy val boundCondition = InterpretedPredicate( @@ -48,7 +50,7 @@ case class LeftSemiJoinBNL( .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) - override def execute() = { + override def execute(): RDD[Row] = { val broadcastedRelation = sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala index ea7babf3be948..a04f2a63b5a55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.{Expression, Row} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} @@ -34,14 +35,14 @@ case class LeftSemiJoinHash( left: SparkPlan, right: SparkPlan) extends BinaryNode with HashJoin { - override val buildSide = BuildRight + override val buildSide: BuildSide = BuildRight - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[ClusteredDistribution] = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - override def output = left.output + override def output: Seq[Attribute] = left.output - override def execute() = { + override def execute(): RDD[Row] = { buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => val hashSet = new java.util.HashSet[Row]() var currentRow: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 418c1c23e5546..a6cd8337c1c3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} @@ -38,10 +40,10 @@ case class ShuffledHashJoin( override def outputPartitioning: Partitioning = left.outputPartitioning - override def requiredChildDistribution = + override def requiredChildDistribution: Seq[ClusteredDistribution] = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - override def execute() = { + override def execute(): RDD[Row] = { buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => val hashed = HashedRelation(buildIter, buildSideKeyGenerator) hashJoin(streamIter, hashed) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 33632b8e82ff9..5b308d88d4cdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} +import org.apache.spark.rdd.RDD + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -48,11 +50,13 @@ private[spark] case class PythonUDF( dataType: DataType, children: Seq[Expression]) extends Expression with SparkLogging { - override def toString = s"PythonUDF#$name(${children.mkString(",")})" + override def toString: String = s"PythonUDF#$name(${children.mkString(",")})" def nullable: Boolean = true - override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") + override def eval(input: Row): PythonUDF.this.EvaluatedType = { + sys.error("PythonUDFs can not be directly evaluated.") + } } /** @@ -63,7 +67,7 @@ private[spark] case class PythonUDF( * multiple child operators. */ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan) = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Skip EvaluatePython nodes. case p: EvaluatePython => p @@ -107,7 +111,7 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { } object EvaluatePython { - def apply(udf: PythonUDF, child: LogicalPlan) = + def apply(udf: PythonUDF, child: LogicalPlan): EvaluatePython = new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) /** @@ -205,10 +209,10 @@ case class EvaluatePython( resultAttribute: AttributeReference) extends logical.UnaryNode { - def output = child.output :+ resultAttribute + def output: Seq[Attribute] = child.output :+ resultAttribute // References should not include the produced attribute. - override def references = udf.references + override def references: AttributeSet = udf.references } /** @@ -219,9 +223,10 @@ case class EvaluatePython( @DeveloperApi case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) extends SparkPlan { - def children = child :: Nil - def execute() = { + def children: Seq[SparkPlan] = child :: Nil + + def execute(): RDD[Row] = { // TODO: Clean up after ourselves? val childResults = child.execute().map(_.copy()).cache() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 87304ce2496b4..3266b972128ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -306,7 +306,8 @@ private[sql] class JDBCRDD( /** * Runs the SQL query against the JDBC driver. */ - override def compute(thePart: Partition, context: TaskContext) = new Iterator[Row] { + override def compute(thePart: Partition, context: TaskContext): Iterator[Row] = new Iterator[Row] + { var closed = false var finished = false var gotNext = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 1778d39c42e2b..df687e6da9bea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.jdbc +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.types.StructType + import scala.collection.mutable.ArrayBuffer import java.sql.DriverManager @@ -122,9 +126,9 @@ private[sql] case class JDBCRelation( extends BaseRelation with PrunedFilteredScan { - override val schema = JDBCRDD.resolveTable(url, table) + override val schema: StructType = JDBCRDD.resolveTable(url, table) - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { val driver: String = DriverManager.getDriver(url).getClass.getCanonicalName JDBCRDD.scanTable( sqlContext.sparkContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index b645199ded18c..b1e363d02edfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ @@ -104,10 +106,10 @@ private[sql] case class JSONRelation( samplingRatio, sqlContext.conf.columnNameOfCorruptRecord))) - override def buildScan() = + override def buildScan(): RDD[Row] = JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.conf.columnNameOfCorruptRecord) - override def insert(data: DataFrame, overwrite: Boolean) = { + override def insert(data: DataFrame, overwrite: Boolean): Unit = { val filesystemPath = new Path(path) val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 7d62f3728f036..f898e4b37a56b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -488,7 +488,7 @@ private[parquet] object CatalystTimestampConverter { // Also we use NanoTime and Int96Values from parquet-examples. // We utilize jodd to convert between NanoTime and Timestamp val parquetTsCalendar = new ThreadLocal[Calendar] - def getCalendar = { + def getCalendar: Calendar = { // this is a cache for the calendar instance. if (parquetTsCalendar.get == null) { parquetTsCalendar.set(Calendar.getInstance(TimeZone.getTimeZone("GMT"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index fd161bae128ad..fcb9513ab66f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -71,16 +71,22 @@ private[sql] case class ParquetRelation( sqlContext.conf.isParquetINT96AsTimestamp) lazy val attributeMap = AttributeMap(output.map(o => o -> o)) - override def newInstance() = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] + override def newInstance(): this.type = { + ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] + } // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case p: ParquetRelation => p.path == path && p.output == output case _ => false } + override def hashCode: Int = { + com.google.common.base.Objects.hashCode(path, output) + } + // TODO: Use data from the footers. override lazy val statistics = Statistics(sizeInBytes = sqlContext.conf.defaultSizeInBytes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 62813a981e685..5130d8ad5e003 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -152,8 +152,8 @@ private[sql] case class ParquetTableScan( if (primitiveRow) { new Iterator[Row] { - def hasNext = iter.hasNext - def next() = { + def hasNext: Boolean = iter.hasNext + def next(): Row = { // We are using CatalystPrimitiveRowConverter and it returns a SpecificMutableRow. val row = iter.next()._2.asInstanceOf[SpecificMutableRow] @@ -171,8 +171,8 @@ private[sql] case class ParquetTableScan( // Create a mutable row since we need to fill in values from partition columns. val mutableRow = new GenericMutableRow(outputSize) new Iterator[Row] { - def hasNext = iter.hasNext - def next() = { + def hasNext: Boolean = iter.hasNext + def next(): Row = { // We are using CatalystGroupConverter and it returns a GenericRow. // Since GenericRow is not mutable, we just cast it to a Row. val row = iter.next()._2.asInstanceOf[Row] @@ -255,7 +255,7 @@ private[sql] case class InsertIntoParquetTable( /** * Inserts all rows into the Parquet file. */ - override def execute() = { + override def execute(): RDD[Row] = { // TODO: currently we do not check whether the "schema"s are compatible // That means if one first creates a table and then INSERTs data with // and incompatible schema the execution will fail. It would be nice @@ -302,7 +302,7 @@ private[sql] case class InsertIntoParquetTable( childRdd } - override def output = child.output + override def output: Seq[Attribute] = child.output /** * Stores the given Row RDD as a Hadoop file. 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 c38b6e8c61d8a..10b8876c1d31c 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 @@ -181,7 +181,7 @@ private[sql] case class ParquetRelation2( private val defaultPartitionName = parameters.getOrElse( ParquetRelation2.DEFAULT_PARTITION_NAME, "__HIVE_DEFAULT_PARTITION__") - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case relation: ParquetRelation2 => // If schema merging is required, we don't compare the actual schemas since they may evolve. val schemaEquality = if (shouldMergeSchemas) { @@ -198,6 +198,23 @@ private[sql] case class ParquetRelation2( case _ => false } + override def hashCode(): Int = { + if (shouldMergeSchemas) { + com.google.common.base.Objects.hashCode( + shouldMergeSchemas: java.lang.Boolean, + paths.toSet, + maybeMetastoreSchema, + maybePartitionSpec) + } else { + com.google.common.base.Objects.hashCode( + shouldMergeSchemas: java.lang.Boolean, + schema, + paths.toSet, + maybeMetastoreSchema, + maybePartitionSpec) + } + } + private[sql] def sparkContext = sqlContext.sparkContext private class MetadataCache { @@ -370,19 +387,19 @@ private[sql] case class ParquetRelation2( @transient private val metadataCache = new MetadataCache metadataCache.refresh() - def partitionSpec = metadataCache.partitionSpec + def partitionSpec: PartitionSpec = metadataCache.partitionSpec - def partitionColumns = metadataCache.partitionSpec.partitionColumns + def partitionColumns: StructType = metadataCache.partitionSpec.partitionColumns - def partitions = metadataCache.partitionSpec.partitions + def partitions: Seq[Partition] = metadataCache.partitionSpec.partitions - def isPartitioned = partitionColumns.nonEmpty + def isPartitioned: Boolean = partitionColumns.nonEmpty private def partitionKeysIncludedInDataSchema = metadataCache.partitionKeysIncludedInParquetSchema private def parquetSchema = metadataCache.parquetSchema - override def schema = metadataCache.schema + override def schema: StructType = metadataCache.schema private def isSummaryFile(file: Path): Boolean = { file.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE || @@ -425,8 +442,10 @@ private[sql] case class ParquetRelation2( .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) if (isPartitioned) { - def percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 - logInfo(s"Reading $percentRead% of partitions") + logInfo { + val percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 + s"Reading $percentRead% of partitions" + } } val requiredColumns = output.map(_.name) @@ -703,7 +722,7 @@ private[sql] object ParquetRelation2 { private[parquet] def mergeMetastoreParquetSchema( metastoreSchema: StructType, parquetSchema: StructType): StructType = { - def schemaConflictMessage = + def schemaConflictMessage: String = s"""Converting Hive Metastore Parquet, but detected conflicting schemas. Metastore schema: |${metastoreSchema.prettyJson} | diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala index e24475292ceaf..70bcca7526aae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala @@ -26,7 +26,7 @@ private[parquet] class NanoTime extends Serializable { private var julianDay = 0 private var timeOfDayNanos = 0L - def set(julianDay: Int, timeOfDayNanos: Long) = { + def set(julianDay: Int, timeOfDayNanos: Long): this.type = { this.julianDay = julianDay this.timeOfDayNanos = timeOfDayNanos this @@ -45,11 +45,11 @@ private[parquet] class NanoTime extends Serializable { Binary.fromByteBuffer(buf) } - def writeValue(recordConsumer: RecordConsumer) { + def writeValue(recordConsumer: RecordConsumer): Unit = { recordConsumer.addBinary(toBinary) } - override def toString = + override def toString: String = "NanoTime{julianDay=" + julianDay + ", timeOfDayNanos=" + timeOfDayNanos + "}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala index 12b59ba20bb10..f374abffdd505 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala @@ -30,24 +30,28 @@ private[sql] case class LogicalRelation(relation: BaseRelation) override val output: Seq[AttributeReference] = relation.schema.toAttributes // Logical Relations are distinct if they have different output for the sake of transformations. - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case l @ LogicalRelation(otherRelation) => relation == otherRelation && output == l.output case _ => false } - override def sameResult(otherPlan: LogicalPlan) = otherPlan match { + override def hashCode: Int = { + com.google.common.base.Objects.hashCode(relation, output) + } + + override def sameResult(otherPlan: LogicalPlan): Boolean = otherPlan match { case LogicalRelation(otherRelation) => relation == otherRelation case _ => false } - @transient override lazy val statistics = Statistics( + @transient override lazy val statistics: Statistics = Statistics( sizeInBytes = BigInt(relation.sizeInBytes) ) /** Used to lookup original attribute capitalization */ - val attributeMap = AttributeMap(output.map(o => (o, o))) + val attributeMap: AttributeMap[AttributeReference] = AttributeMap(output.map(o => (o, o))) - def newInstance() = LogicalRelation(relation).asInstanceOf[this.type] + def newInstance(): this.type = LogicalRelation(relation).asInstanceOf[this.type] - override def simpleString = s"Relation[${output.mkString(",")}] $relation" + override def simpleString: String = s"Relation[${output.mkString(",")}] $relation" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 0e540dad81283..9bbe06e59ba30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -27,7 +27,7 @@ private[sql] case class InsertIntoDataSource( overwrite: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] val data = DataFrame(sqlContext, query) // Apply the schema of the existing table to the new data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 76754a6ce4617..d57406645eefa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -362,7 +362,7 @@ private[sql] case class CreateTableUsingAsSelect( mode: SaveMode, options: Map[String, String], child: LogicalPlan) extends UnaryNode { - override def output = Seq.empty[Attribute] + override def output: Seq[Attribute] = Seq.empty[Attribute] // TODO: Override resolved after we support databaseName. // override lazy val resolved = databaseName != None && childrenResolved } @@ -373,7 +373,7 @@ private[sql] case class CreateTempTableUsing( provider: String, options: Map[String, String]) extends RunnableCommand { - def run(sqlContext: SQLContext) = { + def run(sqlContext: SQLContext): Seq[Row] = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerDataFrameAsTable( DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) @@ -388,7 +388,7 @@ private[sql] case class CreateTempTableUsingAsSelect( options: Map[String, String], query: LogicalPlan) extends RunnableCommand { - def run(sqlContext: SQLContext) = { + def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df) sqlContext.registerDataFrameAsTable( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index cfa58f1442218..5a78001117d1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -53,7 +53,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { def castAndRenameChildOutput( insertInto: InsertIntoTable, expectedOutput: Seq[Attribute], - child: LogicalPlan) = { + child: LogicalPlan): InsertIntoTable = { val newChildOutput = expectedOutput.zip(child.output).map { case (expected, actual) => val needCast = !expected.dataType.sameType(actual.dataType) @@ -79,7 +79,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { * A rule to do various checks before inserting into or writing to a data source table. */ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => Unit) { - def failAnalysis(msg: String) = { throw new AnalysisException(msg) } + def failAnalysis(msg: String): Unit = { throw new AnalysisException(msg) } def apply(plan: LogicalPlan): Unit = { plan.foreach { From 25e271d9fbb3394931d23822a1b2020e9d9b46b3 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Fri, 20 Mar 2015 17:14:09 -0700 Subject: [PATCH 476/817] [SPARK-6025] [MLlib] Add helper method evaluateEachIteration to extract learning curve Added evaluateEachIteration to allow the user to manually extract the error for each iteration of GradientBoosting. The internal optimisation can be dealt with later. Author: MechCoder Closes #4906 from MechCoder/spark-6025 and squashes the following commits: 67146ab [MechCoder] Minor 352001f [MechCoder] Minor 6e8aa10 [MechCoder] Made the following changes Used mapPartition instead of map Refactored computeError and unpersisted broadcast variables bc99ac6 [MechCoder] Refactor the method and stuff dbda033 [MechCoder] [SPARK-6025] Add helper method evaluateEachIteration to extract learning curve --- docs/mllib-ensembles.md | 4 +- .../spark/mllib/tree/loss/AbsoluteError.scala | 17 ++---- .../spark/mllib/tree/loss/LogLoss.scala | 20 ++----- .../apache/spark/mllib/tree/loss/Loss.scala | 14 ++++- .../spark/mllib/tree/loss/SquaredError.scala | 17 ++---- .../mllib/tree/model/treeEnsembleModels.scala | 54 +++++++++++++++++++ .../tree/GradientBoostedTreesSuite.scala | 16 +++++- 7 files changed, 96 insertions(+), 46 deletions(-) diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index cbfb682609af3..7521fb14a7bd6 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -464,8 +464,8 @@ first one being the training dataset and the second being the validation dataset The training is stopped when the improvement in the validation error is not more than a certain tolerance (supplied by the `validationTol` argument in `BoostingStrategy`). In practice, the validation error decreases initially and later increases. There might be cases in which the validation error does not change monotonically, -and the user is advised to set a large enough negative tolerance and examine the validation curve to to tune the number of -iterations. +and the user is advised to set a large enough negative tolerance and examine the validation curve using `evaluateEachIteration` +(which gives the error or loss per iteration) to tune the number of iterations. ### Examples diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index d1bde15e6b150..793dd664c5d5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -47,18 +47,9 @@ object AbsoluteError extends Loss { if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 } - /** - * Method to calculate loss of the base learner for the gradient boosting calculation. - * Note: This method is not used by the gradient boosting algorithm but is useful for debugging - * purposes. - * @param model Ensemble model - * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return Mean absolute error of model on data - */ - override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - data.map { y => - val err = model.predict(y.features) - y.label - math.abs(err) - }.mean() + override def computeError(prediction: Double, label: Double): Double = { + val err = label - prediction + math.abs(err) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 55213e695638c..51b1aed167b66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -50,20 +50,10 @@ object LogLoss extends Loss { - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) } - /** - * Method to calculate loss of the base learner for the gradient boosting calculation. - * Note: This method is not used by the gradient boosting algorithm but is useful for debugging - * purposes. - * @param model Ensemble model - * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return Mean log loss of model on data - */ - override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - data.map { case point => - val prediction = model.predict(point.features) - val margin = 2.0 * point.label * prediction - // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. - 2.0 * MLUtils.log1pExp(-margin) - }.mean() + override def computeError(prediction: Double, label: Double): Double = { + val margin = 2.0 * label * prediction + // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. + 2.0 * MLUtils.log1pExp(-margin) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index e1169d9f66ea4..357869ff6b333 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -47,6 +47,18 @@ trait Loss extends Serializable { * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * @return Measure of model error on data */ - def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double + def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { + data.map(point => computeError(model.predict(point.features), point.label)).mean() + } + + /** + * Method to calculate loss when the predictions are already known. + * Note: This method is used in the method evaluateEachIteration to avoid recomputing the + * predicted values from previously fit trees. + * @param prediction Predicted label. + * @param label True label. + * @return Measure of model error on datapoint. + */ + def computeError(prediction: Double, label: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index 50ecaa2f86f35..b990707ca4525 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -47,18 +47,9 @@ object SquaredError extends Loss { 2.0 * (model.predict(point.features) - point.label) } - /** - * Method to calculate loss of the base learner for the gradient boosting calculation. - * Note: This method is not used by the gradient boosting algorithm but is useful for debugging - * purposes. - * @param model Ensemble model - * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return Mean squared error of model on data - */ - override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - data.map { y => - val err = model.predict(y.features) - y.label - err * err - }.mean() + override def computeError(prediction: Double, label: Double): Double = { + val err = prediction - label + err * err } + } 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 f160852c69c77..1950254b2aa6d 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 @@ -28,9 +28,11 @@ import org.apache.spark.{Logging, SparkContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ +import org.apache.spark.mllib.tree.loss.Loss import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext @@ -108,6 +110,58 @@ class GradientBoostedTreesModel( } override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + + /** + * Method to compute error or loss for every iteration of gradient boosting. + * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @param loss evaluation metric. + * @return an array with index i having the losses or errors for the ensemble + * containing the first i+1 trees + */ + def evaluateEachIteration( + data: RDD[LabeledPoint], + loss: Loss): Array[Double] = { + + val sc = data.sparkContext + val remappedData = algo match { + case Classification => data.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + case _ => data + } + + val numIterations = trees.length + val evaluationArray = Array.fill(numIterations)(0.0) + + var predictionAndError: RDD[(Double, Double)] = remappedData.map { i => + val pred = treeWeights(0) * trees(0).predict(i.features) + val error = loss.computeError(pred, i.label) + (pred, error) + } + evaluationArray(0) = predictionAndError.values.mean() + + // Avoid the model being copied across numIterations. + val broadcastTrees = sc.broadcast(trees) + val broadcastWeights = sc.broadcast(treeWeights) + + (1 until numIterations).map { nTree => + predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => + val currentTree = broadcastTrees.value(nTree) + val currentTreeWeight = broadcastWeights.value(nTree) + iter.map { + case (point, (pred, error)) => { + val newPred = pred + currentTree.predict(point.features) * currentTreeWeight + val newError = loss.computeError(newPred, point.label) + (newPred, newError) + } + } + } + evaluationArray(nTree) = predictionAndError.values.mean() + } + + broadcastTrees.unpersist() + broadcastWeights.unpersist() + evaluationArray + } + } object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index b437aeaaf0547..55b0bac7d49fe 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -175,10 +175,11 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { new BoostingStrategy(treeStrategy, loss, numIterations, validationTol = 0.0) val gbtValidate = new GradientBoostedTrees(boostingStrategy) .runWithValidation(trainRdd, validateRdd) - assert(gbtValidate.numTrees !== numIterations) + val numTrees = gbtValidate.numTrees + assert(numTrees !== numIterations) // Test that it performs better on the validation dataset. - val gbt = GradientBoostedTrees.train(trainRdd, boostingStrategy) + val gbt = new GradientBoostedTrees(boostingStrategy).run(trainRdd) val (errorWithoutValidation, errorWithValidation) = { if (algo == Classification) { val remappedRdd = validateRdd.map(x => new LabeledPoint(2 * x.label - 1, x.features)) @@ -188,6 +189,17 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { } } assert(errorWithValidation <= errorWithoutValidation) + + // Test that results from evaluateEachIteration comply with runWithValidation. + // Note that convergenceTol is set to 0.0 + val evaluationArray = gbt.evaluateEachIteration(validateRdd, loss) + assert(evaluationArray.length === numIterations) + assert(evaluationArray(numTrees) > evaluationArray(numTrees - 1)) + var i = 1 + while (i < numTrees) { + assert(evaluationArray(i) <= evaluationArray(i - 1)) + i += 1 + } } } } From bc37c9743e065a0c756363c7b70e88f22a6e6edd Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sat, 21 Mar 2015 10:53:04 +0800 Subject: [PATCH 477/817] [SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful Do the same check as #4610 for ParquetRelation2. Author: Yanbo Liang Closes #5107 from yanboliang/spark-5821-parquet and squashes the following commits: 7092c8d [Yanbo Liang] ParquetRelation2 CTAS should check if delete is successful --- .../apache/spark/sql/parquet/newParquet.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 10b8876c1d31c..fbe7a419feb52 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 @@ -611,13 +611,22 @@ private[sql] case class ParquetRelation2( val destinationPath = new Path(paths.head) if (overwrite) { - try { - destinationPath.getFileSystem(conf).delete(destinationPath, true) - } catch { - case e: IOException => + val fs = destinationPath.getFileSystem(conf) + if (fs.exists(destinationPath)) { + var success: Boolean = false + try { + success = fs.delete(destinationPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${destinationPath.toString} prior" + + s" to writing to Parquet table:\n${e.toString}") + } + if (!success) { throw new IOException( s"Unable to clear output directory ${destinationPath.toString} prior" + - s" to writing to Parquet file:\n${e.toString}") + s" to writing to Parquet table.") + } } } From 937c1e5503963e67a5412be993d30dbec6fc9883 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 21 Mar 2015 11:18:45 +0800 Subject: [PATCH 478/817] [SPARK-6315] [SQL] Also tries the case class string parser while reading Parquet schema When writing Parquet files, Spark 1.1.x persists the schema string into Parquet metadata with the result of `StructType.toString`, which was then deprecated in Spark 1.2 by a schema string in JSON format. But we still need to take the old schema format into account while reading Parquet files. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5034) Author: Cheng Lian Closes #5034 from liancheng/spark-6315 and squashes the following commits: a182f58 [Cheng Lian] Adds a regression test b9c6dbe [Cheng Lian] Also tries the case class string parser while reading Parquet schema --- .../apache/spark/sql/parquet/newParquet.scala | 23 +++++++++- .../spark/sql/parquet/ParquetIOSuite.scala | 42 +++++++++++++++++-- 2 files changed, 60 insertions(+), 5 deletions(-) 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 fbe7a419feb52..410600b0529d3 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 @@ -681,7 +681,7 @@ private[sql] case class ParquetRelation2( } } -private[sql] object ParquetRelation2 { +private[sql] object ParquetRelation2 extends Logging { // Whether we should merge schemas collected from all Parquet part-files. val MERGE_SCHEMA = "mergeSchema" @@ -701,7 +701,26 @@ private[sql] object ParquetRelation2 { .getKeyValueMetaData .toMap .get(RowReadSupport.SPARK_METADATA_KEY) - .map(DataType.fromJson(_).asInstanceOf[StructType]) + .flatMap { serializedSchema => + // Don't throw even if we failed to parse the serialized Spark schema. Just fallback to + // whatever is available. + Try(DataType.fromJson(serializedSchema)) + .recover { case _: Throwable => + logInfo( + s"Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + + "falling back to the deprecated DataType.fromCaseClassString parser.") + DataType.fromCaseClassString(serializedSchema) + } + .recover { case cause: Throwable => + logWarning( + s"""Failed to parse serialized Spark schema in Parquet key-value metadata: + |\t$serializedSchema + """.stripMargin, + cause) + } + .map(_.asInstanceOf[StructType]) + .toOption + } maybeSparkSchema.getOrElse { // Falls back to Parquet schema if Spark SQL schema is absent. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index a70b3c7ce48d3..5438095addeaf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -28,8 +28,8 @@ import parquet.example.data.simple.SimpleGroup import parquet.example.data.{Group, GroupWriter} import parquet.hadoop.api.WriteSupport import parquet.hadoop.api.WriteSupport.WriteContext -import parquet.hadoop.metadata.CompressionCodecName -import parquet.hadoop.{ParquetFileWriter, ParquetWriter} +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData, CompressionCodecName} +import parquet.hadoop.{Footer, ParquetFileWriter, ParquetWriter} import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf, SaveMode} // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport @@ -330,6 +330,42 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } + test("SPARK-6315 regression test") { + // Spark 1.1 and prior versions write Spark schema as case class string into Parquet metadata. + // This has been deprecated by JSON format since 1.2. Notice that, 1.3 further refactored data + // types API, and made StructType.fields an array. This makes the result of StructType.toString + // different from prior versions: there's no "Seq" wrapping the fields part in the string now. + val sparkSchema = + "StructType(Seq(StructField(a,BooleanType,false),StructField(b,IntegerType,false)))" + + // The Parquet schema is intentionally made different from the Spark schema. Because the new + // Parquet data source simply falls back to the Parquet schema once it fails to parse the Spark + // schema. By making these two different, we are able to assert the old style case class string + // is parsed successfully. + val parquetSchema = MessageTypeParser.parseMessageType( + """message root { + | required int32 c; + |} + """.stripMargin) + + withTempPath { location => + val extraMetadata = Map(RowReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString) + val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, "Spark") + val path = new Path(location.getCanonicalPath) + + ParquetFileWriter.writeMetadataFile( + sparkContext.hadoopConfiguration, + path, + new Footer(path, new ParquetMetadata(fileMetadata, Nil)) :: Nil) + + assertResult(parquetFile(path.toString).schema) { + StructType( + StructField("a", BooleanType, nullable = false) :: + StructField("b", IntegerType, nullable = false) :: + Nil) + } + } + } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From e5d2c37c68ac00a57c2542e62d1c5b4ca267c89e Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sat, 21 Mar 2015 11:23:28 +0800 Subject: [PATCH 479/817] [SPARK-5821] [SQL] JSON CTAS command should throw error message when delete path failure When using "CREATE TEMPORARY TABLE AS SELECT" to create JSON table, we first delete the path file or directory and then generate a new directory with the same name. But if only read permission was granted, the delete failed. Here we just throwing an error message to let users know what happened. ParquetRelation2 may also hit this problem. I think to restrict JSONRelation and ParquetRelation2 must base on directory is more reasonable for access control. Maybe I can do it in follow up works. Author: Yanbo Liang Author: Yanbo Liang Closes #4610 from yanboliang/jsonInsertImprovements and squashes the following commits: c387fce [Yanbo Liang] fix typos 42d7fb6 [Yanbo Liang] add unittest & fix output format 46f0d9d [Yanbo Liang] Update JSONRelation.scala e2df8d5 [Yanbo Liang] check path exisit when write 79f7040 [Yanbo Liang] Update JSONRelation.scala e4bc229 [Yanbo Liang] Update JSONRelation.scala 5a42d83 [Yanbo Liang] JSONRelation CTAS should check if delete is successful --- .../apache/spark/sql/json/JSONRelation.scala | 36 +++++++++++++++---- .../sources/CreateTableAsSelectSuite.scala | 25 ++++++++++++- 2 files changed, 53 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index b1e363d02edfe..f4c99b4b56606 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -68,9 +68,23 @@ private[sql] class DefaultSource mode match { case SaveMode.Append => sys.error(s"Append mode is not supported by ${this.getClass.getCanonicalName}") - case SaveMode.Overwrite => - fs.delete(filesystemPath, true) + case SaveMode.Overwrite => { + var success: Boolean = false + try { + success = fs.delete(filesystemPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${filesystemPath.toString} prior" + + s" to writing to JSON table:\n${e.toString}") + } + if (!success) { + throw new IOException( + s"Unable to clear output directory ${filesystemPath.toString} prior" + + s" to writing to JSON table.") + } true + } case SaveMode.ErrorIfExists => sys.error(s"path $path already exists.") case SaveMode.Ignore => false @@ -114,13 +128,21 @@ private[sql] case class JSONRelation( val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) if (overwrite) { - try { - fs.delete(filesystemPath, true) - } catch { - case e: IOException => + if (fs.exists(filesystemPath)) { + var success: Boolean = false + try { + success = fs.delete(filesystemPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${filesystemPath.toString} prior" + + s" to writing to JSON table:\n${e.toString}") + } + if (!success) { throw new IOException( s"Unable to clear output directory ${filesystemPath.toString} prior" - + s" to INSERT OVERWRITE a JSON table:\n${e.toString}") + + s" to writing to JSON table.") + } } // Write the data. data.toJSON.saveAsTextFile(path) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 2975a7fee4c96..20a23b3bd6aa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import java.io.File +import java.io.{IOException, File} import org.apache.spark.sql.AnalysisException import org.scalatest.BeforeAndAfterAll @@ -62,6 +62,29 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { dropTempTable("jsonTable") } + test("CREATE TEMPORARY TABLE AS SELECT based on the file without write permission") { + val childPath = new File(path.toString, "child") + path.mkdir() + childPath.createNewFile() + path.setWritable(false) + + val e = intercept[IOException] { + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + sql("SELECT a, b FROM jsonTable").collect() + } + assert(e.getMessage().contains("Unable to clear output directory")) + + path.setWritable(true) + } + test("create a table, drop it and create another one with the same name") { sql( s""" From 52dd4b2b277eb48bc89db9b21d25f5e836c1d348 Mon Sep 17 00:00:00 2001 From: x1- Date: Sat, 21 Mar 2015 13:22:34 -0700 Subject: [PATCH 480/817] [SPARK-5320][SQL]Add statistics method at NoRelation (override super). Because of no statistics override, in spute of super class say 'LeafNode must override'. fix issue [SPARK-5320: Joins on simple table created using select gives error](https://issues.apache.org/jira/browse/SPARK-5320) Author: x1- Closes #5105 from x1-/SPARK-5320 and squashes the following commits: e561aac [x1-] Add statistics method at NoRelation (override super). --- .../sql/catalyst/plans/logical/basicOperators.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 1e7b449d75b80..384fe53a68362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -289,6 +289,15 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { case object NoRelation extends LeafNode { override def output = Nil + + /** + * Computes [[Statistics]] for this plan. The default implementation assumes the output + * cardinality is the product of of all child plan's cardinality, i.e. applies in the case + * of cartesian joins. + * + * [[LeafNode]]s must override this. + */ + override def statistics: Statistics = Statistics(sizeInBytes = 1) } case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { From ee569a0c7171d149eee52877def902378eaf695e Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Sat, 21 Mar 2015 13:24:24 -0700 Subject: [PATCH 481/817] [SPARK-5680][SQL] Sum function on all null values, should return zero SELECT sum('a'), avg('a'), variance('a'), std('a') FROM src; Should give output as 0.0 NULL NULL NULL This fixes hive udaf_number_format.q Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #4466 from gvramana/sum_fix and squashes the following commits: 42e14d1 [Venkata Ramana Gollamudi] Added comments 39415c0 [Venkata Ramana Gollamudi] Handled the partitioned Sum expression scenario df66515 [Venkata Ramana Gollamudi] code style fix 4be2606 [Venkata Ramana Gollamudi] Add udaf_number_format to whitelist and golden answer 330fd64 [Venkata Ramana Gollamudi] fix sum function for all null data --- .../sql/catalyst/expressions/aggregates.scala | 68 ++++++++++++++++++- .../execution/HiveCompatibilitySuite.scala | 1 + ..._format-0-eff4ef3c207d14d5121368f294697964 | 0 ..._format-1-4a03c4328565c60ca99689239f07fb16 | 1 + 4 files changed, 67 insertions(+), 3 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 create mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 735b7488fdcbd..5297d1e31246c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -346,13 +346,13 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case DecimalType.Fixed(_, _) => val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() SplitEvaluation( - Cast(Sum(partialSum.toAttribute), dataType), + Cast(CombineSum(partialSum.toAttribute), dataType), partialSum :: Nil) case _ => val partialSum = Alias(Sum(child), "PartialSum")() SplitEvaluation( - Sum(partialSum.toAttribute), + CombineSum(partialSum.toAttribute), partialSum :: Nil) } } @@ -360,6 +360,30 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ override def newInstance() = new SumFunction(child, this) } +/** + * Sum should satisfy 3 cases: + * 1) sum of all null values = zero + * 2) sum for table column with no data = null + * 3) sum of column with null and not null values = sum of not null values + * Require separate CombineSum Expression and function as it has to distinguish "No data" case + * versus "data equals null" case, while aggregating results and at each partial expression.i.e., + * Combining PartitionLevel InputData + * <-- null + * Zero <-- Zero <-- null + * + * <-- null <-- no data + * null <-- null <-- no data + */ +case class CombineSum(child: Expression) extends AggregateExpression { + def this() = this(null) + + override def children = child :: Nil + override def nullable = true + override def dataType = child.dataType + override def toString = s"CombineSum($child)" + override def newInstance() = new CombineSumFunction(child, this) +} + case class SumDistinct(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -565,7 +589,8 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr private val sum = MutableLiteral(null, calcType) - private val addFunction = Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum)) + private val addFunction = + Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) override def update(input: Row): Unit = { sum.update(addFunction, input) @@ -580,6 +605,43 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr } } +case class CombineSumFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + private val calcType = + expr.dataType match { + case DecimalType.Fixed(_, _) => + DecimalType.Unlimited + case _ => + expr.dataType + } + + private val zero = Cast(Literal(0), calcType) + + private val sum = MutableLiteral(null, calcType) + + private val addFunction = + Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) + + override def update(input: Row): Unit = { + val result = expr.eval(input) + // partial sum result can be null only when no input rows present + if(result != null) { + sum.update(addFunction, input) + } + } + + override def eval(input: Row): Any = { + expr.dataType match { + case DecimalType.Fixed(_, _) => + Cast(sum, dataType).eval(null) + case _ => sum.eval(null) + } + } +} + case class SumDistinctFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 5180a7f09d80f..2ae9d018e1b1b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -800,6 +800,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udaf_covar_pop", "udaf_covar_samp", "udaf_histogram_numeric", + "udaf_number_format", "udf2", "udf5", "udf6", diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 new file mode 100644 index 0000000000000..c6f275a0db131 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 @@ -0,0 +1 @@ +0.0 NULL NULL NULL From 94a102acb80a7c77f57409ece1f8dbbba791b774 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 21 Mar 2015 13:27:53 -0700 Subject: [PATCH 482/817] [SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser. This PR creates a trait `DataTypeParser` used to parse data types. This trait aims to be single place to provide the functionality of parsing data types' string representation. It is currently mixed in with `DDLParser` and `SqlParser`. It is also used to parse the data type for `DataFrame.cast` and to convert Hive metastore's data type string back to a `DataType`. JIRA: https://issues.apache.org/jira/browse/SPARK-6250 Author: Yin Huai Closes #5078 from yhuai/ddlKeywords and squashes the following commits: 0e66097 [Yin Huai] Special handle struct<>. fea6012 [Yin Huai] Style. c9733fb [Yin Huai] Create a trait to parse data types. --- .../apache/spark/sql/catalyst/SqlParser.scala | 27 +--- .../spark/sql/types/DataTypeParser.scala | 115 +++++++++++++++++ .../spark/sql/types/DataTypeParserSuite.scala | 116 ++++++++++++++++++ .../scala/org/apache/spark/sql/Column.scala | 15 +-- .../org/apache/spark/sql/sources/ddl.scala | 80 +----------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 +- 6 files changed, 241 insertions(+), 120 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 54ab13ca352d2..ea7d44a3723d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.types._ * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ -class SqlParser extends AbstractSparkSQLParser { +class SqlParser extends AbstractSparkSQLParser with DataTypeParser { def parseExpression(input: String): Expression = { // Initialize the Keywords. @@ -61,11 +61,8 @@ class SqlParser extends AbstractSparkSQLParser { protected val CAST = Keyword("CAST") protected val COALESCE = Keyword("COALESCE") protected val COUNT = Keyword("COUNT") - protected val DATE = Keyword("DATE") - protected val DECIMAL = Keyword("DECIMAL") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") - protected val DOUBLE = Keyword("DOUBLE") protected val ELSE = Keyword("ELSE") protected val END = Keyword("END") protected val EXCEPT = Keyword("EXCEPT") @@ -78,7 +75,6 @@ class SqlParser extends AbstractSparkSQLParser { protected val IF = Keyword("IF") protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") - protected val INT = Keyword("INT") protected val INSERT = Keyword("INSERT") protected val INTERSECT = Keyword("INTERSECT") protected val INTO = Keyword("INTO") @@ -105,13 +101,11 @@ class SqlParser extends AbstractSparkSQLParser { protected val SELECT = Keyword("SELECT") protected val SEMI = Keyword("SEMI") protected val SQRT = Keyword("SQRT") - protected val STRING = Keyword("STRING") protected val SUBSTR = Keyword("SUBSTR") protected val SUBSTRING = Keyword("SUBSTRING") protected val SUM = Keyword("SUM") protected val TABLE = Keyword("TABLE") protected val THEN = Keyword("THEN") - protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") protected val UNION = Keyword("UNION") protected val UPPER = Keyword("UPPER") @@ -315,7 +309,9 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val cast: Parser[Expression] = - CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { case exp ~ t => Cast(exp, t) } + CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { + case exp ~ t => Cast(exp, t) + } protected lazy val literal: Parser[Literal] = ( numericLiteral @@ -387,19 +383,4 @@ class SqlParser extends AbstractSparkSQLParser { (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) } - - protected lazy val dataType: Parser[DataType] = - ( STRING ^^^ StringType - | TIMESTAMP ^^^ TimestampType - | DOUBLE ^^^ DoubleType - | fixedDecimalType - | DECIMAL ^^^ DecimalType.Unlimited - | DATE ^^^ DateType - | INT ^^^ IntegerType - ) - - protected lazy val fixedDecimalType: Parser[DataType] = - (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala new file mode 100644 index 0000000000000..89278f7dbc806 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types + +import scala.language.implicitConversions +import scala.util.matching.Regex +import scala.util.parsing.combinator.syntactical.StandardTokenParsers + +import org.apache.spark.sql.catalyst.SqlLexical + +/** + * This is a data type parser that can be used to parse string representations of data types + * provided in SQL queries. This parser is mixed in with DDLParser and SqlParser. + */ +private[sql] trait DataTypeParser extends StandardTokenParsers { + + // This is used to create a parser from a regex. We are using regexes for data type strings + // since these strings can be also used as column names or field names. + import lexical.Identifier + implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", + { case Identifier(str) if regex.unapplySeq(str).isDefined => str } + ) + + protected lazy val primitiveType: Parser[DataType] = + "(?i)string".r ^^^ StringType | + "(?i)float".r ^^^ FloatType | + "(?i)int".r ^^^ IntegerType | + "(?i)tinyint".r ^^^ ByteType | + "(?i)smallint".r ^^^ ShortType | + "(?i)double".r ^^^ DoubleType | + "(?i)bigint".r ^^^ LongType | + "(?i)binary".r ^^^ BinaryType | + "(?i)boolean".r ^^^ BooleanType | + fixedDecimalType | + "(?i)decimal".r ^^^ DecimalType.Unlimited | + "(?i)date".r ^^^ DateType | + "(?i)timestamp".r ^^^ TimestampType | + varchar + + protected lazy val fixedDecimalType: Parser[DataType] = + ("(?i)decimal".r ~> "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { + case precision ~ scale => + DecimalType(precision.toInt, scale.toInt) + } + + protected lazy val varchar: Parser[DataType] = + "(?i)varchar".r ~> "(" ~> (numericLit <~ ")") ^^^ StringType + + protected lazy val arrayType: Parser[DataType] = + "(?i)array".r ~> "<" ~> dataType <~ ">" ^^ { + case tpe => ArrayType(tpe) + } + + protected lazy val mapType: Parser[DataType] = + "(?i)map".r ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { + case t1 ~ _ ~ t2 => MapType(t1, t2) + } + + protected lazy val structField: Parser[StructField] = + ident ~ ":" ~ dataType ^^ { + case name ~ _ ~ tpe => StructField(name, tpe, nullable = true) + } + + protected lazy val structType: Parser[DataType] = + ("(?i)struct".r ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { + case fields => new StructType(fields.toArray) + }) | + ("(?i)struct".r ~ "<>" ^^^ StructType(Nil)) + + protected lazy val dataType: Parser[DataType] = + arrayType | + mapType | + structType | + primitiveType + + def toDataType(dataTypeString: String): DataType = synchronized { + phrase(dataType)(new lexical.Scanner(dataTypeString)) match { + case Success(result, _) => result + case failure: NoSuccess => throw new DataTypeException(failMessage(dataTypeString)) + } + } + + private def failMessage(dataTypeString: String): String = { + s"Unsupported dataType: $dataTypeString. If you have a struct and a field name of it has " + + "any special characters, please use backticks (`) to quote that field name, e.g. `x+y`. " + + "Please note that backtick itself is not supported in a field name." + } +} + +private[sql] object DataTypeParser { + lazy val dataTypeParser = new DataTypeParser { + override val lexical = new SqlLexical + } + + def apply(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString) +} + +/** The exception thrown from the [[DataTypeParser]]. */ +protected[sql] class DataTypeException(message: String) extends Exception(message) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala new file mode 100644 index 0000000000000..1ba21b64603ac --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala @@ -0,0 +1,116 @@ +/* +* 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.types + +import org.scalatest.FunSuite + +class DataTypeParserSuite extends FunSuite { + + def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = { + test(s"parse ${dataTypeString.replace("\n", "")}") { + assert(DataTypeParser(dataTypeString) === expectedDataType) + } + } + + def unsupported(dataTypeString: String): Unit = { + test(s"$dataTypeString is not supported") { + intercept[DataTypeException](DataTypeParser(dataTypeString)) + } + } + + checkDataType("int", IntegerType) + checkDataType("BooLean", BooleanType) + checkDataType("tinYint", ByteType) + checkDataType("smallINT", ShortType) + checkDataType("INT", IntegerType) + checkDataType("bigint", LongType) + checkDataType("float", FloatType) + checkDataType("dOUBle", DoubleType) + checkDataType("decimal(10, 5)", DecimalType(10, 5)) + checkDataType("decimal", DecimalType.Unlimited) + checkDataType("DATE", DateType) + checkDataType("timestamp", TimestampType) + checkDataType("string", StringType) + checkDataType("varchAr(20)", StringType) + checkDataType("BINARY", BinaryType) + + checkDataType("array", ArrayType(DoubleType, true)) + checkDataType("Array>", ArrayType(MapType(IntegerType, ByteType, true), true)) + checkDataType( + "array>", + ArrayType(StructType(StructField("tinYint", ByteType, true) :: Nil), true) + ) + checkDataType("MAP", MapType(IntegerType, StringType, true)) + checkDataType("MAp>", MapType(IntegerType, ArrayType(DoubleType), true)) + checkDataType( + "MAP>", + MapType(IntegerType, StructType(StructField("varchar", StringType, true) :: Nil), true) + ) + + checkDataType( + "struct", + StructType( + StructField("intType", IntegerType, true) :: + StructField("ts", TimestampType, true) :: Nil) + ) + // It is fine to use the data type string as the column name. + checkDataType( + "Struct", + StructType( + StructField("int", IntegerType, true) :: + StructField("timestamp", TimestampType, true) :: Nil) + ) + checkDataType( + """ + |struct< + | struct:struct, + | MAP:Map, + | arrAy:Array> + """.stripMargin, + StructType( + StructField("struct", + StructType( + StructField("deciMal", DecimalType.Unlimited, true) :: + StructField("anotherDecimal", DecimalType(5, 2), true) :: Nil), true) :: + StructField("MAP", MapType(TimestampType, StringType), true) :: + StructField("arrAy", ArrayType(DoubleType, true), true) :: Nil) + ) + // A column name can be a reserved word in our DDL parser and SqlParser. + checkDataType( + "Struct", + StructType( + StructField("TABLE", StringType, true) :: + StructField("CASE", BooleanType, true) :: Nil) + ) + // Use backticks to quote column names having special characters. + checkDataType( + "struct<`x+y`:int, `!@#$%^&*()`:string, `1_2.345<>:\"`:varchar(20)>", + StructType( + StructField("x+y", IntegerType, true) :: + StructField("!@#$%^&*()", StringType, true) :: + StructField("1_2.345<>:\"", StringType, true) :: Nil) + ) + // Empty struct. + checkDataType("strUCt<>", StructType(Nil)) + + unsupported("it is not a data type") + unsupported("struct") + unsupported("struct") + unsupported("struct<`x``y` int>") +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index b7a13a1b26802..ec7d15f5bc4e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -624,20 +624,7 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def cast(to: String): Column = cast(to.toLowerCase match { - case "string" | "str" => StringType - case "boolean" => BooleanType - case "byte" => ByteType - case "short" => ShortType - case "int" => IntegerType - case "long" => LongType - case "float" => FloatType - case "double" => DoubleType - case "decimal" => DecimalType.Unlimited - case "date" => DateType - case "timestamp" => TimestampType - case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") - }) + def cast(to: String): Column = cast(DataTypeParser(to)) /** * Returns an ordering used in sorting. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index d57406645eefa..d2e807d3a69b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -34,7 +34,8 @@ import org.apache.spark.util.Utils * A parser for foreign DDL commands. */ private[sql] class DDLParser( - parseQuery: String => LogicalPlan) extends AbstractSparkSQLParser with Logging { + parseQuery: String => LogicalPlan) + extends AbstractSparkSQLParser with DataTypeParser with Logging { def apply(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = { try { @@ -46,14 +47,6 @@ private[sql] class DDLParser( } } - def parseType(input: String): DataType = { - lexical.initialize(reservedWords) - phrase(dataType)(new lexical.Scanner(input)) match { - case Success(r, x) => r - case x => throw new DDLException(s"Unsupported dataType: $x") - } - } - // Keyword is a convention with AbstractSparkSQLParser, which will scan all of the `Keyword` // properties via reflection the class in runtime for constructing the SqlLexical object protected val CREATE = Keyword("CREATE") @@ -70,24 +63,6 @@ private[sql] class DDLParser( protected val COMMENT = Keyword("COMMENT") protected val REFRESH = Keyword("REFRESH") - // Data types. - protected val STRING = Keyword("STRING") - protected val BINARY = Keyword("BINARY") - protected val BOOLEAN = Keyword("BOOLEAN") - protected val TINYINT = Keyword("TINYINT") - protected val SMALLINT = Keyword("SMALLINT") - protected val INT = Keyword("INT") - protected val BIGINT = Keyword("BIGINT") - protected val FLOAT = Keyword("FLOAT") - protected val DOUBLE = Keyword("DOUBLE") - protected val DECIMAL = Keyword("DECIMAL") - protected val DATE = Keyword("DATE") - protected val TIMESTAMP = Keyword("TIMESTAMP") - protected val VARCHAR = Keyword("VARCHAR") - protected val ARRAY = Keyword("ARRAY") - protected val MAP = Keyword("MAP") - protected val STRUCT = Keyword("STRUCT") - protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable | refreshTable protected def start: Parser[LogicalPlan] = ddl @@ -189,58 +164,9 @@ private[sql] class DDLParser( new MetadataBuilder().putString(COMMENT.str.toLowerCase, comment).build() case None => Metadata.empty } - StructField(columnName, typ, nullable = true, meta) - } - - protected lazy val primitiveType: Parser[DataType] = - STRING ^^^ StringType | - BINARY ^^^ BinaryType | - BOOLEAN ^^^ BooleanType | - TINYINT ^^^ ByteType | - SMALLINT ^^^ ShortType | - INT ^^^ IntegerType | - BIGINT ^^^ LongType | - FLOAT ^^^ FloatType | - DOUBLE ^^^ DoubleType | - fixedDecimalType | // decimal with precision/scale - DECIMAL ^^^ DecimalType.Unlimited | // decimal with no precision/scale - DATE ^^^ DateType | - TIMESTAMP ^^^ TimestampType | - VARCHAR ~ "(" ~ numericLit ~ ")" ^^^ StringType - - protected lazy val fixedDecimalType: Parser[DataType] = - (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) - } - - protected lazy val arrayType: Parser[DataType] = - ARRAY ~> "<" ~> dataType <~ ">" ^^ { - case tpe => ArrayType(tpe) - } - protected lazy val mapType: Parser[DataType] = - MAP ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { - case t1 ~ _ ~ t2 => MapType(t1, t2) - } - - protected lazy val structField: Parser[StructField] = - ident ~ ":" ~ dataType ^^ { - case fieldName ~ _ ~ tpe => StructField(fieldName, tpe, nullable = true) + StructField(columnName, typ, nullable = true, meta) } - - protected lazy val structType: Parser[DataType] = - (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { - case fields => StructType(fields) - }) | - (STRUCT ~> "<>" ^^ { - case fields => StructType(Nil) - }) - - private[sql] lazy val dataType: Parser[DataType] = - arrayType | - mapType | - structType | - primitiveType } private[sql] object ResolvedDataSource { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fe86bd206a71c..949a4e54e6c30 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -756,7 +756,7 @@ private[hive] case class MetastoreRelation implicit class SchemaAttribute(f: FieldSchema) { def toAttribute = AttributeReference( f.getName, - sqlContext.ddlParser.parseType(f.getType), + HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true )(qualifiers = Seq(alias.getOrElse(tableName))) @@ -779,11 +779,7 @@ private[hive] case class MetastoreRelation private[hive] object HiveMetastoreTypes { - protected val ddlParser = new DDLParser(HiveQl.parseSql(_)) - - def toDataType(metastoreType: String): DataType = synchronized { - ddlParser.parseType(metastoreType) - } + def toDataType(metastoreType: String): DataType = DataTypeParser(metastoreType) def toMetastoreType(dt: DataType): String = dt match { case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" From b6090f902e6ec24923b4dde4aabc9076956521c1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 21 Mar 2015 14:30:04 -0700 Subject: [PATCH 483/817] [SPARK-6428][SQL] Added explicit type for all public methods for Hive module Author: Reynold Xin Closes #5108 from rxin/hive-public-type and squashes the following commits: a320328 [Reynold Xin] [SPARK-6428][SQL] Added explicit type for all public methods for Hive module. --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 6 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 21 ++++----- .../org/apache/spark/sql/hive/HiveQl.scala | 10 ++--- .../spark/sql/hive/HiveStrategies.scala | 6 +-- .../apache/spark/sql/hive/TableReader.scala | 2 +- .../hive/execution/CreateTableAsSelect.scala | 2 +- .../execution/DescribeHiveTableCommand.scala | 2 +- .../hive/execution/HiveNativeCommand.scala | 4 +- .../sql/hive/execution/HiveTableScan.scala | 5 ++- .../hive/execution/InsertIntoHiveTable.scala | 4 +- .../hive/execution/ScriptTransformation.scala | 5 ++- .../spark/sql/hive/execution/commands.scala | 8 ++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 45 +++++++++++-------- .../spark/sql/hive/hiveWriterContainers.scala | 4 +- .../apache/spark/sql/hive/test/TestHive.scala | 13 ++++-- 16 files changed, 79 insertions(+), 62 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 895688ab2ec2e..6272cdedb3e48 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -194,8 +194,8 @@ private[hive] object SparkSQLCLIDriver { val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) - def promptWithCurrentDB = s"$prompt$currentDB" - def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + def promptWithCurrentDB: String = s"$prompt$currentDB" + def continuedPromptWithDBSpaces: String = continuedPrompt + ReflectionUtils.invokeStatic( classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) var currentPrompt = promptWithCurrentDB diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index a5c435fdfa778..c06c2e396bbc1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -183,7 +183,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient - protected lazy val outputBuffer = new java.io.OutputStream { + protected lazy val outputBuffer = new java.io.OutputStream { var pos: Int = 0 var buffer = new Array[Int](10240) def write(i: Int): Unit = { @@ -191,7 +191,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { pos = (pos + 1) % buffer.size } - override def toString = { + override def toString: String = { val (end, start) = buffer.splitAt(pos) val input = new java.io.InputStream { val iterator = (start ++ end).iterator @@ -227,7 +227,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry { - def caseSensitive = false + def caseSensitive: Boolean = false } /* An analyzer that uses the Hive metastore. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 949a4e54e6c30..4c5eb48661f7d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -58,7 +58,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // TODO: Use this everywhere instead of tuples or databaseName, tableName,. /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) { - def toLowerCase = QualifiedTableName(database.toLowerCase, name.toLowerCase) + def toLowerCase: QualifiedTableName = QualifiedTableName(database.toLowerCase, name.toLowerCase) } /** A cache of Spark SQL data source tables that have been accessed. */ @@ -629,7 +629,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with castChildOutput(p, table, child) } - def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { + def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) + : LogicalPlan = { val childOutputDataTypes = child.output.map(_.dataType) val tableOutputDataTypes = (table.attributes ++ table.partitionKeys).take(child.output.length).map(_.dataType) @@ -667,7 +668,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ override def unregisterTable(tableIdentifier: Seq[String]): Unit = ??? - override def unregisterAllTables() = {} + override def unregisterAllTables(): Unit = {} } /** @@ -682,10 +683,10 @@ private[hive] case class InsertIntoHiveTable( overwrite: Boolean) extends LogicalPlan { - override def children = child :: Nil - override def output = child.output + override def children: Seq[LogicalPlan] = child :: Nil + override def output: Seq[Attribute] = child.output - override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => childAttr.dataType.sameType(tableAttr.dataType) } } @@ -704,13 +705,13 @@ private[hive] case class MetastoreRelation // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - @transient val hiveQlTable = new Table(table) + @transient val hiveQlTable: Table = new Table(table) - @transient val hiveQlPartitions = partitions.map { p => + @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p => new Partition(hiveQlTable, p) } - @transient override lazy val statistics = Statistics( + @transient override lazy val statistics: Statistics = Statistics( sizeInBytes = { val totalSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize) val rawDataSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize) @@ -754,7 +755,7 @@ private[hive] case class MetastoreRelation ) implicit class SchemaAttribute(f: FieldSchema) { - def toAttribute = AttributeReference( + def toAttribute: AttributeReference = AttributeReference( f.getName, HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index ced99cd082614..51775eb4cd6a0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -196,8 +196,8 @@ private[hive] object HiveQl { * Right now this function only checks the name, type, text and children of the node * for equality. */ - def checkEquals(other: ASTNode) { - def check(field: String, f: ASTNode => Any) = if (f(n) != f(other)) { + def checkEquals(other: ASTNode): Unit = { + def check(field: String, f: ASTNode => Any): Unit = if (f(n) != f(other)) { sys.error(s"$field does not match for trees. " + s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") } @@ -209,7 +209,7 @@ private[hive] object HiveQl { val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]] val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]] leftChildren zip rightChildren foreach { - case (l,r) => l checkEquals r + case (l, r) => l checkEquals r } } } @@ -269,7 +269,7 @@ private[hive] object HiveQl { } /** Creates LogicalPlan for a given VIEW */ - def createPlanForView(view: Table, alias: Option[String]) = alias match { + def createPlanForView(view: Table, alias: Option[String]): Subquery = alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText)) @@ -323,7 +323,7 @@ private[hive] object HiveQl { clauses } - def getClause(clauseName: String, nodeList: Seq[Node]) = + def getClause(clauseName: String, nodeList: Seq[Node]): Node = getClauseOption(clauseName, nodeList).getOrElse(sys.error( s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index e63cea60457d9..5f7e897295117 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -58,9 +58,9 @@ private[hive] trait HiveStrategies { @Experimental object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: DataFrame) { - def lowerCase = DataFrame(s.sqlContext, s.logicalPlan) + def lowerCase: DataFrame = DataFrame(s.sqlContext, s.logicalPlan) - def addPartitioningAttributes(attrs: Seq[Attribute]) = { + def addPartitioningAttributes(attrs: Seq[Attribute]): DataFrame = { // Don't add the partitioning key if its already present in the data. if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) { s @@ -75,7 +75,7 @@ private[hive] trait HiveStrategies { } implicit class PhysicalPlanHacks(originalPlan: SparkPlan) { - def fakeOutput(newOutput: Seq[Attribute]) = + def fakeOutput(newOutput: Seq[Attribute]): OutputFaker = OutputFaker( originalPlan.output.map(a => newOutput.find(a.name.toLowerCase == _.name.toLowerCase) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index f22c9eaeedc7d..af309c0c6ce2c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -175,7 +175,7 @@ class HadoopTableReader( relation.partitionKeys.contains(attr) } - def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow) = { + def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow): Unit = { partitionKeyAttrs.foreach { case (attr, ordinal) => val partOrdinal = relation.partitionKeys.indexOf(attr) row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index a0c91cbc4e86f..fade9e5852eaa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -45,7 +45,7 @@ case class CreateTableAsSelect( allowExisting: Boolean, desc: Option[CreateTableDesc]) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { // Create Hive Table diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index d0510aa342796..6fce69b58b85e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -37,7 +37,7 @@ case class DescribeHiveTableCommand( override val output: Seq[Attribute], isExtended: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala index 9636da206087f..60a9bb630d0d9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala @@ -26,9 +26,9 @@ import org.apache.spark.sql.types.StringType private[hive] case class HiveNativeCommand(sql: String) extends RunnableCommand { - override def output = + override def output: Seq[AttributeReference] = Seq(AttributeReference("result", StringType, nullable = false)()) - override def run(sqlContext: SQLContext) = + override def run(sqlContext: SQLContext): Seq[Row] = sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 5b3cf2861e8ef..0a5f19eee7105 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ @@ -128,11 +129,11 @@ case class HiveTableScan( } } - override def execute() = if (!relation.hiveQlTable.isPartitioned) { + override def execute(): RDD[Row] = if (!relation.hiveQlTable.isPartitioned) { hadoopReader.makeRDDForTable(relation.hiveQlTable) } else { hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) } - override def output = attributes + override def output: Seq[Attribute] = attributes } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index ba5c8e028a151..da53d30354551 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{UnaryNode, SparkPlan} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} @@ -58,7 +58,7 @@ case class InsertIntoHiveTable( serializer } - def output = child.output + def output: Seq[Attribute] = child.output def saveAsHiveFile( rdd: RDD[Row], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0c9aee33985bc..8efed7f0299bf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ @@ -51,9 +52,9 @@ case class ScriptTransformation( ioschema: HiveScriptIOSchema)(@transient sc: HiveContext) extends UnaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs: Seq[HiveContext] = sc :: Nil - def execute() = { + def execute(): RDD[Row] = { child.execute().mapPartitions { iter => val cmd = List("/bin/bash", "-c", script) val builder = new ProcessBuilder(cmd) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 63ad145a6a980..4345ffbf30f77 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType private[hive] case class AnalyzeTable(tableName: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.asInstanceOf[HiveContext].analyze(tableName) Seq.empty[Row] } @@ -52,7 +52,7 @@ case class DropTable( tableName: String, ifExists: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] val ifExistsClause = if (ifExists) "IF EXISTS " else "" try { @@ -75,7 +75,7 @@ case class DropTable( private[hive] case class AddJar(path: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) @@ -86,7 +86,7 @@ case class AddJar(path: String) extends RunnableCommand { private[hive] case class AddFile(path: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD FILE $path") hiveContext.sparkContext.addFile(path) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 4a702d96563d5..bfe43373d9534 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -45,7 +45,7 @@ import scala.collection.JavaConversions._ private[hive] abstract class HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { - def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) + def getFunctionInfo(name: String): FunctionInfo = FunctionRegistry.getFunctionInfo(name) def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is @@ -78,7 +78,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre type EvaluatedType = Any type UDFType = UDF - def nullable = true + override def nullable: Boolean = true @transient lazy val function = funcWrapper.createFunction[UDFType]() @@ -96,7 +96,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre udfType != null && udfType.deterministic() } - override def foldable = isUDFDeterministic && children.forall(_.foldable) + override def foldable: Boolean = isUDFDeterministic && children.forall(_.foldable) // Create parameter converters @transient @@ -110,7 +110,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre method.getGenericReturnType(), ObjectInspectorOptions.JAVA) @transient - protected lazy val cached = new Array[AnyRef](children.length) + protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) // TODO: Finish input output types. override def eval(input: Row): Any = { @@ -120,17 +120,19 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre returnInspector) } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } } // Adapter from Catalyst ExpressionResult to Hive DeferredObject private[hive] class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject with HiveInspectors { private var func: () => Any = _ - def set(func: () => Any) { + def set(func: () => Any): Unit = { this.func = func } - override def prepare(i: Int) = {} + override def prepare(i: Int): Unit = {} override def get(): AnyRef = wrap(func(), oi) } @@ -139,7 +141,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr type UDFType = GenericUDF type EvaluatedType = Any - def nullable = true + override def nullable: Boolean = true @transient lazy val function = funcWrapper.createFunction[UDFType]() @@ -158,7 +160,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr (udfType != null && udfType.deterministic()) } - override def foldable = + override def foldable: Boolean = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] @transient @@ -182,7 +184,9 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr unwrap(function.evaluate(deferedObjects), returnInspector) } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } } private[hive] case class HiveGenericUdaf( @@ -209,9 +213,11 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } - def newInstance() = new HiveUdafFunction(funcWrapper, children, this) + def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this) } /** It is used as a wrapper for the hive functions which uses UDAF interface */ @@ -240,10 +246,11 @@ private[hive] case class HiveUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } - def newInstance() = - new HiveUdafFunction(funcWrapper, children, this, true) + def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this, true) } /** @@ -314,21 +321,23 @@ private[hive] case class HiveGenericUdtf( collected += unwrap(input, outputInspector).asInstanceOf[Row] } - def collectRows() = { + def collectRows(): Seq[Row] = { val toCollect = collected collected = new ArrayBuffer[Row] toCollect } } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } } /** * Resolve Udtfs Alias. */ private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan) = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { case p @ Project(projectList, _) if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 1 => throw new TreeNodeException(p, "only single Generator supported for SELECT clause") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index f136e43acc8f2..ba2bf67aed684 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -222,7 +222,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( s"/$col=$colString" }.mkString - def newWriter = { + def newWriter(): FileSinkOperator.RecordWriter = { val newFileSinkDesc = new FileSinkDesc( fileSinkConf.getDirName + dynamicPartPath, fileSinkConf.getTableInfo, @@ -246,6 +246,6 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( Reporter.NULL) } - writers.getOrElseUpdate(dynamicPartPath, newWriter) + writers.getOrElseUpdate(dynamicPartPath, newWriter()) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index b4aee78046383..dc61e9d2e3522 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -155,8 +155,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution(HiveQl.parseSql(hql)) { - def hiveExec() = runSqlHive(hql) - override def toString = hql + "\n" + super.toString + def hiveExec(): Seq[String] = runSqlHive(hql) + override def toString: String = hql + "\n" + super.toString } /** @@ -186,7 +186,9 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { case class TestTable(name: String, commands: (()=>Unit)*) protected[hive] implicit class SqlCmd(sql: String) { - def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit + def cmd: () => Unit = { + () => new HiveQLQueryExecution(sql).stringResult(): Unit + } } /** @@ -194,7 +196,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { * demand when a query are run against it. */ lazy val testTables = new mutable.HashMap[String, TestTable]() - def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) + + def registerTestTable(testTable: TestTable): Unit = { + testTables += (testTable.name -> testTable) + } // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java From 9b1e1f20d4498bda72dd53a832110883a7ca41b5 Mon Sep 17 00:00:00 2001 From: ypcat Date: Sun, 22 Mar 2015 15:49:13 +0800 Subject: [PATCH 484/817] [SPARK-6408] [SQL] Fix JDBCRDD filtering string literals Author: ypcat Author: Pei-Lun Lee Closes #5087 from ypcat/spark-6408 and squashes the following commits: 1becc16 [ypcat] [SPARK-6408] [SQL] styling 1bc4455 [ypcat] [SPARK-6408] [SQL] move nested function outside e57fa4a [ypcat] [SPARK-6408] [SQL] fix test case 245ab6f [ypcat] [SPARK-6408] [SQL] add test cases for filtering quoted strings 8962534 [Pei-Lun Lee] [SPARK-6408] [SQL] Fix filtering string literals --- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 19 ++++++++++++++----- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 12 ++++++++++-- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 3266b972128ea..76f8593180e85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.jdbc import java.sql.{Connection, DriverManager, ResultSet, ResultSetMetaData, SQLException} +import org.apache.commons.lang.StringEscapeUtils.escapeSql import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow} @@ -226,16 +227,24 @@ private[sql] class JDBCRDD( if (sb.length == 0) "1" else sb.substring(1) } + /** + * Converts value to SQL expression. + */ + private def compileValue(value: Any): Any = value match { + case stringValue: String => s"'${escapeSql(stringValue)}'" + case _ => value + } + /** * Turns a single Filter into a String representing a SQL expression. * Returns null for an unhandled filter. */ private def compileFilter(f: Filter): String = f match { - case EqualTo(attr, value) => s"$attr = $value" - case LessThan(attr, value) => s"$attr < $value" - case GreaterThan(attr, value) => s"$attr > $value" - case LessThanOrEqual(attr, value) => s"$attr <= $value" - case GreaterThanOrEqual(attr, value) => s"$attr >= $value" + case EqualTo(attr, value) => s"$attr = ${compileValue(value)}" + case LessThan(attr, value) => s"$attr < ${compileValue(value)}" + case GreaterThan(attr, value) => s"$attr > ${compileValue(value)}" + case LessThanOrEqual(attr, value) => s"$attr <= ${compileValue(value)}" + case GreaterThanOrEqual(attr, value) => s"$attr >= ${compileValue(value)}" case _ => null } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index cd737c0b62767..5eb6ab2e92e8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -24,6 +24,7 @@ import java.util.{Calendar, GregorianCalendar} import org.apache.spark.sql.test._ import org.scalatest.{FunSuite, BeforeAndAfter} import TestSQLContext._ +import TestSQLContext.implicits._ class JDBCSuite extends FunSuite with BeforeAndAfter { val url = "jdbc:h2:mem:testdb0" @@ -38,7 +39,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() - conn.prepareStatement("insert into test.people values ('joe', 3)").executeUpdate() + conn.prepareStatement("insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() conn.commit() sql( @@ -129,13 +130,20 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size == 0) assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size == 2) assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size == 1) + assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size == 1) + assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size == 2) + assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size == 2) + } + + test("SELECT * WHERE (quoted strings)") { + assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size == 1) } test("SELECT first field") { val names = sql("SELECT NAME FROM foobar").collect().map(x => x.getString(0)).sortWith(_ < _) assert(names.size == 3) assert(names(0).equals("fred")) - assert(names(1).equals("joe")) + assert(names(1).equals("joe 'foo' \"bar\"")) assert(names(2).equals("mary")) } From b9fe504b497cfa509310b4045de4873739c76667 Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Sun, 22 Mar 2015 11:54:23 +0000 Subject: [PATCH 485/817] [SPARK-6448] Make history server log parse exceptions This helped me to debug a parse error that was due to the event log format changing recently. Author: Ryan Williams Closes #5122 from ryan-williams/histerror and squashes the following commits: 5831656 [Ryan Williams] line length c3742ae [Ryan Williams] Make history server log parse exceptions --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 7fde02040927d..db7c499661319 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -233,7 +233,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } catch { case e: Exception => logError( - s"Exception encountered when attempting to load application log ${fileStatus.getPath}") + s"Exception encountered when attempting to load application log ${fileStatus.getPath}", + e) None } }.toSeq.sortWith(compareAppInfo) From ab4f516fbe63e24e076c68f4933a171a72b6f1fd Mon Sep 17 00:00:00 2001 From: Hangchen Yu Date: Sun, 22 Mar 2015 15:51:10 +0000 Subject: [PATCH 486/817] [SPARK-6455] [docs] Correct some mistakes and typos Correct some typos. Correct a mistake in lib/PageRank.scala. The first PageRank implementation uses standalone Graph interface, but the second uses Pregel interface. It may mislead the code viewers. Author: Hangchen Yu Closes #5128 from yuhc/master and squashes the following commits: 53e5432 [Hangchen Yu] Merge branch 'master' of https://github.com/yuhc/spark 67b77b5 [Hangchen Yu] [SPARK-6455] [docs] Correct some mistakes and typos 206f2dc [Hangchen Yu] Correct some mistakes and typos. --- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 4 ++-- graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala | 4 ++-- .../src/main/scala/org/apache/spark/graphx/lib/PageRank.scala | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index dc8b4789c4b61..86f611d55aa8a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -113,7 +113,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Collect the neighbor vertex attributes for each vertex. * * @note This function could be highly inefficient on power-law - * graphs where high degree vertices may force a large ammount of + * graphs where high degree vertices may force a large amount of * information to be collected to a single location. * * @param edgeDirection the direction along which to collect @@ -187,7 +187,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali /** * Join the vertices with an RDD and then apply a function from the - * the vertex and RDD entry to a new vertex value. The input table + * vertex and RDD entry to a new vertex value. The input table * should contain at most one entry for each vertex. If no entry is * provided the map function is skipped and the old value is used. * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 5e55620147df8..01b013ff716fc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -78,8 +78,8 @@ object Pregel extends Logging { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the on - * the first iteration + * @param initialMsg the message each vertex will receive at the first + * iteration * * @param maxIterations the maximum number of iterations to run for * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index e139959c3f5c1..ca3b513821e13 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -25,7 +25,7 @@ import org.apache.spark.graphx._ /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. * - * The first implementation uses the [[Pregel]] interface and runs PageRank for a fixed number + * The first implementation uses the standalone [[Graph]] interface and runs PageRank for a fixed number * of iterations: * {{{ * var PR = Array.fill(n)( 1.0 ) @@ -38,7 +38,7 @@ import org.apache.spark.graphx._ * } * }}} * - * The second implementation uses the standalone [[Graph]] interface and runs PageRank until + * The second implementation uses the [[Pregel]] interface and runs PageRank until * convergence: * * {{{ From adb2ff752fa8bda54c969b60a3168d87cd70237d Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Sun, 22 Mar 2015 15:53:18 +0000 Subject: [PATCH 487/817] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Moved Suites from o.a.s.s.mesos to o.a.s.s.cluster.mesos Author: Jongyoul Lee Closes #5126 from jongyoul/SPARK-6453 and squashes the following commits: 4f24a3e [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed imports orders 8ab149d [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Moved Suites from o.a.s.s.mesos to o.a.s.s.cluster.mesos --- .../mesos/MesosSchedulerBackendSuite.scala | 13 ++++++------- .../mesos/MesosTaskLaunchDataSuite.scala | 4 +--- 2 files changed, 7 insertions(+), 10 deletions(-) rename core/src/test/scala/org/apache/spark/scheduler/{ => cluster}/mesos/MesosSchedulerBackendSuite.scala (98%) rename core/src/test/scala/org/apache/spark/scheduler/{ => cluster}/mesos/MesosTaskLaunchDataSuite.scala (92%) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index afbaa9ade811f..f1a4380d349b3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import java.util @@ -24,21 +24,20 @@ import java.util.Collections import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.Scalar -import org.mockito.Mockito._ +import org.apache.mesos.Protos._ +import org.apache.mesos.SchedulerDriver import org.mockito.Matchers._ +import org.mockito.Mockito._ import org.mockito.{ArgumentCaptor, Matchers} import org.scalatest.FunSuite import org.scalatest.mock.MockitoSugar -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala similarity index 92% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala index 86a42a7398e4d..eebcba40f8a1c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala @@ -15,14 +15,12 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import org.scalatest.FunSuite -import org.apache.spark.scheduler.cluster.mesos.MesosTaskLaunchData - class MesosTaskLaunchDataSuite extends FunSuite { test("serialize and deserialize data must be same") { val serializedTask = ByteBuffer.allocate(40) From 6ef48632fbf3e6659ceacaab1dbb8be8238d4d33 Mon Sep 17 00:00:00 2001 From: Kamil Smuga Date: Sun, 22 Mar 2015 15:56:25 +0000 Subject: [PATCH 488/817] SPARK-6454 [DOCS] Fix links to pyspark api Author: Kamil Smuga Author: stderr Closes #5120 from kamilsmuga/master and squashes the following commits: fee3281 [Kamil Smuga] more python api links fixed for docs 13240cb [Kamil Smuga] resolved merge conflicts with upstream/master 6649b3b [Kamil Smuga] fix broken docs links to Python API 92f03d7 [stderr] Fix links to pyspark api --- docs/mllib-data-types.md | 8 ++++---- docs/mllib-naive-bayes.md | 6 +++--- docs/mllib-statistics.md | 10 +++++----- docs/programming-guide.md | 12 ++++++------ docs/sql-programming-guide.md | 2 +- 5 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index fe6c1bf7bfd99..4f2a2f71048f7 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -78,13 +78,13 @@ MLlib recognizes the following types as dense vectors: and the following as sparse vectors: -* MLlib's [`SparseVector`](api/python/pyspark.mllib.linalg.SparseVector-class.html). +* MLlib's [`SparseVector`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.SparseVector). * SciPy's [`csc_matrix`](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) with a single column We recommend using NumPy arrays over lists for efficiency, and using the factory methods implemented -in [`Vectors`](api/python/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. +in [`Vectors`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Vector) to create sparse vectors. {% highlight python %} import numpy as np @@ -151,7 +151,7 @@ LabeledPoint neg = new LabeledPoint(1.0, Vectors.sparse(3, new int[] {0, 2}, new
    A labeled point is represented by -[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html). +[`LabeledPoint`](api/python/pyspark.mllib.html#pyspark.mllib.regression.LabeledPoint). {% highlight python %} from pyspark.mllib.linalg import SparseVector @@ -211,7 +211,7 @@ JavaRDD examples =
    -[`MLUtils.loadLibSVMFile`](api/python/pyspark.mllib.util.MLUtils-class.html) reads training +[`MLUtils.loadLibSVMFile`](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) reads training examples stored in LIBSVM format. {% highlight python %} diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 55b8f2ce6c364..a83472f5be52e 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -106,11 +106,11 @@ NaiveBayesModel sameModel = NaiveBayesModel.load(sc.sc(), "myModelPath");
    -[NaiveBayes](api/python/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +[NaiveBayes](api/python/pyspark.mllib.html#pyspark.mllib.classification.NaiveBayes) implements multinomial naive Bayes. It takes an RDD of -[LabeledPoint](api/python/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +[LabeledPoint](api/python/pyspark.mllib.html#pyspark.mllib.regression.LabeledPoint) and an optionally smoothing parameter `lambda` as input, and output a -[NaiveBayesModel](api/python/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +[NaiveBayesModel](api/python/pyspark.mllib.html#pyspark.mllib.classification.NaiveBayesModel), which can be used for evaluation and prediction. Note that the Python API does not yet support model save/load but will in the future. diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index ca8c29218f52d..887eae7f4f07b 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -81,8 +81,8 @@ System.out.println(summary.numNonzeros()); // number of nonzeros in each column
    -[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of -[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +[`colStats()`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics.colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.html#pyspark.mllib.stat.MultivariateStatisticalSummary), which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the total count. @@ -169,7 +169,7 @@ Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson");
    -[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +[`Statistics`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) provides methods to calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. @@ -258,7 +258,7 @@ JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); {% endhighlight %}
    -[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +[`sampleByKey()`](api/python/pyspark.html#pyspark.RDD.sampleByKey) allows users to sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of keys. @@ -476,7 +476,7 @@ JavaDoubleRDD v = u.map(
    -[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +[`RandomRDDs`](api/python/pyspark.mllib.html#pyspark.mllib.random.RandomRDDs) provides factory methods to generate random double RDDs or vector RDDs. The following example generates a random double RDD, whose values follows the standard normal distribution `N(0, 1)`, and then map it to `N(1, 4)`. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index eda3a95426182..5fe832b6fa100 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -142,8 +142,8 @@ JavaSparkContext sc = new JavaSparkContext(conf);
    -The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.context.SparkContext-class.html) object, which tells Spark -how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.conf.SparkConf-class.html) object +The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.html#pyspark.SparkConf) object that contains information about your application. {% highlight python %} @@ -912,7 +912,7 @@ The following table lists some of the common transformations supported by Spark. RDD API doc ([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.rdd.RDD-class.html)) + [Python](api/python/pyspark.html#pyspark.RDD)) and pair RDD functions doc ([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) @@ -1025,7 +1025,7 @@ The following table lists some of the common actions supported by Spark. Refer t RDD API doc ([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.rdd.RDD-class.html)) + [Python](api/python/pyspark.html#pyspark.RDD)) and pair RDD functions doc ([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) @@ -1105,7 +1105,7 @@ replicate it across nodes, or store it off-heap in [Tachyon](http://tachyon-proj These levels are set by passing a `StorageLevel` object ([Scala](api/scala/index.html#org.apache.spark.storage.StorageLevel), [Java](api/java/index.html?org/apache/spark/storage/StorageLevel.html), -[Python](api/python/pyspark.storagelevel.StorageLevel-class.html)) +[Python](api/python/pyspark.html#pyspark.StorageLevel)) to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The full set of storage levels is: @@ -1374,7 +1374,7 @@ scala> accum.value {% endhighlight %} While this code used the built-in support for accumulators of type Int, programmers can also -create their own types by subclassing [AccumulatorParam](api/python/pyspark.accumulators.AccumulatorParam-class.html). +create their own types by subclassing [AccumulatorParam](api/python/pyspark.html#pyspark.AccumulatorParam). The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class representing mathematical vectors, we could write: diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2cbb4c967eb81..a7d35741a48c3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -56,7 +56,7 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
    The entry point into all relational functionality in Spark is the -[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one +[`SQLContext`](api/python/pyspark.sql.html#pyspark.sql.SQLContext) class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight python %} From a41b9c6004cfee84bd56dfa1faf5a0cf084551ae Mon Sep 17 00:00:00 2001 From: Calvin Jia Date: Sun, 22 Mar 2015 11:11:29 -0700 Subject: [PATCH 489/817] [SPARK-6122][Core] Upgrade Tachyon client version to 0.6.1. Changes the Tachyon client version from 0.5 to 0.6 in spark core and distribution script. New dependencies in Tachyon 0.6.0 include commons-codec:commons-codec:jar:1.5:compile io.netty:netty-all:jar:4.0.23.Final:compile These are already in spark core. Author: Calvin Jia Closes #4867 from calvinjia/upgrade_tachyon_0.6.0 and squashes the following commits: eed9230 [Calvin Jia] Update tachyon version to 0.6.1. 11907b3 [Calvin Jia] Use TachyonURI for tachyon paths instead of strings. 71bf441 [Calvin Jia] Upgrade Tachyon client version to 0.6.0. --- core/pom.xml | 2 +- .../spark/storage/TachyonBlockManager.scala | 27 +++++++++---------- .../scala/org/apache/spark/util/Utils.scala | 4 ++- make-distribution.sh | 2 +- 4 files changed, 18 insertions(+), 17 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 6cd1965ec37c2..868834dd505ef 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -275,7 +275,7 @@ org.tachyonproject tachyon-client - 0.5.0 + 0.6.1 org.apache.hadoop diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index af873034215a9..2ab6a8f3ec1d4 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -20,8 +20,8 @@ package org.apache.spark.storage import java.text.SimpleDateFormat import java.util.{Date, Random} -import tachyon.client.TachyonFS -import tachyon.client.TachyonFile +import tachyon.TachyonURI +import tachyon.client.{TachyonFile, TachyonFS} import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode @@ -40,7 +40,7 @@ private[spark] class TachyonBlockManager( val master: String) extends Logging { - val client = if (master != null && master != "") TachyonFS.get(master) else null + val client = if (master != null && master != "") TachyonFS.get(new TachyonURI(master)) else null if (client == null) { logError("Failed to connect to the Tachyon as the master address is not configured") @@ -60,11 +60,11 @@ private[spark] class TachyonBlockManager( addShutdownHook() def removeFile(file: TachyonFile): Boolean = { - client.delete(file.getPath(), false) + client.delete(new TachyonURI(file.getPath()), false) } def fileExists(file: TachyonFile): Boolean = { - client.exist(file.getPath()) + client.exist(new TachyonURI(file.getPath())) } def getFile(filename: String): TachyonFile = { @@ -81,7 +81,7 @@ private[spark] class TachyonBlockManager( if (old != null) { old } else { - val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) + val path = new TachyonURI(s"${tachyonDirs(dirId)}/${"%02x".format(subDirId)}") client.mkdir(path) val newDir = client.getFile(path) subDirs(dirId)(subDirId) = newDir @@ -89,7 +89,7 @@ private[spark] class TachyonBlockManager( } } } - val filePath = subDir + "/" + filename + val filePath = new TachyonURI(s"$subDir/$filename") if(!client.exist(filePath)) { client.createFile(filePath) } @@ -101,7 +101,7 @@ private[spark] class TachyonBlockManager( // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. private def createTachyonDirs(): Array[TachyonFile] = { - logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") + logDebug(s"Creating tachyon directories at root dirs '$rootDirs'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") rootDirs.split(",").map { rootDir => var foundLocalDir = false @@ -113,22 +113,21 @@ private[spark] class TachyonBlockManager( tries += 1 try { tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId + val path = new TachyonURI(s"$rootDir/spark-tachyon-$tachyonDirId") if (!client.exist(path)) { foundLocalDir = client.mkdir(path) tachyonDir = client.getFile(path) } } catch { case e: Exception => - logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) + logWarning(s"Attempt $tries to create tachyon dir $tachyonDir failed", e) } } if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + - rootDir) + logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create tachyon dir in $rootDir") System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) } - logInfo("Created tachyon directory at " + tachyonDir) + logInfo(s"Created tachyon directory at $tachyonDir") tachyonDir } } @@ -145,7 +144,7 @@ private[spark] class TachyonBlockManager( } } catch { case e: Exception => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) + logError(s"Exception while deleting tachyon spark dir: $tachyonDir", e) } } client.close() diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fa56bb09e4e5c..91d833295e376 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -42,6 +42,8 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ + +import tachyon.TachyonURI import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ @@ -970,7 +972,7 @@ private[spark] object Utils extends Logging { * Delete a file or directory and its contents recursively. */ def deleteRecursively(dir: TachyonFile, client: TachyonFS) { - if (!client.delete(dir.getPath(), true)) { + if (!client.delete(new TachyonURI(dir.getPath()), true)) { throw new IOException("Failed to delete the tachyon dir: " + dir) } } diff --git a/make-distribution.sh b/make-distribution.sh index 9ed1abfe8c598..8162fe94c1af0 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -32,7 +32,7 @@ SPARK_HOME="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$SPARK_HOME/dist" SPARK_TACHYON=false -TACHYON_VERSION="0.5.0" +TACHYON_VERSION="0.6.1" TACHYON_TGZ="tachyon-${TACHYON_VERSION}-bin.tar.gz" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/${TACHYON_TGZ}" From 7a0da47708b0e6b117b5c1a35aa3e93b8a914d5f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 22 Mar 2015 12:08:15 -0700 Subject: [PATCH 490/817] [HOTFIX] Build break due to https://github.com/apache/spark/pull/5128 --- .../src/main/scala/org/apache/spark/graphx/lib/PageRank.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index ca3b513821e13..570440ba4441f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -25,8 +25,8 @@ import org.apache.spark.graphx._ /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. * - * The first implementation uses the standalone [[Graph]] interface and runs PageRank for a fixed number - * of iterations: + * The first implementation uses the standalone [[Graph]] interface and runs PageRank + * for a fixed number of iterations: * {{{ * var PR = Array.fill(n)( 1.0 ) * val oldPR = Array.fill(n)( 1.0 ) From 2bf40c58e6e89e061783c999204107069df17f73 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Sun, 22 Mar 2015 20:00:08 +0000 Subject: [PATCH 491/817] [SPARK-6337][Documentation, SQL]Spark 1.3 doc fixes Author: vinodkc Closes #5112 from vinodkc/spark_1.3_doc_fixes and squashes the following commits: 2c6aee6 [vinodkc] Spark 1.3 doc fixes --- docs/sql-programming-guide.md | 7 +++++-- mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala | 2 +- .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a7d35741a48c3..6a333fdb562a7 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -509,8 +509,11 @@ val people = sc.textFile("examples/src/main/resources/people.txt") // The schema is encoded in a string val schemaString = "name age" -// Import Spark SQL data types and Row. -import org.apache.spark.sql._ +// Import Row. +import org.apache.spark.sql.Row; + +// Import Spark SQL data types +import org.apache.spark.sql.types.{StructType,StructField,StringType}; // Generate the schema based on the string of schema val schema = diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index 5bbcd2e080e07..c4a36103303a2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.types.StructType abstract class PipelineStage extends Serializable with Logging { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * * Derives the output schema from the input schema and parameters. * The schema describes the columns and types of the data. 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 8b8f86c4127e0..5aece166aad22 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 @@ -89,7 +89,7 @@ private[sql] object DataFrame { * val people = sqlContext.parquetFile("...") * val department = sqlContext.parquetFile("...") * - * people.filter("age" > 30) + * people.filter("age > 30") * .join(department, people("deptId") === department("id")) * .groupBy(department("name"), "gender") * .agg(avg(people("salary")), max(people("age"))) From 4659468f369d69e7f777130e5e3b4c5d47a624f1 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 23 Mar 2015 11:46:16 +0800 Subject: [PATCH 492/817] [SPARK-4985] [SQL] parquet support for date type This PR might have some issues with #3732 , and this would have merge conflicts with #3820 so the review can be delayed till that 2 were merged. Author: Daoyuan Wang Closes #3822 from adrian-wang/parquetdate and squashes the following commits: 2c5d54d [Daoyuan Wang] add a test case faef887 [Daoyuan Wang] parquet support for primitive date 97e9080 [Daoyuan Wang] parquet support for date type --- .../spark/sql/parquet/ParquetConverter.scala | 12 ++++++++++++ .../spark/sql/parquet/ParquetTableSupport.scala | 2 ++ .../apache/spark/sql/parquet/ParquetTypes.scala | 4 ++++ .../apache/spark/sql/parquet/ParquetIOSuite.scala | 15 +++++++++++++++ .../spark/sql/parquet/ParquetSchemaSuite.scala | 3 ++- 5 files changed, 35 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index f898e4b37a56b..43ca359b51735 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -127,6 +127,12 @@ private[sql] object CatalystConverter { parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.JvmType]) } } + case DateType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addInt(value: Int): Unit = + parent.updateDate(fieldIndex, value.asInstanceOf[DateType.JvmType]) + } + } case d: DecimalType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = @@ -192,6 +198,9 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = updateField(fieldIndex, value) + protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = + updateField(fieldIndex, value) + protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = updateField(fieldIndex, value) @@ -388,6 +397,9 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = current.setInt(fieldIndex, value) + override protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = + current.update(fieldIndex, value) + override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = current.setLong(fieldIndex, value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 19bfba34b8f4a..5a1b15490d273 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -212,6 +212,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) + case DateType => writer.addInteger(value.asInstanceOf[Int]) case d: DecimalType => if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { sys.error(s"Unsupported datatype $d, cannot write to consumer") @@ -358,6 +359,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { case DoubleType => writer.addDouble(record.getDouble(index)) case FloatType => writer.addFloat(record.getFloat(index)) case BooleanType => writer.addBoolean(record.getBoolean(index)) + case DateType => writer.addInteger(record.getInt(index)) case TimestampType => writeTimestamp(record(index).asInstanceOf[java.sql.Timestamp]) case d: DecimalType => if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 5209581fa8357..da668f068613b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -64,6 +64,8 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetPrimitiveTypeName.BOOLEAN => BooleanType case ParquetPrimitiveTypeName.DOUBLE => DoubleType case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 + if originalType == ParquetOriginalType.DATE => DateType case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType case ParquetPrimitiveTypeName.INT96 if int96AsTimestamp => TimestampType @@ -222,6 +224,8 @@ private[parquet] object ParquetTypesConverter extends Logging { // There is no type for Byte or Short so we promote them to INT32. case ShortType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) case ByteType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) + case DateType => Some(ParquetTypeInfo( + ParquetPrimitiveTypeName.INT32, Some(ParquetOriginalType.DATE))) case LongType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT64)) case TimestampType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT96)) case DecimalType.Fixed(precision, scale) if precision <= 18 => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 5438095addeaf..203bc79f153dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -135,6 +135,21 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } + test("date type") { + def makeDateRDD(): DataFrame = + sparkContext + .parallelize(0 to 1000) + .map(i => Tuple1(DateUtils.toJavaDate(i))) + .toDF() + .select($"_1") + + withTempPath { dir => + val data = makeDateRDD() + data.saveAsParquetFile(dir.getCanonicalPath) + checkAnswer(parquetFile(dir.getCanonicalPath), data.collect().toSeq) + } + } + test("map") { val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) checkParquetFile(data) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index ad880e2bc3679..321832cd43211 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -57,7 +57,7 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { |} """.stripMargin) - testSchema[(Byte, Short, Int, Long)]( + testSchema[(Byte, Short, Int, Long, java.sql.Date)]( "logical integral types", """ |message root { @@ -65,6 +65,7 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { | required int32 _2 (INT_16); | required int32 _3 (INT_32); | required int64 _4 (INT_64); + | optional int32 _5 (DATE); |} """.stripMargin) From e566fe5982bac5d24e6be76e5d7d6270544a85e6 Mon Sep 17 00:00:00 2001 From: q00251598 Date: Mon, 23 Mar 2015 12:06:13 +0800 Subject: [PATCH 493/817] [SPARK-6397][SQL] Check the missingInput simply Author: q00251598 Closes #5082 from watermen/sql-missingInput and squashes the following commits: 25766b9 [q00251598] Check the missingInput simply --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 5 ++--- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 2 ++ 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 4e8fc892f3eea..fb975ee5e7296 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -85,9 +85,8 @@ class CheckAnalysis { cleaned.foreach(checkValidAggregateExpression) - case o if o.children.nonEmpty && - !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => - val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") + case o if o.children.nonEmpty && o.missingInput.nonEmpty => + val missingAttributes = o.missingInput.map(_.prettyString).mkString(",") val input = o.inputSet.map(_.prettyString).mkString(",") failAnalysis(s"resolved attributes $missingAttributes missing from $input") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 384fe53a68362..a94b2d2095d12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -191,6 +191,8 @@ case class Expand( val sizeInBytes = child.statistics.sizeInBytes * projections.length Statistics(sizeInBytes = sizeInBytes) } + + override def missingInput = super.missingInput.filter(_.name != VirtualColumn.groupingIdName) } trait GroupingAnalytics extends UnaryNode { From bf044def4c3a37a0fd4d5e70c2d57685cfd9fd71 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 23 Mar 2015 12:15:19 +0800 Subject: [PATCH 494/817] Revert "[SPARK-6397][SQL] Check the missingInput simply" This reverts commit e566fe5982bac5d24e6be76e5d7d6270544a85e6. --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 5 +++-- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 2 -- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fb975ee5e7296..4e8fc892f3eea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -85,8 +85,9 @@ class CheckAnalysis { cleaned.foreach(checkValidAggregateExpression) - case o if o.children.nonEmpty && o.missingInput.nonEmpty => - val missingAttributes = o.missingInput.map(_.prettyString).mkString(",") + case o if o.children.nonEmpty && + !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => + val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") val input = o.inputSet.map(_.prettyString).mkString(",") failAnalysis(s"resolved attributes $missingAttributes missing from $input") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index a94b2d2095d12..384fe53a68362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -191,8 +191,6 @@ case class Expand( val sizeInBytes = child.statistics.sizeInBytes * projections.length Statistics(sizeInBytes = sizeInBytes) } - - override def missingInput = super.missingInput.filter(_.name != VirtualColumn.groupingIdName) } trait GroupingAnalytics extends UnaryNode { From 9f3273bd9c919f6c48a95383b3d5be357c89998c Mon Sep 17 00:00:00 2001 From: Yadong Qi Date: Mon, 23 Mar 2015 18:16:49 +0800 Subject: [PATCH 495/817] [SPARK-6397][SQL] Check the missingInput simply https://github.com/apache/spark/pull/5082 /cc liancheng Author: Yadong Qi Closes #5132 from watermen/sql-missingInput-new and squashes the following commits: 1e5bdc5 [Yadong Qi] Check the missingInput simply --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 5 ++--- .../org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 4e8fc892f3eea..fb975ee5e7296 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -85,9 +85,8 @@ class CheckAnalysis { cleaned.foreach(checkValidAggregateExpression) - case o if o.children.nonEmpty && - !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => - val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") + case o if o.children.nonEmpty && o.missingInput.nonEmpty => + val missingAttributes = o.missingInput.map(_.prettyString).mkString(",") val input = o.inputSet.map(_.prettyString).mkString(",") failAnalysis(s"resolved attributes $missingAttributes missing from $input") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 17a88e07de15f..400a6b2825c10 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} +import org.apache.spark.sql.catalyst.expressions.{VirtualColumn, Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType} @@ -48,7 +48,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy * Subclasses should override this method if they produce attributes internally as it is used by * assertions designed to prevent the construction of invalid plans. */ - def missingInput: AttributeSet = references -- inputSet + def missingInput: AttributeSet = (references -- inputSet) + .filter(_.name != VirtualColumn.groupingIdName) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. From 474d1320c9b93c501710ad1cfa836b8284562a2c Mon Sep 17 00:00:00 2001 From: MechCoder Date: Mon, 23 Mar 2015 13:30:21 -0700 Subject: [PATCH 496/817] [SPARK-6308] [MLlib] [Sql] Override TypeName in VectorUDT and MatrixUDT Author: MechCoder Closes #5118 from MechCoder/spark-6308 and squashes the following commits: 6c8ffab [MechCoder] Add test for simpleString b966242 [MechCoder] [SPARK-6308] [MLlib][Sql] VectorUDT is displayed as vecto in dtypes --- .../src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala | 2 ++ .../src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 2 ++ .../scala/org/apache/spark/mllib/linalg/MatricesSuite.scala | 2 ++ .../test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala | 2 ++ 4 files changed, 8 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 849f44295f089..d1a174063caba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -187,6 +187,8 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { override def hashCode(): Int = 1994 + override def typeName: String = "matrix" + private[spark] override def asNullable: MatrixUDT = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 2cda9b252ee06..328dbe2ce11fa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -185,6 +185,8 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def hashCode: Int = 7919 + override def typeName: String = "vector" + private[spark] override def asNullable: VectorUDT = this } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 96f677db3f377..0d2cec58e2c03 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -436,5 +436,7 @@ class MatricesSuite extends FunSuite { Seq(dm1, dm2, dm3, sm1, sm2, sm3).foreach { mat => assert(mat.toArray === mUDT.deserialize(mUDT.serialize(mat)).toArray) } + assert(mUDT.typeName == "matrix") + assert(mUDT.simpleString == "matrix") } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 5def899cea117..2839c4c289b2d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -187,6 +187,8 @@ class VectorsSuite extends FunSuite { for (v <- Seq(dv0, dv1, sv0, sv1)) { assert(v === udt.deserialize(udt.serialize(v))) } + assert(udt.typeName == "vector") + assert(udt.simpleString == "vector") } test("fromBreeze") { From 6cd7058b369ec8d01938961f148734ee9eaf76de Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 23 Mar 2015 15:08:39 -0700 Subject: [PATCH 497/817] Revert "[SPARK-6122][Core] Upgrade Tachyon client version to 0.6.1." This reverts commit a41b9c6004cfee84bd56dfa1faf5a0cf084551ae. --- core/pom.xml | 2 +- .../spark/storage/TachyonBlockManager.scala | 27 ++++++++++--------- .../scala/org/apache/spark/util/Utils.scala | 4 +-- make-distribution.sh | 2 +- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 868834dd505ef..6cd1965ec37c2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -275,7 +275,7 @@ org.tachyonproject tachyon-client - 0.6.1 + 0.5.0 org.apache.hadoop diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index 2ab6a8f3ec1d4..af873034215a9 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -20,8 +20,8 @@ package org.apache.spark.storage import java.text.SimpleDateFormat import java.util.{Date, Random} -import tachyon.TachyonURI -import tachyon.client.{TachyonFile, TachyonFS} +import tachyon.client.TachyonFS +import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode @@ -40,7 +40,7 @@ private[spark] class TachyonBlockManager( val master: String) extends Logging { - val client = if (master != null && master != "") TachyonFS.get(new TachyonURI(master)) else null + val client = if (master != null && master != "") TachyonFS.get(master) else null if (client == null) { logError("Failed to connect to the Tachyon as the master address is not configured") @@ -60,11 +60,11 @@ private[spark] class TachyonBlockManager( addShutdownHook() def removeFile(file: TachyonFile): Boolean = { - client.delete(new TachyonURI(file.getPath()), false) + client.delete(file.getPath(), false) } def fileExists(file: TachyonFile): Boolean = { - client.exist(new TachyonURI(file.getPath())) + client.exist(file.getPath()) } def getFile(filename: String): TachyonFile = { @@ -81,7 +81,7 @@ private[spark] class TachyonBlockManager( if (old != null) { old } else { - val path = new TachyonURI(s"${tachyonDirs(dirId)}/${"%02x".format(subDirId)}") + val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) client.mkdir(path) val newDir = client.getFile(path) subDirs(dirId)(subDirId) = newDir @@ -89,7 +89,7 @@ private[spark] class TachyonBlockManager( } } } - val filePath = new TachyonURI(s"$subDir/$filename") + val filePath = subDir + "/" + filename if(!client.exist(filePath)) { client.createFile(filePath) } @@ -101,7 +101,7 @@ private[spark] class TachyonBlockManager( // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. private def createTachyonDirs(): Array[TachyonFile] = { - logDebug(s"Creating tachyon directories at root dirs '$rootDirs'") + logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") rootDirs.split(",").map { rootDir => var foundLocalDir = false @@ -113,21 +113,22 @@ private[spark] class TachyonBlockManager( tries += 1 try { tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - val path = new TachyonURI(s"$rootDir/spark-tachyon-$tachyonDirId") + val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId if (!client.exist(path)) { foundLocalDir = client.mkdir(path) tachyonDir = client.getFile(path) } } catch { case e: Exception => - logWarning(s"Attempt $tries to create tachyon dir $tachyonDir failed", e) + logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) } } if (!foundLocalDir) { - logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create tachyon dir in $rootDir") + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + + rootDir) System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) } - logInfo(s"Created tachyon directory at $tachyonDir") + logInfo("Created tachyon directory at " + tachyonDir) tachyonDir } } @@ -144,7 +145,7 @@ private[spark] class TachyonBlockManager( } } catch { case e: Exception => - logError(s"Exception while deleting tachyon spark dir: $tachyonDir", e) + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } client.close() diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 91d833295e376..fa56bb09e4e5c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -42,8 +42,6 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ - -import tachyon.TachyonURI import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ @@ -972,7 +970,7 @@ private[spark] object Utils extends Logging { * Delete a file or directory and its contents recursively. */ def deleteRecursively(dir: TachyonFile, client: TachyonFS) { - if (!client.delete(new TachyonURI(dir.getPath()), true)) { + if (!client.delete(dir.getPath(), true)) { throw new IOException("Failed to delete the tachyon dir: " + dir) } } diff --git a/make-distribution.sh b/make-distribution.sh index 8162fe94c1af0..9ed1abfe8c598 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -32,7 +32,7 @@ SPARK_HOME="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$SPARK_HOME/dist" SPARK_TACHYON=false -TACHYON_VERSION="0.6.1" +TACHYON_VERSION="0.5.0" TACHYON_TGZ="tachyon-${TACHYON_VERSION}-bin.tar.gz" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/${TACHYON_TGZ}" From bfd3ee9f76aaab3dcde71d92e2b8ca60a0e42262 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Mon, 23 Mar 2015 17:00:27 -0700 Subject: [PATCH 498/817] [SPARK-6124] Support jdbc connection properties in OPTIONS part of the query One more thing if this PR is considered to be OK - it might make sense to add extra .jdbc() API's that take Properties to SQLContext. Author: Volodymyr Lyubinets Closes #4859 from vlyubin/jdbcProperties and squashes the following commits: 7a8cfda [Volodymyr Lyubinets] Support jdbc connection properties in OPTIONS part of the query --- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 14 +++-- .../apache/spark/sql/jdbc/JDBCRelation.scala | 19 ++++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 55 +++++++++++++------ 3 files changed, 59 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 76f8593180e85..463e1dcc268bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.jdbc import java.sql.{Connection, DriverManager, ResultSet, ResultSetMetaData, SQLException} +import java.util.Properties import org.apache.commons.lang.StringEscapeUtils.escapeSql import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} @@ -90,9 +91,9 @@ private[sql] object JDBCRDD extends Logging { * @throws SQLException if the table specification is garbage. * @throws SQLException if the table contains an unsupported type. */ - def resolveTable(url: String, table: String): StructType = { + def resolveTable(url: String, table: String, properties: Properties): StructType = { val quirks = DriverQuirks.get(url) - val conn: Connection = DriverManager.getConnection(url) + val conn: Connection = DriverManager.getConnection(url, properties) try { val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0").executeQuery() try { @@ -147,7 +148,7 @@ private[sql] object JDBCRDD extends Logging { * * @return A function that loads the driver and connects to the url. */ - def getConnector(driver: String, url: String): () => Connection = { + def getConnector(driver: String, url: String, properties: Properties): () => Connection = { () => { try { if (driver != null) Class.forName(driver) @@ -156,7 +157,7 @@ private[sql] object JDBCRDD extends Logging { logWarning(s"Couldn't find class $driver", e); } } - DriverManager.getConnection(url) + DriverManager.getConnection(url, properties) } } /** @@ -179,6 +180,7 @@ private[sql] object JDBCRDD extends Logging { schema: StructType, driver: String, url: String, + properties: Properties, fqTable: String, requiredColumns: Array[String], filters: Array[Filter], @@ -189,7 +191,7 @@ private[sql] object JDBCRDD extends Logging { return new JDBCRDD( sc, - getConnector(driver, url), + getConnector(driver, url, properties), prunedSchema, fqTable, requiredColumns, @@ -361,7 +363,7 @@ private[sql] class JDBCRDD( var ans = 0L var j = 0 while (j < bytes.size) { - ans = 256*ans + (255 & bytes(j)) + ans = 256 * ans + (255 & bytes(j)) j = j + 1; } mutableRow.setLong(i, ans) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index df687e6da9bea..4fa84dc076f7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -17,16 +17,17 @@ package org.apache.spark.sql.jdbc -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.types.StructType +import java.sql.DriverManager +import java.util.Properties import scala.collection.mutable.ArrayBuffer -import java.sql.DriverManager import org.apache.spark.Partition +import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType /** * Data corresponding to one partition of a JDBCRDD. @@ -115,18 +116,21 @@ private[sql] class DefaultSource extends RelationProvider { numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) - JDBCRelation(url, table, parts)(sqlContext) + val properties = new Properties() // Additional properties that we will pass to getConnection + parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) + JDBCRelation(url, table, parts, properties)(sqlContext) } } private[sql] case class JDBCRelation( url: String, table: String, - parts: Array[Partition])(@transient val sqlContext: SQLContext) + parts: Array[Partition], + properties: Properties = new Properties())(@transient val sqlContext: SQLContext) extends BaseRelation with PrunedFilteredScan { - override val schema: StructType = JDBCRDD.resolveTable(url, table) + override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { val driver: String = DriverManager.getDriver(url).getClass.getCanonicalName @@ -135,6 +139,7 @@ private[sql] case class JDBCRelation( schema, driver, url, + properties, table, requiredColumns, filters, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 5eb6ab2e92e8b..592ed4b23b7d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -19,22 +19,31 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal import java.sql.DriverManager -import java.util.{Calendar, GregorianCalendar} +import java.util.{Calendar, GregorianCalendar, Properties} import org.apache.spark.sql.test._ +import org.h2.jdbc.JdbcSQLException import org.scalatest.{FunSuite, BeforeAndAfter} import TestSQLContext._ import TestSQLContext.implicits._ class JDBCSuite extends FunSuite with BeforeAndAfter { val url = "jdbc:h2:mem:testdb0" + val urlWithUserAndPass = "jdbc:h2:mem:testdb0;user=testUser;password=testPass" var conn: java.sql.Connection = null val testBytes = Array[Byte](99.toByte, 134.toByte, 135.toByte, 200.toByte, 205.toByte) before { Class.forName("org.h2.Driver") - conn = DriverManager.getConnection(url) + // Extra properties that will be specified for our database. We need these to test + // usage of parameters from OPTIONS clause in queries. + val properties = new Properties() + properties.setProperty("user", "testUser") + properties.setProperty("password", "testPass") + properties.setProperty("rowId", "false") + + conn = DriverManager.getConnection(url, properties) conn.prepareStatement("create schema test").executeUpdate() conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() @@ -46,15 +55,15 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE foobar |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.PEOPLE') + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) sql( s""" |CREATE TEMPORARY TABLE parts |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', - |partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', + | partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.inttypes (a INT, b BOOLEAN, c TINYINT, " @@ -68,12 +77,12 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE inttypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.INTTYPES') + |OPTIONS (url '$url', dbtable 'TEST.INTTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.strtypes (a BINARY(20), b VARCHAR(20), " + "c VARCHAR_IGNORECASE(20), d CHAR(20), e BLOB, f CLOB)").executeUpdate() - var stmt = conn.prepareStatement("insert into test.strtypes values (?, ?, ?, ?, ?, ?)") + val stmt = conn.prepareStatement("insert into test.strtypes values (?, ?, ?, ?, ?, ?)") stmt.setBytes(1, testBytes) stmt.setString(2, "Sensitive") stmt.setString(3, "Insensitive") @@ -85,7 +94,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE strtypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.STRTYPES') + |OPTIONS (url '$url', dbtable 'TEST.STRTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.timetypes (a TIME, b DATE, c TIMESTAMP)" @@ -97,7 +106,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE timetypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES') + |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) @@ -112,7 +121,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE flttypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES') + |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. @@ -174,16 +183,17 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("Basic API") { - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE").collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect.size == 3) } test("Partitioning via JDBCPartitioningInfo API") { - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", "THEID", 0, 4, 3).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3) + .collect.size == 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect.size == 3) } test("H2 integral types") { @@ -216,7 +226,6 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(rows(0).getString(5).equals("I am a clob!")) } - test("H2 time types") { val rows = sql("SELECT * FROM timetypes").collect() val cal = new GregorianCalendar(java.util.Locale.ROOT) @@ -246,17 +255,31 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { .equals(new BigDecimal("123456789012345.54321543215432100000"))) } - test("SQL query as table name") { sql( s""" |CREATE TEMPORARY TABLE hack |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)') + |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)', + | user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) val rows = sql("SELECT * FROM hack").collect() assert(rows(0).getDouble(0) == 1.00000011920928955) // Yes, I meant ==. // For some reason, H2 computes this square incorrectly... assert(math.abs(rows(0).getDouble(1) - 1.00000023841859331) < 1e-12) } + + test("Pass extra properties via OPTIONS") { + // We set rowId to false during setup, which means that _ROWID_ column should be absent from + // all tables. If rowId is true (default), the query below doesn't throw an exception. + intercept[JdbcSQLException] { + sql( + s""" + |CREATE TEMPORARY TABLE abc + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable '(SELECT _ROWID_ FROM test.people)', + | user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + } + } } From 4ce2782a61e23ed0326faac2ee97a9bd36ec8963 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 23 Mar 2015 23:41:06 -0700 Subject: [PATCH 499/817] [SPARK-6428] Added explicit types for all public methods in core. Author: Reynold Xin Closes #5125 from rxin/core-explicit-type and squashes the following commits: f471415 [Reynold Xin] Revert style checker changes. 81b66e4 [Reynold Xin] Code review feedback. a7533e3 [Reynold Xin] Mima excludes. 1d795f5 [Reynold Xin] [SPARK-6428] Added explicit types for all public methods in core. --- .../scala/org/apache/spark/Accumulators.scala | 23 +++-- .../scala/org/apache/spark/Dependency.scala | 6 +- .../scala/org/apache/spark/FutureAction.scala | 4 +- .../org/apache/spark/HeartbeatReceiver.scala | 2 +- .../org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/Partitioner.scala | 4 +- .../apache/spark/SerializableWritable.scala | 6 +- .../scala/org/apache/spark/SparkConf.scala | 2 +- .../scala/org/apache/spark/SparkContext.scala | 48 +++++----- .../scala/org/apache/spark/TaskState.scala | 4 +- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/api/java/JavaUtils.scala | 35 ++++---- .../apache/spark/api/python/PythonRDD.scala | 39 ++++---- .../apache/spark/api/python/SerDeUtil.scala | 2 +- .../WriteInputFormatTestDataGenerator.scala | 27 +++--- .../apache/spark/broadcast/Broadcast.scala | 2 +- .../spark/broadcast/BroadcastManager.scala | 2 +- .../spark/broadcast/HttpBroadcast.scala | 4 +- .../broadcast/HttpBroadcastFactory.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../broadcast/TorrentBroadcastFactory.scala | 2 +- .../org/apache/spark/deploy/Client.scala | 4 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../apache/spark/deploy/DeployMessage.scala | 2 +- .../spark/deploy/FaultToleranceTest.scala | 2 +- .../apache/spark/deploy/JsonProtocol.scala | 15 ++-- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../spark/deploy/SparkSubmitArguments.scala | 2 +- .../spark/deploy/client/AppClient.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 4 +- .../spark/deploy/history/HistoryServer.scala | 10 +-- .../master/FileSystemPersistenceEngine.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 2 +- .../deploy/master/RecoveryModeFactory.scala | 15 +++- .../spark/deploy/master/WorkerInfo.scala | 2 +- .../master/ZooKeeperPersistenceEngine.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 2 +- .../spark/deploy/worker/DriverRunner.scala | 22 +++-- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../apache/spark/executor/ExecutorActor.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 90 +++++++++---------- .../spark/input/PortableDataStream.scala | 17 ++-- .../spark/mapred/SparkHadoopMapRedUtil.scala | 2 +- .../mapreduce/SparkHadoopMapReduceUtil.scala | 2 +- .../apache/spark/metrics/MetricsSystem.scala | 3 +- .../spark/metrics/sink/MetricsServlet.scala | 18 ++-- .../org/apache/spark/metrics/sink/Sink.scala | 4 +- .../spark/network/nio/BlockMessageArray.scala | 6 +- .../spark/network/nio/BufferMessage.scala | 10 +-- .../apache/spark/network/nio/Connection.scala | 8 +- .../spark/network/nio/ConnectionId.scala | 4 +- .../spark/network/nio/ConnectionManager.scala | 2 +- .../network/nio/ConnectionManagerId.scala | 2 +- .../apache/spark/network/nio/Message.scala | 6 +- .../spark/network/nio/MessageChunk.scala | 6 +- .../network/nio/MessageChunkHeader.scala | 4 +- .../apache/spark/partial/PartialResult.scala | 2 +- .../org/apache/spark/rdd/CartesianRDD.scala | 2 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 10 +-- .../org/apache/spark/rdd/HadoopRDD.scala | 13 ++- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 10 ++- .../apache/spark/rdd/MapPartitionsRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/ParallelCollectionRDD.scala | 2 +- .../spark/rdd/PartitionPruningRDD.scala | 10 ++- .../scala/org/apache/spark/rdd/PipedRDD.scala | 8 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 12 +-- .../org/apache/spark/rdd/ShuffledRDD.scala | 4 +- .../org/apache/spark/rdd/SubtractedRDD.scala | 2 +- .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../spark/rdd/ZippedPartitionsRDD.scala | 2 +- .../spark/scheduler/AccumulableInfo.scala | 7 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/EventLoggingListener.scala | 61 ++++++++----- .../apache/spark/scheduler/JobLogger.scala | 2 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../scheduler/OutputCommitCoordinator.scala | 2 +- .../apache/spark/scheduler/ResultTask.scala | 2 +- .../spark/scheduler/ShuffleMapTask.scala | 4 +- .../spark/scheduler/SparkListener.scala | 4 +- .../org/apache/spark/scheduler/Stage.scala | 2 +- .../apache/spark/scheduler/TaskLocation.scala | 19 ++-- .../spark/scheduler/TaskSchedulerImpl.scala | 16 ++-- .../spark/scheduler/TaskSetManager.scala | 13 +-- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/YarnSchedulerBackend.scala | 2 +- .../scheduler/cluster/mesos/MemoryUtils.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/scheduler/local/LocalBackend.scala | 4 +- .../spark/serializer/JavaSerializer.scala | 5 +- .../spark/serializer/KryoSerializer.scala | 2 +- .../shuffle/FileShuffleBlockManager.scala | 4 +- .../shuffle/IndexShuffleBlockManager.scala | 4 +- .../org/apache/spark/storage/BlockId.scala | 34 +++---- .../apache/spark/storage/BlockManagerId.scala | 8 +- .../spark/storage/BlockManagerMaster.scala | 2 +- .../storage/BlockManagerMasterActor.scala | 4 +- .../storage/BlockManagerSlaveActor.scala | 2 +- .../spark/storage/BlockObjectWriter.scala | 17 ++-- .../apache/spark/storage/FileSegment.scala | 4 +- .../org/apache/spark/storage/RDDInfo.scala | 4 +- .../apache/spark/storage/StorageLevel.scala | 16 ++-- .../spark/storage/StorageStatusListener.scala | 7 +- .../spark/storage/TachyonFileSegment.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 2 +- .../scala/org/apache/spark/ui/UIUtils.scala | 6 +- .../apache/spark/ui/UIWorkloadGenerator.scala | 4 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 10 +-- .../spark/ui/jobs/JobProgressListener.scala | 14 +-- .../org/apache/spark/ui/jobs/JobsTab.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 10 ++- .../org/apache/spark/ui/jobs/StagesTab.scala | 6 +- .../org/apache/spark/ui/jobs/UIData.scala | 10 +-- .../apache/spark/ui/storage/StorageTab.scala | 12 +-- .../spark/util/CompletionIterator.scala | 10 +-- .../org/apache/spark/util/Distribution.scala | 2 +- .../org/apache/spark/util/ManualClock.scala | 30 +++---- .../apache/spark/util/MetadataCleaner.scala | 7 +- .../org/apache/spark/util/MutablePair.scala | 2 +- .../apache/spark/util/ParentClassLoader.scala | 2 +- .../spark/util/SerializableBuffer.scala | 2 +- .../org/apache/spark/util/StatCounter.scala | 4 +- .../util/TimeStampedWeakValueHashMap.scala | 6 +- .../scala/org/apache/spark/util/Utils.scala | 30 +++---- .../apache/spark/util/collection/BitSet.scala | 4 +- .../collection/ExternalAppendOnlyMap.scala | 4 +- .../util/collection/ExternalSorter.scala | 2 +- .../spark/util/collection/OpenHashMap.scala | 6 +- .../spark/util/collection/OpenHashSet.scala | 4 +- .../collection/PrimitiveKeyOpenHashMap.scala | 8 +- .../apache/spark/util/collection/Utils.scala | 2 +- .../spark/util/logging/FileAppender.scala | 4 +- .../spark/util/random/RandomSampler.scala | 44 +++++---- .../util/random/StratifiedSamplingUtils.scala | 2 +- .../spark/util/random/XORShiftRandom.scala | 2 +- project/MimaExcludes.scala | 8 +- scalastyle-config.xml | 4 +- 142 files changed, 597 insertions(+), 526 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index bcf832467f00b..330df1d59a9b1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,8 +18,6 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} -import java.util.concurrent.atomic.AtomicLong -import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -109,7 +107,7 @@ class Accumulable[R, T] ( * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. */ - def localValue = value_ + def localValue: R = value_ /** * Set the accumulator's value; only allowed on master. @@ -137,7 +135,7 @@ class Accumulable[R, T] ( Accumulators.register(this, false) } - override def toString = if (value_ == null) "null" else value_.toString + override def toString: String = if (value_ == null) "null" else value_.toString } /** @@ -257,22 +255,22 @@ object AccumulatorParam { implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double) = 0.0 + def zero(initialValue: Double): Double = 0.0 } implicit object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int) = 0 + def zero(initialValue: Int): Int = 0 } implicit object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0L + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L } implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float) = t1 + t2 - def zero(initialValue: Float) = 0f + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f } // TODO: Add AccumulatorParams for other types, e.g. lists and strings @@ -351,6 +349,7 @@ private[spark] object Accumulators extends Logging { } } - def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue) - def stringifyValue(value: Any) = "%s".format(value) + def stringifyPartialValue(partialValue: Any): String = "%s".format(partialValue) + + def stringifyValue(value: Any): String = "%s".format(value) } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 9a7cd4523e5ab..fc8cdde9348ee 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -74,7 +74,7 @@ class ShuffleDependency[K, V, C]( val mapSideCombine: Boolean = false) extends Dependency[Product2[K, V]] { - override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] + override def rdd: RDD[Product2[K, V]] = _rdd.asInstanceOf[RDD[Product2[K, V]]] val shuffleId: Int = _rdd.context.newShuffleId() @@ -91,7 +91,7 @@ class ShuffleDependency[K, V, C]( */ @DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { - override def getParents(partitionId: Int) = List(partitionId) + override def getParents(partitionId: Int): List[Int] = List(partitionId) } @@ -107,7 +107,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { - override def getParents(partitionId: Int) = { + override def getParents(partitionId: Int): List[Int] = { if (partitionId >= outStart && partitionId < outStart + length) { List(partitionId - outStart + inStart) } else { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index e97a7375a267b..91f9ef8ce7185 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -168,7 +168,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } } - def jobIds = Seq(jobWaiter.jobId) + def jobIds: Seq[Int] = Seq(jobWaiter.jobId) } @@ -276,7 +276,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { override def value: Option[Try[T]] = p.future.value - def jobIds = jobs + def jobIds: Seq[Int] = jobs } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 69178da1a7773..715f292f03469 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -65,7 +65,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule super.preStart() } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case Heartbeat(executorId, taskMetrics, blockManagerId) => val unknownExecutor = !scheduler.executorHeartbeatReceived( executorId, taskMetrics, blockManagerId) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6e4edc7c80d7a..c9426c5de23a2 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -43,7 +43,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster extends Actor with ActorLogReceive with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = sender.path.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index e53a78ead2c0e..b8d244408bc5b 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -76,7 +76,7 @@ object Partitioner { * produce an unexpected or incorrect result. */ class HashPartitioner(partitions: Int) extends Partitioner { - def numPartitions = partitions + def numPartitions: Int = partitions def getPartition(key: Any): Int = key match { case null => 0 @@ -154,7 +154,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } - def numPartitions = rangeBounds.length + 1 + def numPartitions: Int = rangeBounds.length + 1 private var binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 55cb25946c2ad..cb2cae185256a 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -28,8 +28,10 @@ import org.apache.spark.util.Utils @DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { - def value = t - override def toString = t.toString + + def value: T = t + + override def toString: String = t.toString private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 2ca19f53d2f07..0c123c96b8d7b 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -133,7 +133,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } /** Set multiple parameters together */ - def setAll(settings: Traversable[(String, String)]) = { + def setAll(settings: Traversable[(String, String)]): SparkConf = { this.settings.putAll(settings.toMap.asJava) this } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 228ff715fe7cb..a70be16f77eeb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -986,7 +986,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli union(Seq(first) ++ rest) /** Get an RDD that has no partitions or elements. */ - def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + def emptyRDD[T: ClassTag]: EmptyRDD[T] = new EmptyRDD[T](this) // Methods for creating shared variables @@ -994,7 +994,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `+=` method. Only the driver can access the accumulator's `value`. */ - def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = + def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { val acc = new Accumulator(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) @@ -1006,7 +1006,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the * driver can access the accumulator's `value`. */ - def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = { + def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) + : Accumulator[T] = { val acc = new Accumulator(initialValue, param, Some(name)) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1018,7 +1019,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = { + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1031,7 +1033,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = { + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param, Some(name)) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1209,7 +1212,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId) /** The version of Spark on which this application is running. */ - def version = SPARK_VERSION + def version: String = SPARK_VERSION /** * Return a map from the slave to the max memory available for caching and the remaining @@ -1659,7 +1662,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - def getCheckpointDir = checkpointDir + def getCheckpointDir: Option[String] = checkpointDir /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: Int = { @@ -1900,28 +1903,28 @@ object SparkContext extends Logging { "backward compatibility.", "1.3.0") object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double) = 0.0 + def zero(initialValue: Double): Double = 0.0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int) = 0 + def zero(initialValue: Int): Int = 0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0L + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float) = t1 + t2 - def zero(initialValue: Float) = 0f + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f } // The following deprecated functions have already been moved to `object RDD` to @@ -1931,18 +1934,18 @@ object SparkContext extends Logging { @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): PairRDDFunctions[K, V] = RDD.rddToPairRDDFunctions(rdd) - } @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) + def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]): AsyncRDDActions[T] = + RDD.rddToAsyncRDDActions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = { + rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { val kf = implicitly[K => Writable] val vf = implicitly[V => Writable] // Set the Writable class to null and `SequenceFileRDDFunctions` will use Reflection to get it @@ -1954,16 +1957,17 @@ object SparkContext extends Logging { @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(K, V)]): OrderedRDDFunctions[K, V, (K, V)] = RDD.rddToOrderedRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) + def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions = + RDD.doubleRDDToDoubleRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]): DoubleRDDFunctions = RDD.numericRDDToDoubleRDDFunctions(rdd) // The following deprecated functions have already been moved to `object WritableFactory` to @@ -2134,7 +2138,7 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_N_REGEX(threads) => - def localCpuCount = Runtime.getRuntime.availableProcessors() + def localCpuCount: Int = Runtime.getRuntime.availableProcessors() // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads. val threadCount = if (threads == "*") localCpuCount else threads.toInt if (threadCount <= 0) { @@ -2146,7 +2150,7 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - def localCpuCount = Runtime.getRuntime.availableProcessors() + def localCpuCount: Int = Runtime.getRuntime.availableProcessors() // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index c415fe99b105e..fe19f07e32d1b 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -27,9 +27,9 @@ private[spark] object TaskState extends Enumeration { type TaskState = Value - def isFailed(state: TaskState) = (LOST == state) || (FAILED == state) + def isFailed(state: TaskState): Boolean = (LOST == state) || (FAILED == state) - def isFinished(state: TaskState) = FINISHED_STATES.contains(state) + def isFinished(state: TaskState): Boolean = FINISHED_STATES.contains(state) def toMesos(state: TaskState): MesosTaskState = state match { case LAUNCHING => MesosTaskState.TASK_STARTING diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 35b324ba6f573..398ca41e16151 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -107,7 +107,7 @@ private[spark] object TestUtils { private class JavaSourceFromString(val name: String, val code: String) extends SimpleJavaFileObject(createURI(name), SOURCE) { - override def getCharContent(ignoreEncodingErrors: Boolean) = code + override def getCharContent(ignoreEncodingErrors: Boolean): String = code } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 3e9beb670f7ad..18ccd625fc8d1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -179,7 +179,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) - override def toString = rdd.toString + override def toString: String = rdd.toString /** Assign a name to this RDD */ def setName(name: String): JavaRDD[T] = { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 6d6ed693be752..3be6783bba49d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -108,7 +108,7 @@ class JavaSparkContext(val sc: SparkContext) private[spark] val env = sc.env - def statusTracker = new JavaSparkStatusTracker(sc) + def statusTracker: JavaSparkStatusTracker = new JavaSparkStatusTracker(sc) def isLocal: java.lang.Boolean = sc.isLocal diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 71b26737b8c02..8f9647eea9e25 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.api.java +import java.util.Map.Entry + import com.google.common.base.Optional import java.{util => ju} @@ -30,8 +32,8 @@ private[spark] object JavaUtils { } // Workaround for SPARK-3926 / SI-8911 - def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = - new SerializableMapWrapper(underlying) + def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]): SerializableMapWrapper[A, B] + = new SerializableMapWrapper(underlying) // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper, // but implements java.io.Serializable. It can't just be subclassed to make it @@ -40,36 +42,33 @@ private[spark] object JavaUtils { class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) extends ju.AbstractMap[A, B] with java.io.Serializable { self => - override def size = underlying.size + override def size: Int = underlying.size override def get(key: AnyRef): B = try { - underlying get key.asInstanceOf[A] match { - case None => null.asInstanceOf[B] - case Some(v) => v - } + underlying.getOrElse(key.asInstanceOf[A], null.asInstanceOf[B]) } catch { case ex: ClassCastException => null.asInstanceOf[B] } override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] { - def size = self.size + override def size: Int = self.size - def iterator = new ju.Iterator[ju.Map.Entry[A, B]] { + override def iterator: ju.Iterator[ju.Map.Entry[A, B]] = new ju.Iterator[ju.Map.Entry[A, B]] { val ui = underlying.iterator var prev : Option[A] = None - def hasNext = ui.hasNext + def hasNext: Boolean = ui.hasNext - def next() = { - val (k, v) = ui.next + def next(): Entry[A, B] = { + val (k, v) = ui.next() prev = Some(k) new ju.Map.Entry[A, B] { import scala.util.hashing.byteswap32 - def getKey = k - def getValue = v - def setValue(v1 : B) = self.put(k, v1) - override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) - override def equals(other: Any) = other match { + override def getKey: A = k + override def getValue: B = v + override def setValue(v1 : B): B = self.put(k, v1) + override def hashCode: Int = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) + override def equals(other: Any): Boolean = other match { case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue case _ => false } @@ -81,7 +80,7 @@ private[spark] object JavaUtils { case Some(k) => underlying match { case mm: mutable.Map[A, _] => - mm remove k + mm.remove(k) prev = None case _ => throw new UnsupportedOperationException("remove") diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 4c71b69069eb3..19f4c95fcad74 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -54,9 +54,11 @@ private[spark] class PythonRDD( val bufferSize = conf.getInt("spark.buffer.size", 65536) val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true) - override def getPartitions = firstParent.partitions + override def getPartitions: Array[Partition] = firstParent.partitions - override val partitioner = if (preservePartitoning) firstParent.partitioner else None + override val partitioner: Option[Partitioner] = { + if (preservePartitoning) firstParent.partitioner else None + } override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis @@ -92,7 +94,7 @@ private[spark] class PythonRDD( // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = new Iterator[Array[Byte]] { - def next(): Array[Byte] = { + override def next(): Array[Byte] = { val obj = _nextObj if (hasNext) { _nextObj = read() @@ -175,7 +177,7 @@ private[spark] class PythonRDD( var _nextObj = read() - def hasNext = _nextObj != null + override def hasNext: Boolean = _nextObj != null } new InterruptibleIterator(context, stdoutIterator) } @@ -303,11 +305,10 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. * This is used by PySpark's shuffle operations. */ -private class PairwiseRDD(prev: RDD[Array[Byte]]) extends - RDD[(Long, Array[Byte])](prev) { - override def getPartitions = prev.partitions - override val partitioner = prev.partitioner - override def compute(split: Partition, context: TaskContext) = +private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Long, Array[Byte])](prev) { + override def getPartitions: Array[Partition] = prev.partitions + override val partitioner: Option[Partitioner] = prev.partitioner + override def compute(split: Partition, context: TaskContext): Iterator[(Long, Array[Byte])] = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (Utils.deserializeLongValue(a), b) case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) @@ -435,7 +436,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, minSplits: Int, - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] @@ -462,7 +463,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, @@ -488,7 +489,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, @@ -505,7 +506,7 @@ private[spark] object PythonRDD extends Logging { inputFormatClass: String, keyClass: String, valueClass: String, - conf: Configuration) = { + conf: Configuration): RDD[(K, V)] = { val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] @@ -531,7 +532,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = hadoopRDDFromClassNames[K, V, F](sc, @@ -557,7 +558,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = hadoopRDDFromClassNames[K, V, F](sc, @@ -686,7 +687,7 @@ private[spark] object PythonRDD extends Logging { pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, path: String, - compressionCodecClass: String) = { + compressionCodecClass: String): Unit = { saveAsHadoopFile( pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat", null, null, null, null, new java.util.HashMap(), compressionCodecClass) @@ -711,7 +712,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - compressionCodecClass: String) = { + compressionCodecClass: String): Unit = { val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) @@ -741,7 +742,7 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String]): Unit = { val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) @@ -766,7 +767,7 @@ private[spark] object PythonRDD extends Logging { confAsMap: java.util.HashMap[String, String], keyConverterClass: String, valueConverterClass: String, - useNewAPI: Boolean) = { + useNewAPI: Boolean): Unit = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized), keyConverterClass, valueConverterClass, new JavaToWritableConverter) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index fb52a960e0765..257491e90dd66 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -84,7 +84,7 @@ private[spark] object SerDeUtil extends Logging { private var initialized = false // This should be called before trying to unpickle array.array from Python // In cluster mode, this should be put in closure - def initialize() = { + def initialize(): Unit = { synchronized{ if (!initialized) { Unpickler.registerConstructor("array", "array", new ArrayConstructor()) diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index cf289fb3ae39f..8f30ff9202c83 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -18,38 +18,37 @@ package org.apache.spark.api.python import java.io.{DataOutput, DataInput} +import java.{util => ju} import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat + +import org.apache.spark.SparkException import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.{SparkContext, SparkException} /** * A class to test Pyrolite serialization on the Scala side, that will be deserialized * in Python - * @param str - * @param int - * @param double */ case class TestWritable(var str: String, var int: Int, var double: Double) extends Writable { def this() = this("", 0, 0.0) - def getStr = str + def getStr: String = str def setStr(str: String) { this.str = str } - def getInt = int + def getInt: Int = int def setInt(int: Int) { this.int = int } - def getDouble = double + def getDouble: Double = double def setDouble(double: Double) { this.double = double } - def write(out: DataOutput) = { + def write(out: DataOutput): Unit = { out.writeUTF(str) out.writeInt(int) out.writeDouble(double) } - def readFields(in: DataInput) = { + def readFields(in: DataInput): Unit = { str = in.readUTF() int = in.readInt() double = in.readDouble() @@ -57,28 +56,28 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten } private[python] class TestInputKeyConverter extends Converter[Any, Any] { - override def convert(obj: Any) = { + override def convert(obj: Any): Char = { obj.asInstanceOf[IntWritable].get().toChar } } private[python] class TestInputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ - override def convert(obj: Any) = { + override def convert(obj: Any): ju.List[Double] = { val m = obj.asInstanceOf[MapWritable] seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq) } } private[python] class TestOutputKeyConverter extends Converter[Any, Any] { - override def convert(obj: Any) = { + override def convert(obj: Any): Text = { new Text(obj.asInstanceOf[Int].toString) } } private[python] class TestOutputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ - override def convert(obj: Any) = { + override def convert(obj: Any): DoubleWritable = { new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head) } } @@ -86,7 +85,7 @@ private[python] class TestOutputValueConverter extends Converter[Any, Any] { private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable]) private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] { - override def convert(obj: Any) = obj match { + override def convert(obj: Any): DoubleArrayWritable = obj match { case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] => val daw = new DoubleArrayWritable daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_))) diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index a5ea478f231d7..12d79f6ed311b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -146,5 +146,5 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo } } - override def toString = "Broadcast(" + id + ")" + override def toString: String = "Broadcast(" + id + ")" } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index 8f8a0b11f9f2e..685313ac009ba 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -58,7 +58,7 @@ private[spark] class BroadcastManager( private val nextBroadcastId = new AtomicLong(0) - def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = { + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = { broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 1444c0dd3d2d6..74ccfa6d3c9a3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -160,7 +160,7 @@ private[broadcast] object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } - def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) + def getFile(id: Long): File = new File(broadcastDir, BroadcastBlockId(id).name) private def write(id: Long, value: Any) { val file = getFile(id) @@ -222,7 +222,7 @@ private[broadcast] object HttpBroadcast extends Logging { * If removeFromDriver is true, also remove these persisted blocks on the driver * and delete the associated broadcast file. */ - def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = synchronized { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) if (removeFromDriver) { val file = getFile(id) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala index c7ef02d572a19..cf3ae36f27949 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -31,7 +31,7 @@ class HttpBroadcastFactory extends BroadcastFactory { HttpBroadcast.initialize(isDriver, conf, securityMgr) } - override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] = new HttpBroadcast[T](value_, isLocal, id) override def stop() { HttpBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 94142d33369c7..23b02e60338fb 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -222,7 +222,7 @@ private object TorrentBroadcast extends Logging { * Remove all persisted blocks associated with this torrent broadcast on the executors. * If removeFromDriver is true, also remove these persisted blocks on the driver. */ - def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = { logDebug(s"Unpersisting TorrentBroadcast $id") SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index fb024c12094f2..96d8dd79908c8 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -30,7 +30,7 @@ class TorrentBroadcastFactory extends BroadcastFactory { override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { } - override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = { + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] = { new TorrentBroadcast[T](value_, id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 237d26fc6bd0e..65238af2caa24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -38,7 +38,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) var masterActor: ActorSelection = _ val timeout = AkkaUtils.askTimeout(conf) - override def preStart() = { + override def preStart(): Unit = { masterActor = context.actorSelection( Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system))) @@ -118,7 +118,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) } } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case SubmitDriverResponse(success, driverId, message) => println(message) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 53bc62aff7395..5cbac787dceeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -42,7 +42,7 @@ private[deploy] class ClientArguments(args: Array[String]) { var memory: Int = DEFAULT_MEMORY var cores: Int = DEFAULT_CORES private var _driverOptions = ListBuffer[String]() - def driverOptions = _driverOptions.toSeq + def driverOptions: Seq[String] = _driverOptions.toSeq // kill parameters var driverId: String = "" diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 7f600d89604a2..0997507d016f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -162,7 +162,7 @@ private[deploy] object DeployMessages { Utils.checkHost(host, "Required hostname") assert (port > 0) - def uri = "spark://" + host + ":" + port + def uri: String = "spark://" + host + ":" + port def restUri: Option[String] = restPort.map { p => "spark://" + host + ":" + p } } diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 5668b53fc6f4f..a7c89276a045e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -426,7 +426,7 @@ private object SparkDocker { } private class DockerId(val id: String) { - override def toString = id + override def toString: String = id } private object Docker extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 458a7c3a455de..dfc5b97e6a6c8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import org.json4s.JsonAST.JObject import org.json4s.JsonDSL._ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -24,7 +25,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner private[deploy] object JsonProtocol { - def writeWorkerInfo(obj: WorkerInfo) = { + def writeWorkerInfo(obj: WorkerInfo): JObject = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ ("port" -> obj.port) ~ @@ -39,7 +40,7 @@ private[deploy] object JsonProtocol { ("lastheartbeat" -> obj.lastHeartbeat) } - def writeApplicationInfo(obj: ApplicationInfo) = { + def writeApplicationInfo(obj: ApplicationInfo): JObject = { ("starttime" -> obj.startTime) ~ ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ @@ -51,7 +52,7 @@ private[deploy] object JsonProtocol { ("duration" -> obj.duration) } - def writeApplicationDescription(obj: ApplicationDescription) = { + def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ ("memoryperslave" -> obj.memoryPerSlave) ~ @@ -59,14 +60,14 @@ private[deploy] object JsonProtocol { ("command" -> obj.command.toString) } - def writeExecutorRunner(obj: ExecutorRunner) = { + def writeExecutorRunner(obj: ExecutorRunner): JObject = { ("id" -> obj.execId) ~ ("memory" -> obj.memory) ~ ("appid" -> obj.appId) ~ ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - def writeDriverInfo(obj: DriverInfo) = { + def writeDriverInfo(obj: DriverInfo): JObject = { ("id" -> obj.id) ~ ("starttime" -> obj.startTime.toString) ~ ("state" -> obj.state.toString) ~ @@ -74,7 +75,7 @@ private[deploy] object JsonProtocol { ("memory" -> obj.desc.mem) } - def writeMasterState(obj: MasterStateResponse) = { + def writeMasterState(obj: MasterStateResponse): JObject = { ("url" -> obj.uri) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ ("cores" -> obj.workers.map(_.cores).sum) ~ @@ -87,7 +88,7 @@ private[deploy] object JsonProtocol { ("status" -> obj.status.toString) } - def writeWorkerState(obj: WorkerStateResponse) = { + def writeWorkerState(obj: WorkerStateResponse): JObject = { ("id" -> obj.workerId) ~ ("masterurl" -> obj.masterUrl) ~ ("masterwebuiurl" -> obj.masterWebUiUrl) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e0a32fb65cd51..c2568eb4b60ac 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -193,7 +193,7 @@ class SparkHadoopUtil extends Logging { * that file. */ def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { - def recurse(path: Path) = { + def recurse(path: Path): Array[FileStatus] = { val (directories, leaves) = fs.listStatus(path).partition(_.isDir) leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4f506be63fe59..660307d19eab4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -777,7 +777,7 @@ private[deploy] object SparkSubmitUtils { } /** A nice function to use in tests as well. Values are dummy strings. */ - def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + def getModuleDescriptor: DefaultModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 2250d5a28e4ef..6eb73c43470a5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -252,7 +252,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S master.startsWith("spark://") && deployMode == "cluster" } - override def toString = { + override def toString: String = { s"""Parsed arguments: | master $master | deployMode $deployMode diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 2d24083a77b73..3b729725257ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -116,7 +116,7 @@ private[spark] class AppClient( masterAkkaUrls.map(AddressFromURIString(_).hostPort).contains(remoteUrl.hostPort) } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ registered = true diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index db7c499661319..80c9c13ddec1e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -93,7 +93,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def getRunner(operateFun: () => Unit): Runnable = { new Runnable() { - override def run() = Utils.tryOrExit { + override def run(): Unit = Utils.tryOrExit { operateFun() } } @@ -141,7 +141,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } - override def getListing() = applications.values + override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values override def getAppUI(appId: String): Option[SparkUI] = { try { 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 af483d560b33e..72f6048239297 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 @@ -61,7 +61,7 @@ class HistoryServer( private val appCache = CacheBuilder.newBuilder() .maximumSize(retainedApplications) .removalListener(new RemovalListener[String, SparkUI] { - override def onRemoval(rm: RemovalNotification[String, SparkUI]) = { + override def onRemoval(rm: RemovalNotification[String, SparkUI]): Unit = { detachSparkUI(rm.getValue()) } }) @@ -149,14 +149,14 @@ class HistoryServer( * * @return List of all known applications. */ - def getApplicationList() = provider.getListing() + def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing() /** * Returns the provider configuration to show in the listing page. * * @return A map with the provider's configuration. */ - def getProviderConfig() = provider.getConfig() + def getProviderConfig(): Map[String, String] = provider.getConfig() } @@ -195,9 +195,7 @@ object HistoryServer extends Logging { server.bind() Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") { - override def run() = { - server.stop() - } + override def run(): Unit = server.stop() }) // Wait until the end of the world... or if the HistoryServer process is manually stopped diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index d2d30bfd7fcba..32499b3a784a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -48,7 +48,7 @@ private[master] class FileSystemPersistenceEngine( new File(dir + File.separator + name).delete() } - override def read[T: ClassTag](prefix: String) = { + override def read[T: ClassTag](prefix: String): Seq[T] = { val files = new File(dir).listFiles().filter(_.getName.startsWith(prefix)) files.map(deserializeFromFile[T]) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 1b42121c8db05..80506621f4d24 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -204,7 +204,7 @@ private[master] class Master( self ! RevokedLeadership } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 1583bf1f60032..351db8fab2041 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -51,20 +51,27 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial */ private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { + val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") - def createPersistenceEngine() = { + def createPersistenceEngine(): PersistenceEngine = { logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) } - def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) + def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { + new MonarchyLeaderAgent(master) + } } private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) { - def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) - def createLeaderElectionAgent(master: LeaderElectable) = + def createPersistenceEngine(): PersistenceEngine = { + new ZooKeeperPersistenceEngine(conf, serializer) + } + + def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { new ZooKeeperLeaderElectionAgent(master, conf) + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index e94aae93e4495..9b3d48c6edc84 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -104,7 +104,7 @@ private[spark] class WorkerInfo( "http://" + this.publicAddress + ":" + this.webUiPort } - def setState(state: WorkerState.Value) = { + def setState(state: WorkerState.Value): Unit = { this.state = state } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 1ac6677ad2b6d..a285783f72000 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -46,7 +46,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializat zk.delete().forPath(WORKING_DIR + "/" + name) } - override def read[T: ClassTag](prefix: String) = { + override def read[T: ClassTag](prefix: String): Seq[T] = { val file = zk.getChildren.forPath(WORKING_DIR).filter(_.startsWith(prefix)) file.map(deserializeFromFile[T]).flatten } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index dee2e4a447c6e..46509e39c0f23 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -95,7 +95,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { // For now we only show driver information if the user has submitted drivers to the cluster. // This is until we integrate the notion of drivers and applications in the UI. - def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0 + def hasDrivers: Boolean = activeDrivers.length > 0 || completedDrivers.length > 0 val content =
    diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 27a9eabb1ede7..e0948e16ef354 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -56,8 +56,14 @@ private[deploy] class DriverRunner( private var finalExitCode: Option[Int] = None // Decoupled for testing - def setClock(_clock: Clock) = clock = _clock - def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + def setClock(_clock: Clock): Unit = { + clock = _clock + } + + def setSleeper(_sleeper: Sleeper): Unit = { + sleeper = _sleeper + } + private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) @@ -155,7 +161,7 @@ private[deploy] class DriverRunner( private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) { builder.directory(baseDir) - def initialize(process: Process) = { + def initialize(process: Process): Unit = { // Redirect stdout and stderr to files val stdout = new File(baseDir, "stdout") CommandUtils.redirectStream(process.getInputStream, stdout) @@ -169,8 +175,8 @@ private[deploy] class DriverRunner( runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, - supervise: Boolean) { + def runCommandWithRetry( + command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Unit = { // Time to wait between submission retries. var waitSeconds = 1 // A run of this many seconds resets the exponential back-off. @@ -216,8 +222,8 @@ private[deploy] trait ProcessBuilderLike { } private[deploy] object ProcessBuilderLike { - def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike { - def start() = processBuilder.start() - def command = processBuilder.command() + def apply(processBuilder: ProcessBuilder): ProcessBuilderLike = new ProcessBuilderLike { + override def start(): Process = processBuilder.start() + override def command: Seq[String] = processBuilder.command() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c1b0a295f9f74..c4c24a7866aa3 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -275,7 +275,7 @@ private[worker] class Worker( } } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 09d866fb0cd90..e0790274d7d3e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -50,7 +50,7 @@ private[spark] class WorkerWatcher(workerUrl: String) private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dd19e4947db1e..b5205d4e997ae 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -62,7 +62,7 @@ private[spark] class CoarseGrainedExecutorBackend( .map(e => (e._1.substring(prefix.length).toLowerCase, e._2)) } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala index 41925f7e97e84..3e47d13f7545d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala @@ -33,7 +33,7 @@ private[spark] case object TriggerThreadDump private[spark] class ExecutorActor(executorId: String) extends Actor with ActorLogReceive with Logging { - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case TriggerThreadDump => sender ! Utils.getThreadDump() } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 07b152651dedf..06152f16ae618 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,13 +17,10 @@ package org.apache.spark.executor -import java.util.concurrent.atomic.AtomicLong - -import org.apache.spark.executor.DataReadMethod.DataReadMethod - import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} /** @@ -44,14 +41,14 @@ class TaskMetrics extends Serializable { * Host's name the task runs on */ private var _hostname: String = _ - def hostname = _hostname + def hostname: String = _hostname private[spark] def setHostname(value: String) = _hostname = value /** * Time taken on the executor to deserialize this task */ private var _executorDeserializeTime: Long = _ - def executorDeserializeTime = _executorDeserializeTime + def executorDeserializeTime: Long = _executorDeserializeTime private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value @@ -59,14 +56,14 @@ class TaskMetrics extends Serializable { * Time the executor spends actually running the task (including fetching shuffle data) */ private var _executorRunTime: Long = _ - def executorRunTime = _executorRunTime + def executorRunTime: Long = _executorRunTime private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value /** * The number of bytes this task transmitted back to the driver as the TaskResult */ private var _resultSize: Long = _ - def resultSize = _resultSize + def resultSize: Long = _resultSize private[spark] def setResultSize(value: Long) = _resultSize = value @@ -74,31 +71,31 @@ class TaskMetrics extends Serializable { * Amount of time the JVM spent in garbage collection while executing this task */ private var _jvmGCTime: Long = _ - def jvmGCTime = _jvmGCTime + def jvmGCTime: Long = _jvmGCTime private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value /** * Amount of time spent serializing the task result */ private var _resultSerializationTime: Long = _ - def resultSerializationTime = _resultSerializationTime + def resultSerializationTime: Long = _resultSerializationTime private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value /** * The number of in-memory bytes spilled by this task */ private var _memoryBytesSpilled: Long = _ - def memoryBytesSpilled = _memoryBytesSpilled - private[spark] def incMemoryBytesSpilled(value: Long) = _memoryBytesSpilled += value - private[spark] def decMemoryBytesSpilled(value: Long) = _memoryBytesSpilled -= value + def memoryBytesSpilled: Long = _memoryBytesSpilled + private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value + private[spark] def decMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled -= value /** * The number of on-disk bytes spilled by this task */ private var _diskBytesSpilled: Long = _ - def diskBytesSpilled = _diskBytesSpilled - def incDiskBytesSpilled(value: Long) = _diskBytesSpilled += value - def decDiskBytesSpilled(value: Long) = _diskBytesSpilled -= value + def diskBytesSpilled: Long = _diskBytesSpilled + def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value + def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read @@ -106,7 +103,7 @@ class TaskMetrics extends Serializable { */ private var _inputMetrics: Option[InputMetrics] = None - def inputMetrics = _inputMetrics + def inputMetrics: Option[InputMetrics] = _inputMetrics /** * This should only be used when recreating TaskMetrics, not when updating input metrics in @@ -128,7 +125,7 @@ class TaskMetrics extends Serializable { */ private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None - def shuffleReadMetrics = _shuffleReadMetrics + def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics /** * This should only be used when recreating TaskMetrics, not when updating read metrics in @@ -177,17 +174,18 @@ class TaskMetrics extends Serializable { * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, * we can store all the different inputMetrics (one per readMethod). */ - private[spark] def getInputMetricsForReadMethod( - readMethod: DataReadMethod): InputMetrics = synchronized { - _inputMetrics match { - case None => - val metrics = new InputMetrics(readMethod) - _inputMetrics = Some(metrics) - metrics - case Some(metrics @ InputMetrics(method)) if method == readMethod => - metrics - case Some(InputMetrics(method)) => - new InputMetrics(readMethod) + private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): InputMetrics = { + synchronized { + _inputMetrics match { + case None => + val metrics = new InputMetrics(readMethod) + _inputMetrics = Some(metrics) + metrics + case Some(metrics @ InputMetrics(method)) if method == readMethod => + metrics + case Some(InputMetrics(method)) => + new InputMetrics(readMethod) + } } } @@ -256,14 +254,14 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { */ private var _bytesRead: Long = _ def bytesRead: Long = _bytesRead - def incBytesRead(bytes: Long) = _bytesRead += bytes + def incBytesRead(bytes: Long): Unit = _bytesRead += bytes /** * Total records read. */ private var _recordsRead: Long = _ def recordsRead: Long = _recordsRead - def incRecordsRead(records: Long) = _recordsRead += records + def incRecordsRead(records: Long): Unit = _recordsRead += records /** * Invoke the bytesReadCallback and mutate bytesRead. @@ -293,15 +291,15 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { * Total bytes written */ private var _bytesWritten: Long = _ - def bytesWritten = _bytesWritten - private[spark] def setBytesWritten(value : Long) = _bytesWritten = value + def bytesWritten: Long = _bytesWritten + private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value /** * Total records written */ private var _recordsWritten: Long = 0L - def recordsWritten = _recordsWritten - private[spark] def setRecordsWritten(value: Long) = _recordsWritten = value + def recordsWritten: Long = _recordsWritten + private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value } /** @@ -314,7 +312,7 @@ class ShuffleReadMetrics extends Serializable { * Number of remote blocks fetched in this shuffle by this task */ private var _remoteBlocksFetched: Int = _ - def remoteBlocksFetched = _remoteBlocksFetched + def remoteBlocksFetched: Int = _remoteBlocksFetched private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value @@ -322,7 +320,7 @@ class ShuffleReadMetrics extends Serializable { * Number of local blocks fetched in this shuffle by this task */ private var _localBlocksFetched: Int = _ - def localBlocksFetched = _localBlocksFetched + def localBlocksFetched: Int = _localBlocksFetched private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value @@ -332,7 +330,7 @@ class ShuffleReadMetrics extends Serializable { * still not finished processing block A, it is not considered to be blocking on block B. */ private var _fetchWaitTime: Long = _ - def fetchWaitTime = _fetchWaitTime + def fetchWaitTime: Long = _fetchWaitTime private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value @@ -340,7 +338,7 @@ class ShuffleReadMetrics extends Serializable { * Total number of remote bytes read from the shuffle by this task */ private var _remoteBytesRead: Long = _ - def remoteBytesRead = _remoteBytesRead + def remoteBytesRead: Long = _remoteBytesRead private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value @@ -348,24 +346,24 @@ class ShuffleReadMetrics extends Serializable { * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ private var _localBytesRead: Long = _ - def localBytesRead = _localBytesRead + def localBytesRead: Long = _localBytesRead private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value /** * Total bytes fetched in the shuffle by this task (both remote and local). */ - def totalBytesRead = _remoteBytesRead + _localBytesRead + def totalBytesRead: Long = _remoteBytesRead + _localBytesRead /** * Number of blocks fetched in this shuffle by this task (remote or local) */ - def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched + def totalBlocksFetched: Int = _remoteBlocksFetched + _localBlocksFetched /** * Total number of records read from the shuffle by this task */ private var _recordsRead: Long = _ - def recordsRead = _recordsRead + def recordsRead: Long = _recordsRead private[spark] def incRecordsRead(value: Long) = _recordsRead += value private[spark] def decRecordsRead(value: Long) = _recordsRead -= value } @@ -380,7 +378,7 @@ class ShuffleWriteMetrics extends Serializable { * Number of bytes written for the shuffle by this task */ @volatile private var _shuffleBytesWritten: Long = _ - def shuffleBytesWritten = _shuffleBytesWritten + def shuffleBytesWritten: Long = _shuffleBytesWritten private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value @@ -388,7 +386,7 @@ class ShuffleWriteMetrics extends Serializable { * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ @volatile private var _shuffleWriteTime: Long = _ - def shuffleWriteTime= _shuffleWriteTime + def shuffleWriteTime: Long = _shuffleWriteTime private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value @@ -396,7 +394,7 @@ class ShuffleWriteMetrics extends Serializable { * Total number of records written to the shuffle by this task */ @volatile private var _shuffleRecordsWritten: Long = _ - def shuffleRecordsWritten = _shuffleRecordsWritten + def shuffleRecordsWritten: Long = _shuffleRecordsWritten private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 593a62b3e3b32..6cda7772f77bc 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -73,16 +73,16 @@ private[spark] abstract class StreamBasedRecordReader[T]( private var key = "" private var value: T = null.asInstanceOf[T] - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = {} + override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {} + override def close(): Unit = {} - override def getProgress = if (processed) 1.0f else 0.0f + override def getProgress: Float = if (processed) 1.0f else 0.0f - override def getCurrentKey = key + override def getCurrentKey: String = key - override def getCurrentValue = value + override def getCurrentValue: T = value - override def nextKeyValue = { + override def nextKeyValue: Boolean = { if (!processed) { val fileIn = new PortableDataStream(split, context, index) value = parseStream(fileIn) @@ -119,7 +119,8 @@ private[spark] class StreamRecordReader( * The format for the PortableDataStream files */ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) + : CombineFileRecordReader[String, PortableDataStream] = { new CombineFileRecordReader[String, PortableDataStream]( split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) } @@ -204,7 +205,7 @@ class PortableDataStream( /** * Close the file (if it is currently open) */ - def close() = { + def close(): Unit = { if (isOpen) { try { fileIn.close() diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 21b782edd2a9e..87c2aa481095d 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -52,7 +52,7 @@ trait SparkHadoopMapRedUtil { jobId: Int, isMap: Boolean, taskId: Int, - attemptId: Int) = { + attemptId: Int): TaskAttemptID = { new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) } diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 3340673f91156..cfd20392d12f1 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -45,7 +45,7 @@ trait SparkHadoopMapReduceUtil { jobId: Int, isMap: Boolean, taskId: Int, - attemptId: Int) = { + attemptId: Int): TaskAttemptID = { val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID") try { // First, attempt to use the old-style constructor that takes a boolean isMap diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 345db36630fd5..9150ad35712a1 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} @@ -84,7 +85,7 @@ private[spark] class MetricsSystem private ( /** * Get any UI handlers used by this metrics system; can only be called after start(). */ - def getServletHandlers = { + def getServletHandlers: Array[ServletContextHandler] = { require(running, "Can only call getServletHandlers on a running MetricsSystem") metricsServlet.map(_.getHandlers).getOrElse(Array()) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 2f65bc8b46609..0c2e212a33074 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,8 +30,12 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, - securityMgr: SecurityManager) extends Sink { +private[spark] class MetricsServlet( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink { + val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" @@ -45,10 +49,12 @@ private[spark] class MetricsServlet(val property: Properties, val registry: Metr val mapper = new ObjectMapper().registerModule( new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample)) - def getHandlers = Array[ServletContextHandler]( - createServletHandler(servletPath, - new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) - ) + def getHandlers: Array[ServletContextHandler] = { + Array[ServletContextHandler]( + createServletHandler(servletPath, + new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) + ) + } def getMetricsSnapshot(request: HttpServletRequest): String = { mapper.writeValueAsString(registry) diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 0d83d8c425ca4..9fad4e7deacb6 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -18,7 +18,7 @@ package org.apache.spark.metrics.sink private[spark] trait Sink { - def start: Unit - def stop: Unit + def start(): Unit + def stop(): Unit def report(): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index a1a2c00ed1542..1ba25aa74aa02 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -32,11 +32,11 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int): BlockMessage = blockMessages(i) - def iterator = blockMessages.iterator + def iterator: Iterator[BlockMessage] = blockMessages.iterator - def length = blockMessages.length + def length: Int = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala index 3b245c5c7a4f3..9a9e22b0c2366 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala @@ -31,9 +31,9 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val initialSize = currentSize() var gotChunkForSendingOnce = false - def size = initialSize + def size: Int = initialSize - def currentSize() = { + def currentSize(): Int = { if (buffers == null || buffers.isEmpty) { 0 } else { @@ -100,11 +100,11 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: buffers.foreach(_.flip) } - def hasAckId() = (ackId != 0) + def hasAckId(): Boolean = ackId != 0 - def isCompletelyReceived() = !buffers(0).hasRemaining + def isCompletelyReceived: Boolean = !buffers(0).hasRemaining - override def toString = { + override def toString: String = { if (hasAckId) { "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" } else { diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index c2d9578be7ebb..04eb2bf9ba4ab 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -101,9 +101,11 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, socketRemoteConnectionManagerId } - def key() = channel.keyFor(selector) + def key(): SelectionKey = channel.keyFor(selector) - def getRemoteAddress() = channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + def getRemoteAddress(): InetSocketAddress = { + channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + } // Returns whether we have to register for further reads or not. def read(): Boolean = { @@ -280,7 +282,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, /* channel.socket.setSendBufferSize(256 * 1024) */ - override def getRemoteAddress() = address + override def getRemoteAddress(): InetSocketAddress = address val DEFAULT_INTEREST = SelectionKey.OP_READ diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala index 764dc5e5503ed..b3b281ff465f1 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala @@ -18,7 +18,9 @@ package org.apache.spark.network.nio private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString: String = { + connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + } } private[nio] object ConnectionId { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index ee22c6656e69e..741fe3e1ea750 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -188,7 +188,7 @@ private[nio] class ConnectionManager( private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() private val selectorThread = new Thread("connection-manager-thread") { - override def run() = ConnectionManager.this.run() + override def run(): Unit = ConnectionManager.this.run() } selectorThread.setDaemon(true) // start this thread last, since it invokes run(), which accesses members above diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala index cbb37ec5ced1f..1cd13d887c6f6 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala @@ -26,7 +26,7 @@ private[nio] case class ConnectionManagerId(host: String, port: Int) { Utils.checkHost(host) assert (port > 0) - def toSocketAddress() = new InetSocketAddress(host, port) + def toSocketAddress(): InetSocketAddress = new InetSocketAddress(host, port) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala index fb4a979b824c3..85d2fe2bf9c20 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -42,7 +42,9 @@ private[nio] abstract class Message(val typ: Long, val id: Int) { def timeTaken(): String = (finishTime - startTime).toString + " ms" - override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" + override def toString: String = { + this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" + } } @@ -51,7 +53,7 @@ private[nio] object Message { var lastId = 1 - def getNewId() = synchronized { + def getNewId(): Int = synchronized { lastId += 1 if (lastId == 0) { lastId += 1 diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala index 278c5ac356ef2..a4568e849fa13 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala @@ -24,9 +24,9 @@ import scala.collection.mutable.ArrayBuffer private[nio] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { - val size = if (buffer == null) 0 else buffer.remaining + val size: Int = if (buffer == null) 0 else buffer.remaining - lazy val buffers = { + lazy val buffers: ArrayBuffer[ByteBuffer] = { val ab = new ArrayBuffer[ByteBuffer]() ab += header.buffer if (buffer != null) { @@ -35,7 +35,7 @@ class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { ab } - override def toString = { + override def toString: String = { "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala index 6e20f291c5cec..7b3da4bb9d5ee 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala @@ -50,8 +50,10 @@ private[nio] class MessageChunkHeader( flip.asInstanceOf[ByteBuffer] } - override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + + override def toString: String = { + "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + " and sizes " + totalSize + " / " + chunkSize + " bytes, securityNeg: " + securityNeg + } } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index cadd0c7ed19ba..53c4b32c95ab3 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -99,7 +99,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { case None => "(partial: " + initialValue + ")" } } - def getFinalValueInternal() = PartialResult.this.getFinalValueInternal().map(f) + def getFinalValueInternal(): Option[T] = PartialResult.this.getFinalValueInternal().map(f) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 1cbd684224b7c..9059eb13bb5d8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -70,7 +70,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( (rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)).distinct } - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[(T, U)] = { val currSplit = split.asInstanceOf[CartesianPartition] for (x <- rdd1.iterator(currSplit.s1, context); y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index b073eba8a1574..5117ccfabfcc2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -186,7 +186,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: override val isEmpty = !it.hasNext // initializes/resets to start iterating from the beginning - def resetIterator() = { + def resetIterator(): Iterator[(String, Partition)] = { val iterators = (0 to 2).map( x => prev.partitions.iterator.flatMap(p => { if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None @@ -196,10 +196,10 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - def hasNext(): Boolean = { !isEmpty } + override def hasNext: Boolean = { !isEmpty } // return the next preferredLocation of some partition of the RDD - def next(): (String, Partition) = { + override def next(): (String, Partition) = { if (it.hasNext) { it.next() } else { @@ -237,7 +237,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: val rotIt = new LocationIterator(prev) // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext()) { + if (!rotIt.hasNext) { (1 to targetLen).foreach(x => groupArr += PartitionGroup()) return } @@ -343,7 +343,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: private case class PartitionGroup(prefLoc: Option[String] = None) { var arr = mutable.ArrayBuffer[Partition]() - def size = arr.size + def size: Int = arr.size } private object PartitionGroup { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 486e86ce1bb19..f77abac42b623 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -215,8 +215,7 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + 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 @@ -240,7 +239,7 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - override def getNext() = { + override def getNext(): (K, V) = { try { finished = !reader.next(key, value) } catch { @@ -337,11 +336,11 @@ private[spark] object HadoopRDD extends Logging { * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. */ - def getCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.get(key) + def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key) - def containsCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.containsKey(key) + def containsCachedMetadata(key: String): Boolean = SparkEnv.get.hadoopJobMetadata.containsKey(key) - def putCachedMetadata(key: String, value: Any) = + private def putCachedMetadata(key: String, value: Any): Unit = SparkEnv.get.hadoopJobMetadata.put(key, value) /** Add Hadoop configuration specific to a single partition and attempt. */ @@ -371,7 +370,7 @@ private[spark] object HadoopRDD extends Logging { override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[U] = { val partition = split.asInstanceOf[HadoopPartition] val inputSplit = partition.inputSplit.value f(inputSplit, firstParent[T].iterator(split, context)) diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index e2267861e79df..0c28f045e46e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.sql.{Connection, ResultSet} +import java.sql.{PreparedStatement, Connection, ResultSet} import scala.reflect.ClassTag @@ -28,8 +28,9 @@ import org.apache.spark.util.NextIterator import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { - override def index = idx + override def index: Int = idx } + // TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. @@ -70,7 +71,8 @@ class JdbcRDD[T: ClassTag]( }).toArray } - override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + override def compute(thePart: Partition, context: TaskContext): Iterator[T] = new NextIterator[T] + { context.addTaskCompletionListener{ context => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() @@ -88,7 +90,7 @@ class JdbcRDD[T: ClassTag]( stmt.setLong(2, part.upper) val rs = stmt.executeQuery() - override def getNext: T = { + override def getNext(): T = { if (rs.next()) { mapRow(rs) } else { diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 4883fb828814c..a838aac6e8d1a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -31,6 +31,6 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[U] = f(context, split.index, firstParent[T].iterator(split, context)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 7fb94840df99c..2ab967f4bb313 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -238,7 +238,7 @@ private[spark] object NewHadoopRDD { override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[U] = { val partition = split.asInstanceOf[NewHadoopPartition] val inputSplit = partition.serializableHadoopSplit.value f(inputSplit, firstParent[T].iterator(split, context)) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index f12d0cffaba34..e2394e28f8d26 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -98,7 +98,7 @@ private[spark] class ParallelCollectionRDD[T: ClassTag]( slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray } - override def compute(s: Partition, context: TaskContext) = { + override def compute(s: Partition, context: TaskContext): Iterator[T] = { new InterruptibleIterator(context, s.asInstanceOf[ParallelCollectionPartition[T]].iterator) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index f781a8d776f2a..a00f4c1cdff91 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -40,7 +40,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF .filter(s => partitionFilterFunc(s.index)).zipWithIndex .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } - override def getParents(partitionId: Int) = { + override def getParents(partitionId: Int): List[Int] = { List(partitions(partitionId).asInstanceOf[PartitionPruningRDDPartition].parentSplit.index) } } @@ -59,8 +59,10 @@ class PartitionPruningRDD[T: ClassTag]( @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { - override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator( - split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context) + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + firstParent[T].iterator( + split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context) + } override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions @@ -74,7 +76,7 @@ object PartitionPruningRDD { * Create a PartitionPruningRDD. This function can be used to create the PartitionPruningRDD * when its type T is not known at compile time. */ - def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = { + def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean): PartitionPruningRDD[T] = { new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index ed79032893d33..dc60d48927624 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -149,10 +149,10 @@ private[spark] class PipedRDD[T: ClassTag]( }.start() // Return an iterator that read lines from the process's stdout - val lines = Source.fromInputStream(proc.getInputStream).getLines + val lines = Source.fromInputStream(proc.getInputStream).getLines() new Iterator[String] { - def next() = lines.next() - def hasNext = { + def next(): String = lines.next() + def hasNext: Boolean = { if (lines.hasNext) { true } else { @@ -162,7 +162,7 @@ private[spark] class PipedRDD[T: ClassTag]( } // cleanup task working directory if used - if (workInTaskDirectory == true) { + if (workInTaskDirectory) { scala.util.control.Exception.ignoring(classOf[IOException]) { Utils.deleteRecursively(new File(taskDirectory)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a4c74ed03e330..ddbfd5624e741 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -186,7 +186,7 @@ abstract class RDD[T: ClassTag]( } /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ - def getStorageLevel = storageLevel + def getStorageLevel: StorageLevel = storageLevel // Our dependencies and partitions will be gotten by calling subclass's methods below, and will // be overwritten when we're checkpointed @@ -746,13 +746,13 @@ abstract class RDD[T: ClassTag]( def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => new Iterator[(T, U)] { - def hasNext = (thisIter.hasNext, otherIter.hasNext) match { + def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { case (true, true) => true case (false, false) => false case _ => throw new SparkException("Can only zip RDDs with " + "same number of elements in each partition") } - def next = (thisIter.next, otherIter.next) + def next(): (T, U) = (thisIter.next(), otherIter.next()) } } } @@ -868,8 +868,8 @@ abstract class RDD[T: ClassTag]( // Our partitioner knows how to handle T (which, since we have a partitioner, is // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples val p2 = new Partitioner() { - override def numPartitions = p.numPartitions - override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) + override def numPartitions: Int = p.numPartitions + override def getPartition(k: Any): Int = p.getPartition(k.asInstanceOf[(Any, _)]._1) } // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies // anyway, and when calling .keys, will not have a partitioner set, even though @@ -1394,7 +1394,7 @@ abstract class RDD[T: ClassTag]( } /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ - def context = sc + def context: SparkContext = sc /** * Private API for changing an RDD's ClassTag. diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index d9fe6847254fa..2dc47f95937cb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,14 +17,12 @@ package org.apache.spark.rdd -import scala.reflect.ClassTag - import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { - override val index = idx + override val index: Int = idx override def hashCode(): Int = idx } diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index ed24ea22a661c..c27f435eb9c5a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -105,7 +105,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( seq } } - def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match { + def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit): Unit = dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index aece683ff3199..4239e7e22af89 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -44,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag]( var parentPartition: Partition = rdd.partitions(parentRddPartitionIndex) - def preferredLocations() = rdd.preferredLocations(parentPartition) + def preferredLocations(): Seq[String] = rdd.preferredLocations(parentPartition) override val index: Int = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 95b2dd954e9f4..d0be304762e1f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -32,7 +32,7 @@ private[spark] class ZippedPartitionsPartition( override val index: Int = idx var partitionValues = rdds.map(rdd => rdd.partitions(idx)) - def partitions = partitionValues + def partitions: Seq[Partition] = partitionValues @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index fa83372bb4d11..e0edd7d4ae968 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -39,8 +39,11 @@ class AccumulableInfo ( } object AccumulableInfo { - def apply(id: Long, name: String, update: Option[String], value: String) = + def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = { new AccumulableInfo(id, name, update, value) + } - def apply(id: Long, name: String, value: String) = new AccumulableInfo(id, name, None, value) + def apply(id: Long, name: String, value: String): AccumulableInfo = { + new AccumulableInfo(id, name, None, value) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8feac6cb6b7a1..b405bd3338e7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -946,7 +946,7 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) - def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { + def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None): Unit = { val serviceTime = stage.latestInfo.submissionTime match { case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) case _ => "Unknown" diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 34fa6d27c3a45..c0d889360ae99 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -149,47 +149,60 @@ private[spark] class EventLoggingListener( } // Events that do not trigger a flush - override def onStageSubmitted(event: SparkListenerStageSubmitted) = - logEvent(event) - override def onTaskStart(event: SparkListenerTaskStart) = - logEvent(event) - override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = - logEvent(event) - override def onTaskEnd(event: SparkListenerTaskEnd) = - logEvent(event) - override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = - logEvent(event) + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event) + + override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) + + override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = logEvent(event) + + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = logEvent(event) + + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = logEvent(event) // Events that trigger a flush - override def onStageCompleted(event: SparkListenerStageCompleted) = - logEvent(event, flushLogger = true) - override def onJobStart(event: SparkListenerJobStart) = - logEvent(event, flushLogger = true) - override def onJobEnd(event: SparkListenerJobEnd) = + override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { logEvent(event, flushLogger = true) - override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) = + } + + override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true) + + override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) + + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = { logEvent(event, flushLogger = true) - override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved) = + } + + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { logEvent(event, flushLogger = true) - override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = + } + + override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { logEvent(event, flushLogger = true) - override def onApplicationStart(event: SparkListenerApplicationStart) = + } + + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { logEvent(event, flushLogger = true) - override def onApplicationEnd(event: SparkListenerApplicationEnd) = + } + + override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { logEvent(event, flushLogger = true) - override def onExecutorAdded(event: SparkListenerExecutorAdded) = + } + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) - override def onExecutorRemoved(event: SparkListenerExecutorRemoved) = + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) + } // No-op because logging every update would be overkill - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } /** * Stop logging events. The event log file will be renamed so that it loses the * ".inprogress" suffix. */ - def stop() = { + def stop(): Unit = { writer.foreach(_.close()) val target = new Path(logPath) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 8aa528ac573d0..e55b76c36cc5f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -57,7 +57,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener private val stageIdToJobId = new HashMap[Int, Int] private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } createLogDir() diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 29879b374b801..382b09422a4a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -34,7 +34,7 @@ private[spark] class JobWaiter[T]( @volatile private var _jobFinished = totalTasks == 0 - def jobFinished = _jobFinished + def jobFinished: Boolean = _jobFinished // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero // partition RDDs), we set the jobResult directly to JobSucceeded. diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 759df023a6dcf..a3caa9f000c89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -160,7 +160,7 @@ private[spark] object OutputCommitCoordinator { class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator) extends Actor with ActorLogReceive with Logging { - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case AskPermissionToCommitOutput(stage, partition, taskAttempt) => sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt) case StopCoordinator => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 4a9ff918afe25..e074ce6ebff0b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -64,5 +64,5 @@ private[spark] class ResultTask[T, U]( // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs - override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" + override def toString: String = "ResultTask(" + stageId + ", " + partitionId + ")" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 79709089c0da4..fd0d484b45460 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -47,7 +47,7 @@ private[spark] class ShuffleMapTask( /** A constructor used only in test suites. This does not require passing in an RDD. */ def this(partitionId: Int) { - this(0, null, new Partition { override def index = 0 }, null) + this(0, null, new Partition { override def index: Int = 0 }, null) } @transient private val preferredLocs: Seq[TaskLocation] = { @@ -83,5 +83,5 @@ private[spark] class ShuffleMapTask( override def preferredLocations: Seq[TaskLocation] = preferredLocs - override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partitionId) + override def toString: String = "ShuffleMapTask(%d, %d)".format(stageId, partitionId) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 52720d48ca67f..b711ff209af94 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -300,7 +300,7 @@ private[spark] object StatsReportListener extends Logging { } def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { - def f(d: Double) = format.format(d) + def f(d: Double): String = format.format(d) showDistribution(heading, dOpt, f _) } @@ -346,7 +346,7 @@ private[spark] object StatsReportListener extends Logging { /** * Reformat a time interval in milliseconds to a prettier format for output */ - def millisToString(ms: Long) = { + def millisToString(ms: Long): String = { val (size, units) = if (ms > hours) { (ms.toDouble / hours, "hours") diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index cc13f57a49b89..4cbc6e84a6bdd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -133,7 +133,7 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - override def toString = "Stage " + id + override def toString: String = "Stage " + id override def hashCode(): Int = id diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 10c685f29d3ac..da07ce2c6ea49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -29,23 +29,22 @@ private[spark] sealed trait TaskLocation { /** * A location that includes both a host and an executor id on that host. */ -private [spark] case class ExecutorCacheTaskLocation(override val host: String, - val executorId: String) extends TaskLocation { -} +private [spark] +case class ExecutorCacheTaskLocation(override val host: String, executorId: String) + extends TaskLocation /** * A location on a host. */ private [spark] case class HostTaskLocation(override val host: String) extends TaskLocation { - override def toString = host + override def toString: String = host } /** * A location on a host that is cached by HDFS. */ -private [spark] case class HDFSCacheTaskLocation(override val host: String) - extends TaskLocation { - override def toString = TaskLocation.inMemoryLocationTag + host +private [spark] case class HDFSCacheTaskLocation(override val host: String) extends TaskLocation { + override def toString: String = TaskLocation.inMemoryLocationTag + host } private[spark] object TaskLocation { @@ -54,14 +53,16 @@ private[spark] object TaskLocation { // confusion. See RFC 952 and RFC 1123 for information about the format of hostnames. val inMemoryLocationTag = "hdfs_cache_" - def apply(host: String, executorId: String) = new ExecutorCacheTaskLocation(host, executorId) + def apply(host: String, executorId: String): TaskLocation = { + new ExecutorCacheTaskLocation(host, executorId) + } /** * Create a TaskLocation from a string returned by getPreferredLocations. * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the * location is cached. */ - def apply(str: String) = { + def apply(str: String): TaskLocation = { val hstr = str.stripPrefix(inMemoryLocationTag) if (hstr.equals(str)) { new HostTaskLocation(str) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index f33fd4450b2a6..076b36e86c0ce 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -373,17 +373,17 @@ private[spark] class TaskSchedulerImpl( } def handleSuccessfulTask( - taskSetManager: TaskSetManager, - tid: Long, - taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager: TaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]): Unit = synchronized { taskSetManager.handleSuccessfulTask(tid, taskResult) } def handleFailedTask( - taskSetManager: TaskSetManager, - tid: Long, - taskState: TaskState, - reason: TaskEndReason) = synchronized { + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: TaskEndReason): Unit = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { // Need to revive offers again now that the task set manager state has been updated to @@ -423,7 +423,7 @@ private[spark] class TaskSchedulerImpl( starvationTimer.cancel() } - override def defaultParallelism() = backend.defaultParallelism() + override def defaultParallelism(): Int = backend.defaultParallelism() // Check for speculatable tasks in all our active jobs. def checkSpeculatableTasks() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 529237f0d35dc..d509881c74fef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.nio.ByteBuffer import java.util.Arrays +import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -29,6 +30,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -97,7 +99,8 @@ private[spark] class TaskSetManager( var calculatedTasks = 0 val runningTasksSet = new HashSet[Long] - override def runningTasks = runningTasksSet.size + + override def runningTasks: Int = runningTasksSet.size // True once no more tasks should be launched for this task set manager. TaskSetManagers enter // the zombie state once at least one attempt of each task has completed successfully, or if the @@ -168,9 +171,9 @@ private[spark] class TaskSetManager( var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level - override def schedulableQueue = null + override def schedulableQueue: ConcurrentLinkedQueue[Schedulable] = null - override def schedulingMode = SchedulingMode.NONE + override def schedulingMode: SchedulingMode = SchedulingMode.NONE var emittedTaskSizeWarning = false @@ -585,7 +588,7 @@ private[spark] class TaskSetManager( /** * Marks the task as getting result and notifies the DAG Scheduler */ - def handleTaskGettingResult(tid: Long) = { + def handleTaskGettingResult(tid: Long): Unit = { val info = taskInfos(tid) info.markGettingResult() sched.dagScheduler.taskGettingResult(info) @@ -612,7 +615,7 @@ private[spark] class TaskSetManager( /** * Marks the task as successful and notifies the DAGScheduler that a task has ended. */ - def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = { + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = { val info = taskInfos(tid) val index = info.index info.markSuccessful() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 87ebf31139ce9..5d258d9da4d1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -85,7 +85,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } - def receiveWithLogging = { + def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisterExecutor(executorId, hostPort, cores, logUrls) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) if (executorDataMap.contains(executorId)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index f14aaeea0a25c..5a38ad9f2b12c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -109,7 +109,7 @@ private[spark] abstract class YarnSchedulerBackend( context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RegisterClusterManager => logInfo(s"ApplicationMaster registered as $sender") amActor = Some(sender) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala index aa3ec0f8cfb9c..8df4f3b554c41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -24,7 +24,7 @@ private[spark] object MemoryUtils { val OVERHEAD_FRACTION = 0.10 val OVERHEAD_MINIMUM = 384 - def calculateTotalMemory(sc: SparkContext) = { + def calculateTotalMemory(sc: SparkContext): Int = { sc.conf.getInt("spark.mesos.executor.memoryOverhead", math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 06bb527522141..b381436839227 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -387,7 +387,7 @@ private[spark] class MesosSchedulerBackend( } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8) override def applicationId(): String = Option(appId).getOrElse { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index d95426d918e19..eb3f999b5b375 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -59,7 +59,7 @@ private[spark] class LocalActor( private val executor = new Executor( localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case ReviveOffers => reviveOffers() @@ -117,7 +117,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! ReviveOffers } - override def defaultParallelism() = + override def defaultParallelism(): Int = scheduler.conf.getInt("spark.default.parallelism", totalCores) override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 1baa0e009f3ae..dfbde7c8a1b0d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -59,9 +59,10 @@ private[spark] class JavaSerializationStream( } private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) -extends DeserializationStream { + extends DeserializationStream { + private val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass) = + override def resolveClass(desc: ObjectStreamClass): Class[_] = Class.forName(desc.getName, false, loader) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index dc7aa99738c17..f83bcaa5cc09e 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -60,7 +60,7 @@ class KryoSerializer(conf: SparkConf) .split(',') .filter(!_.isEmpty) - def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 7de2f9cbb2866..660df00bc32f5 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -106,7 +106,7 @@ class FileShuffleBlockManager(conf: SparkConf) * when the writers are closed successfully */ def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer, - writeMetrics: ShuffleWriteMetrics) = { + writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleId) @@ -268,7 +268,7 @@ object FileShuffleBlockManager { new PrimitiveVector[Long]() } - def apply(bucketId: Int) = files(bucketId) + def apply(bucketId: Int): File = files(bucketId) def recordMapOutput(mapId: Int, offsets: Array[Long], lengths: Array[Long]) { assert(offsets.length == lengths.length) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index b292587d37028..87fd161e06c85 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -80,7 +80,7 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { * end of the output file. This will be used by getBlockLocation to figure out where each block * begins and ends. * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]) = { + def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) try { @@ -121,5 +121,5 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { } } - override def stop() = {} + override def stop(): Unit = {} } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 1f012941c85ab..c186fd360fef6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -35,13 +35,13 @@ sealed abstract class BlockId { def name: String // convenience methods - def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None - def isRDD = isInstanceOf[RDDBlockId] - def isShuffle = isInstanceOf[ShuffleBlockId] - def isBroadcast = isInstanceOf[BroadcastBlockId] + def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None + def isRDD: Boolean = isInstanceOf[RDDBlockId] + def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] + def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] - override def toString = name - override def hashCode = name.hashCode + override def toString: String = name + override def hashCode: Int = name.hashCode override def equals(other: Any): Boolean = other match { case o: BlockId => getClass == o.getClass && name.equals(o.name) case _ => false @@ -50,54 +50,54 @@ sealed abstract class BlockId { @DeveloperApi case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { - def name = "rdd_" + rddId + "_" + splitIndex + override def name: String = "rdd_" + rddId + "_" + splitIndex } // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getBlockData(). @DeveloperApi case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } @DeveloperApi case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" } @DeveloperApi case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { - def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) + override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } @DeveloperApi case class TaskResultBlockId(taskId: Long) extends BlockId { - def name = "taskresult_" + taskId + override def name: String = "taskresult_" + taskId } @DeveloperApi case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { - def name = "input-" + streamId + "-" + uniqueId + override def name: String = "input-" + streamId + "-" + uniqueId } /** Id associated with temporary local data managed as blocks. Not serializable. */ private[spark] case class TempLocalBlockId(id: UUID) extends BlockId { - def name = "temp_local_" + id + override def name: String = "temp_local_" + id } /** Id associated with temporary shuffle data managed as blocks. Not serializable. */ private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId { - def name = "temp_shuffle_" + id + override def name: String = "temp_shuffle_" + id } // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { - def name = "test_" + id + override def name: String = "test_" + id } @DeveloperApi @@ -112,7 +112,7 @@ object BlockId { val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ - def apply(id: String) = id match { + def apply(id: String): BlockId = id match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b177a59c721df..a6f1ebf325a7c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -77,11 +77,11 @@ class BlockManagerId private ( @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = s"BlockManagerId($executorId, $host, $port)" + override def toString: String = s"BlockManagerId($executorId, $host, $port)" override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port - override def equals(that: Any) = that match { + override def equals(that: Any): Boolean = that match { case id: BlockManagerId => executorId == id.executorId && port == id.port && host == id.host case _ => @@ -100,10 +100,10 @@ private[spark] object BlockManagerId { * @param port Port of the block manager. * @return A new [[org.apache.spark.storage.BlockManagerId]]. */ - def apply(execId: String, host: String, port: Int) = + def apply(execId: String, host: String, port: Int): BlockManagerId = getCachedBlockManagerId(new BlockManagerId(execId, host, port)) - def apply(in: ObjectInput) = { + def apply(in: ObjectInput): BlockManagerId = { val obj = new BlockManagerId() obj.readExternal(in) getCachedBlockManagerId(obj) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 654796f23c96e..061964826f08b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -79,7 +79,7 @@ class BlockManagerMaster( * Check if block manager master has a block. Note that this can be used to check for only * those blocks that are reported to block manager master. */ - def contains(blockId: BlockId) = { + def contains(blockId: BlockId): Boolean = { !getLocations(blockId).isEmpty } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 787b0f96bec32..5b5328016124e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -52,7 +52,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) sender ! true @@ -421,7 +421,7 @@ private[spark] class BlockManagerInfo( // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) + def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId)) def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 8462871e798a5..52fb896c4e21f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -38,7 +38,7 @@ class BlockManagerSlaveActor( import context.dispatcher // Operations that involve removing blocks may be slow and should be done asynchronously - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RemoveBlock(blockId) => doAsync[Boolean]("removing block " + blockId, sender) { blockManager.removeBlock(blockId) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 81164178b9e8e..f703e50b6b0ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -82,11 +82,13 @@ private[spark] class DiskBlockObjectWriter( { /** Intercepts write calls and tracks total time spent writing. Not thread safe. */ private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream { - def write(i: Int): Unit = callWithTiming(out.write(i)) - override def write(b: Array[Byte]) = callWithTiming(out.write(b)) - override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len)) - override def close() = out.close() - override def flush() = out.flush() + override def write(i: Int): Unit = callWithTiming(out.write(i)) + override def write(b: Array[Byte]): Unit = callWithTiming(out.write(b)) + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + callWithTiming(out.write(b, off, len)) + } + override def close(): Unit = out.close() + override def flush(): Unit = out.flush() } /** The file channel, used for repositioning / truncating the file. */ @@ -141,8 +143,9 @@ private[spark] class DiskBlockObjectWriter( if (syncWrites) { // Force outstanding writes to disk and track how long it takes objOut.flush() - def sync = fos.getFD.sync() - callWithTiming(sync) + callWithTiming { + fos.getFD.sync() + } } objOut.close() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 132502b75f8cd..95e2d688d9b17 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -24,5 +24,7 @@ import java.io.File * based off an offset and a length. */ private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { - override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) + override def toString: String = { + "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 120c327a7e580..0186eb30a1905 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -36,7 +36,7 @@ class RDDInfo( def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0 - override def toString = { + override def toString: String = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + "MemorySize: %s; TachyonSize: %s; DiskSize: %s").format( @@ -44,7 +44,7 @@ class RDDInfo( bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) } - override def compare(that: RDDInfo) = { + override def compare(that: RDDInfo): Int = { this.id - that.id } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index e5e1cf5a69a19..134abea866218 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -50,11 +50,11 @@ class StorageLevel private( def this() = this(false, true, false, false) // For deserialization - def useDisk = _useDisk - def useMemory = _useMemory - def useOffHeap = _useOffHeap - def deserialized = _deserialized - def replication = _replication + def useDisk: Boolean = _useDisk + def useMemory: Boolean = _useMemory + def useOffHeap: Boolean = _useOffHeap + def deserialized: Boolean = _deserialized + def replication: Int = _replication assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") @@ -80,7 +80,7 @@ class StorageLevel private( false } - def isValid = (useMemory || useDisk || useOffHeap) && (replication > 0) + def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0) def toInt: Int = { var ret = 0 @@ -183,7 +183,7 @@ object StorageLevel { useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, - replication: Int) = { + replication: Int): StorageLevel = { getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) } @@ -197,7 +197,7 @@ object StorageLevel { useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, - replication: Int = 1) = { + replication: Int = 1): StorageLevel = { getCachedStorageLevel(new StorageLevel(useDisk, useMemory, false, deserialized, replication)) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index def49e80a3605..7d75929b96f75 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,7 +19,6 @@ package org.apache.spark.storage import scala.collection.mutable -import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ @@ -32,7 +31,7 @@ class StorageStatusListener extends SparkListener { // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() - def storageStatusList = executorIdToStorageStatus.values.toSeq + def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { @@ -56,7 +55,7 @@ class StorageStatusListener extends SparkListener { } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { @@ -67,7 +66,7 @@ class StorageStatusListener extends SparkListener { } } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { updateStorageStatus(unpersistRDD.rddId) } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala index b86abbda1d3e7..65fa81704c365 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala @@ -24,5 +24,7 @@ import tachyon.client.TachyonFile * a length. */ private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) { - override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) + override def toString: String = { + "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 0c24ad2760e08..adfa6bbada256 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -60,7 +60,7 @@ private[spark] class SparkUI private ( } initialize() - def getAppName = appName + def getAppName: String = appName /** Set the app name for this UI. */ def setAppName(name: String) { diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index b5022fe853c49..f07864141a21c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -149,9 +149,11 @@ private[spark] object UIUtils extends Logging { } } - def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource + def prependBaseUri(basePath: String = "", resource: String = ""): String = { + uiRoot + basePath + resource + } - def commonHeaderNodes = { + def commonHeaderNodes: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index fc1844600f1cb..19ac7a826e306 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -51,7 +51,7 @@ private[spark] object UIWorkloadGenerator { val nJobSet = args(2).toInt val sc = new SparkContext(conf) - def setProperties(s: String) = { + def setProperties(s: String): Unit = { if(schedulingMode == SchedulingMode.FAIR) { sc.setLocalProperty("spark.scheduler.pool", s) } @@ -59,7 +59,7 @@ private[spark] object UIWorkloadGenerator { } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) - def nextFloat() = new Random().nextFloat() + def nextFloat(): Float = new Random().nextFloat() val jobs = Seq[(String, () => Long)]( ("Count", baseData.count), diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 3afd7ef07d7c9..69053fe44d7e4 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.ExceptionFailure import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.storage.{StorageStatus, StorageStatusListener} import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { @@ -55,19 +55,19 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp val executorToShuffleWrite = HashMap[String, Long]() val executorToLogUrls = HashMap[String, Map[String, String]]() - def storageStatusList = storageStatusListener.storageStatusList + def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) = synchronized { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap } - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 937d95a934b59..949e80d30f5eb 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -73,7 +73,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Misc: val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]() - def blockManagerIds = executorIdToBlockManagerId.values.toSeq + def blockManagerIds: Seq[BlockManagerId] = executorIdToBlockManagerId.values.toSeq var schedulingMode: Option[SchedulingMode] = None @@ -146,7 +146,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { val jobGroup = for ( props <- Option(jobStart.properties); group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) @@ -182,7 +182,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized { val jobData = activeJobs.remove(jobEnd.jobId).getOrElse { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) @@ -219,7 +219,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { val stage = stageCompleted.stageInfo stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { @@ -260,7 +260,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val stage = stageSubmitted.stageInfo activeStages(stage.stageId) = stage pendingStages.remove(stage.stageId) @@ -288,7 +288,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val taskInfo = taskStart.taskInfo if (taskInfo != null) { val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { @@ -312,7 +312,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // stageToTaskInfos already has the updated status. } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task // completion event is for. Let's just drop it here. This means we might have some speculation diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index b2bbfdee56946..7ffcf291b5cc6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -24,7 +24,7 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { val sc = parent.sc val killEnabled = parent.killEnabled - def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) val listener = parent.jobProgressListener attachPage(new AllJobsPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 110f8780a9a12..e03442894c5cc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.xml.{Node, Unparsed} +import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils @@ -170,7 +170,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
    val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo) = {acc.name}{acc.value} + def accumulableRow(acc: AccumulableInfo): Elem = + {acc.name}{acc.value} val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, accumulables.values.toSeq) @@ -293,10 +294,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val schedulerDelayQuantiles = schedulerDelayTitle +: getFormattedTimeQuantiles(schedulerDelays) - def getFormattedSizeQuantiles(data: Seq[Double]) = + def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) - def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = { + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) + : Seq[Elem] = { val recordDist = getDistributionQuantiles(records).iterator getDistributionQuantiles(data).map(d => {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 937261de00e3a..1bd2d87e00796 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -32,10 +32,10 @@ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages" attachPage(new StagePage(this)) attachPage(new PoolPage(this)) - def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - def handleKillRequest(request: HttpServletRequest) = { - if ((killEnabled) && (parent.securityManager.checkModifyPermissions(request.getRemoteUser))) { + def handleKillRequest(request: HttpServletRequest): Unit = { + if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index dbf1ceeda1878..711a3697bda15 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -94,11 +94,11 @@ private[jobs] object UIData { var taskData = new HashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] - def hasInput = inputBytes > 0 - def hasOutput = outputBytes > 0 - def hasShuffleRead = shuffleReadTotalBytes > 0 - def hasShuffleWrite = shuffleWriteBytes > 0 - def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 + def hasInput: Boolean = inputBytes > 0 + def hasOutput: Boolean = outputBytes > 0 + def hasShuffleRead: Boolean = shuffleReadTotalBytes > 0 + def hasShuffleWrite: Boolean = shuffleWriteBytes > 0 + def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 && diskBytesSpilled > 0 } /** diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index a81291d505583..045bd784990d1 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -40,10 +40,10 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing - def storageStatusList = storageStatusListener.storageStatusList + def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ - def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { @@ -56,19 +56,19 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar * Assumes the storage status list is fully up-to-date. This implies the corresponding * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. */ - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val metrics = taskEnd.taskMetrics if (metrics != null && metrics.updatedBlocks.isDefined) { updateRDDInfo(metrics.updatedBlocks.get) } } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { // Remove all partitions that are no longer cached in current completed stage val completedRddIds = stageCompleted.stageInfo.rddInfos.map(r => r.id).toSet _rddInfoMap.retain { case (id, info) => @@ -76,7 +76,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar } } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { _rddInfoMap.remove(unpersistRDD.rddId) } } diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 390310243ee0a..9044aaeef2d48 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -27,8 +27,8 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat // scalastyle:on private[this] var completed = false - def next() = sub.next() - def hasNext = { + def next(): A = sub.next() + def hasNext: Boolean = { val r = sub.hasNext if (!r && !completed) { completed = true @@ -37,13 +37,13 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat r } - def completion() + def completion(): Unit } private[spark] object CompletionIterator { - def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A,I] = { + def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A, I] = { new CompletionIterator[A,I](sub) { - def completion() = completionFunction + def completion(): Unit = completionFunction } } } diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index a465298c8c5ab..9aea8efa38c7a 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -57,7 +57,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va out.println } - def statCounter = StatCounter(data.slice(startIdx, endIdx)) + def statCounter: StatCounter = StatCounter(data.slice(startIdx, endIdx)) /** * print a summary of this distribution to the given PrintStream. diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index cf89c1782fd67..1718554061985 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -39,31 +39,27 @@ private[spark] class ManualClock(private var time: Long) extends Clock { /** * @param timeToSet new time (in milliseconds) that the clock should represent */ - def setTime(timeToSet: Long) = - synchronized { - time = timeToSet - notifyAll() - } + def setTime(timeToSet: Long): Unit = synchronized { + time = timeToSet + notifyAll() + } /** * @param timeToAdd time (in milliseconds) to add to the clock's time */ - def advance(timeToAdd: Long) = - synchronized { - time += timeToAdd - notifyAll() - } + def advance(timeToAdd: Long): Unit = synchronized { + time += timeToAdd + notifyAll() + } /** * @param targetTime block until the clock time is set or advanced to at least this time * @return current time reported by the clock when waiting finishes */ - def waitTillTime(targetTime: Long): Long = - synchronized { - while (time < targetTime) { - wait(100) - } - getTimeMillis() + def waitTillTime(targetTime: Long): Long = synchronized { + while (time < targetTime) { + wait(100) } - + getTimeMillis() + } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index ac40f19ed6799..375ed430bde45 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -67,14 +67,15 @@ private[spark] object MetadataCleanerType extends Enumeration { type MetadataCleanerType = Value - def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = - "spark.cleaner.ttl." + which.toString + def systemProperty(which: MetadataCleanerType.MetadataCleanerType): String = { + "spark.cleaner.ttl." + which.toString + } } // TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { - def getDelaySeconds(conf: SparkConf) = { + def getDelaySeconds(conf: SparkConf): Int = { conf.getInt("spark.cleaner.ttl", -1) } diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 74fa77b68de0b..dad888548ed10 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -43,7 +43,7 @@ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef this } - override def toString = "(" + _1 + "," + _2 + ")" + override def toString: String = "(" + _1 + "," + _2 + ")" override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]] } diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 6d8d9e8da3678..73d126ff6254e 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -22,7 +22,7 @@ package org.apache.spark.util */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { - override def findClass(name: String) = { + override def findClass(name: String): Class[_] = { super.findClass(name) } diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index 770ff9d5ad6ae..a06b6f84ef11b 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -27,7 +27,7 @@ import java.nio.channels.Channels */ private[spark] class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable { - def value = buffer + def value: ByteBuffer = buffer private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index d80eed455c427..8586da1996cf3 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -141,8 +141,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { object StatCounter { /** Build a StatCounter from a list of values. */ - def apply(values: TraversableOnce[Double]) = new StatCounter(values) + def apply(values: TraversableOnce[Double]): StatCounter = new StatCounter(values) /** Build a StatCounter from a list of values passed as variable-length arguments. */ - def apply(values: Double*) = new StatCounter(values) + def apply(values: Double*): StatCounter = new StatCounter(values) } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala index f5be5856c2109..310c0c109416c 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -82,7 +82,7 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo this } - override def update(key: A, value: B) = this += ((key, value)) + override def update(key: A, value: B): Unit = this += ((key, value)) override def apply(key: A): B = internalMap.apply(key) @@ -92,14 +92,14 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo override def size: Int = internalMap.size - override def foreach[U](f: ((A, B)) => U) = nonNullReferenceMap.foreach(f) + override def foreach[U](f: ((A, B)) => U): Unit = nonNullReferenceMap.foreach(f) def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) def toMap: Map[A, B] = iterator.toMap /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ - def clearOldValues(threshTime: Long) = internalMap.clearOldValues(threshTime) + def clearOldValues(threshTime: Long): Unit = internalMap.clearOldValues(threshTime) /** Remove entries with values that are no longer strongly reachable. */ def clearNullValues() { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fa56bb09e4e5c..d9a671687aad0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -85,7 +85,7 @@ private[spark] object Utils extends Logging { def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { val bis = new ByteArrayInputStream(bytes) val ois = new ObjectInputStream(bis) { - override def resolveClass(desc: ObjectStreamClass) = + override def resolveClass(desc: ObjectStreamClass): Class[_] = Class.forName(desc.getName, false, loader) } ois.readObject.asInstanceOf[T] @@ -106,11 +106,10 @@ private[spark] object Utils extends Logging { /** Serialize via nested stream using specific serializer */ def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)( - f: SerializationStream => Unit) = { + f: SerializationStream => Unit): Unit = { val osWrapper = ser.serializeStream(new OutputStream { - def write(b: Int) = os.write(b) - - override def write(b: Array[Byte], off: Int, len: Int) = os.write(b, off, len) + override def write(b: Int): Unit = os.write(b) + override def write(b: Array[Byte], off: Int, len: Int): Unit = os.write(b, off, len) }) try { f(osWrapper) @@ -121,10 +120,9 @@ private[spark] object Utils extends Logging { /** Deserialize via nested stream using specific serializer */ def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)( - f: DeserializationStream => Unit) = { + f: DeserializationStream => Unit): Unit = { val isWrapper = ser.deserializeStream(new InputStream { - def read(): Int = is.read() - + override def read(): Int = is.read() override def read(b: Array[Byte], off: Int, len: Int): Int = is.read(b, off, len) }) try { @@ -137,7 +135,7 @@ private[spark] object Utils extends Logging { /** * Get the ClassLoader which loaded Spark. */ - def getSparkClassLoader = getClass.getClassLoader + def getSparkClassLoader: ClassLoader = getClass.getClassLoader /** * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that @@ -146,7 +144,7 @@ private[spark] object Utils extends Logging { * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently * active loader when setting up ClassLoader delegation chains. */ - def getContextOrSparkClassLoader = + def getContextOrSparkClassLoader: ClassLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) /** Determines whether the provided class is loadable in the current thread. */ @@ -155,12 +153,14 @@ private[spark] object Utils extends Logging { } /** Preferred alternative to Class.forName(className) */ - def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader) + def classForName(className: String): Class[_] = { + Class.forName(className, true, getContextOrSparkClassLoader) + } /** * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ - def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { + def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput): Unit = { if (bb.hasArray) { out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) } else { @@ -1557,7 +1557,7 @@ private[spark] object Utils extends Logging { /** Return the class name of the given object, removing all dollar signs */ - def getFormattedClassName(obj: AnyRef) = { + def getFormattedClassName(obj: AnyRef): String = { obj.getClass.getSimpleName.replace("$", "") } @@ -1570,7 +1570,7 @@ private[spark] object Utils extends Logging { } /** Return an empty JSON object */ - def emptyJson = JObject(List[JField]()) + def emptyJson: JsonAST.JObject = JObject(List[JField]()) /** * Return a Hadoop FileSystem with the scheme encoded in the given path. @@ -1618,7 +1618,7 @@ private[spark] object Utils extends Logging { /** * Indicates whether Spark is currently running unit tests. */ - def isTesting = { + def isTesting: Boolean = { sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") } diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index af1f64649f354..f79e8e0491ea1 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -156,10 +156,10 @@ class BitSet(numBits: Int) extends Serializable { /** * Get an iterator over the set bits. */ - def iterator = new Iterator[Int] { + def iterator: Iterator[Int] = new Iterator[Int] { var ind = nextSetBit(0) override def hasNext: Boolean = ind >= 0 - override def next() = { + override def next(): Int = { val tmp = ind ind = nextSetBit(ind + 1) tmp diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 8a0f5a602de12..9ff4744593d4d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -159,7 +159,7 @@ class ExternalAppendOnlyMap[K, V, C]( val batchSizes = new ArrayBuffer[Long] // Flush the disk writer's contents to disk, and update relevant variables - def flush() = { + def flush(): Unit = { val w = writer writer = null w.commitAndClose() @@ -355,7 +355,7 @@ class ExternalAppendOnlyMap[K, V, C]( val pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { - def isEmpty = pairs.length == 0 + def isEmpty: Boolean = pairs.length == 0 // Invalid if there are no more pairs in this stream def minKeyHash: Int = { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index d69f2d9048055..3262e670c2030 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -283,7 +283,7 @@ private[spark] class ExternalSorter[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables. // The writer is closed at the end of this process, and cannot be reused. - def flush() = { + def flush(): Unit = { val w = writer writer = null w.commitAndClose() diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index b8de4ff9aa494..c52591b352340 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -109,7 +109,7 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = -1 var nextPair: (K, V) = computeNextPair() @@ -132,9 +132,9 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 4e363b74f4bef..c80057f95e0b2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -85,7 +85,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( protected var _bitset = new BitSet(_capacity) - def getBitSet = _bitset + def getBitSet: BitSet = _bitset // Init of the array in constructor (instead of in declaration) to work around a Scala compiler // specialization bug that would generate two arrays (one for Object and one for specialized T). @@ -183,7 +183,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( /** Return the value at the specified position. */ def getValue(pos: Int): T = _data(pos) - def iterator = new Iterator[T] { + def iterator: Iterator[T] = new Iterator[T] { var pos = nextPos(0) override def hasNext: Boolean = pos != INVALID_POS override def next(): T = { diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index 2e1ef06cbc4e1..61e22642761f0 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -46,7 +46,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, private var _oldValues: Array[V] = null - override def size = _keySet.size + override def size: Int = _keySet.size /** Get the value for a given key */ def apply(k: K): V = { @@ -87,7 +87,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = 0 var nextPair: (K, V) = computeNextPair() @@ -103,9 +103,9 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala index c5268c0fae0ef..bdbca00a00622 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala @@ -32,7 +32,7 @@ private[spark] object Utils { */ def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = { val ordering = new GuavaOrdering[T] { - override def compare(l: T, r: T) = ord.compare(l, r) + override def compare(l: T, r: T): Int = ord.compare(l, r) } collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator } diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 1d5467060623c..14b6ba4af489a 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -121,7 +121,7 @@ private[spark] object FileAppender extends Logging { val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) - def createTimeBasedAppender() = { + def createTimeBasedAppender(): FileAppender = { val validatedParams: Option[(Long, String)] = rollingInterval match { case "daily" => logInfo(s"Rolling executor logs enabled for $file with daily rolling") @@ -149,7 +149,7 @@ private[spark] object FileAppender extends Logging { } } - def createSizeBasedAppender() = { + def createSizeBasedAppender(): FileAppender = { rollingSizeBytes match { case IntParam(bytes) => logInfo(s"Rolling executor logs enabled for $file with rolling every $bytes bytes") diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 76e7a2760bcd1..786b97ad7b9ec 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -105,7 +105,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals private val rng: Random = new XORShiftRandom - override def setSeed(seed: Long) = rng.setSeed(seed) + override def setSeed(seed: Long): Unit = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { if (ub - lb <= 0.0) { @@ -131,7 +131,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals def cloneComplement(): BernoulliCellSampler[T] = new BernoulliCellSampler[T](lb, ub, !complement) - override def clone = new BernoulliCellSampler[T](lb, ub, complement) + override def clone: BernoulliCellSampler[T] = new BernoulliCellSampler[T](lb, ub, complement) } @@ -153,7 +153,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T private val rng: Random = RandomSampler.newDefaultRNG - override def setSeed(seed: Long) = rng.setSeed(seed) + override def setSeed(seed: Long): Unit = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { if (fraction <= 0.0) { @@ -167,7 +167,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T } } - override def clone = new BernoulliSampler[T](fraction) + override def clone: BernoulliSampler[T] = new BernoulliSampler[T](fraction) } @@ -209,7 +209,7 @@ class PoissonSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] } } - override def clone = new PoissonSampler[T](fraction) + override def clone: PoissonSampler[T] = new PoissonSampler[T](fraction) } @@ -228,15 +228,18 @@ class GapSamplingIterator[T: ClassTag]( val arrayClass = Array.empty[T].iterator.getClass val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass data.getClass match { - case `arrayClass` => ((n: Int) => { data = data.drop(n) }) - case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) - case _ => ((n: Int) => { + case `arrayClass` => + (n: Int) => { data = data.drop(n) } + case `arrayBufferClass` => + (n: Int) => { data = data.drop(n) } + case _ => + (n: Int) => { var j = 0 while (j < n && data.hasNext) { data.next() j += 1 } - }) + } } } @@ -244,21 +247,21 @@ class GapSamplingIterator[T: ClassTag]( override def next(): T = { val r = data.next() - advance + advance() r } private val lnq = math.log1p(-f) /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ - private def advance: Unit = { + private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) val k = (math.log(u) / lnq).toInt iterDrop(k) } /** advance to first sample as part of object construction. */ - advance + advance() // Attempting to invoke this closer to the top with other object initialization // was causing it to break in strange ways, so I'm invoking it last, which seems to // work reliably. @@ -279,15 +282,18 @@ class GapSamplingReplacementIterator[T: ClassTag]( val arrayClass = Array.empty[T].iterator.getClass val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass data.getClass match { - case `arrayClass` => ((n: Int) => { data = data.drop(n) }) - case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) - case _ => ((n: Int) => { + case `arrayClass` => + (n: Int) => { data = data.drop(n) } + case `arrayBufferClass` => + (n: Int) => { data = data.drop(n) } + case _ => + (n: Int) => { var j = 0 while (j < n && data.hasNext) { data.next() j += 1 } - }) + } } } @@ -300,7 +306,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( override def next(): T = { val r = v rep -= 1 - if (rep <= 0) advance + if (rep <= 0) advance() r } @@ -309,7 +315,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is * q is the probabililty of Poisson(0; f) */ - private def advance: Unit = { + private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) val k = (math.log(u) / (-f)).toInt iterDrop(k) @@ -343,7 +349,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( } /** advance to first sample as part of object construction. */ - advance + advance() // Attempting to invoke this closer to the top with other object initialization // was causing it to break in strange ways, so I'm invoking it last, which seems to // work reliably. diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 2ae308dacf1ae..9e29bf9d61f17 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -311,7 +311,7 @@ private[random] class AcceptanceResult(var numItems: Long = 0L, var numAccepted: var acceptBound: Double = Double.NaN // upper bound for accepting item instantly var waitListBound: Double = Double.NaN // upper bound for adding item to waitlist - def areBoundsEmpty = acceptBound.isNaN || waitListBound.isNaN + def areBoundsEmpty: Boolean = acceptBound.isNaN || waitListBound.isNaN def merge(other: Option[AcceptanceResult]): Unit = { if (other.isDefined) { diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 467b890fb4bb9..c4a7b4441c85c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -83,7 +83,7 @@ private[spark] object XORShiftRandom { * @return Map of execution times for {@link java.util.Random java.util.Random} * and XORShift */ - def benchmark(numIters: Int) = { + def benchmark(numIters: Int): Map[String, Long] = { val seed = 1L val million = 1e6.toInt diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 328d59485a731..56f5dbe53fad4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -44,7 +44,13 @@ object MimaExcludes { // the maven-generated artifacts in 1.3. excludePackage("org.spark-project.jetty"), MimaBuild.excludeSparkPackage("unused"), - ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional") + ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.rdd.JdbcRDD.compute"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast") ) case v if v.startsWith("1.3") => diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 0ff521706c71a..459a5035d4984 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -137,9 +137,9 @@ - + - + From 1afcf773d0cafdfd9bf106fdc5c429ed2ba3dd36 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 24 Mar 2015 01:12:11 -0700 Subject: [PATCH 500/817] [SPARK-6452] [SQL] Checks for missing attributes and unresolved operator for all types of operator In `CheckAnalysis`, `Filter` and `Aggregate` are checked in separate case clauses, thus never hit those clauses for unresolved operators and missing input attributes. This PR also removes the `prettyString` call when generating error message for missing input attributes. Because result of `prettyString` doesn't contain expression ID, and may give confusing messages like > resolved attributes a missing from a cc rxin [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5129) Author: Cheng Lian Closes #5129 from liancheng/spark-6452 and squashes the following commits: 52cdc69 [Cheng Lian] Addresses comments 029f9bd [Cheng Lian] Checks for missing attributes and unresolved operator for all types of operator --- .../sql/catalyst/analysis/CheckAnalysis.scala | 15 ++++++++++----- .../spark/sql/catalyst/plans/QueryPlan.scala | 7 +++++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 18 ++++++++++++++++++ 3 files changed, 33 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fb975ee5e7296..425e1e41cbf21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -63,7 +63,7 @@ class CheckAnalysis { s"filter expression '${f.condition.prettyString}' " + s"of type ${f.condition.dataType.simpleString} is not a boolean.") - case aggregatePlan@Aggregate(groupingExprs, aggregateExprs, child) => + case Aggregate(groupingExprs, aggregateExprs, child) => def checkValidAggregateExpression(expr: Expression): Unit = expr match { case _: AggregateExpression => // OK case e: Attribute if !groupingExprs.contains(e) => @@ -85,13 +85,18 @@ class CheckAnalysis { cleaned.foreach(checkValidAggregateExpression) + case _ => // Fallbacks to the following checks + } + + operator match { case o if o.children.nonEmpty && o.missingInput.nonEmpty => - val missingAttributes = o.missingInput.map(_.prettyString).mkString(",") - val input = o.inputSet.map(_.prettyString).mkString(",") + val missingAttributes = o.missingInput.mkString(",") + val input = o.inputSet.mkString(",") - failAnalysis(s"resolved attributes $missingAttributes missing from $input") + failAnalysis( + s"resolved attribute(s) $missingAttributes missing from $input " + + s"in operator ${operator.simpleString}") - // Catch all case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 400a6b2825c10..48191f31198f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -47,9 +47,12 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy * Attributes that are referenced by expressions but not provided by this nodes children. * Subclasses should override this method if they produce attributes internally as it is used by * assertions designed to prevent the construction of invalid plans. + * + * Note that virtual columns should be excluded. Currently, we only support the grouping ID + * virtual column. */ - def missingInput: AttributeSet = (references -- inputSet) - .filter(_.name != VirtualColumn.groupingIdName) + def missingInput: AttributeSet = + (references -- inputSet).filter(_.name != VirtualColumn.groupingIdName) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index c1dd5aa913ddc..359aec4a7b5ab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -199,4 +199,22 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(pl(3).dataType == DecimalType.Unlimited) assert(pl(4).dataType == DoubleType) } + + test("SPARK-6452 regression test") { + // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) + val plan = + Aggregate( + Nil, + Alias(Sum(AttributeReference("a", StringType)(exprId = ExprId(1))), "b")() :: Nil, + LocalRelation( + AttributeReference("a", StringType)(exprId = ExprId(2)))) + + assert(plan.resolved) + + val message = intercept[AnalysisException] { + caseSensitiveAnalyze(plan) + }.getMessage + + assert(message.contains("resolved attribute(s) a#1 missing from a#2")) + } } From 37fac1dcd2b0f2845d2952a417fcf85d40351f57 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 24 Mar 2015 10:33:04 +0000 Subject: [PATCH 501/817] [SPARK-6477][Build]: Run MIMA tests before the Spark test suite This moves the MIMA checks to before the full Spark test suite such that, if new PR's fail the MIMA check, they will return much faster having not run the entire test suite. This is preferable to the current scenario where a user would have to wait until the entire test suite completes before realizing it failed on a MIMA check in which case, once the MIMA issues are fixed, the user would have to resubmit and rerun the full test suite again. Author: Brennon York Closes #5145 from brennonyork/SPARK-6477 and squashes the following commits: 12b0aee [Brennon York] updated to put the mima checks before the spark test suite --- dev/run-tests | 18 +++++++++--------- dev/run-tests-codes.sh | 6 +++--- dev/run-tests-jenkins | 4 ++-- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index d6935a61c6d29..561d7fc9e7b1f 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -178,6 +178,15 @@ CURRENT_BLOCK=$BLOCK_BUILD fi } +echo "" +echo "=========================================================================" +echo "Detecting binary incompatibilities with MiMa" +echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_MIMA + +./dev/mima + echo "" echo "=========================================================================" echo "Running Spark unit tests" @@ -227,12 +236,3 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS ./python/run-tests - -echo "" -echo "=========================================================================" -echo "Detecting binary incompatibilities with MiMa" -echo "=========================================================================" - -CURRENT_BLOCK=$BLOCK_MIMA - -./dev/mima diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh index 1348e0609dda4..8ab6db6925d6e 100644 --- a/dev/run-tests-codes.sh +++ b/dev/run-tests-codes.sh @@ -22,6 +22,6 @@ readonly BLOCK_RAT=11 readonly BLOCK_SCALA_STYLE=12 readonly BLOCK_PYTHON_STYLE=13 readonly BLOCK_BUILD=14 -readonly BLOCK_SPARK_UNIT_TESTS=15 -readonly BLOCK_PYSPARK_UNIT_TESTS=16 -readonly BLOCK_MIMA=17 +readonly BLOCK_MIMA=15 +readonly BLOCK_SPARK_UNIT_TESTS=16 +readonly BLOCK_PYSPARK_UNIT_TESTS=17 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 5f4000e83925c..3a937b637e003 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -199,12 +199,12 @@ done failing_test="Python style tests" elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then failing_test="to build" + elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then + failing_test="MiMa tests" elif [ "$test_result" -eq "$BLOCK_SPARK_UNIT_TESTS" ]; then failing_test="Spark unit tests" elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then failing_test="PySpark unit tests" - elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then - failing_test="MiMa tests" else failing_test="some tests" fi From c12312f8b16bb8f9355d5f9e786c5a608863eb01 Mon Sep 17 00:00:00 2001 From: Cong Yue Date: Tue, 24 Mar 2015 12:56:13 +0000 Subject: [PATCH 502/817] Update the command to use IPython notebook As for "notebook --pylab inline" is not supported any more, update the related documentation for this. Author: Cong Yue Closes #5111 from yuecong/patch-1 and squashes the following commits: 872df76 [Cong Yue] Update the command to use IPython notebook --- docs/programming-guide.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 5fe832b6fa100..f5b775da7930a 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -237,9 +237,13 @@ You can customize the `ipython` command by setting `PYSPARK_DRIVER_PYTHON_OPTS`. the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: {% highlight bash %} -$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark +$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook" ./bin/pyspark {% endhighlight %} +After the IPython Notebook server is launched, you can create a new "Python 2" notebook from +the "Files" tab. Inside the notebook, you can input the command `%pylab inline` as part of +your notebook before you start to try Spark from the IPython notebook. +
    From b293afc42c370da0ff308c16ba4af81289ea6f89 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 24 Mar 2015 13:48:33 +0000 Subject: [PATCH 503/817] [SPARK-6473] [core] Do not try to figure out Scala version if not needed... .... Author: Marcelo Vanzin Closes #5143 from vanzin/SPARK-6473 and squashes the following commits: a2e5e2d [Marcelo Vanzin] [SPARK-6473] [core] Do not try to figure out Scala version if not needed. --- .../org/apache/spark/launcher/AbstractCommandBuilder.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index dc90e9e987234..2da5f7278729e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -147,7 +147,6 @@ void addOptionString(List cmd, String options) { */ List buildClassPath(String appClassPath) throws IOException { String sparkHome = getSparkHome(); - String scala = getScalaVersion(); List cp = new ArrayList(); addToClassPath(cp, getenv("SPARK_CLASSPATH")); @@ -158,6 +157,7 @@ List buildClassPath(String appClassPath) throws IOException { boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); boolean isTesting = "1".equals(getenv("SPARK_TESTING")); if (prependClasses || isTesting) { + String scala = getScalaVersion(); List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", "yarn", "launcher"); @@ -182,7 +182,7 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - String assembly = findAssembly(scala); + String assembly = findAssembly(); addToClassPath(cp, assembly); // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus @@ -330,7 +330,7 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } - private String findAssembly(String scalaVersion) { + private String findAssembly() { String sparkHome = getSparkHome(); File libdir; if (new File(sparkHome, "RELEASE").isFile()) { @@ -338,7 +338,7 @@ private String findAssembly(String scalaVersion) { checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", libdir.getAbsolutePath()); } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); } final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); From 85cf0636825d1997d64d0bdc04618f29b7222da1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 24 Mar 2015 16:13:25 +0000 Subject: [PATCH 504/817] [SPARK-5559] [Streaming] [Test] Remove oppotunity we met flakiness when running FlumeStreamSuite When we run FlumeStreamSuite on Jenkins, sometimes we get error like as follows. sbt.ForkMain$ForkError: The code passed to eventually never returned normally. Attempted 52 times over 10.094849836 seconds. Last failure message: Error connecting to localhost/127.0.0.1:23456. at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307) at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478) at org.apache.spark.streaming.flume.FlumeStreamSuite.writeAndVerify(FlumeStreamSuite.scala:116) at org.apache.spark.streaming.flume.FlumeStreamSuite.org$apache$spark$streaming$flume$FlumeStreamSuite$$testFlumeStream(FlumeStreamSuite.scala:74) at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply$mcV$sp(FlumeStreamSuite.scala:66) at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply(FlumeStreamSuite.scala:66) at org.apache.spark.streaming.flume.FlumeStreamSuite$$anonfun$3.apply(FlumeStreamSuite.scala:66) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.scalatest.Suite$class.withFixture(Suite.scala:1122) at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) This error is caused by check-then-act logic when it find free-port . /** Find a free port */ private def findFreePort(): Int = { Utils.startServiceOnPort(23456, (trialPort: Int) => { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) }, conf)._2 } Removing the check-then-act is not easy but we can reduce the chance of having the error by choosing random value for initial port instead of 23456. Author: Kousuke Saruta Closes #4337 from sarutak/SPARK-5559 and squashes the following commits: 16f109f [Kousuke Saruta] Added `require` to Utils#startServiceOnPort c39d8b6 [Kousuke Saruta] Merge branch 'SPARK-5559' of github.com:sarutak/spark into SPARK-5559 1610ba2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559 33357e3 [Kousuke Saruta] Changed "findFreePort" method in MQTTStreamSuite and FlumeStreamSuite so that it can choose valid random port a9029fe [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559 9489ef9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5559 8212e42 [Kousuke Saruta] Modified default port used in FlumeStreamSuite from 23456 to random value --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++++ .../org/apache/spark/streaming/flume/FlumeStreamSuite.scala | 5 +++-- .../org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala | 4 +++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d9a671687aad0..0b5a914e7dbbf 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1876,6 +1876,10 @@ private[spark] object Utils extends Logging { startService: Int => (T, Int), conf: SparkConf, serviceName: String = ""): (T, Int) = { + + require(startPort == 0 || (1024 <= startPort && startPort < 65536), + "startPort should be between 1024 and 65535 (inclusive), or 0 for a random free port.") + val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" val maxRetries = portMaxRetries(conf) for (offset <- 0 to maxRetries) { diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 322de7bf2fed8..51d273af8da84 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -28,6 +28,7 @@ import scala.language.postfixOps import com.google.common.base.Charsets import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.commons.lang3.RandomUtils import org.apache.flume.source.avro import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.jboss.netty.channel.ChannelPipeline @@ -40,7 +41,6 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} -import org.apache.spark.streaming.scheduler.{StreamingListener, StreamingListenerReceiverStarted} import org.apache.spark.util.Utils class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { @@ -76,7 +76,8 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L /** Find a free port */ private def findFreePort(): Int = { - Utils.startServiceOnPort(23456, (trialPort: Int) => { + val candidatePort = RandomUtils.nextInt(1024, 65536) + Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 0f3298af6234a..24d78ecb3a97d 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import org.apache.activemq.broker.{TransportConnector, BrokerService} +import org.apache.commons.lang3.RandomUtils import org.eclipse.paho.client.mqttv3._ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence @@ -113,7 +114,8 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { } private def findFreePort(): Int = { - Utils.startServiceOnPort(23456, (trialPort: Int) => { + val candidatePort = RandomUtils.nextInt(1024, 65536) + Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) From 08d452801195cc6cf0697a594e98cd4778f358ee Mon Sep 17 00:00:00 2001 From: Peter Rudenko Date: Tue, 24 Mar 2015 16:33:38 +0000 Subject: [PATCH 505/817] [ML][docs][minor] Define LabeledDocument/Document classes in CV example To easier copy/paste Cross-Validation example code snippet need to define LabeledDocument/Document in it, since they difined in a previous example. Author: Peter Rudenko Closes #5135 from petro-rudenko/patch-3 and squashes the following commits: 5190c75 [Peter Rudenko] Fix primitive types for java examples. 1d35383 [Peter Rudenko] [SQL][docs][minor] Define LabeledDocument/Document classes in CV example --- docs/ml-guide.md | 51 ++++++++++++++++++++++++++++++++++++++++-------- 1 file changed, 43 insertions(+), 8 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index da6aef7f14c4c..c08c76d226713 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -408,31 +408,31 @@ import org.apache.spark.sql.SQLContext; // Labeled and unlabeled instance types. // Spark SQL can infer schema from Java Beans. public class Document implements Serializable { - private Long id; + private long id; private String text; - public Document(Long id, String text) { + public Document(long id, String text) { this.id = id; this.text = text; } - public Long getId() { return this.id; } - public void setId(Long id) { this.id = id; } + public long getId() { return this.id; } + public void setId(long id) { this.id = id; } public String getText() { return this.text; } public void setText(String text) { this.text = text; } } public class LabeledDocument extends Document implements Serializable { - private Double label; + private double label; - public LabeledDocument(Long id, String text, Double label) { + public LabeledDocument(long id, String text, double label) { super(id, text); this.label = label; } - public Double getLabel() { return this.label; } - public void setLabel(Double label) { this.label = label; } + public double getLabel() { return this.label; } + public void setLabel(double label) { this.label = label; } } // Set up contexts. @@ -565,6 +565,11 @@ import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from case classes. +case class LabeledDocument(id: Long, text: String, label: Double) +case class Document(id: Long, text: String) + val conf = new SparkConf().setAppName("CrossValidatorExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) @@ -655,6 +660,36 @@ import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from Java Beans. +public class Document implements Serializable { + private long id; + private String text; + + public Document(long id, String text) { + this.id = id; + this.text = text; + } + + public long getId() { return this.id; } + public void setId(long id) { this.id = id; } + + public String getText() { return this.text; } + public void setText(String text) { this.text = text; } +} + +public class LabeledDocument extends Document implements Serializable { + private double label; + + public LabeledDocument(long id, String text, double label) { + super(id, text); + this.label = label; + } + + public double getLabel() { return this.label; } + public void setLabel(double label) { this.label = label; } +} + SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); JavaSparkContext jsc = new JavaSparkContext(conf); SQLContext jsql = new SQLContext(jsc); From a1d1529daebee30b76b954d16a30849407f795d1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Mar 2015 10:11:27 -0700 Subject: [PATCH 506/817] [SPARK-6475][SQL] recognize array types when infer data types from JavaBeans Right now if there is a array field in a JavaBean, the user wold see an exception in `createDataFrame`. liancheng Author: Xiangrui Meng Closes #5146 from mengxr/SPARK-6475 and squashes the following commits: 51e87e5 [Xiangrui Meng] validate schemas 4f2df5e [Xiangrui Meng] recognize array types when infer data types from JavaBeans --- .../org/apache/spark/sql/SQLContext.scala | 80 ++++++++++++------- .../apache/spark/sql/JavaDataFrameSuite.java | 41 +++++++++- 2 files changed, 89 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index dc9912b52dcab..e59cf9b9e037b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1210,38 +1210,56 @@ class SQLContext(@transient val sparkContext: SparkContext) * Returns a Catalyst Schema for the given java bean class. */ protected def getSchema(beanClass: Class[_]): Seq[AttributeReference] = { + val (dataType, _) = inferDataType(beanClass) + dataType.asInstanceOf[StructType].fields.map { f => + AttributeReference(f.name, f.dataType, f.nullable)() + } + } + + /** + * Infers the corresponding SQL data type of a Java class. + * @param clazz Java class + * @return (SQL data type, nullable) + */ + private def inferDataType(clazz: Class[_]): (DataType, Boolean) = { // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. - val beanInfo = Introspector.getBeanInfo(beanClass) - - // Note: The ordering of elements may differ from when the schema is inferred in Scala. - // This is because beanInfo.getPropertyDescriptors gives no guarantees about - // element ordering. - val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") - fields.map { property => - val (dataType, nullable) = property.getPropertyType match { - case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) - case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) - case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) - case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) - case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) - } - AttributeReference(property.getName, dataType, nullable)() + clazz match { + case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) + + case c: Class[_] if c.isArray => + val (dataType, nullable) = inferDataType(c.getComponentType) + (ArrayType(dataType, nullable), true) + + case _ => + val beanInfo = Introspector.getBeanInfo(clazz) + val properties = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + val fields = properties.map { property => + val (dataType, nullable) = inferDataType(property.getPropertyType) + new StructField(property.getName, dataType, nullable) + } + (new StructType(fields), true) } } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 2d586f784ac5a..1ff2d5a190521 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -17,29 +17,39 @@ package test.org.apache.spark.sql; +import java.io.Serializable; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; +import org.apache.spark.sql.test.TestSQLContext; import org.apache.spark.sql.test.TestSQLContext$; -import static org.apache.spark.sql.functions.*; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.functions.*; public class JavaDataFrameSuite { + private transient JavaSparkContext jsc; private transient SQLContext context; @Before public void setUp() { // Trigger static initializer of TestData TestData$.MODULE$.testData(); + jsc = new JavaSparkContext(TestSQLContext.sparkContext()); context = TestSQLContext$.MODULE$; } @After public void tearDown() { + jsc = null; context = null; } @@ -90,4 +100,33 @@ public void testShow() { df.show(); df.show(1000); } + + public static class Bean implements Serializable { + private double a = 0.0; + private Integer[] b = new Integer[]{0, 1}; + + public double getA() { + return a; + } + + public Integer[] getB() { + return b; + } + } + + @Test + public void testCreateDataFrameFromJavaBeans() { + Bean bean = new Bean(); + JavaRDD rdd = jsc.parallelize(Arrays.asList(bean)); + DataFrame df = context.createDataFrame(rdd, Bean.class); + StructType schema = df.schema(); + Assert.assertEquals(new StructField("a", DoubleType$.MODULE$, false, Metadata.empty()), + schema.apply("a")); + Assert.assertEquals( + new StructField("b", new ArrayType(IntegerType$.MODULE$, true), true, Metadata.empty()), + schema.apply("b")); + Row first = df.select("a", "b").first(); + Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); + Assert.assertArrayEquals(bean.getB(), first.getAs(1)); + } } From 6bdddb6f6ffbd1bee4c45880904e9561b18764a7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Mar 2015 12:08:19 -0700 Subject: [PATCH 507/817] [SPARK-6361][SQL] support adding a column with metadata in DF This is used by ML pipelines to embed ML attributes in columns created by ML transformers/estimators. marmbrus Author: Xiangrui Meng Closes #5151 from mengxr/SPARK-6361 and squashes the following commits: bb30de3 [Xiangrui Meng] support adding a column with metadata in DF --- .../expressions/namedExpressions.scala | 20 ++++++++++++------- .../scala/org/apache/spark/sql/Column.scala | 13 ++++++++++++ .../spark/sql/ColumnExpressionSuite.scala | 15 +++++++++++--- 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 17f7f9fe51376..3dd7d38847b44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -95,9 +95,12 @@ abstract class Attribute extends NamedExpression { * @param name the name to be associated with the result of computing [[child]]. * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this * alias. Auto-assigned if left blank. + * @param explicitMetadata Explicit metadata associated with this alias that overwrites child's. */ -case class Alias(child: Expression, name: String) - (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) +case class Alias(child: Expression, name: String)( + val exprId: ExprId = NamedExpression.newExprId, + val qualifiers: Seq[String] = Nil, + val explicitMetadata: Option[Metadata] = None) extends NamedExpression with trees.UnaryNode[Expression] { override type EvaluatedType = Any @@ -107,9 +110,11 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable override def metadata: Metadata = { - child match { - case named: NamedExpression => named.metadata - case _ => Metadata.empty + explicitMetadata.getOrElse { + child match { + case named: NamedExpression => named.metadata + case _ => Metadata.empty + } } } @@ -123,11 +128,12 @@ case class Alias(child: Expression, name: String) override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" - override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + override protected final def otherCopyArgs = exprId :: qualifiers :: explicitMetadata :: Nil override def equals(other: Any): Boolean = other match { case a: Alias => - name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers + name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers && + explicitMetadata == a.explicitMetadata case _ => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index ec7d15f5bc4e7..2ae47e07d45ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -594,6 +594,19 @@ class Column(protected[sql] val expr: Expression) { */ def as(alias: Symbol): Column = Alias(expr, alias.name)() + /** + * Gives the column an alias with metadata. + * {{{ + * val metadata: Metadata = ... + * df.select($"colA".as("colB", metadata)) + * }}} + * + * @group expr_ops + */ + def as(alias: String, metadata: Metadata): Column = { + Alias(expr, alias)(explicitMetadata = Some(metadata)) + } + /** * Casts the column to a different data type. * {{{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a53ae97d6243a..bc8fae100db6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.NamedExpression -import org.apache.spark.sql.catalyst.plans.logical.{Project, NoRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{BooleanType, IntegerType, StructField, StructType} +import org.apache.spark.sql.types._ class ColumnExpressionSuite extends QueryTest { @@ -322,4 +320,15 @@ class ColumnExpressionSuite extends QueryTest { assert('key.desc == 'key.desc) assert('key.desc != 'key.asc) } + + test("alias with metadata") { + val metadata = new MetadataBuilder() + .putString("originName", "value") + .build() + val schema = testData + .select($"*", col("value").as("abc", metadata)) + .schema + assert(schema("value").metadata === Metadata.empty) + assert(schema("abc").metadata === metadata) + } } From 32efadd0500f10bddf2ae8456c9e719ec52940f1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 12:09:02 -0700 Subject: [PATCH 508/817] [SPARK-6459][SQL] Warn when constructing trivially true equals predicate For example, one might expect the following code to work, but it does not. Now you will at least get a warning with a suggestion to use aliases. ```scala val df = sqlContext.load(path, "parquet") val txns = df.groupBy("cust_id").agg($"cust_id", countDistinct($"day_num").as("txns")) val spend = df.groupBy("cust_id").agg($"cust_id", sum($"extended_price").as("spend")) val rmJoin = txns.join(spend, txns("cust_id") === spend("cust_id"), "inner") ``` Author: Michael Armbrust Closes #5163 from marmbrus/selfJoinError and squashes the following commits: 16c1f0b [Michael Armbrust] fix visibility 1b57e8d [Michael Armbrust] Warn when constructing trivially true equals predicate --- .../main/scala/org/apache/spark/sql/Column.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 2ae47e07d45ec..3cd7adf8cab5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.language.implicitConversions import org.apache.spark.annotation.Experimental +import org.apache.spark.Logging import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar, UnresolvedGetField} @@ -46,7 +47,7 @@ private[sql] object Column { * @groupname Ungrouped Support functions for DataFrames. */ @Experimental -class Column(protected[sql] val expr: Expression) { +class Column(protected[sql] val expr: Expression) extends Logging { def this(name: String) = this(name match { case "*" => UnresolvedStar(None) @@ -109,7 +110,15 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def === (other: Any): Column = EqualTo(expr, lit(other).expr) + def === (other: Any): Column = { + val right = lit(other).expr + if (this.expr == right) { + logWarning( + s"Constructing trivially true equals predicate, '${this.expr} = $right'. " + + "Perhaps you need to use aliases.") + } + EqualTo(expr, right) + } /** * Equality test. From 26c6ce3d2947df5a294b1ad4a22fae5d31d06c19 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 12:10:30 -0700 Subject: [PATCH 509/817] [SPARK-6437][SQL] Use completion iterator to close external sorter Otherwise we will leak files when spilling occurs. Author: Michael Armbrust Closes #5161 from marmbrus/cleanupAfterSort and squashes the following commits: cb13d3c [Michael Armbrust] hint to inferencer cdebdf5 [Michael Armbrust] Use completion iterator to close external sorter --- .../org/apache/spark/sql/execution/basicOperators.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 20c9bc3e75542..1f5251a20376f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.util.MutablePair +import org.apache.spark.util.{CompletionIterator, MutablePair} import org.apache.spark.util.collection.ExternalSorter /** @@ -194,7 +194,9 @@ case class ExternalSort( val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) sorter.insertAll(iterator.map(r => (r, null))) - sorter.iterator.map(_._1) + val baseIterator = sorter.iterator.map(_._1) + // TODO(marmbrus): The complex type signature below thwarts inference for no reason. + CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop()) }, preservesPartitioning = true) } From 3fa3d121dfec60f9768d3859e8450ee482b2d4e8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 12:28:01 -0700 Subject: [PATCH 510/817] [SPARK-6054][SQL] Fix transformations of TreeNodes that hold StructTypes Due to a recent change that made `StructType` a `Seq` we started inadvertently turning `StructType`s into generic `Traversable` when attempting nested tree transformations. In this PR we explicitly avoid descending into `DataType`s to avoid this bug. Author: Michael Armbrust Closes #5157 from marmbrus/udfFix and squashes the following commits: 26f7087 [Michael Armbrust] Fix transformations of TreeNodes that hold StructTypes --- .../spark/sql/catalyst/plans/QueryPlan.scala | 2 ++ .../spark/sql/catalyst/trees/TreeNode.scala | 20 ++++++++++++++++--- .../scala/org/apache/spark/sql/UDFSuite.scala | 6 ++++++ 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 48191f31198f3..bd9291e9ba5d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -85,6 +85,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case Some(e: Expression) => Some(transformExpressionDown(e)) case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionDown(e) case other => other @@ -117,6 +118,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case Some(e: Expression) => Some(transformExpressionUp(e)) case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionUp(e) case other => other diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index f84ffe4e176cc..0ae9f6b2965d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.types.DataType /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) @@ -220,6 +221,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { Some(arg) } case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case args: Traversable[_] => args.map { case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) @@ -276,6 +278,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { Some(arg) } case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case args: Traversable[_] => args.map { case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) @@ -307,10 +310,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param newArgs the new product arguments. */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { + val defaultCtor = + getClass.getConstructors + .find(_.getParameterTypes.size != 0) + .headOption + .getOrElse(sys.error(s"No valid constructor for $nodeName")) + try { CurrentOrigin.withOrigin(origin) { // Skip no-arg constructors that are just there for kryo. - val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head if (otherCopyArgs.isEmpty) { defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] } else { @@ -320,8 +328,14 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } catch { case e: java.lang.IllegalArgumentException => throw new TreeNodeException( - this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName? " - + s"Exception message: ${e.getMessage}.") + this, + s""" + |Failed to copy node. + |Is otherCopyArgs specified correctly for $nodeName. + |Exception message: ${e.getMessage} + |ctor: $defaultCtor? + |args: ${newArgs.mkString(", ")} + """.stripMargin) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index be105c6e83594..d615542ab50a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -50,4 +50,10 @@ class UDFSuite extends QueryTest { .select($"ret.f1").head().getString(0) assert(result === "test") } + + test("udf that is transformed") { + udf.register("makeStruct", (x: Int, y: Int) => (x, y)) + // 1 + 1 is constant folded causing a transformation. + assert(sql("SELECT makeStruct(1 + 1, 2)").first().getAs[Row](0) === Row(2, 2)) + } } From 046c1e2aa459147bf592371bb9fb7a65edb182e7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 13:22:46 -0700 Subject: [PATCH 511/817] [SPARK-6375][SQL] Fix formatting of error messages. Author: Michael Armbrust Closes #5155 from marmbrus/errorMessages and squashes the following commits: b898188 [Michael Armbrust] Fix formatting of error messages. --- .../apache/spark/sql/AnalysisException.scala | 6 +++++ .../sql/catalyst/analysis/Analyzer.scala | 5 ++-- .../spark/sql/catalyst/analysis/package.scala | 8 ++++++ .../sql/catalyst/analysis/unresolved.scala | 4 +++ .../expressions/namedExpressions.scala | 7 ++++++ .../catalyst/plans/logical/LogicalPlan.scala | 3 ++- .../spark/sql/hive/ErrorPositionSuite.scala | 25 +++++++++++++++++-- 7 files changed, 53 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 15add84878ecf..34fedead44db3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -30,6 +30,12 @@ class AnalysisException protected[sql] ( val startPosition: Option[Int] = None) extends Exception with Serializable { + def withPosition(line: Option[Int], startPosition: Option[Int]) = { + val newException = new AnalysisException(message, line, startPosition) + newException.setStackTrace(getStackTrace) + newException + } + override def getMessage: String = { val lineAnnotation = line.map(l => s" line $l").getOrElse("") val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 92d3db077c5e1..c93af79795bc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -175,7 +175,7 @@ class Analyzer(catalog: Catalog, catalog.lookupRelation(u.tableIdentifier, u.alias) } catch { case _: NoSuchTableException => - u.failAnalysis(s"no such table ${u.tableIdentifier}") + u.failAnalysis(s"no such table ${u.tableName}") } } @@ -275,7 +275,8 @@ class Analyzer(catalog: Catalog, q.asInstanceOf[GroupingAnalytics].gid case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolveChildren(name, resolver).getOrElse(u) + val result = + withPosition(u) { q.resolveChildren(name, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index e95f19e69ed43..a7d3a8ee7deb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -42,4 +42,12 @@ package object analysis { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition) } } + + /** Catches any AnalysisExceptions thrown by `f` and attaches `t`'s position if any. */ + def withPosition[A](t: TreeNode[_])(f: => A) = { + try f catch { + case a: AnalysisException => + throw a.withPosition(t.origin.line, t.origin.startPosition) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a7cd4124e56f3..ad5172c0349eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -36,6 +36,10 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str case class UnresolvedRelation( tableIdentifier: Seq[String], alias: Option[String] = None) extends LeafNode { + + /** Returns a `.` separated name for this relation. */ + def tableName = tableIdentifier.mkString(".") + override def output = Nil override lazy val resolved = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 3dd7d38847b44..08361d043b6ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -41,6 +41,13 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId + /** + * Returns a dot separated fully qualified name for this attribute. Given that there can be + * multiple qualifiers, it is possible that there are other possible way to refer to this + * attribute. + */ + def qualifiedName: String = (qualifiers.headOption.toSeq :+ name).mkString(".") + /** * All possible qualifiers for the expression. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 8c4f09b58a4f2..0f8b144ccc113 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -208,8 +208,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // More than one match. case ambiguousReferences => + val referenceNames = ambiguousReferences.map(_._1.qualifiedName).mkString(", ") throw new AnalysisException( - s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") + s"Reference '$name' is ambiguous, could be: $referenceNames.") } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index f04437c595bf6..968557c9c4686 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -19,12 +19,29 @@ package org.apache.spark.sql.hive import java.io.{OutputStream, PrintStream} +import scala.util.Try + +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.{AnalysisException, QueryTest} -import scala.util.Try -class ErrorPositionSuite extends QueryTest { +class ErrorPositionSuite extends QueryTest with BeforeAndAfter { + + before { + Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes") + } + + positionTest("ambiguous attribute reference 1", + "SELECT a from dupAttributes", "a") + + positionTest("ambiguous attribute reference 2", + "SELECT a, b from dupAttributes", "a") + + positionTest("ambiguous attribute reference 3", + "SELECT b, a from dupAttributes", "a") positionTest("unresolved attribute 1", "SELECT x FROM src", "x") @@ -127,6 +144,10 @@ class ErrorPositionSuite extends QueryTest { val error = intercept[AnalysisException] { quietly(sql(query)) } + + assert(!error.getMessage.contains("Seq(")) + assert(!error.getMessage.contains("List(")) + val (line, expectedLineNum) = query.split("\n").zipWithIndex.collect { case (l, i) if l.contains(token) => (l, i + 1) }.headOption.getOrElse(sys.error(s"Invalid test. Token $token not in $query")) From cbeaf9ebab31a0bcbca884d4db7a791fd9edbff3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 14:08:20 -0700 Subject: [PATCH 512/817] [SPARK-6376][SQL] Avoid eliminating subqueries until optimization Previously it was okay to throw away subqueries after analysis, as we would never try to use that tree for resolution again. However, with eager analysis in `DataFrame`s this can cause errors for queries such as: ```scala val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str") df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count() ``` As a result, in this PR we defer the elimination of subqueries until the optimization phase. Author: Michael Armbrust Closes #5160 from marmbrus/subqueriesInDfs and squashes the following commits: a9bb262 [Michael Armbrust] Update Optimizer.scala 27d25bf [Michael Armbrust] fix hive tests 9137e03 [Michael Armbrust] add type 81cd597 [Michael Armbrust] Avoid eliminating subqueries until optimization --- .../spark/sql/catalyst/analysis/Analyzer.scala | 4 +--- .../apache/spark/sql/catalyst/dsl/package.scala | 4 ++-- .../spark/sql/catalyst/optimizer/Optimizer.scala | 4 ++++ .../sql/catalyst/plans/logical/LogicalPlan.scala | 16 ++++++++++------ .../sql/catalyst/analysis/AnalysisSuite.scala | 8 ++++++-- .../org/apache/spark/sql/DataFrameSuite.scala | 7 +++++++ .../scala/org/apache/spark/sql/JoinSuite.scala | 4 ++-- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 2 +- 9 files changed, 34 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c93af79795bc7..13d2ae4c6fcb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -64,9 +64,7 @@ class Analyzer(catalog: Catalog, UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: typeCoercionRules ++ - extendedResolutionRules : _*), - Batch("Remove SubQueries", fixedPoint, - EliminateSubQueries) + extendedResolutionRules : _*) ) /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 51a09ac0e1249..7f5f617812919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} @@ -289,7 +289,7 @@ package object dsl { InsertIntoTable( analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) - def analyze = analysis.SimpleAnalyzer(logicalPlan) + def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer(logicalPlan)) } object plans { // scalastyle:ignore diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1a75fcf3545bd..74edaacc4f609 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.immutable.HashSet +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.FullOuter @@ -32,6 +33,9 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] object DefaultOptimizer extends Optimizer { val batches = + // SubQueries are only needed for analysis and can be removed before execution. + Batch("Remove SubQueries", FixedPoint(100), + EliminateSubQueries) :: Batch("Combine Limits", FixedPoint(100), CombineLimits) :: Batch("ConstantFolding", FixedPoint(100), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 0f8b144ccc113..b01a61d7bf8d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, Resolver} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, Resolver} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode @@ -73,12 +73,16 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * can do better should override this function. */ def sameResult(plan: LogicalPlan): Boolean = { - plan.getClass == this.getClass && - plan.children.size == children.size && { - logDebug(s"[${cleanArgs.mkString(", ")}] == [${plan.cleanArgs.mkString(", ")}]") - cleanArgs == plan.cleanArgs + val cleanLeft = EliminateSubQueries(this) + val cleanRight = EliminateSubQueries(plan) + + cleanLeft.getClass == cleanRight.getClass && + cleanLeft.children.size == cleanRight.children.size && { + logDebug( + s"[${cleanRight.cleanArgs.mkString(", ")}] == [${cleanLeft.cleanArgs.mkString(", ")}]") + cleanRight.cleanArgs == cleanLeft.cleanArgs } && - (plan.children, children).zipped.forall(_ sameResult _) + (cleanLeft.children, cleanRight.children).zipped.forall(_ sameResult _) } /** Args that have cleaned such that differences in expression id should not affect equality */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 359aec4a7b5ab..756cd36f05c8c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -32,9 +32,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseInsensitiveCatalog = new SimpleCatalog(false) val caseSensitiveAnalyzer = - new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } val caseInsensitiveAnalyzer = - new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } val checkAnalysis = new CheckAnalysis diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ff441ef26f9c0..c30ed694a62f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -108,6 +108,13 @@ class DataFrameSuite extends QueryTest { ) } + test("self join with aliases") { + val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str") + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + test("explode") { val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters") val df2 = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index dd0948ad824be..e4dee87849fd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -34,7 +34,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") - val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.analyzed + val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.optimizedPlan val planned = planner.HashJoin(join) assert(planned.size === 1) } @@ -109,7 +109,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("multiple-key equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") - val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.analyzed + val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.optimizedPlan val planned = planner.HashJoin(join) assert(planned.size === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index ff2e6ea9ea51d..e5ad0bf552073 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -579,7 +579,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row(3) :: Row(4) :: Nil ) - table("test_parquet_ctas").queryExecution.analyzed match { + table("test_parquet_ctas").queryExecution.optimizedPlan match { case LogicalRelation(p: ParquetRelation2) => // OK case _ => fail( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index d891c4e8903d9..8a31bd03092d1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -292,7 +292,7 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { Seq(Row(1, "str1")) ) - table("test_parquet_ctas").queryExecution.analyzed match { + table("test_parquet_ctas").queryExecution.optimizedPlan match { case LogicalRelation(p: ParquetRelation2) => // OK case _ => fail( From a8f51b82968147abebbe61b8b68b066d21a0c6e6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Mar 2015 14:10:56 -0700 Subject: [PATCH 513/817] [SPARK-6458][SQL] Better error messages for invalid data sources Avoid unclear match errors and use `AnalysisException`. Author: Michael Armbrust Closes #5158 from marmbrus/dataSourceError and squashes the following commits: af9f82a [Michael Armbrust] Yins comment 90c6ba4 [Michael Armbrust] Better error messages for invalid data sources --- .../scala/org/apache/spark/sql/sources/ddl.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index d2e807d3a69b6..eb46b46ca5bf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -21,7 +21,7 @@ import scala.language.existentials import scala.language.implicitConversions import org.apache.spark.Logging -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} +import org.apache.spark.sql.{AnalysisException, SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -204,19 +204,25 @@ private[sql] object ResolvedDataSource { provider: String, options: Map[String, String]): ResolvedDataSource = { val clazz: Class[_] = lookupDataSource(provider) + def className = clazz.getCanonicalName val relation = userSpecifiedSchema match { case Some(schema: StructType) => clazz.newInstance() match { case dataSource: SchemaRelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) case dataSource: org.apache.spark.sql.sources.RelationProvider => - sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") + throw new AnalysisException(s"$className does not allow user-specified schemas.") + case _ => + throw new AnalysisException(s"$className is not a RelationProvider.") } case None => clazz.newInstance() match { case dataSource: RelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => - sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") + throw new AnalysisException( + s"A schema needs to be specified when using $className.") + case _ => + throw new AnalysisException(s"$className is not a RelationProvider.") } } new ResolvedDataSource(clazz, relation) From 7215aa745590a3eec9c1ff35d28194235a550db7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 24 Mar 2015 14:38:20 -0700 Subject: [PATCH 514/817] [SPARK-6209] Clean up connections in ExecutorClassLoader after failing to load classes (master branch PR) ExecutorClassLoader does not ensure proper cleanup of network connections that it opens. If it fails to load a class, it may leak partially-consumed InputStreams that are connected to the REPL's HTTP class server, causing that server to exhaust its thread pool, which can cause the entire job to hang. See [SPARK-6209](https://issues.apache.org/jira/browse/SPARK-6209) for more details, including a bug reproduction. This patch fixes this issue by ensuring proper cleanup of these resources. It also adds logging for unexpected error cases. This PR is an extended version of #4935 and adds a regression test. Author: Josh Rosen Closes #4944 from JoshRosen/executorclassloader-leak-master-branch and squashes the following commits: e0e3c25 [Josh Rosen] Wrap try block around getReponseCode; re-enable keep-alive by closing error stream 961c284 [Josh Rosen] Roll back changes that were added to get the regression test to fail 7ee2261 [Josh Rosen] Add a failing regression test e2d70a3 [Josh Rosen] Properly clean up after errors in ExecutorClassLoader --- repl/pom.xml | 5 ++ .../spark/repl/ExecutorClassLoader.scala | 85 +++++++++++++++---- .../spark/repl/ExecutorClassLoaderSuite.scala | 70 ++++++++++++++- 3 files changed, 140 insertions(+), 20 deletions(-) diff --git a/repl/pom.xml b/repl/pom.xml index edfa1c7f2c29c..03053b4c3b287 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -84,6 +84,11 @@ scalacheck_${scala.binary.version} test + + org.mockito + mockito-all + test + diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 9805609120005..004941d5f50ae 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -17,9 +17,10 @@ package org.apache.spark.repl -import java.io.{ByteArrayOutputStream, InputStream, FileNotFoundException} -import java.net.{URI, URL, URLEncoder} -import java.util.concurrent.{Executors, ExecutorService} +import java.io.{IOException, ByteArrayOutputStream, InputStream} +import java.net.{HttpURLConnection, URI, URL, URLEncoder} + +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} @@ -43,6 +44,9 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader val parentLoader = new ParentClassLoader(parent) + // Allows HTTP connect and read timeouts to be controlled for testing / debugging purposes + private[repl] var httpUrlConnectionTimeoutMillis: Int = -1 + // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { if (Set("http", "https", "ftp").contains(uri.getScheme)) { @@ -71,30 +75,66 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader } } + private def getClassFileInputStreamFromHttpServer(pathInDirectory: String): InputStream = { + val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { + val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) + val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) + newuri.toURL + } else { + new URL(classUri + "/" + urlEncode(pathInDirectory)) + } + val connection: HttpURLConnection = Utils.setupSecureURLConnection(url.openConnection(), + SparkEnv.get.securityManager).asInstanceOf[HttpURLConnection] + // Set the connection timeouts (for testing purposes) + if (httpUrlConnectionTimeoutMillis != -1) { + connection.setConnectTimeout(httpUrlConnectionTimeoutMillis) + connection.setReadTimeout(httpUrlConnectionTimeoutMillis) + } + connection.connect() + try { + if (connection.getResponseCode != 200) { + // Close the error stream so that the connection is eligible for re-use + try { + connection.getErrorStream.close() + } catch { + case ioe: IOException => + logError("Exception while closing error stream", ioe) + } + throw new ClassNotFoundException(s"Class file not found at URL $url") + } else { + connection.getInputStream + } + } catch { + case NonFatal(e) if !e.isInstanceOf[ClassNotFoundException] => + connection.disconnect() + throw e + } + } + + private def getClassFileInputStreamFromFileSystem(pathInDirectory: String): InputStream = { + val path = new Path(directory, pathInDirectory) + if (fileSystem.exists(path)) { + fileSystem.open(path) + } else { + throw new ClassNotFoundException(s"Class file not found at path $path") + } + } + def findClassLocally(name: String): Option[Class[_]] = { + val pathInDirectory = name.replace('.', '/') + ".class" + var inputStream: InputStream = null try { - val pathInDirectory = name.replace('.', '/') + ".class" - val inputStream = { + inputStream = { if (fileSystem != null) { - fileSystem.open(new Path(directory, pathInDirectory)) + getClassFileInputStreamFromFileSystem(pathInDirectory) } else { - val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { - val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) - val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) - newuri.toURL - } else { - new URL(classUri + "/" + urlEncode(pathInDirectory)) - } - - Utils.setupSecureURLConnection(url.openConnection(), SparkEnv.get.securityManager) - .getInputStream + getClassFileInputStreamFromHttpServer(pathInDirectory) } } val bytes = readAndTransformClass(name, inputStream) - inputStream.close() Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: FileNotFoundException => + case e: ClassNotFoundException => // We did not find the class logDebug(s"Did not load class $name from REPL class server at $uri", e) None @@ -102,6 +142,15 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader // Something bad happened while checking if the class exists logError(s"Failed to check existence of class $name on REPL class server at $uri", e) None + } finally { + if (inputStream != null) { + try { + inputStream.close() + } catch { + case e: Exception => + logError("Exception while closing inputStream", e) + } + } } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 6a79e76a34db8..c709cde740748 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -20,13 +20,25 @@ package org.apache.spark.repl import java.io.File import java.net.{URL, URLClassLoader} +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.scalatest.concurrent.Interruptor +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.mock.MockitoSugar +import org.mockito.Mockito._ -import org.apache.spark.{SparkConf, TestUtils} +import org.apache.spark._ import org.apache.spark.util.Utils -class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { +class ExecutorClassLoaderSuite + extends FunSuite + with BeforeAndAfterAll + with MockitoSugar + with Logging { val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") @@ -34,6 +46,7 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { var tempDir2: File = _ var url1: String = _ var urls2: Array[URL] = _ + var classServer: HttpServer = _ override def beforeAll() { super.beforeAll() @@ -47,8 +60,12 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { override def afterAll() { super.afterAll() + if (classServer != null) { + classServer.stop() + } Utils.deleteRecursively(tempDir1) Utils.deleteRecursively(tempDir2) + SparkEnv.set(null) } test("child first") { @@ -83,4 +100,53 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { } } + test("failing to fetch classes from HTTP server should not leak resources (SPARK-6209)") { + // This is a regression test for SPARK-6209, a bug where each failed attempt to load a class + // from the driver's class server would leak a HTTP connection, causing the class server's + // thread / connection pool to be exhausted. + val conf = new SparkConf() + val securityManager = new SecurityManager(conf) + classServer = new HttpServer(conf, tempDir1, securityManager) + classServer.start() + // ExecutorClassLoader uses SparkEnv's SecurityManager, so we need to mock this + val mockEnv = mock[SparkEnv] + when(mockEnv.securityManager).thenReturn(securityManager) + SparkEnv.set(mockEnv) + // Create an ExecutorClassLoader that's configured to load classes from the HTTP server + val parentLoader = new URLClassLoader(Array.empty, null) + val classLoader = new ExecutorClassLoader(conf, classServer.uri, parentLoader, false) + classLoader.httpUrlConnectionTimeoutMillis = 500 + // Check that this class loader can actually load classes that exist + val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + // Try to perform a full GC now, since GC during the test might mask resource leaks + System.gc() + // When the original bug occurs, the test thread becomes blocked in a classloading call + // and does not respond to interrupts. Therefore, use a custom ScalaTest interruptor to + // shut down the HTTP server when the test times out + val interruptor: Interruptor = new Interruptor { + override def apply(thread: Thread): Unit = { + classServer.stop() + classServer = null + thread.interrupt() + } + } + def tryAndFailToLoadABunchOfClasses(): Unit = { + // The number of trials here should be much larger than Jetty's thread / connection limit + // in order to expose thread or connection leaks + for (i <- 1 to 1000) { + if (Thread.currentThread().isInterrupted) { + throw new InterruptedException() + } + // Incorporate the iteration number into the class name in order to avoid any response + // caching that might be added in the future + intercept[ClassNotFoundException] { + classLoader.loadClass(s"ReplFakeClassDoesNotExist$i").newInstance() + } + } + } + failAfter(10 seconds)(tryAndFailToLoadABunchOfClasses())(interruptor) + } + } From 73348012d4ce6c9db85dfb48d51026efe5051c73 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 Mar 2015 16:03:55 -0700 Subject: [PATCH 515/817] [SPARK-6428][SQL] Added explicit types for all public methods in catalyst I think after this PR, we can finally turn the rule on. There are still some smaller ones that need to be fixed, but those are easier. Author: Reynold Xin Closes #5162 from rxin/catalyst-explicit-types and squashes the following commits: e7eac03 [Reynold Xin] [SPARK-6428][SQL] Added explicit types for all public methods in catalyst. --- .../sql/catalyst/AbstractSparkSQLParser.scala | 8 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/analysis/Catalog.scala | 22 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../catalyst/analysis/FunctionRegistry.scala | 14 +- .../spark/sql/catalyst/analysis/package.scala | 2 +- .../sql/catalyst/analysis/unresolved.scala | 78 +++---- .../spark/sql/catalyst/dsl/package.scala | 202 +++++++++--------- .../catalyst/expressions/AttributeMap.scala | 4 +- .../catalyst/expressions/AttributeSet.scala | 24 ++- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 10 +- .../sql/catalyst/expressions/Expression.scala | 14 +- .../sql/catalyst/expressions/Projection.scala | 52 ++--- .../spark/sql/catalyst/expressions/Rand.scala | 13 +- .../sql/catalyst/expressions/ScalaUdf.scala | 4 +- .../sql/catalyst/expressions/SortOrder.scala | 7 +- .../expressions/SpecificMutableRow.scala | 80 +++---- .../sql/catalyst/expressions/aggregates.scala | 165 +++++++------- .../sql/catalyst/expressions/arithmetic.scala | 66 +++--- .../expressions/codegen/CodeGenerator.scala | 2 +- .../catalyst/expressions/complexTypes.scala | 26 +-- .../expressions/decimalFunctions.scala | 12 +- .../sql/catalyst/expressions/generators.scala | 6 +- .../sql/catalyst/expressions/literals.scala | 13 +- .../expressions/namedExpressions.scala | 51 +++-- .../catalyst/expressions/nullFunctions.scala | 21 +- .../sql/catalyst/expressions/predicates.scala | 80 +++---- .../spark/sql/catalyst/expressions/rows.scala | 18 +- .../spark/sql/catalyst/expressions/sets.scala | 35 +-- .../expressions/stringOperations.scala | 37 ++-- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../sql/catalyst/planning/patterns.scala | 14 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 6 +- .../plans/logical/basicOperators.scala | 46 ++-- .../catalyst/plans/logical/partitioning.scala | 11 +- .../plans/physical/partitioning.scala | 24 +-- .../spark/sql/catalyst/trees/TreeNode.scala | 27 ++- .../spark/sql/catalyst/trees/package.scala | 4 +- .../spark/sql/catalyst/util/package.scala | 6 +- 40 files changed, 626 insertions(+), 586 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala index 366be00473d1c..3823584287741 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala @@ -26,7 +26,7 @@ import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.plans.logical._ private[sql] object KeywordNormalizer { - def apply(str: String) = str.toLowerCase() + def apply(str: String): String = str.toLowerCase() } private[sql] abstract class AbstractSparkSQLParser @@ -42,7 +42,7 @@ private[sql] abstract class AbstractSparkSQLParser } protected case class Keyword(str: String) { - def normalize = KeywordNormalizer(str) + def normalize: String = KeywordNormalizer(str) def parser: Parser[String] = normalize } @@ -81,7 +81,7 @@ private[sql] abstract class AbstractSparkSQLParser class SqlLexical extends StdLexical { case class FloatLit(chars: String) extends Token { - override def toString = chars + override def toString: String = chars } /* This is a work around to support the lazy setting */ @@ -120,7 +120,7 @@ class SqlLexical extends StdLexical { | failure("illegal character") ) - override def identChar = letter | elem('_') + override def identChar: Parser[Elem] = letter | elem('_') override def whitespace: Parser[Any] = ( whitespaceChar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 13d2ae4c6fcb6..44eceb0b372e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -168,7 +168,7 @@ class Analyzer(catalog: Catalog, * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation) = { + def getTable(u: UnresolvedRelation): LogicalPlan = { try { catalog.lookupRelation(u.tableIdentifier, u.alias) } catch { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 9e6e2912e0622..5eb7dff0cede8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -86,12 +86,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { tables += ((getDbTableName(tableIdent), plan)) } - override def unregisterTable(tableIdentifier: Seq[String]) = { + override def unregisterTable(tableIdentifier: Seq[String]): Unit = { val tableIdent = processTableIdentifier(tableIdentifier) tables -= getDbTableName(tableIdent) } - override def unregisterAllTables() = { + override def unregisterAllTables(): Unit = { tables.clear() } @@ -147,8 +147,8 @@ trait OverrideCatalog extends Catalog { } abstract override def lookupRelation( - tableIdentifier: Seq[String], - alias: Option[String] = None): LogicalPlan = { + tableIdentifier: Seq[String], + alias: Option[String] = None): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val overriddenTable = overrides.get(getDBTable(tableIdent)) val tableWithQualifers = overriddenTable.map(r => Subquery(tableIdent.last, r)) @@ -205,15 +205,15 @@ trait OverrideCatalog extends Catalog { */ object EmptyCatalog extends Catalog { - val caseSensitive: Boolean = true + override val caseSensitive: Boolean = true - def tableExists(tableIdentifier: Seq[String]): Boolean = { + override def tableExists(tableIdentifier: Seq[String]): Boolean = { throw new UnsupportedOperationException } - def lookupRelation( - tableIdentifier: Seq[String], - alias: Option[String] = None) = { + override def lookupRelation( + tableIdentifier: Seq[String], + alias: Option[String] = None): LogicalPlan = { throw new UnsupportedOperationException } @@ -221,11 +221,11 @@ object EmptyCatalog extends Catalog { throw new UnsupportedOperationException } - def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { + override def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { throw new UnsupportedOperationException } - def unregisterTable(tableIdentifier: Seq[String]): Unit = { + override def unregisterTable(tableIdentifier: Seq[String]): Unit = { throw new UnsupportedOperationException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 425e1e41cbf21..40472a1cbb3b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -33,7 +33,7 @@ class CheckAnalysis { */ val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil - def failAnalysis(msg: String) = { + def failAnalysis(msg: String): Nothing = { throw new AnalysisException(msg) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 9f334f6d42ad1..c43ea55899695 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -35,7 +35,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry { val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) - def registerFunction(name: String, builder: FunctionBuilder) = { + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { functionBuilders.put(name, builder) } @@ -47,7 +47,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry { class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistry { val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) - def registerFunction(name: String, builder: FunctionBuilder) = { + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { functionBuilders.put(name, builder) } @@ -61,13 +61,15 @@ class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistr * functions are already filled in and the analyser needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { - def registerFunction(name: String, builder: FunctionBuilder) = ??? + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { + throw new UnsupportedOperationException + } - def lookupFunction(name: String, children: Seq[Expression]): Expression = { + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } - def caseSensitive: Boolean = ??? + override def caseSensitive: Boolean = throw new UnsupportedOperationException } /** @@ -76,7 +78,7 @@ object EmptyFunctionRegistry extends FunctionRegistry { * TODO move this into util folder? */ object StringKeyHashMap { - def apply[T](caseSensitive: Boolean) = caseSensitive match { + def apply[T](caseSensitive: Boolean): StringKeyHashMap[T] = caseSensitive match { case false => new StringKeyHashMap[T](_.toLowerCase) case true => new StringKeyHashMap[T](identity) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index a7d3a8ee7deb3..c61c395cb4bb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -38,7 +38,7 @@ package object analysis { implicit class AnalysisErrorAt(t: TreeNode[_]) { /** Fails the analysis at the point where a specific tree node was parsed. */ - def failAnalysis(msg: String) = { + def failAnalysis(msg: String): Nothing = { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index ad5172c0349eb..300e9ba187bc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.types.DataType /** * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully @@ -38,9 +39,10 @@ case class UnresolvedRelation( alias: Option[String] = None) extends LeafNode { /** Returns a `.` separated name for this relation. */ - def tableName = tableIdentifier.mkString(".") + def tableName: String = tableIdentifier.mkString(".") + + override def output: Seq[Attribute] = Nil - override def output = Nil override lazy val resolved = false } @@ -48,16 +50,16 @@ case class UnresolvedRelation( * Holds the name of an attribute that has yet to be resolved. */ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { - override def exprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this - override def withNullability(newNullability: Boolean) = this - override def withQualifiers(newQualifiers: Seq[String]) = this - override def withName(newName: String) = UnresolvedAttribute(name) + override def newInstance(): UnresolvedAttribute = this + override def withNullability(newNullability: Boolean): UnresolvedAttribute = this + override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this + override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute(name) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -67,16 +69,16 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo } case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { - override def dataType = throw new UnresolvedException(this, "dataType") - override def foldable = throw new UnresolvedException(this, "foldable") - override def nullable = throw new UnresolvedException(this, "nullable") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"'$name(${children.mkString(",")})" + override def toString: String = s"'$name(${children.mkString(",")})" } /** @@ -86,17 +88,17 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E trait Star extends Attribute with trees.LeafNode[Expression] { self: Product => - override def name = throw new UnresolvedException(this, "name") - override def exprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def name: String = throw new UnresolvedException(this, "name") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this - override def withNullability(newNullability: Boolean) = this - override def withQualifiers(newQualifiers: Seq[String]) = this - override def withName(newName: String) = this + override def newInstance(): Star = this + override def withNullability(newNullability: Boolean): Star = this + override def withQualifiers(newQualifiers: Seq[String]): Star = this + override def withName(newName: String): Star = this // Star gets expanded at runtime so we never evaluate a Star. override def eval(input: Row = null): EvaluatedType = @@ -129,7 +131,7 @@ case class UnresolvedStar(table: Option[String]) extends Star { } } - override def toString = table.map(_ + ".").getOrElse("") + "*" + override def toString: String = table.map(_ + ".").getOrElse("") + "*" } /** @@ -144,25 +146,25 @@ case class UnresolvedStar(table: Option[String]) extends Star { case class MultiAlias(child: Expression, names: Seq[String]) extends Attribute with trees.UnaryNode[Expression] { - override def name = throw new UnresolvedException(this, "name") + override def name: String = throw new UnresolvedException(this, "name") - override def exprId = throw new UnresolvedException(this, "exprId") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this + override def newInstance(): MultiAlias = this - override def withNullability(newNullability: Boolean) = this + override def withNullability(newNullability: Boolean): MultiAlias = this - override def withQualifiers(newQualifiers: Seq[String]) = this + override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this - override def withName(newName: String) = this + override def withName(newName: String): MultiAlias = this override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -179,17 +181,17 @@ case class MultiAlias(child: Expression, names: Seq[String]) */ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star { override def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = expressions - override def toString = expressions.mkString("ResolvedStar(", ", ", ")") + override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") } case class UnresolvedGetField(child: Expression, fieldName: String) extends UnaryExpression { - override def dataType = throw new UnresolvedException(this, "dataType") - override def foldable = throw new UnresolvedException(this, "foldable") - override def nullable = throw new UnresolvedException(this, "nullable") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"$child.$fieldName" + override def toString: String = s"$child.$fieldName" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 7f5f617812919..145f062dd6817 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -61,60 +61,60 @@ package object dsl { trait ImplicitOperators { def expr: Expression - def unary_- = UnaryMinus(expr) - def unary_! = Not(expr) - def unary_~ = BitwiseNot(expr) - - def + (other: Expression) = Add(expr, other) - def - (other: Expression) = Subtract(expr, other) - def * (other: Expression) = Multiply(expr, other) - def / (other: Expression) = Divide(expr, other) - def % (other: Expression) = Remainder(expr, other) - def & (other: Expression) = BitwiseAnd(expr, other) - def | (other: Expression) = BitwiseOr(expr, other) - def ^ (other: Expression) = BitwiseXor(expr, other) - - def && (other: Expression) = And(expr, other) - def || (other: Expression) = Or(expr, other) - - def < (other: Expression) = LessThan(expr, other) - def <= (other: Expression) = LessThanOrEqual(expr, other) - def > (other: Expression) = GreaterThan(expr, other) - def >= (other: Expression) = GreaterThanOrEqual(expr, other) - def === (other: Expression) = EqualTo(expr, other) - def <=> (other: Expression) = EqualNullSafe(expr, other) - def !== (other: Expression) = Not(EqualTo(expr, other)) - - def in(list: Expression*) = In(expr, list) - - def like(other: Expression) = Like(expr, other) - def rlike(other: Expression) = RLike(expr, other) - def contains(other: Expression) = Contains(expr, other) - def startsWith(other: Expression) = StartsWith(expr, other) - def endsWith(other: Expression) = EndsWith(expr, other) - def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + def unary_- : Expression= UnaryMinus(expr) + def unary_! : Predicate = Not(expr) + def unary_~ : Expression = BitwiseNot(expr) + + def + (other: Expression): Expression = Add(expr, other) + def - (other: Expression): Expression = Subtract(expr, other) + def * (other: Expression): Expression = Multiply(expr, other) + def / (other: Expression): Expression = Divide(expr, other) + def % (other: Expression): Expression = Remainder(expr, other) + def & (other: Expression): Expression = BitwiseAnd(expr, other) + def | (other: Expression): Expression = BitwiseOr(expr, other) + def ^ (other: Expression): Expression = BitwiseXor(expr, other) + + def && (other: Expression): Predicate = And(expr, other) + def || (other: Expression): Predicate = Or(expr, other) + + def < (other: Expression): Predicate = LessThan(expr, other) + def <= (other: Expression): Predicate = LessThanOrEqual(expr, other) + def > (other: Expression): Predicate = GreaterThan(expr, other) + def >= (other: Expression): Predicate = GreaterThanOrEqual(expr, other) + def === (other: Expression): Predicate = EqualTo(expr, other) + def <=> (other: Expression): Predicate = EqualNullSafe(expr, other) + def !== (other: Expression): Predicate = Not(EqualTo(expr, other)) + + def in(list: Expression*): Expression = In(expr, list) + + def like(other: Expression): Expression = Like(expr, other) + def rlike(other: Expression): Expression = RLike(expr, other) + def contains(other: Expression): Expression = Contains(expr, other) + def startsWith(other: Expression): Expression = StartsWith(expr, other) + def endsWith(other: Expression): Expression = EndsWith(expr, other) + def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression = Substring(expr, pos, len) - def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression = Substring(expr, pos, len) - def isNull = IsNull(expr) - def isNotNull = IsNotNull(expr) + def isNull: Predicate = IsNull(expr) + def isNotNull: Predicate = IsNotNull(expr) - def getItem(ordinal: Expression) = GetItem(expr, ordinal) - def getField(fieldName: String) = UnresolvedGetField(expr, fieldName) + def getItem(ordinal: Expression): Expression = GetItem(expr, ordinal) + def getField(fieldName: String): UnresolvedGetField = UnresolvedGetField(expr, fieldName) - def cast(to: DataType) = Cast(expr, to) + def cast(to: DataType): Expression = Cast(expr, to) - def asc = SortOrder(expr, Ascending) - def desc = SortOrder(expr, Descending) + def asc: SortOrder = SortOrder(expr, Ascending) + def desc: SortOrder = SortOrder(expr, Descending) - def as(alias: String) = Alias(expr, alias)() - def as(alias: Symbol) = Alias(expr, alias.name)() + def as(alias: String): NamedExpression = Alias(expr, alias)() + def as(alias: Symbol): NamedExpression = Alias(expr, alias.name)() } trait ExpressionConversions { implicit class DslExpression(e: Expression) extends ImplicitOperators { - def expr = e + def expr: Expression = e } implicit def booleanToLiteral(b: Boolean): Literal = Literal(b) @@ -144,94 +144,100 @@ package object dsl { } } - def sum(e: Expression) = Sum(e) - def sumDistinct(e: Expression) = SumDistinct(e) - def count(e: Expression) = Count(e) - def countDistinct(e: Expression*) = CountDistinct(e) - def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) - def avg(e: Expression) = Average(e) - def first(e: Expression) = First(e) - def last(e: Expression) = Last(e) - def min(e: Expression) = Min(e) - def max(e: Expression) = Max(e) - def upper(e: Expression) = Upper(e) - def lower(e: Expression) = Lower(e) - def sqrt(e: Expression) = Sqrt(e) - def abs(e: Expression) = Abs(e) - - implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } + def sum(e: Expression): Expression = Sum(e) + def sumDistinct(e: Expression): Expression = SumDistinct(e) + def count(e: Expression): Expression = Count(e) + def countDistinct(e: Expression*): Expression = CountDistinct(e) + def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression = + ApproxCountDistinct(e, rsd) + def avg(e: Expression): Expression = Average(e) + def first(e: Expression): Expression = First(e) + def last(e: Expression): Expression = Last(e) + def min(e: Expression): Expression = Min(e) + def max(e: Expression): Expression = Max(e) + def upper(e: Expression): Expression = Upper(e) + def lower(e: Expression): Expression = Lower(e) + def sqrt(e: Expression): Expression = Sqrt(e) + def abs(e: Expression): Expression = Abs(e) + + implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s: String = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { override def expr: Expression = Literal(s) - def attr = analysis.UnresolvedAttribute(s) + def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s) } abstract class ImplicitAttribute extends ImplicitOperators { def s: String - def expr = attr - def attr = analysis.UnresolvedAttribute(s) + def expr: UnresolvedAttribute = attr + def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s) /** Creates a new AttributeReference of type boolean */ - def boolean = AttributeReference(s, BooleanType, nullable = true)() + def boolean: AttributeReference = AttributeReference(s, BooleanType, nullable = true)() /** Creates a new AttributeReference of type byte */ - def byte = AttributeReference(s, ByteType, nullable = true)() + def byte: AttributeReference = AttributeReference(s, ByteType, nullable = true)() /** Creates a new AttributeReference of type short */ - def short = AttributeReference(s, ShortType, nullable = true)() + def short: AttributeReference = AttributeReference(s, ShortType, nullable = true)() /** Creates a new AttributeReference of type int */ - def int = AttributeReference(s, IntegerType, nullable = true)() + def int: AttributeReference = AttributeReference(s, IntegerType, nullable = true)() /** Creates a new AttributeReference of type long */ - def long = AttributeReference(s, LongType, nullable = true)() + def long: AttributeReference = AttributeReference(s, LongType, nullable = true)() /** Creates a new AttributeReference of type float */ - def float = AttributeReference(s, FloatType, nullable = true)() + def float: AttributeReference = AttributeReference(s, FloatType, nullable = true)() /** Creates a new AttributeReference of type double */ - def double = AttributeReference(s, DoubleType, nullable = true)() + def double: AttributeReference = AttributeReference(s, DoubleType, nullable = true)() /** Creates a new AttributeReference of type string */ - def string = AttributeReference(s, StringType, nullable = true)() + def string: AttributeReference = AttributeReference(s, StringType, nullable = true)() /** Creates a new AttributeReference of type date */ - def date = AttributeReference(s, DateType, nullable = true)() + def date: AttributeReference = AttributeReference(s, DateType, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal = AttributeReference(s, DecimalType.Unlimited, nullable = true)() + def decimal: AttributeReference = + AttributeReference(s, DecimalType.Unlimited, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal(precision: Int, scale: Int) = + def decimal(precision: Int, scale: Int): AttributeReference = AttributeReference(s, DecimalType(precision, scale), nullable = true)() /** Creates a new AttributeReference of type timestamp */ - def timestamp = AttributeReference(s, TimestampType, nullable = true)() + def timestamp: AttributeReference = AttributeReference(s, TimestampType, nullable = true)() /** Creates a new AttributeReference of type binary */ - def binary = AttributeReference(s, BinaryType, nullable = true)() + def binary: AttributeReference = AttributeReference(s, BinaryType, nullable = true)() /** Creates a new AttributeReference of type array */ - def array(dataType: DataType) = AttributeReference(s, ArrayType(dataType), nullable = true)() + def array(dataType: DataType): AttributeReference = + AttributeReference(s, ArrayType(dataType), nullable = true)() /** Creates a new AttributeReference of type map */ def map(keyType: DataType, valueType: DataType): AttributeReference = map(MapType(keyType, valueType)) - def map(mapType: MapType) = AttributeReference(s, mapType, nullable = true)() + + def map(mapType: MapType): AttributeReference = + AttributeReference(s, mapType, nullable = true)() /** Creates a new AttributeReference of type struct */ def struct(fields: StructField*): AttributeReference = struct(StructType(fields)) - def struct(structType: StructType) = AttributeReference(s, structType, nullable = true)() + def struct(structType: StructType): AttributeReference = + AttributeReference(s, structType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { - def notNull = a.withNullability(false) - def nullable = a.withNullability(true) + def notNull: AttributeReference = a.withNullability(false) + def nullable: AttributeReference = a.withNullability(true) // Protobuf terminology - def required = a.withNullability(false) + def required: AttributeReference = a.withNullability(false) - def at(ordinal: Int) = BoundReference(ordinal, a.dataType, a.nullable) + def at(ordinal: Int): BoundReference = BoundReference(ordinal, a.dataType, a.nullable) } } @@ -241,23 +247,23 @@ package object dsl { abstract class LogicalPlanFunctions { def logicalPlan: LogicalPlan - def select(exprs: NamedExpression*) = Project(exprs, logicalPlan) + def select(exprs: NamedExpression*): LogicalPlan = Project(exprs, logicalPlan) - def where(condition: Expression) = Filter(condition, logicalPlan) + def where(condition: Expression): LogicalPlan = Filter(condition, logicalPlan) - def limit(limitExpr: Expression) = Limit(limitExpr, logicalPlan) + def limit(limitExpr: Expression): LogicalPlan = Limit(limitExpr, logicalPlan) def join( otherPlan: LogicalPlan, joinType: JoinType = Inner, - condition: Option[Expression] = None) = + condition: Option[Expression] = None): LogicalPlan = Join(logicalPlan, otherPlan, joinType, condition) - def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, true, logicalPlan) + def orderBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, true, logicalPlan) - def sortBy(sortExprs: SortOrder*) = Sort(sortExprs, false, logicalPlan) + def sortBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, false, logicalPlan) - def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { + def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): LogicalPlan = { val aliasedExprs = aggregateExprs.map { case ne: NamedExpression => ne case e => Alias(e, e.toString)() @@ -265,27 +271,27 @@ package object dsl { Aggregate(groupingExprs, aliasedExprs, logicalPlan) } - def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan) + def subquery(alias: Symbol): LogicalPlan = Subquery(alias.name, logicalPlan) - def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan) + def unionAll(otherPlan: LogicalPlan): LogicalPlan = Union(logicalPlan, otherPlan) - def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) = + def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean): LogicalPlan = Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan) def sample( fraction: Double, withReplacement: Boolean = true, - seed: Int = (math.random * 1000).toInt) = + seed: Int = (math.random * 1000).toInt): LogicalPlan = Sample(fraction, withReplacement, seed, logicalPlan) def generate( generator: Generator, join: Boolean = false, outer: Boolean = false, - alias: Option[String] = None) = + alias: Option[String] = None): LogicalPlan = Generate(generator, join, outer, None, logicalPlan) - def insertInto(tableName: String, overwrite: Boolean = false) = + def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) @@ -294,12 +300,14 @@ package object dsl { object plans { // scalastyle:ignore implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions { - def writeToFile(path: String) = WriteToFile(path, logicalPlan) + def writeToFile(path: String): LogicalPlan = WriteToFile(path, logicalPlan) } } case class ScalaUdfBuilder[T: TypeTag](f: AnyRef) { - def call(args: Expression*) = ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args) + def call(args: Expression*): ScalaUdf = { + ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args) + } } // scalastyle:off diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index 82e760b6c6916..96a11e352ec50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -23,7 +23,9 @@ package org.apache.spark.sql.catalyst.expressions * of the name, or the expected nullability). */ object AttributeMap { - def apply[A](kvs: Seq[(Attribute, A)]) = new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) + def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = { + new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) + } } class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index adaeab0b5c027..f9ae85a5cfc1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -19,27 +19,27 @@ package org.apache.spark.sql.catalyst.expressions protected class AttributeEquals(val a: Attribute) { - override def hashCode() = a match { + override def hashCode(): Int = a match { case ar: AttributeReference => ar.exprId.hashCode() case a => a.hashCode() } - override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match { + override def equals(other: Any): Boolean = (a, other.asInstanceOf[AttributeEquals].a) match { case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId case (a1, a2) => a1 == a2 } } object AttributeSet { - def apply(a: Attribute) = - new AttributeSet(Set(new AttributeEquals(a))) + def apply(a: Attribute): AttributeSet = new AttributeSet(Set(new AttributeEquals(a))) /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ - def apply(baseSet: Seq[Expression]) = + def apply(baseSet: Seq[Expression]): AttributeSet = { new AttributeSet( baseSet .flatMap(_.references) .map(new AttributeEquals(_)).toSet) + } } /** @@ -57,7 +57,7 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) extends Traversable[Attribute] with Serializable { /** Returns true if the members of this AttributeSet and other are the same. */ - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) case _ => false } @@ -81,32 +81,34 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in * `other`. */ - def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + def subsetOf(other: AttributeSet): Boolean = baseSet.subsetOf(other.baseSet) /** * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found * in `other`. */ - def --(other: Traversable[NamedExpression]) = + def --(other: Traversable[NamedExpression]): AttributeSet = new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) /** * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found * in `other`. */ - def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + def ++(other: AttributeSet): AttributeSet = new AttributeSet(baseSet ++ other.baseSet) /** * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to * true. */ - override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + override def filter(f: Attribute => Boolean): AttributeSet = + new AttributeSet(baseSet.filter(ae => f(ae.a))) /** * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in * `this` and `other`. */ - def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + def intersect(other: AttributeSet): AttributeSet = + new AttributeSet(baseSet.intersect(other.baseSet)) override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 76a9f08dea85f..2225621dbaabd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -32,7 +32,7 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) type EvaluatedType = Any - override def toString = s"input[$ordinal]" + override def toString: String = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index b1bc858478ee1..9bde74ac22669 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -29,9 +29,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w override lazy val resolved = childrenResolved && resolve(child.dataType, dataType) - override def foldable = child.foldable + override def foldable: Boolean = child.foldable - override def nullable = forceNullable(child.dataType, dataType) || child.nullable + override def nullable: Boolean = forceNullable(child.dataType, dataType) || child.nullable private[this] def forceNullable(from: DataType, to: DataType) = (from, to) match { case (StringType, _: NumericType) => true @@ -103,7 +103,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } - override def toString = s"CAST($child, $dataType)" + override def toString: String = s"CAST($child, $dataType)" type EvaluatedType = Any @@ -430,14 +430,14 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w object Cast { // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { - override def initialValue() = { + override def initialValue(): SimpleDateFormat = { new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } } // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { - override def initialValue() = { + override def initialValue(): SimpleDateFormat = { new SimpleDateFormat("yyyy-MM-dd") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 6ad39b8372cfb..4e3bbc06a5b4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -65,7 +65,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if all the children of this expression have been resolved to a specific schema * and false if any still contains any unresolved placeholders. */ - def childrenResolved = !children.exists(!_.resolved) + def childrenResolved: Boolean = !children.exists(!_.resolved) /** * Returns a string representation of this expression that does not have developer centric @@ -84,9 +84,9 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express def symbol: String - override def foldable = left.foldable && right.foldable + override def foldable: Boolean = left.foldable && right.foldable - override def toString = s"($left $symbol $right)" + override def toString: String = s"($left $symbol $right)" } abstract class LeafExpression extends Expression with trees.LeafNode[Expression] { @@ -104,8 +104,8 @@ abstract class UnaryExpression extends Expression with trees.UnaryNode[Expressio case class GroupExpression(children: Seq[Expression]) extends Expression { self: Product => type EvaluatedType = Seq[Any] - override def eval(input: Row): EvaluatedType = ??? - override def nullable = false - override def foldable = false - override def dataType = ??? + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = false + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index db5d897ee569f..c2866cd955409 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -40,7 +40,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { new GenericRow(outputArray) } - override def toString = s"Row => [${exprArray.mkString(",")}]" + override def toString: String = s"Row => [${exprArray.mkString(",")}]" } /** @@ -107,12 +107,12 @@ class JoinedRow extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -142,7 +142,7 @@ class JoinedRow extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -153,7 +153,7 @@ class JoinedRow extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -207,12 +207,12 @@ class JoinedRow2 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -242,7 +242,7 @@ class JoinedRow2 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -253,7 +253,7 @@ class JoinedRow2 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -301,12 +301,12 @@ class JoinedRow3 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -336,7 +336,7 @@ class JoinedRow3 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -347,7 +347,7 @@ class JoinedRow3 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -395,12 +395,12 @@ class JoinedRow4 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -430,7 +430,7 @@ class JoinedRow4 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -441,7 +441,7 @@ class JoinedRow4 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -489,12 +489,12 @@ class JoinedRow5 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -524,7 +524,7 @@ class JoinedRow5 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -535,7 +535,7 @@ class JoinedRow5 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index b2c6d3029031d..f5fea3f015dc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -18,16 +18,19 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Random -import org.apache.spark.sql.types.DoubleType + +import org.apache.spark.sql.types.{DataType, DoubleType} case object Rand extends LeafExpression { - override def dataType = DoubleType - override def nullable = false + override def dataType: DataType = DoubleType + override def nullable: Boolean = false private[this] lazy val rand = new Random - override def eval(input: Row = null) = rand.nextDouble().asInstanceOf[EvaluatedType] + override def eval(input: Row = null): EvaluatedType = { + rand.nextDouble().asInstanceOf[EvaluatedType] + } - override def toString = "RAND()" + override def toString: String = "RAND()" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 8a36c6810790d..1fd5ce342b2ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -29,9 +29,9 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def nullable = true + override def nullable: Boolean = true - override def toString = s"scalaUDF(${children.mkString(",")})" + override def toString: String = s"scalaUDF(${children.mkString(",")})" // scalastyle:off diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d00b2ac09745c..83074eb1e6310 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.types.DataType abstract sealed class SortDirection case object Ascending extends SortDirection @@ -31,12 +32,12 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def dataType = child.dataType - override def nullable = child.nullable + override def dataType: DataType = child.dataType + override def nullable: Boolean = child.nullable // SortOrder itself is never evaluated. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" + override def toString: String = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 21d714c9a8c3b..47b6f358ed1b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -62,126 +62,126 @@ abstract class MutableValue extends Serializable { var isNull: Boolean = true def boxed: Any def update(v: Any) - def copy(): this.type + def copy(): MutableValue } final class MutableInt extends MutableValue { var value: Int = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Int] + value = v.asInstanceOf[Int] } - def copy() = { + override def copy(): MutableInt = { val newCopy = new MutableInt newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableInt] } } final class MutableFloat extends MutableValue { var value: Float = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Float] + value = v.asInstanceOf[Float] } - def copy() = { + override def copy(): MutableFloat = { val newCopy = new MutableFloat newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableFloat] } } final class MutableBoolean extends MutableValue { var value: Boolean = false - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Boolean] + value = v.asInstanceOf[Boolean] } - def copy() = { + override def copy(): MutableBoolean = { val newCopy = new MutableBoolean newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableBoolean] } } final class MutableDouble extends MutableValue { var value: Double = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Double] + value = v.asInstanceOf[Double] } - def copy() = { + override def copy(): MutableDouble = { val newCopy = new MutableDouble newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableDouble] } } final class MutableShort extends MutableValue { var value: Short = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Short] } - def copy() = { + override def copy(): MutableShort = { val newCopy = new MutableShort newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableShort] } } final class MutableLong extends MutableValue { var value: Long = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Long] } - def copy() = { + override def copy(): MutableLong = { val newCopy = new MutableLong newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableLong] } } final class MutableByte extends MutableValue { var value: Byte = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Byte] } - def copy() = { + override def copy(): MutableByte = { val newCopy = new MutableByte newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableByte] } } final class MutableAny extends MutableValue { var value: Any = _ - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Any] + value = v.asInstanceOf[Any] } - def copy() = { + override def copy(): MutableAny = { val newCopy = new MutableAny newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableAny] } } @@ -234,9 +234,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR if (value == null) setNullAt(ordinal) else values(ordinal).update(value) } - override def setString(ordinal: Int, value: String) = update(ordinal, value) + override def setString(ordinal: Int, value: String): Unit = update(ordinal, value) - override def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int): String = apply(ordinal).asInstanceOf[String] override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 5297d1e31246c..30da4faa3f1c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -79,27 +79,29 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def nullable = base.nullable - override def dataType = base.dataType + override def nullable: Boolean = base.nullable + override def dataType: DataType = base.dataType def update(input: Row): Unit // Do we really need this? - override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + override def newInstance(): AggregateFunction = { + makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + } } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"MIN($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"MIN($child)" override def asPartial: SplitEvaluation = { val partialMin = Alias(Min(child), "PartialMin")() SplitEvaluation(Min(partialMin.toAttribute), partialMin :: Nil) } - override def newInstance() = new MinFunction(child, this) + override def newInstance(): MinFunction = new MinFunction(child, this) } case class MinFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -121,16 +123,16 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"MAX($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"MAX($child)" override def asPartial: SplitEvaluation = { val partialMax = Alias(Max(child), "PartialMax")() SplitEvaluation(Max(partialMax.toAttribute), partialMax :: Nil) } - override def newInstance() = new MaxFunction(child, this) + override def newInstance(): MaxFunction = new MaxFunction(child, this) } case class MaxFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -152,29 +154,29 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"COUNT($child)" + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"COUNT($child)" override def asPartial: SplitEvaluation = { val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Coalesce(Seq(Sum(partialCount.toAttribute), Literal(0L))), partialCount :: Nil) } - override def newInstance() = new CountFunction(child, this) + override def newInstance(): CountFunction = new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate { def this() = this(null) - override def children = expressions + override def children: Seq[Expression] = expressions - override def nullable = false - override def dataType = LongType - override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" - override def newInstance() = new CountDistinctFunction(expressions, this) + override def nullable: Boolean = false + override def dataType: DataType = LongType + override def toString: String = s"COUNT(DISTINCT ${expressions.mkString(",")})" + override def newInstance(): CountDistinctFunction = new CountDistinctFunction(expressions, this) - override def asPartial = { + override def asPartial: SplitEvaluation = { val partialSet = Alias(CollectHashSet(expressions), "partialSets")() SplitEvaluation( CombineSetsAndCount(partialSet.toAttribute), @@ -185,11 +187,11 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression { def this() = this(null) - override def children = expressions - override def nullable = false - override def dataType = ArrayType(expressions.head.dataType) - override def toString = s"AddToHashSet(${expressions.mkString(",")})" - override def newInstance() = new CollectHashSetFunction(expressions, this) + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: ArrayType = ArrayType(expressions.head.dataType) + override def toString: String = s"AddToHashSet(${expressions.mkString(",")})" + override def newInstance(): CollectHashSetFunction = new CollectHashSetFunction(expressions, this) } case class CollectHashSetFunction( @@ -219,11 +221,13 @@ case class CollectHashSetFunction( case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression { def this() = this(null) - override def children = inputSet :: Nil - override def nullable = false - override def dataType = LongType - override def toString = s"CombineAndCount($inputSet)" - override def newInstance() = new CombineSetsAndCountFunction(inputSet, this) + override def children: Seq[Expression] = inputSet :: Nil + override def nullable: Boolean = false + override def dataType: DataType = LongType + override def toString: String = s"CombineAndCount($inputSet)" + override def newInstance(): CombineSetsAndCountFunction = { + new CombineSetsAndCountFunction(inputSet, this) + } } case class CombineSetsAndCountFunction( @@ -249,27 +253,31 @@ case class CombineSetsAndCountFunction( case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = child.dataType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" - override def newInstance() = new ApproxCountDistinctPartitionFunction(child, this, relativeSD) + override def nullable: Boolean = false + override def dataType: DataType = child.dataType + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance(): ApproxCountDistinctPartitionFunction = { + new ApproxCountDistinctPartitionFunction(child, this, relativeSD) + } } case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" - override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD) + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance(): ApproxCountDistinctMergeFunction = { + new ApproxCountDistinctMergeFunction(child, this, relativeSD) + } } case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" override def asPartial: SplitEvaluation = { val partialCount = @@ -280,14 +288,14 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) partialCount :: Nil) } - override def newInstance() = new CountDistinctFunction(child :: Nil, this) + override def newInstance(): CountDistinctFunction = new CountDistinctFunction(child :: Nil, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true + override def nullable: Boolean = true - override def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 4, scale + 4) // Add 4 digits after decimal point, like Hive case DecimalType.Unlimited => @@ -296,7 +304,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN DoubleType } - override def toString = s"AVG($child)" + override def toString: String = s"AVG($child)" override def asPartial: SplitEvaluation = { child.dataType match { @@ -323,14 +331,14 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } } - override def newInstance() = new AverageFunction(child, this) + override def newInstance(): AverageFunction = new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true + override def nullable: Boolean = true - override def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive case DecimalType.Unlimited => @@ -339,7 +347,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ child.dataType } - override def toString = s"SUM($child)" + override def toString: String = s"SUM($child)" override def asPartial: SplitEvaluation = { child.dataType match { @@ -357,7 +365,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ } } - override def newInstance() = new SumFunction(child, this) + override def newInstance(): SumFunction = new SumFunction(child, this) } /** @@ -377,19 +385,19 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class CombineSum(child: Expression) extends AggregateExpression { def this() = this(null) - override def children = child :: Nil - override def nullable = true - override def dataType = child.dataType - override def toString = s"CombineSum($child)" - override def newInstance() = new CombineSumFunction(child, this) + override def children: Seq[Expression] = child :: Nil + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"CombineSum($child)" + override def newInstance(): CombineSumFunction = new CombineSumFunction(child, this) } case class SumDistinct(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def this() = this(null) - override def nullable = true - override def dataType = child.dataType match { + override def nullable: Boolean = true + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive case DecimalType.Unlimited => @@ -397,10 +405,10 @@ case class SumDistinct(child: Expression) case _ => child.dataType } - override def toString = s"SUM(DISTINCT ${child})" - override def newInstance() = new SumDistinctFunction(child, this) + override def toString: String = s"SUM(DISTINCT $child)" + override def newInstance(): SumDistinctFunction = new SumDistinctFunction(child, this) - override def asPartial = { + override def asPartial: SplitEvaluation = { val partialSet = Alias(CollectHashSet(child :: Nil), "partialSets")() SplitEvaluation( CombineSetsAndSum(partialSet.toAttribute, this), @@ -411,11 +419,13 @@ case class SumDistinct(child: Expression) case class CombineSetsAndSum(inputSet: Expression, base: Expression) extends AggregateExpression { def this() = this(null, null) - override def children = inputSet :: Nil - override def nullable = true - override def dataType = base.dataType - override def toString = s"CombineAndSum($inputSet)" - override def newInstance() = new CombineSetsAndSumFunction(inputSet, this) + override def children: Seq[Expression] = inputSet :: Nil + override def nullable: Boolean = true + override def dataType: DataType = base.dataType + override def toString: String = s"CombineAndSum($inputSet)" + override def newInstance(): CombineSetsAndSumFunction = { + new CombineSetsAndSumFunction(inputSet, this) + } } case class CombineSetsAndSumFunction( @@ -449,9 +459,9 @@ case class CombineSetsAndSumFunction( } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"FIRST($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"FIRST($child)" override def asPartial: SplitEvaluation = { val partialFirst = Alias(First(child), "PartialFirst")() @@ -459,14 +469,14 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance() = new FirstFunction(child, this) + override def newInstance(): FirstFunction = new FirstFunction(child, this) } case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references - override def nullable = true - override def dataType = child.dataType - override def toString = s"LAST($child)" + override def references: AttributeSet = child.references + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"LAST($child)" override def asPartial: SplitEvaluation = { val partialLast = Alias(Last(child), "PartialLast")() @@ -474,7 +484,7 @@ case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode Last(partialLast.toAttribute), partialLast :: Nil) } - override def newInstance() = new LastFunction(child, this) + override def newInstance(): LastFunction = new LastFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) @@ -713,6 +723,7 @@ case class LastFunction(expr: Expression, base: AggregateExpression) extends Agg result = input } - override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row]) - else null + override def eval(input: Row): Any = { + if (result != null) expr.eval(result.asInstanceOf[Row]) else null + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 00b0d3c683fe2..1f6526ef66c56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"-$child" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"-$child" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -47,10 +47,10 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { case class Sqrt(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = DoubleType - override def foldable = child.foldable - def nullable = true - override def toString = s"SQRT($child)" + override def dataType: DataType = DoubleType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = true + override def toString: String = s"SQRT($child)" lazy val numeric = child.dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -74,14 +74,14 @@ abstract class BinaryArithmetic extends BinaryExpression { type EvaluatedType = Any - def nullable = left.nullable || right.nullable + def nullable: Boolean = left.nullable || right.nullable override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType && !DecimalType.isFixed(left.dataType) - def dataType = { + def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") @@ -108,7 +108,7 @@ abstract class BinaryArithmetic extends BinaryExpression { } case class Add(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "+" + override def symbol: String = "+" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -131,7 +131,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic { } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "-" + override def symbol: String = "-" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -154,7 +154,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "*" + override def symbol: String = "*" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -177,9 +177,9 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "/" + override def symbol: String = "/" - override def nullable = true + override def nullable: Boolean = true lazy val div: (Any, Any) => Any = dataType match { case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div @@ -203,9 +203,9 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "%" + override def symbol: String = "%" - override def nullable = true + override def nullable: Boolean = true lazy val integral = dataType match { case i: IntegralType => i.integral.asInstanceOf[Integral[Any]] @@ -232,7 +232,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet * A function that calculates bitwise and(&) of two numbers. */ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "&" + override def symbol: String = "&" lazy val and: (Any, Any) => Any = dataType match { case ByteType => @@ -253,7 +253,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme * A function that calculates bitwise or(|) of two numbers. */ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "|" + override def symbol: String = "|" lazy val or: (Any, Any) => Any = dataType match { case ByteType => @@ -274,7 +274,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet * A function that calculates bitwise xor(^) of two numbers. */ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "^" + override def symbol: String = "^" lazy val xor: (Any, Any) => Any = dataType match { case ByteType => @@ -297,10 +297,10 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme case class BitwiseNot(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"~$child" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"~$child" lazy val not: (Any) => Any = dataType match { case ByteType => @@ -327,17 +327,17 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any - override def foldable = left.foldable && right.foldable + override def foldable: Boolean = left.foldable && right.foldable - override def nullable = left.nullable && right.nullable + override def nullable: Boolean = left.nullable && right.nullable - override def children = left :: right :: Nil + override def children: Seq[Expression] = left :: right :: Nil override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType - override def dataType = { + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") @@ -366,7 +366,7 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { } } - override def toString = s"MaxOf($left, $right)" + override def toString: String = s"MaxOf($left, $right)" } /** @@ -375,10 +375,10 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { case class Abs(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"Abs($child)" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"Abs($child)" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index e48b8cde20eda..d1abf3c0b64a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -91,7 +91,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val startTime = System.nanoTime() val result = create(in) val endTime = System.nanoTime() - def timeMs = (endTime - startTime).toDouble / 1000000 + def timeMs: Double = (endTime - startTime).toDouble / 1000000 logInfo(s"Code generated expression $in in $timeMs ms") result } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 68051a2a2007e..3fd78db297462 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -27,12 +27,12 @@ import org.apache.spark.sql.types._ case class GetItem(child: Expression, ordinal: Expression) extends Expression { type EvaluatedType = Any - val children = child :: ordinal :: Nil + val children: Seq[Expression] = child :: ordinal :: Nil /** `Null` is returned for invalid ordinals. */ - override def nullable = true - override def foldable = child.foldable && ordinal.foldable + override def nullable: Boolean = true + override def foldable: Boolean = child.foldable && ordinal.foldable - def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case ArrayType(dt, _) => dt case MapType(_, vt, _) => vt } @@ -40,7 +40,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { childrenResolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - override def toString = s"$child[$ordinal]" + override def toString: String = s"$child[$ordinal]" override def eval(input: Row): Any = { val value = child.eval(input) @@ -75,8 +75,8 @@ trait GetField extends UnaryExpression { self: Product => type EvaluatedType = Any - override def foldable = child.foldable - override def toString = s"$child.${field.name}" + override def foldable: Boolean = child.foldable + override def toString: String = s"$child.${field.name}" def field: StructField } @@ -86,8 +86,8 @@ trait GetField extends UnaryExpression { */ case class StructGetField(child: Expression, field: StructField, ordinal: Int) extends GetField { - def dataType = field.dataType - override def nullable = child.nullable || field.nullable + override def dataType: DataType = field.dataType + override def nullable: Boolean = child.nullable || field.nullable override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Row] @@ -101,8 +101,8 @@ case class StructGetField(child: Expression, field: StructField, ordinal: Int) e case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, containsNull: Boolean) extends GetField { - def dataType = ArrayType(field.dataType, containsNull) - override def nullable = child.nullable + override def dataType: DataType = ArrayType(field.dataType, containsNull) + override def nullable: Boolean = child.nullable override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Seq[Row]] @@ -120,7 +120,7 @@ case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, co case class CreateArray(children: Seq[Expression]) extends Expression { override type EvaluatedType = Any - override def foldable = !children.exists(!_.foldable) + override def foldable: Boolean = !children.exists(!_.foldable) lazy val childTypes = children.map(_.dataType).distinct @@ -140,5 +140,5 @@ case class CreateArray(children: Seq[Expression]) extends Expression { children.map(_.eval(input)) } - override def toString = s"Array(${children.mkString(",")})" + override def toString: String = s"Array(${children.mkString(",")})" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index 83d8c1d42bca4..adb94df7d1c7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -24,9 +24,9 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { override type EvaluatedType = Any override def dataType: DataType = LongType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"UnscaledValue($child)" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"UnscaledValue($child)" override def eval(input: Row): Any = { val childResult = child.eval(input) @@ -43,9 +43,9 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un override type EvaluatedType = Decimal override def dataType: DataType = DecimalType(precision, scale) - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"MakeDecimal($child,$precision,$scale)" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"MakeDecimal($child,$precision,$scale)" override def eval(input: Row): Decimal = { val childResult = child.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 0983d274def3f..860b72fad38b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -45,7 +45,7 @@ abstract class Generator extends Expression { override lazy val dataType = ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))) - override def nullable = false + override def nullable: Boolean = false /** * Should be overridden by specific generators. Called only once for each instance to ensure @@ -89,7 +89,7 @@ case class UserDefinedGenerator( function(inputRow(input)) } - override def toString = s"UserDefinedGenerator(${children.mkString(",")})" + override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" } /** @@ -130,5 +130,5 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - override def toString() = s"explode($child)" + override def toString: String = s"explode($child)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 9ff66563c8164..19f3fc9c2291a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -64,14 +64,13 @@ object IntegerLiteral { case class Literal(value: Any, dataType: DataType) extends LeafExpression { - override def foldable = true - def nullable = value == null + override def foldable: Boolean = true + override def nullable: Boolean = value == null - - override def toString = if (value != null) value.toString else "null" + override def toString: String = if (value != null) value.toString else "null" type EvaluatedType = Any - override def eval(input: Row):Any = value + override def eval(input: Row): Any = value } // TODO: Specialize @@ -79,9 +78,9 @@ case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean extends LeafExpression { type EvaluatedType = Any - def update(expression: Expression, input: Row) = { + def update(expression: Expression, input: Row): Unit = { value = expression.eval(input) } - override def eval(input: Row) = value + override def eval(input: Row): Any = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 08361d043b6ed..bcbcbeb31c7b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees.LeafNode import org.apache.spark.sql.types._ object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() - def newExprId = ExprId(curId.getAndIncrement()) + def newExprId: ExprId = ExprId(curId.getAndIncrement()) def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType) } @@ -79,13 +80,13 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => - override def references = AttributeSet(this) + override def references: AttributeSet = AttributeSet(this) def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute - def toAttribute = this + def toAttribute: Attribute = this def newInstance(): Attribute } @@ -112,10 +113,10 @@ case class Alias(child: Expression, name: String)( override type EvaluatedType = Any - override def eval(input: Row) = child.eval(input) + override def eval(input: Row): Any = child.eval(input) - override def dataType = child.dataType - override def nullable = child.nullable + override def dataType: DataType = child.dataType + override def nullable: Boolean = child.nullable override def metadata: Metadata = { explicitMetadata.getOrElse { child match { @@ -125,7 +126,7 @@ case class Alias(child: Expression, name: String)( } } - override def toAttribute = { + override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifiers) } else { @@ -135,7 +136,9 @@ case class Alias(child: Expression, name: String)( override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" - override protected final def otherCopyArgs = exprId :: qualifiers :: explicitMetadata :: Nil + override protected final def otherCopyArgs: Seq[AnyRef] = { + exprId :: qualifiers :: explicitMetadata :: Nil + } override def equals(other: Any): Boolean = other match { case a: Alias => @@ -166,7 +169,7 @@ case class AttributeReference( val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType case _ => false } @@ -180,7 +183,7 @@ case class AttributeReference( h } - override def newInstance() = + override def newInstance(): AttributeReference = AttributeReference(name, dataType, nullable, metadata)(qualifiers = qualifiers) /** @@ -205,7 +208,7 @@ case class AttributeReference( /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ - override def withQualifiers(newQualifiers: Seq[String]) = { + override def withQualifiers(newQualifiers: Seq[String]): AttributeReference = { if (newQualifiers.toSet == qualifiers.toSet) { this } else { @@ -227,20 +230,22 @@ case class AttributeReference( case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { type EvaluatedType = Any - override def toString = name - - override def withNullability(newNullability: Boolean): Attribute = ??? - override def newInstance(): Attribute = ??? - override def withQualifiers(newQualifiers: Seq[String]): Attribute = ??? - override def withName(newName: String): Attribute = ??? - override def qualifiers: Seq[String] = ??? - override def exprId: ExprId = ??? - override def eval(input: Row): EvaluatedType = ??? - override def nullable: Boolean = ??? + override def toString: String = name + + override def withNullability(newNullability: Boolean): Attribute = + throw new UnsupportedOperationException + override def newInstance(): Attribute = throw new UnsupportedOperationException + override def withQualifiers(newQualifiers: Seq[String]): Attribute = + throw new UnsupportedOperationException + override def withName(newName: String): Attribute = throw new UnsupportedOperationException + override def qualifiers: Seq[String] = throw new UnsupportedOperationException + override def exprId: ExprId = throw new UnsupportedOperationException + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = throw new UnsupportedOperationException override def dataType: DataType = NullType } object VirtualColumn { - val groupingIdName = "grouping__id" - def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)() + val groupingIdName: String = "grouping__id" + def newGroupingId: AttributeReference = AttributeReference(groupingIdName, IntegerType, false)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 08b982bc671e7..d1f3d4f4ee9ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -19,22 +19,23 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.types.DataType case class Coalesce(children: Seq[Expression]) extends Expression { type EvaluatedType = Any /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ - def nullable = !children.exists(!_.nullable) + override def nullable: Boolean = !children.exists(!_.nullable) // Coalesce is foldable if all children are foldable. - override def foldable = !children.exists(!_.foldable) + override def foldable: Boolean = !children.exists(!_.foldable) // Only resolved if all the children are of the same type. override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) - override def toString = s"Coalesce(${children.mkString(",")})" + override def toString: String = s"Coalesce(${children.mkString(",")})" - def dataType = if (resolved) { + def dataType: DataType = if (resolved) { children.head.dataType } else { val childTypes = children.map(c => s"$c: ${c.dataType}").mkString(", ") @@ -54,20 +55,20 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - override def foldable = child.foldable - def nullable = false + override def foldable: Boolean = child.foldable + override def nullable: Boolean = false override def eval(input: Row): Any = { child.eval(input) == null } - override def toString = s"IS NULL $child" + override def toString: String = s"IS NULL $child" } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - override def foldable = child.foldable - def nullable = false - override def toString = s"IS NOT NULL $child" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = false + override def toString: String = s"IS NOT NULL $child" override def eval(input: Row): Any = { child.eval(input) != null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 0024ef92c0452..7e47cb3fffe12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.{BinaryType, BooleanType, NativeType} +import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, NativeType} object InterpretedPredicate { def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = @@ -34,7 +34,7 @@ object InterpretedPredicate { trait Predicate extends Expression { self: Product => - def dataType = BooleanType + override def dataType: DataType = BooleanType type EvaluatedType = Any } @@ -72,13 +72,13 @@ trait PredicateHelper { abstract class BinaryPredicate extends BinaryExpression with Predicate { self: Product => - def nullable = left.nullable || right.nullable + override def nullable: Boolean = left.nullable || right.nullable } case class Not(child: Expression) extends UnaryExpression with Predicate { - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"NOT $child" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"NOT $child" override def eval(input: Row): Any = { child.eval(input) match { @@ -92,10 +92,10 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { * Evaluates to `true` if `list` contains `value`. */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { - def children = value +: list + override def children: Seq[Expression] = value +: list - def nullable = true // TODO: Figure out correct nullability semantics of IN. - override def toString = s"$value IN ${list.mkString("(", ",", ")")}" + override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN. + override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}" override def eval(input: Row): Any = { val evaluatedValue = value.eval(input) @@ -110,10 +110,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { case class InSet(value: Expression, hset: Set[Any]) extends Predicate { - def children = value :: Nil + override def children: Seq[Expression] = value :: Nil - def nullable = true // TODO: Figure out correct nullability semantics of IN. - override def toString = s"$value INSET ${hset.mkString("(", ",", ")")}" + override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN. + override def toString: String = s"$value INSET ${hset.mkString("(", ",", ")")}" override def eval(input: Row): Any = { hset.contains(value.eval(input)) @@ -121,7 +121,7 @@ case class InSet(value: Expression, hset: Set[Any]) } case class And(left: Expression, right: Expression) extends BinaryPredicate { - def symbol = "&&" + override def symbol: String = "&&" override def eval(input: Row): Any = { val l = left.eval(input) @@ -143,7 +143,7 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { } case class Or(left: Expression, right: Expression) extends BinaryPredicate { - def symbol = "||" + override def symbol: String = "||" override def eval(input: Row): Any = { val l = left.eval(input) @@ -169,7 +169,8 @@ abstract class BinaryComparison extends BinaryPredicate { } case class EqualTo(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "=" + override def symbol: String = "=" + override def eval(input: Row): Any = { val l = left.eval(input) if (l == null) { @@ -185,8 +186,10 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison } case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<=>" - override def nullable = false + override def symbol: String = "<=>" + + override def nullable: Boolean = false + override def eval(input: Row): Any = { val l = left.eval(input) val r = right.eval(input) @@ -201,9 +204,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<" + override def symbol: String = "<" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") @@ -216,7 +219,7 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -230,9 +233,9 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<=" + override def symbol: String = "<=" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") @@ -245,7 +248,7 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -259,9 +262,9 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { - def symbol = ">" + override def symbol: String = ">" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") @@ -288,9 +291,9 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryCompar } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { - def symbol = ">=" + override def symbol: String = ">=" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") @@ -303,7 +306,7 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -317,13 +320,13 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) - extends Expression { + extends Expression { - def children = predicate :: trueValue :: falseValue :: Nil - override def nullable = trueValue.nullable || falseValue.nullable + override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil + override def nullable: Boolean = trueValue.nullable || falseValue.nullable override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType - def dataType = { + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException( this, @@ -342,7 +345,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } } - override def toString = s"if ($predicate) $trueValue else $falseValue" + override def toString: String = s"if ($predicate) $trueValue else $falseValue" } // scalastyle:off @@ -362,9 +365,10 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi // scalastyle:on case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any - def children = branches - def dataType = { + override def children: Seq[Expression] = branches + + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") } @@ -379,12 +383,12 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { @transient private[this] lazy val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) - override def nullable = { + override def nullable: Boolean = { // If no value is nullable and no elseValue is provided, the whole statement defaults to null. values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) } - override lazy val resolved = { + override lazy val resolved: Boolean = { if (!childrenResolved) { false } else { @@ -415,7 +419,7 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { res } - override def toString = { + override def toString: String = { "CASE" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" case Seq(elseValue) => s" ELSE $elseValue" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index f03d6f71a9fae..8bba26bc4cf7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -44,8 +44,8 @@ trait MutableRow extends Row { */ object EmptyRow extends Row { override def apply(i: Int): Any = throw new UnsupportedOperationException - override def toSeq = Seq.empty - override def length = 0 + override def toSeq: Seq[Any] = Seq.empty + override def length: Int = 0 override def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException override def getInt(i: Int): Int = throw new UnsupportedOperationException override def getLong(i: Int): Long = throw new UnsupportedOperationException @@ -56,7 +56,7 @@ object EmptyRow extends Row { override def getByte(i: Int): Byte = throw new UnsupportedOperationException override def getString(i: Int): String = throw new UnsupportedOperationException override def getAs[T](i: Int): T = throw new UnsupportedOperationException - def copy() = this + override def copy(): Row = this } /** @@ -70,13 +70,13 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { def this(size: Int) = this(new Array[Any](size)) - override def toSeq = values.toSeq + override def toSeq: Seq[Any] = values.toSeq - override def length = values.length + override def length: Int = values.length - override def apply(i: Int) = values(i) + override def apply(i: Int): Any = values(i) - override def isNullAt(i: Int) = values(i) == null + override def isNullAt(i: Int): Boolean = values(i) == null override def getInt(i: Int): Int = { if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") @@ -167,7 +167,7 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { case _ => false } - def copy() = this + override def copy(): Row = this } class GenericRowWithSchema(values: Array[Any], override val schema: StructType) @@ -194,7 +194,7 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { override def update(ordinal: Int, value: Any): Unit = { values(ordinal) = value } - override def copy() = new GenericRow(values.clone()) + override def copy(): Row = new GenericRow(values.clone()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index 3a5bdca1f07c3..35faa00782e80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -26,17 +26,17 @@ import org.apache.spark.util.collection.OpenHashSet case class NewSet(elementType: DataType) extends LeafExpression { type EvaluatedType = Any - def nullable = false + override def nullable: Boolean = false // We are currently only using these Expressions internally for aggregation. However, if we ever // expose these to users we'll want to create a proper type instead of hijacking ArrayType. - def dataType = ArrayType(elementType) + override def dataType: DataType = ArrayType(elementType) - def eval(input: Row): Any = { + override def eval(input: Row): Any = { new OpenHashSet[Any]() } - override def toString = s"new Set($dataType)" + override def toString: String = s"new Set($dataType)" } /** @@ -46,12 +46,13 @@ case class NewSet(elementType: DataType) extends LeafExpression { case class AddItemToSet(item: Expression, set: Expression) extends Expression { type EvaluatedType = Any - def children = item :: set :: Nil + override def children: Seq[Expression] = item :: set :: Nil - def nullable = set.nullable + override def nullable: Boolean = set.nullable - def dataType = set.dataType - def eval(input: Row): Any = { + override def dataType: DataType = set.dataType + + override def eval(input: Row): Any = { val itemEval = item.eval(input) val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] @@ -67,7 +68,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { } } - override def toString = s"$set += $item" + override def toString: String = s"$set += $item" } /** @@ -77,13 +78,13 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { case class CombineSets(left: Expression, right: Expression) extends BinaryExpression { type EvaluatedType = Any - def nullable = left.nullable || right.nullable + override def nullable: Boolean = left.nullable || right.nullable - def dataType = left.dataType + override def dataType: DataType = left.dataType - def symbol = "++=" + override def symbol: String = "++=" - def eval(input: Row): Any = { + override def eval(input: Row): Any = { val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]] if(leftEval != null) { val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]] @@ -109,16 +110,16 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres case class CountSet(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def nullable = child.nullable + override def nullable: Boolean = child.nullable - def dataType = LongType + override def dataType: DataType = LongType - def eval(input: Row): Any = { + override def eval(input: Row): Any = { val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]] if (childEval != null) { childEval.size.toLong } } - override def toString = s"$child.count()" + override def toString: String = s"$child.count()" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index f85ee0a9bb6d8..3cdca4e9dd2d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -33,8 +33,8 @@ trait StringRegexExpression { def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - def nullable: Boolean = left.nullable || right.nullable - def dataType: DataType = BooleanType + override def nullable: Boolean = left.nullable || right.nullable + override def dataType: DataType = BooleanType // try cache the pattern for Literal private lazy val cache: Pattern = right match { @@ -98,11 +98,11 @@ trait CaseConversionExpression { case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - def symbol = "LIKE" + override def symbol: String = "LIKE" // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character - override def escape(v: String) = + override def escape(v: String): String = if (!v.isEmpty) { "(?s)" + (' ' +: v.init).zip(v).flatMap { case (prev, '\\') => "" @@ -129,7 +129,7 @@ case class Like(left: Expression, right: Expression) case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - def symbol = "RLIKE" + override def symbol: String = "RLIKE" override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } @@ -141,7 +141,7 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE override def convert(v: String): String = v.toUpperCase() - override def toString() = s"Upper($child)" + override def toString: String = s"Upper($child)" } /** @@ -151,7 +151,7 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE override def convert(v: String): String = v.toLowerCase() - override def toString() = s"Lower($child)" + override def toString: String = s"Lower($child)" } /** A base trait for functions that compare two strings, returning a boolean. */ @@ -160,7 +160,7 @@ trait StringComparison { type EvaluatedType = Any - def nullable: Boolean = left.nullable || right.nullable + override def nullable: Boolean = left.nullable || right.nullable override def dataType: DataType = BooleanType def compare(l: String, r: String): Boolean @@ -175,9 +175,9 @@ trait StringComparison { } } - def symbol: String = nodeName + override def symbol: String = nodeName - override def toString() = s"$nodeName($left, $right)" + override def toString: String = s"$nodeName($left, $right)" } /** @@ -185,7 +185,7 @@ trait StringComparison { */ case class Contains(left: Expression, right: Expression) extends BinaryExpression with StringComparison { - override def compare(l: String, r: String) = l.contains(r) + override def compare(l: String, r: String): Boolean = l.contains(r) } /** @@ -193,7 +193,7 @@ case class Contains(left: Expression, right: Expression) */ case class StartsWith(left: Expression, right: Expression) extends BinaryExpression with StringComparison { - def compare(l: String, r: String) = l.startsWith(r) + override def compare(l: String, r: String): Boolean = l.startsWith(r) } /** @@ -201,7 +201,7 @@ case class StartsWith(left: Expression, right: Expression) */ case class EndsWith(left: Expression, right: Expression) extends BinaryExpression with StringComparison { - def compare(l: String, r: String) = l.endsWith(r) + override def compare(l: String, r: String): Boolean = l.endsWith(r) } /** @@ -212,17 +212,17 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends type EvaluatedType = Any - override def foldable = str.foldable && pos.foldable && len.foldable + override def foldable: Boolean = str.foldable && pos.foldable && len.foldable - def nullable: Boolean = str.nullable || pos.nullable || len.nullable - def dataType: DataType = { + override def nullable: Boolean = str.nullable || pos.nullable || len.nullable + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved") } if (str.dataType == BinaryType) str.dataType else StringType } - override def children = str :: pos :: len :: Nil + override def children: Seq[Expression] = str :: pos :: len :: Nil @inline def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) @@ -267,7 +267,8 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends } } - override def toString = len match { + override def toString: String = len match { + // TODO: This is broken because max is not an integer value. case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)" case _ => s"SUBSTR($str, $pos, $len)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 74edaacc4f609..c23d3b61887c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -141,7 +141,7 @@ object ColumnPruning extends Rule[LogicalPlan] { condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ - def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) + def pruneJoinChild(c: LogicalPlan): LogicalPlan = prunedChild(c, allReferences) Project(projectList, Join(pruneJoinChild(left), pruneJoinChild(right), joinType, condition)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index b4c445b3badf1..9c8c643f7d17a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -91,16 +91,18 @@ object PhysicalOperation extends PredicateHelper { (None, Nil, other, Map.empty) } - def collectAliases(fields: Seq[Expression]) = fields.collect { + def collectAliases(fields: Seq[Expression]): Map[Attribute, Expression] = fields.collect { case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child }.toMap - def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = expr.transform { - case a @ Alias(ref: AttributeReference, name) => - aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a) + def substitute(aliases: Map[Attribute, Expression])(expr: Expression): Expression = { + expr.transform { + case a @ Alias(ref: AttributeReference, name) => + aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a) - case a: AttributeReference => - aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a) + case a: AttributeReference => + aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index bd9291e9ba5d7..02f7c26a8ab6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -71,7 +71,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpressionDown(e: Expression) = { + @inline def transformExpressionDown(e: Expression): Expression = { val newE = e.transformDown(rule) if (newE.fastEquals(e)) { e @@ -104,7 +104,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpressionUp(e: Expression) = { + @inline def transformExpressionUp(e: Expression): Expression = { val newE = e.transformUp(rule) if (newE.fastEquals(e)) { e @@ -165,5 +165,5 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" - override def simpleString = statePrefix + super.simpleString + override def simpleString: String = statePrefix + super.simpleString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 384fe53a68362..4d9e41a2b5d85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { - def output = projectList.map(_.toAttribute) + override def output: Seq[Attribute] = projectList.map(_.toAttribute) override lazy val resolved: Boolean = { val containsAggregatesOrGenerators = projectList.exists ( _.collect { @@ -66,19 +66,19 @@ case class Generate( } } - override def output = + override def output: Seq[Attribute] = if (join) child.output ++ generatorOutput else generatorOutput } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { // TODO: These aren't really the same attributes as nullability etc might change. - override def output = left.output + override def output: Seq[Attribute] = left.output - override lazy val resolved = + override lazy val resolved: Boolean = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } @@ -94,7 +94,7 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def output = { + override def output: Seq[Attribute] = { joinType match { case LeftSemi => left.output @@ -109,7 +109,7 @@ case class Join( } } - def selfJoinResolved = left.outputSet.intersect(right.outputSet).isEmpty + private def selfJoinResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty // Joins are only resolved if they don't introduce ambiguious expression ids. override lazy val resolved: Boolean = { @@ -118,7 +118,7 @@ case class Join( } case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - def output = left.output + override def output: Seq[Attribute] = left.output } case class InsertIntoTable( @@ -128,10 +128,10 @@ case class InsertIntoTable( overwrite: Boolean) extends LogicalPlan { - override def children = child :: Nil - override def output = child.output + override def children: Seq[LogicalPlan] = child :: Nil + override def output: Seq[Attribute] = child.output - override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) } @@ -143,14 +143,14 @@ case class CreateTableAsSelect[T]( child: LogicalPlan, allowExisting: Boolean, desc: Option[T] = None) extends UnaryNode { - override def output = Seq.empty[Attribute] - override lazy val resolved = databaseName != None && childrenResolved + override def output: Seq[Attribute] = Seq.empty[Attribute] + override lazy val resolved: Boolean = databaseName != None && childrenResolved } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } /** @@ -163,7 +163,7 @@ case class Sort( order: Seq[SortOrder], global: Boolean, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Aggregate( @@ -172,7 +172,7 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { - override def output = aggregateExpressions.map(_.toAttribute) + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) } /** @@ -199,7 +199,7 @@ trait GroupingAnalytics extends UnaryNode { def groupByExprs: Seq[Expression] def aggregations: Seq[NamedExpression] - override def output = aggregations.map(_.toAttribute) + override def output: Seq[Attribute] = aggregations.map(_.toAttribute) } /** @@ -264,7 +264,7 @@ case class Rollup( gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output override lazy val statistics: Statistics = { val limit = limitExpr.eval(null).asInstanceOf[Int] @@ -274,21 +274,21 @@ case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { - override def output = child.output.map(_.withQualifiers(alias :: Nil)) + override def output: Seq[Attribute] = child.output.map(_.withQualifiers(alias :: Nil)) } case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Distinct(child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case object NoRelation extends LeafNode { - override def output = Nil + override def output: Seq[Attribute] = Nil /** * Computes [[Statistics]] for this plan. The default implementation assumes the output @@ -301,5 +301,5 @@ case object NoRelation extends LeafNode { } case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - override def output = left.output + override def output: Seq[Attribute] = left.output } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 72b0c5c8e7a26..e737418d9c3bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} /** * Performs a physical redistribution of the data. Used when the consumer of the query @@ -26,14 +26,11 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} abstract class RedistributeData extends UnaryNode { self: Product => - def output = child.output + override def output: Seq[Attribute] = child.output } case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) - extends RedistributeData { -} + extends RedistributeData case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) - extends RedistributeData { -} - + extends RedistributeData diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 3c3d7a3119064..288c11f69fe22 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder} -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{DataType, IntegerType} /** * Specifies how tuples that share common expressions will be distributed when a query is executed @@ -72,7 +72,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "a single partition.") // TODO: This is not really valid... - def clustering = ordering.map(_.child).toSet + def clustering: Set[Expression] = ordering.map(_.child).toSet } sealed trait Partitioning { @@ -113,7 +113,7 @@ case object SinglePartition extends Partitioning { override def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case SinglePartition => true case _ => false } @@ -124,7 +124,7 @@ case object BroadcastPartitioning extends Partitioning { override def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case SinglePartition => true case _ => false } @@ -139,9 +139,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - override def children = expressions - override def nullable = false - override def dataType = IntegerType + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: DataType = IntegerType private[this] lazy val clusteringSet = expressions.toSet @@ -152,7 +152,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false @@ -178,9 +178,9 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - override def children = ordering - override def nullable = false - override def dataType = IntegerType + override def children: Seq[SortOrder] = ordering + override def nullable: Boolean = false + override def dataType: DataType = IntegerType private[this] lazy val clusteringSet = ordering.map(_.child).toSet @@ -194,7 +194,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 0ae9f6b2965d4..a2df51e598a2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -36,12 +36,12 @@ object CurrentOrigin { override def initialValue: Origin = Origin() } - def get = value.get() - def set(o: Origin) = value.set(o) + def get: Origin = value.get() + def set(o: Origin): Unit = value.set(o) - def reset() = value.set(Origin()) + def reset(): Unit = value.set(Origin()) - def setPosition(line: Int, start: Int) = { + def setPosition(line: Int, start: Int): Unit = { value.set( value.get.copy(line = Some(line), startPosition = Some(start))) } @@ -57,7 +57,7 @@ object CurrentOrigin { abstract class TreeNode[BaseType <: TreeNode[BaseType]] { self: BaseType with Product => - val origin = CurrentOrigin.get + val origin: Origin = CurrentOrigin.get /** Returns a Seq of the children of this node */ def children: Seq[BaseType] @@ -340,12 +340,12 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } /** Returns the name of this type of TreeNode. Defaults to the class name. */ - def nodeName = getClass.getSimpleName + def nodeName: String = getClass.getSimpleName /** * The arguments that should be included in the arg string. Defaults to the `productIterator`. */ - protected def stringArgs = productIterator + protected def stringArgs: Iterator[Any] = productIterator /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { @@ -357,18 +357,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { }.mkString(", ") /** String representation of this node without any children */ - def simpleString = s"$nodeName $argString".trim + def simpleString: String = s"$nodeName $argString".trim override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ - def treeString = generateTreeString(0, new StringBuilder).toString + def treeString: String = generateTreeString(0, new StringBuilder).toString /** * Returns a string representation of the nodes in this tree, where each operator is numbered. * The numbers can be used with [[trees.TreeNode.apply apply]] to easily access specific subtrees. */ - def numberedTreeString = + def numberedTreeString: String = treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n") /** @@ -420,14 +420,14 @@ trait BinaryNode[BaseType <: TreeNode[BaseType]] { def left: BaseType def right: BaseType - def children = Seq(left, right) + def children: Seq[BaseType] = Seq(left, right) } /** * A [[TreeNode]] with no children. */ trait LeafNode[BaseType <: TreeNode[BaseType]] { - def children = Nil + def children: Seq[BaseType] = Nil } /** @@ -435,6 +435,5 @@ trait LeafNode[BaseType <: TreeNode[BaseType]] { */ trait UnaryNode[BaseType <: TreeNode[BaseType]] { def child: BaseType - def children = child :: Nil + def children: Seq[BaseType] = child :: Nil } - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 79a8e06d4b4d4..ea6aa1850db4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -41,11 +41,11 @@ package object trees extends Logging { * A [[TreeNode]] companion for reference equality for Hash based Collection. */ class TreeNodeRef(val obj: TreeNode[_]) { - override def equals(o: Any) = o match { + override def equals(o: Any): Boolean = o match { case that: TreeNodeRef => that.obj.eq(obj) case _ => false } - override def hashCode = if (obj == null) 0 else obj.hashCode + override def hashCode: Int = if (obj == null) 0 else obj.hashCode } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index feed50f9a2a2d..c86214a2aa944 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.Utils package object util { - def fileToString(file: File, encoding: String = "UTF-8") = { + def fileToString(file: File, encoding: String = "UTF-8"): String = { val inStream = new FileInputStream(file) val outStream = new ByteArrayOutputStream try { @@ -45,7 +45,7 @@ package object util { def resourceToString( resource:String, encoding: String = "UTF-8", - classLoader: ClassLoader = Utils.getSparkClassLoader) = { + classLoader: ClassLoader = Utils.getSparkClassLoader): String = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { @@ -93,7 +93,7 @@ package object util { new String(out.toByteArray) } - def stringOrNull(a: AnyRef) = if (a == null) null else a.toString + def stringOrNull(a: AnyRef): String = if (a == null) null else a.toString def benchmark[A](f: => A): A = { val startTime = System.nanoTime() From 6948ab6f8ba836446b005f2cf1cc4abc944c5053 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 24 Mar 2015 16:26:43 -0700 Subject: [PATCH 516/817] [SPARK-6088] Correct how tasks that get remote results are shown in UI. It would be great to fix this for 1.3. since the fix is surgical and it helps understandability for users. cc shivaram pwendell Author: Kay Ousterhout Closes #4839 from kayousterhout/SPARK-6088 and squashes the following commits: 3ab012c [Kay Ousterhout] Update getting result time incrementally, correctly set GET_RESULT status f346b49 [Kay Ousterhout] Typos 748ea6b [Kay Ousterhout] Fixed build failure 84d617c [Kay Ousterhout] [SPARK-6088] Correct how tasks that get remote results are shown in the UI. --- .../org/apache/spark/scheduler/TaskInfo.scala | 8 +++--- .../org/apache/spark/ui/jobs/StagePage.scala | 25 +++++++++++++------ 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6fa1f2c880f7a..132a9ced77700 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -81,9 +81,11 @@ class TaskInfo( def status: String = { if (running) { - "RUNNING" - } else if (gettingResult) { - "GET RESULT" + if (gettingResult) { + "GET RESULT" + } else { + "RUNNING" + } } else if (failed) { "FAILED" } else if (successful) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index e03442894c5cc..797c9404bc449 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -269,11 +269,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(serializationTimes) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => - if (info.gettingResultTime > 0) { - (info.finishTime - info.gettingResultTime).toDouble - } else { - 0.0 - } + getGettingResultTime(info).toDouble } val gettingResultQuantiles = @@ -464,7 +460,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) - val gettingResultTime = info.gettingResultTime + val gettingResultTime = getGettingResultTime(info) val maybeAccumulators = info.accumulables val accumulatorsReadable = maybeAccumulators.map{acc => s"${acc.name}: ${acc.update.get}"} @@ -627,6 +623,19 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {errorSummary}{details} } + private def getGettingResultTime(info: TaskInfo): Long = { + if (info.gettingResultTime > 0) { + if (info.finishTime > 0) { + info.finishTime - info.gettingResultTime + } else { + // The task is still fetching the result. + System.currentTimeMillis - info.gettingResultTime + } + } else { + 0L + } + } + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { val totalExecutionTime = if (info.gettingResult) { @@ -638,6 +647,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } val executorOverhead = (metrics.executorDeserializeTime + metrics.resultSerializationTime) - math.max(0, totalExecutionTime - metrics.executorRunTime - executorOverhead) + math.max( + 0, + totalExecutionTime - metrics.executorRunTime - executorOverhead - getGettingResultTime(info)) } } From d8ccf655f344eed65cdaf5d9252f1b565b8406ca Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 24 Mar 2015 16:29:40 -0700 Subject: [PATCH 517/817] [SPARK-3570] Include time to open files in shuffle write time. Opening shuffle files can be very significant when the disk is contended, especially when using ext3. While writing data to a file can avoid hitting disk (and instead hit the buffer cache), opening a file always involves writing some metadata about the file to disk, so the open time can be a very significant portion of the shuffle write time. In one job I ran recently, the time to write shuffle data to the file was only 4ms for each task, but the time to open the file was about 100x as long (~400ms). When we add metrics about spilled data (#2504), we should ensure that the file open time is also included there. Author: Kay Ousterhout Closes #4550 from kayousterhout/SPARK-3570 and squashes the following commits: ea3a4ae [Kay Ousterhout] Added comment about excluded open time fdc5185 [Kay Ousterhout] Improved comment 42b7e43 [Kay Ousterhout] Fixed parens for nanotime 2423555 [Kay Ousterhout] [SPARK-3570] Include time to open files in shuffle write time. --- .../org/apache/spark/shuffle/FileShuffleBlockManager.scala | 4 ++++ .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 3 +++ .../org/apache/spark/util/collection/ExternalSorter.scala | 5 +++++ 3 files changed, 12 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 660df00bc32f5..d0178dfde6935 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -112,6 +112,7 @@ class FileShuffleBlockManager(conf: SparkConf) private val shuffleState = shuffleStates(shuffleId) private var fileGroup: ShuffleFileGroup = null + val openStartTime = System.nanoTime val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => @@ -135,6 +136,9 @@ class FileShuffleBlockManager(conf: SparkConf) blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize, writeMetrics) } } + // Creating the file to write to and creating a disk writer both involve interacting with + // the disk, so should be included in the shuffle write time. + writeMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) override def releaseWriters(success: Boolean) { if (consolidateShuffleFiles) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index fa2e617762f55..55ea0f17b156a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -63,6 +63,9 @@ private[spark] class SortShuffleWriter[K, V, C]( sorter.insertAll(records) } + // Don't bother including the time to open the merged output file in the shuffle write time, + // because it just opens a single file, so is typically too fast to measure accurately + // (see SPARK-3570). val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) val blockId = shuffleBlockManager.consolidateId(dep.shuffleId, mapId) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 3262e670c2030..b962c101c91da 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -352,6 +352,7 @@ private[spark] class ExternalSorter[K, V, C]( // Create our file writers if we haven't done so yet if (partitionWriters == null) { curWriteMetrics = new ShuffleWriteMetrics() + val openStartTime = System.nanoTime partitionWriters = Array.fill(numPartitions) { // Because these files may be read during shuffle, their compression must be controlled by // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use @@ -359,6 +360,10 @@ private[spark] class ExternalSorter[K, V, C]( val (blockId, file) = diskBlockManager.createTempShuffleBlock() blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics).open() } + // Creating the file to write to and creating a disk writer both involve interacting with + // the disk, and can take a long time in aggregate when we open many files, so should be + // included in the shuffle write time. + curWriteMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) } // No need to sort stuff, just write each element out From f7c3668ee62d5b87a0fdf64614aa45c7409ff585 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 24 Mar 2015 16:41:31 -0700 Subject: [PATCH 518/817] Revert "[SPARK-5771][UI][hotfix] Change Requested Cores into * if default cores is not set" This reverts commit 12135e90549f957962899487cd5eb95badd8976d. --- .../scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 46509e39c0f23..9ab3c8f8fed9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -218,7 +218,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } } - {if (app.requestedCores == Int.MaxValue) "*" else app.requestedCores} + {app.requestedCores} {Utils.megabytesToString(app.desc.memoryPerSlave)} From dd907d1a9df52ffe0a8e1e8dacd837019d11742c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 24 Mar 2015 16:49:27 -0700 Subject: [PATCH 519/817] Revert "[SPARK-5771] Number of Cores in Completed Applications of Standalone Master Web Page always be 0 if sc.stop() is called" This reverts commit dd077abf2e2949fdfec31074b760b587f00efcf2. Conflicts: core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala --- .../spark/deploy/master/ApplicationInfo.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 32 ++++--------------- 2 files changed, 7 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 536aedb6f9fe9..f979ffa16641a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -91,7 +91,7 @@ private[deploy] class ApplicationInfo( } } - private[master] val requestedCores = desc.maxCores.getOrElse(defaultCores) + private val requestedCores = desc.maxCores.getOrElse(defaultCores) private[master] def coresLeft: Int = requestedCores - coresGranted diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 9ab3c8f8fed9c..45412a35e9a7d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -75,16 +75,12 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val activeAppHeaders = Seq("Application ID", "Name", "Cores in Use", - "Cores Requested", "Memory per Node", "Submitted Time", "User", "State", "Duration") + val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per Node", "Submitted Time", + "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse - val activeAppsTable = UIUtils.listingTable(activeAppHeaders, activeAppRow, activeApps) - - val completedAppHeaders = Seq("Application ID", "Name", "Cores Requested", "Memory per Node", - "Submitted Time", "User", "State", "Duration") + val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse - val completedAppsTable = UIUtils.listingTable(completedAppHeaders, completeAppRow, - completedApps) + val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class") @@ -191,7 +187,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { val killLinkUri = s"app/kill?id=${app.id}&terminate=true" @@ -201,7 +197,6 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { (kill) } - {app.id} @@ -210,15 +205,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.desc.name} - { - if (active) { - - {app.coresGranted} - - } - } - {app.requestedCores} + {app.coresGranted} {Utils.megabytesToString(app.desc.memoryPerSlave)} @@ -230,14 +218,6 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def activeAppRow(app: ApplicationInfo): Seq[Node] = { - appRow(app, active = true) - } - - private def completeAppRow(app: ApplicationInfo): Seq[Node] = { - appRow(app, active = false) - } - private def driverRow(driver: DriverInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (driver.state == DriverState.RUNNING || From 05c2214b41f4c0fd17b6f0c62e26398b963efe64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christophe=20Pr=C3=A9aud?= Date: Tue, 24 Mar 2015 17:05:49 -0700 Subject: [PATCH 520/817] [SPARK-6469] Improving documentation on YARN local directories usage MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Clarify the local directories usage in YARN Author: Christophe Préaud Closes #5165 from preaudc/yarn-doc-local-dirs and squashes the following commits: 6912b90 [Christophe Préaud] Fix some formatting issues. 4fa8ec2 [Christophe Préaud] Merge remote-tracking branch 'upstream/master' into yarn-doc-local-dirs eaaf519 [Christophe Préaud] Clarify the local directories usage in YARN 436fb7d [Christophe Préaud] Revert "Clarify the local directories usage in YARN" 876ae5e [Christophe Préaud] Clarify the local directories usage in YARN 608dbfa [Christophe Préaud] Merge remote-tracking branch 'upstream/master' a49a2ce [Christophe Préaud] Merge remote-tracking branch 'upstream/master' 9ba89ca [Christophe Préaud] Ensure that files are fetched atomically 54419ae [Christophe Préaud] Merge remote-tracking branch 'upstream/master' c6a5590 [Christophe Préaud] Revert commit 8ea871f8130b2490f1bad7374a819bf56f0ccbbd 7456a33 [Christophe Préaud] Merge remote-tracking branch 'upstream/master' 8ea871f [Christophe Préaud] Ensure that files are fetched atomically --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 68b1aeb8ebd01..d9f3eb2b74b18 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -274,6 +274,6 @@ If you need a reference to the proper location to put log files in the YARN so t # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. -- The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. +- In `yarn-cluster` mode, the local directories used by the Spark executors and the Spark driver will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. In `yarn-client` mode, the Spark executors will use the local directories configured for YARN while the Spark driver will use those defined in `spark.local.dir`. This is because the Spark driver does not run on the YARN cluster in `yarn-client` mode, only the Spark executors do. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `yarn-cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. From 6930e965e26d39fa6c26ae67a08b4c4d0368d556 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Mar 2015 17:06:22 -0700 Subject: [PATCH 521/817] [SPARK-6512] add contains to OpenHashMap Add `contains` to test whether a key exists in an OpenHashMap. rxin Author: Xiangrui Meng Closes #5171 from mengxr/openhashmap-contains and squashes the following commits: d6e6f1f [Xiangrui Meng] add contains to primitivekeyopenhashmap 748a69b [Xiangrui Meng] add contains to OpenHashMap --- .../org/apache/spark/util/collection/OpenHashMap.scala | 9 +++++++++ .../util/collection/PrimitiveKeyOpenHashMap.scala | 5 +++++ .../spark/util/collection/OpenHashMapSuite.scala | 10 ++++++++++ .../util/collection/PrimitiveKeyOpenHashMapSuite.scala | 7 +++++++ 4 files changed, 31 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index c52591b352340..efc2482c74ddf 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -53,6 +53,15 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( override def size: Int = if (haveNullValue) _keySet.size + 1 else _keySet.size + /** Tests whether this map contains a binding for a key. */ + def contains(k: K): Boolean = { + if (k == null) { + haveNullValue + } else { + _keySet.getPos(k) != OpenHashSet.INVALID_POS + } + } + /** Get the value for a given key */ def apply(k: K): V = { if (k == null) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index 61e22642761f0..b4ec4ea521253 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -48,6 +48,11 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, override def size: Int = _keySet.size + /** Tests whether this map contains a binding for a key. */ + def contains(k: K): Boolean = { + _keySet.getPos(k) != OpenHashSet.INVALID_POS + } + /** Get the value for a given key */ def apply(k: K): V = { val pos = _keySet.getPos(k) diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 6a70877356409..ef890d2ba60f3 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -176,4 +176,14 @@ class OpenHashMapSuite extends FunSuite with Matchers { assert(map(i.toString) === i.toString) } } + + test("contains") { + val map = new OpenHashMap[String, Int](2) + map("a") = 1 + assert(map.contains("a")) + assert(!map.contains("b")) + assert(!map.contains(null)) + map(null) = 0 + assert(map.contains(null)) + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index 8c7df7d73dcd3..caf378fec8b3e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -118,4 +118,11 @@ class PrimitiveKeyOpenHashMapSuite extends FunSuite with Matchers { assert(map(i.toLong) === i.toString) } } + + test("contains") { + val map = new PrimitiveKeyOpenHashMap[Int, Int](1) + map(0) = 0 + assert(map.contains(0)) + assert(!map.contains(1)) + } } From 94598653bc772e71709163db3fed4048aa7f5f75 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 24 Mar 2015 17:08:25 -0700 Subject: [PATCH 522/817] [SPARK-6428][Streaming] Added explicit types for all public methods. Author: Reynold Xin Closes #5110 from rxin/streaming-explicit-type and squashes the following commits: 2c2db32 [Reynold Xin] [SPARK-6428][Streaming] Added explicit types for all public methods. --- .../apache/spark/streaming/Checkpoint.scala | 6 ++-- .../apache/spark/streaming/DStreamGraph.scala | 6 ++-- .../org/apache/spark/streaming/Duration.scala | 15 ++++----- .../org/apache/spark/streaming/Interval.scala | 10 +++--- .../spark/streaming/StreamingContext.scala | 5 +-- .../streaming/api/java/JavaDStreamLike.scala | 3 +- .../streaming/api/python/PythonDStream.scala | 15 +++++---- .../spark/streaming/dstream/DStream.scala | 20 ++++++----- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 4 +-- .../streaming/dstream/FilteredDStream.scala | 2 +- .../dstream/FlatMapValuedDStream.scala | 2 +- .../streaming/dstream/FlatMappedDStream.scala | 2 +- .../streaming/dstream/ForEachDStream.scala | 2 +- .../streaming/dstream/GlommedDStream.scala | 2 +- .../streaming/dstream/InputDStream.scala | 2 +- .../dstream/MapPartitionedDStream.scala | 2 +- .../streaming/dstream/MapValuedDStream.scala | 2 +- .../streaming/dstream/MappedDStream.scala | 2 +- .../dstream/ReducedWindowedDStream.scala | 4 +-- .../streaming/dstream/ShuffledDStream.scala | 2 +- .../streaming/dstream/StateDStream.scala | 2 +- .../dstream/TransformedDStream.scala | 2 +- .../streaming/dstream/UnionDStream.scala | 6 ++-- .../streaming/dstream/WindowedDStream.scala | 2 +- .../rdd/WriteAheadLogBackedBlockRDD.scala | 6 ++-- .../streaming/receiver/ActorReceiver.scala | 18 +++++----- .../streaming/receiver/BlockGenerator.scala | 2 +- .../spark/streaming/receiver/Receiver.scala | 2 +- .../receiver/ReceiverSupervisor.scala | 4 +-- .../receiver/ReceiverSupervisorImpl.scala | 6 ++-- .../spark/streaming/scheduler/Job.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 8 ++--- .../streaming/scheduler/JobScheduler.scala | 2 +- .../spark/streaming/scheduler/JobSet.scala | 11 +++---- .../streaming/scheduler/ReceiverTracker.scala | 11 +++---- .../ui/StreamingJobProgressListener.scala | 33 +++++++++++-------- .../spark/streaming/util/RawTextHelper.scala | 14 ++++---- 38 files changed, 127 insertions(+), 114 deletions(-) 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 db64e11e16304..f73b463d07779 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -67,12 +67,12 @@ object Checkpoint extends Logging { val REGEX = (PREFIX + """([\d]+)([\w\.]*)""").r /** Get the checkpoint file for the given checkpoint time */ - def checkpointFile(checkpointDir: String, checkpointTime: Time) = { + def checkpointFile(checkpointDir: String, checkpointTime: Time): Path = { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds) } /** Get the checkpoint backup file for the given checkpoint time */ - def checkpointBackupFile(checkpointDir: String, checkpointTime: Time) = { + def checkpointBackupFile(checkpointDir: String, checkpointTime: Time): Path = { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } @@ -232,6 +232,8 @@ object CheckpointReader extends Logging { def read(checkpointDir: String, conf: SparkConf, hadoopConf: Configuration): Option[Checkpoint] = { val checkpointPath = new Path(checkpointDir) + + // TODO(rxin): Why is this a def?! def fs = checkpointPath.getFileSystem(hadoopConf) // Try to find the checkpoint files diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 0e285d6088ec1..175140481e5ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -100,11 +100,11 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } } - def getInputStreams() = this.synchronized { inputStreams.toArray } + def getInputStreams(): Array[InputDStream[_]] = this.synchronized { inputStreams.toArray } - def getOutputStreams() = this.synchronized { outputStreams.toArray } + def getOutputStreams(): Array[DStream[_]] = this.synchronized { outputStreams.toArray } - def getReceiverInputStreams() = this.synchronized { + def getReceiverInputStreams(): Array[ReceiverInputDStream[_]] = this.synchronized { inputStreams.filter(_.isInstanceOf[ReceiverInputDStream[_]]) .map(_.asInstanceOf[ReceiverInputDStream[_]]) .toArray diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index a0d8fb5ab93ec..3249bb348981f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -55,7 +55,6 @@ case class Duration (private val millis: Long) { def div(that: Duration): Double = this / that - def isMultipleOf(that: Duration): Boolean = (this.millis % that.millis == 0) @@ -71,7 +70,7 @@ case class Duration (private val millis: Long) { def milliseconds: Long = millis - def prettyPrint = Utils.msDurationToString(millis) + def prettyPrint: String = Utils.msDurationToString(millis) } @@ -80,7 +79,7 @@ case class Duration (private val millis: Long) { * a given number of milliseconds. */ object Milliseconds { - def apply(milliseconds: Long) = new Duration(milliseconds) + def apply(milliseconds: Long): Duration = new Duration(milliseconds) } /** @@ -88,7 +87,7 @@ object Milliseconds { * a given number of seconds. */ object Seconds { - def apply(seconds: Long) = new Duration(seconds * 1000) + def apply(seconds: Long): Duration = new Duration(seconds * 1000) } /** @@ -96,7 +95,7 @@ object Seconds { * a given number of minutes. */ object Minutes { - def apply(minutes: Long) = new Duration(minutes * 60000) + def apply(minutes: Long): Duration = new Duration(minutes * 60000) } // Java-friendlier versions of the objects above. @@ -107,16 +106,16 @@ object Durations { /** * @return [[org.apache.spark.streaming.Duration]] representing given number of milliseconds. */ - def milliseconds(milliseconds: Long) = Milliseconds(milliseconds) + def milliseconds(milliseconds: Long): Duration = Milliseconds(milliseconds) /** * @return [[org.apache.spark.streaming.Duration]] representing given number of seconds. */ - def seconds(seconds: Long) = Seconds(seconds) + def seconds(seconds: Long): Duration = Seconds(seconds) /** * @return [[org.apache.spark.streaming.Duration]] representing given number of minutes. */ - def minutes(minutes: Long) = Minutes(minutes) + def minutes(minutes: Long): Duration = Minutes(minutes) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index ad4f3fdd14ad6..3f5be785e1b1a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -39,18 +39,18 @@ class Interval(val beginTime: Time, val endTime: Time) { this.endTime < that.endTime } - def <= (that: Interval) = (this < that || this == that) + def <= (that: Interval): Boolean = (this < that || this == that) - def > (that: Interval) = !(this <= that) + def > (that: Interval): Boolean = !(this <= that) - def >= (that: Interval) = !(this < that) + def >= (that: Interval): Boolean = !(this < that) - override def toString = "[" + beginTime + ", " + endTime + "]" + override def toString: String = "[" + beginTime + ", " + endTime + "]" } private[streaming] object Interval { - def currentInterval(duration: Duration): Interval = { + def currentInterval(duration: Duration): Interval = { val time = new Time(System.currentTimeMillis) val intervalBegin = time.floor(duration) new Interval(intervalBegin, intervalBegin + duration) 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 543224d4b07bc..f57f295874645 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -188,7 +188,7 @@ class StreamingContext private[streaming] ( /** * Return the associated Spark context */ - def sparkContext = sc + def sparkContext: SparkContext = sc /** * Set each DStreams in this context to remember RDDs it generated in the last given duration. @@ -596,7 +596,8 @@ object StreamingContext extends Logging { @deprecated("Replaced by implicit functions in the DStream companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) + : PairDStreamFunctions[K, V] = { DStream.toPairDStreamFunctions(stream)(kt, vt, ord) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 2eabdd9387913..73030e15c5661 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -415,8 +415,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T implicit val cmv2: ClassTag[V2] = fakeClassTag implicit val cmw: ClassTag[W] = fakeClassTag - def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] = + def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] = { transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd + } dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index 7053f47ec69a2..4c28654ef6413 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -176,11 +176,11 @@ private[python] abstract class PythonDStream( val func = new TransformFunction(pfunc) - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration - val asJavaDStream = JavaDStream.fromDStream(this) + val asJavaDStream: JavaDStream[Array[Byte]] = JavaDStream.fromDStream(this) } /** @@ -212,7 +212,7 @@ private[python] class PythonTransformed2DStream( val func = new TransformFunction(pfunc) - override def dependencies = List(parent, parent2) + override def dependencies: List[DStream[_]] = List(parent, parent2) override def slideDuration: Duration = parent.slideDuration @@ -223,7 +223,7 @@ private[python] class PythonTransformed2DStream( func(Some(rdd1), Some(rdd2), validTime) } - val asJavaDStream = JavaDStream.fromDStream(this) + val asJavaDStream: JavaDStream[Array[Byte]] = JavaDStream.fromDStream(this) } /** @@ -260,12 +260,15 @@ private[python] class PythonReducedWindowedDStream( extends PythonDStream(parent, preduceFunc) { super.persist(StorageLevel.MEMORY_ONLY) - override val mustCheckpoint = true - val invReduceFunc = new TransformFunction(pinvReduceFunc) + override val mustCheckpoint: Boolean = true + + val invReduceFunc: TransformFunction = new TransformFunction(pinvReduceFunc) def windowDuration: Duration = _windowDuration + override def slideDuration: Duration = _slideDuration + override def parentRememberDuration: Duration = rememberDuration + windowDuration override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { 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 b874f561c12eb..795c5aa6d585b 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 @@ -104,7 +104,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def parentRememberDuration = rememberDuration /** Return the StreamingContext associated with this DStream */ - def context = ssc + def context: StreamingContext = ssc /* Set the creation call site */ private[streaming] val creationSite = DStream.getCreationSite() @@ -619,14 +619,16 @@ abstract class DStream[T: ClassTag] ( * operator, so this DStream will be registered as an output stream and there materialized. */ def print(num: Int) { - def foreachFunc = (rdd: RDD[T], time: Time) => { - val firstNum = rdd.take(num + 1) - println ("-------------------------------------------") - println ("Time: " + time) - println ("-------------------------------------------") - firstNum.take(num).foreach(println) - if (firstNum.size > num) println("...") - println() + def foreachFunc: (RDD[T], Time) => Unit = { + (rdd: RDD[T], time: Time) => { + val firstNum = rdd.take(num + 1) + println("-------------------------------------------") + println("Time: " + time) + println("-------------------------------------------") + firstNum.take(num).foreach(println) + if (firstNum.size > num) println("...") + println() + } } new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 0dc72790fbdbd..39fd21342813e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -114,7 +114,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } } - override def toString() = { + override def toString: String = { "[\n" + currentCheckpointFiles.size + " checkpoint files \n" + currentCheckpointFiles.mkString("\n") + "\n]" } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 22de8c02e63c8..66d519171fd76 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -298,7 +298,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] + private def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] override def update(time: Time) { hadoopFiles.clear() @@ -320,7 +320,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( } } - override def toString() = { + override def toString: String = { "[\n" + hadoopFiles.size + " file sets\n" + hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index c81534ae584ea..fcd5216f101af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -27,7 +27,7 @@ class FilteredDStream[T: ClassTag]( filterFunc: T => Boolean ) extends DStream[T](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 658623455498c..9d09a3baf37ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -28,7 +28,7 @@ class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( flatMapValueFunc: V => TraversableOnce[U] ) extends DStream[(K, U)](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index c7bb2833eabb8..475ea2d2d4f38 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -27,7 +27,7 @@ class FlatMappedDStream[T: ClassTag, U: ClassTag]( flatMapFunc: T => Traversable[U] ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 1361c30395b57..685a32e1d280d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -28,7 +28,7 @@ class ForEachDStream[T: ClassTag] ( foreachFunc: (RDD[T], Time) => Unit ) extends DStream[Unit](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index a9bb51f054048..dbb295fe54f71 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -25,7 +25,7 @@ private[streaming] class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index aa1993f0580a8..e652702e213ef 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -61,7 +61,7 @@ abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext) } } - override def dependencies = List() + override def dependencies: List[DStream[_]] = List() override def slideDuration: Duration = { if (ssc == null) throw new Exception("ssc is null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 3d8ee29df1e82..5994bc1e23f2b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -28,7 +28,7 @@ class MapPartitionedDStream[T: ClassTag, U: ClassTag]( preservePartitioning: Boolean ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 7aea1f945d9db..954d2eb4a7b00 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -28,7 +28,7 @@ class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( mapValueFunc: V => U ) extends DStream[(K, U)](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index 02704a8d1c2e0..fa14b2e897c3e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -27,7 +27,7 @@ class MappedDStream[T: ClassTag, U: ClassTag] ( mapFunc: T => U ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index c0a5af0b65cc3..1385ccbf56ee5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -52,7 +52,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( // Reduce each batch of data using reduceByKey which will be further reduced by window // by ReducedWindowedDStream - val reducedStream = parent.reduceByKey(reduceFunc, partitioner) + private val reducedStream = parent.reduceByKey(reduceFunc, partitioner) // Persist RDDs to memory by default as these RDDs are going to be reused. super.persist(StorageLevel.MEMORY_ONLY_SER) @@ -60,7 +60,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( def windowDuration: Duration = _windowDuration - override def dependencies = List(reducedStream) + override def dependencies: List[DStream[_]] = List(reducedStream) override def slideDuration: Duration = _slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 880a89bc36895..7757ccac09a58 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -33,7 +33,7 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( mapSideCombine: Boolean = true ) extends DStream[(K,C)] (parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index ebb04dd35b9a2..de8718d0a80fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -36,7 +36,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( super.persist(StorageLevel.MEMORY_ONLY_SER) - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 71b61856e23c0..5d46ca0715ffd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -32,7 +32,7 @@ class TransformedDStream[U: ClassTag] ( require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") - override def dependencies = parents.toList + override def dependencies: List[DStream[_]] = parents.toList override def slideDuration: Duration = parents.head.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index abbc40befa95b..9405dbaa12329 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -33,17 +33,17 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") - override def dependencies = parents.toList + override def dependencies: List[DStream[_]] = parents.toList override def slideDuration: Duration = parents.head.slideDuration override def compute(validTime: Time): Option[RDD[T]] = { val rdds = new ArrayBuffer[RDD[T]]() - parents.map(_.getOrCompute(validTime)).foreach(_ match { + parents.map(_.getOrCompute(validTime)).foreach { case Some(rdd) => rdds += rdd case None => throw new Exception("Could not generate RDD from a parent for unifying at time " + validTime) - }) + } if (rdds.size > 0) { Some(new UnionRDD(ssc.sc, rdds)) } else { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 775b6bfd065c0..899865a906c27 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -46,7 +46,7 @@ class WindowedDStream[T: ClassTag]( def windowDuration: Duration = _windowDuration - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = _slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index dd1e96334952f..93caa4ba35c7f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -117,8 +117,8 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( override def getPreferredLocations(split: Partition): Seq[String] = { val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] val blockLocations = getBlockIdLocations().get(partition.blockId) - def segmentLocations = HdfsUtils.getFileSegmentLocations( - partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig) - blockLocations.getOrElse(segmentLocations) + blockLocations.getOrElse( + HdfsUtils.getFileSegmentLocations( + partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index a7d63bd4f2dbf..cd309788a7717 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.receiver +import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration._ @@ -25,10 +26,10 @@ import scala.reflect.ClassTag import akka.actor._ import akka.actor.SupervisorStrategy.{Escalate, Restart} + import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.StorageLevel -import java.nio.ByteBuffer import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -149,13 +150,13 @@ private[streaming] class ActorReceiver[T: ClassTag]( class Supervisor extends Actor { override val supervisorStrategy = receiverSupervisorStrategy - val worker = context.actorOf(props, name) + private val worker = context.actorOf(props, name) logInfo("Started receiver worker at:" + worker.path) - val n: AtomicInteger = new AtomicInteger(0) - val hiccups: AtomicInteger = new AtomicInteger(0) + private val n: AtomicInteger = new AtomicInteger(0) + private val hiccups: AtomicInteger = new AtomicInteger(0) - def receive = { + override def receive: PartialFunction[Any, Unit] = { case IteratorData(iterator) => logDebug("received iterator") @@ -189,13 +190,12 @@ private[streaming] class ActorReceiver[T: ClassTag]( } } - def onStart() = { + def onStart(): Unit = { supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) - } - def onStop() = { + def onStop(): Unit = { supervisor ! PoisonPill } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index ee5e639b26d91..42514d8b47dcf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -120,7 +120,7 @@ private[streaming] class BlockGenerator( * `BlockGeneratorListener.onAddData` callback will be called. All received data items * will be periodically pushed into BlockManager. */ - def addDataWithCallback(data: Any, metadata: Any) = synchronized { + def addDataWithCallback(data: Any, metadata: Any): Unit = synchronized { waitToPush() currentBuffer += data listener.onAddData(data, metadata) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 5acf8a9a811ee..5b5a3fe648602 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -245,7 +245,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable * Get the unique identifier the receiver input stream that this * receiver is associated with. */ - def streamId = id + def streamId: Int = id /* * ================= diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 1f0244c251eba..4943f29395d12 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -162,13 +162,13 @@ private[streaming] abstract class ReceiverSupervisor( } /** Check if receiver has been marked for stopping */ - def isReceiverStarted() = { + def isReceiverStarted(): Boolean = { logDebug("state = " + receiverState) receiverState == Started } /** Check if receiver has been marked for stopping */ - def isReceiverStopped() = { + def isReceiverStopped(): Boolean = { logDebug("state = " + receiverState) receiverState == Stopped } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 7d29ed88cfcb4..8f2f1fef76874 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await -import akka.actor.{Actor, Props} +import akka.actor.{ActorRef, Actor, Props} import akka.pattern.ask import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration @@ -83,7 +83,7 @@ private[streaming] class ReceiverSupervisorImpl( private val actor = env.actorSystem.actorOf( Props(new Actor { - override def receive() = { + override def receive: PartialFunction[Any, Unit] = { case StopReceiver => logInfo("Received stop signal") stop("Stopped by driver", None) @@ -92,7 +92,7 @@ private[streaming] class ReceiverSupervisorImpl( cleanupOldBlocks(threshTime) } - def ref = self + def ref: ActorRef = self }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) /** Unique block ids if one wants to add blocks directly */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala index 7e0f6b2cdfc08..30cf87f5b7dd1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -36,5 +36,5 @@ class Job(val time: Time, func: () => _) { id = "streaming job " + time + "." + number } - override def toString = id + override def toString: String = id } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 59488dfb0f8c6..4946806d2ee95 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -82,7 +82,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (eventActor != null) return // generator has already been started eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - def receive = { + override def receive: PartialFunction[Any, Unit] = { case event: JobGeneratorEvent => processEvent(event) } }), "JobGenerator") @@ -111,8 +111,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val pollTime = 100 // To prevent graceful stop to get stuck permanently - def hasTimedOut = { - val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout + def hasTimedOut: Boolean = { + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeout if (timedOut) { logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") } @@ -133,7 +133,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Stopped generation timer") // Wait for the jobs to complete and checkpoints to be written - def haveAllBatchesBeenProcessed = { + def haveAllBatchesBeenProcessed: Boolean = { lastProcessedBatch != null && lastProcessedBatch.milliseconds == stopTime } logInfo("Waiting for jobs to be processed and checkpoints to be written") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 60bc099b27a4c..d6a93acbe711b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -56,7 +56,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logDebug("Starting JobScheduler") eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - def receive = { + override def receive: PartialFunction[Any, Unit] = { case event: JobSchedulerEvent => processEvent(event) } }), "JobScheduler") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index 8c15a75b1b0e0..5b134877d0b2d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -28,8 +28,7 @@ private[streaming] case class JobSet( time: Time, jobs: Seq[Job], - receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty - ) { + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted @@ -48,17 +47,17 @@ case class JobSet( if (hasCompleted) processingEndTime = System.currentTimeMillis() } - def hasStarted = processingStartTime > 0 + def hasStarted: Boolean = processingStartTime > 0 - def hasCompleted = incompleteJobs.isEmpty + def hasCompleted: Boolean = incompleteJobs.isEmpty // Time taken to process all the jobs from the time they started processing // (i.e. not including the time they wait in the streaming scheduler queue) - def processingDelay = processingEndTime - processingStartTime + def processingDelay: Long = processingEndTime - processingStartTime // Time taken to process all the jobs from the time they were submitted // (i.e. including the time they wait in the streaming scheduler queue) - def totalDelay = { + def totalDelay: Long = { processingEndTime - time.milliseconds } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index b36aeb341d25e..98900473138fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -72,7 +72,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private var actor: ActorRef = null /** Start the actor and receiver execution thread. */ - def start() = synchronized { + def start(): Unit = synchronized { if (actor != null) { throw new SparkException("ReceiverTracker already started") } @@ -86,7 +86,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } /** Stop the receiver execution thread. */ - def stop(graceful: Boolean) = synchronized { + def stop(graceful: Boolean): Unit = synchronized { if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers if (!skipReceiverLaunch) receiverExecutor.stop(graceful) @@ -201,7 +201,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Actor to receive messages from the receivers. */ private class ReceiverTrackerActor extends Actor { - def receive = { + override def receive: PartialFunction[Any, Unit] = { case RegisterReceiver(streamId, typ, host, receiverActor) => registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true @@ -244,16 +244,15 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (graceful) { val pollTime = 100 - def done = { receiverInfo.isEmpty && !running } logInfo("Waiting for receiver job to terminate gracefully") - while(!done) { + while (receiverInfo.nonEmpty || running) { Thread.sleep(pollTime) } logInfo("Waited for receiver job to terminate gracefully") } // Check if all the receivers have been deregistered or not - if (!receiverInfo.isEmpty) { + if (receiverInfo.nonEmpty) { logWarning("Not all of the receivers have deregistered, " + receiverInfo) } else { logInfo("All of the receivers have deregistered successfully") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 5ee53a5c5f561..e4bd067cacb77 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -17,9 +17,10 @@ package org.apache.spark.streaming.ui +import scala.collection.mutable.{Queue, HashMap} + import org.apache.spark.streaming.{Time, StreamingContext} import org.apache.spark.streaming.scheduler._ -import scala.collection.mutable.{Queue, HashMap} import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo @@ -59,11 +60,13 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } - override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { - runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + synchronized { + runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + } } - override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = synchronized { runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) @@ -72,19 +75,21 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { - waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) - runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) - completedaBatchInfos.enqueue(batchCompleted.batchInfo) - if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() - totalCompletedBatches += 1L - - batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => - totalProcessedRecords += infos.map(_.numRecords).sum + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + synchronized { + waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) + runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) + completedaBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + totalCompletedBatches += 1L + + batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => + totalProcessedRecords += infos.map(_.numRecords).sum + } } } - def numReceivers = synchronized { + def numReceivers: Int = synchronized { ssc.graph.getReceiverInputStreams().size } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index a73d6f3bf0661..4d968f8bfa7a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -18,9 +18,7 @@ package org.apache.spark.streaming.util import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.util.collection.OpenHashMap -import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { @@ -71,7 +69,7 @@ object RawTextHelper { var count = 0 while(data.hasNext) { - value = data.next + value = data.next() if (value != null) { count += 1 if (len == 0) { @@ -108,9 +106,13 @@ object RawTextHelper { } } - def add(v1: Long, v2: Long) = (v1 + v2) + def add(v1: Long, v2: Long): Long = { + v1 + v2 + } - def subtract(v1: Long, v2: Long) = (v1 - v2) + def subtract(v1: Long, v2: Long): Long = { + v1 - v2 + } - def max(v1: Long, v2: Long) = math.max(v1, v2) + def max(v1: Long, v2: Long): Long = math.max(v1, v2) } From c14ddd97ed662a8429b9b9078bd7c1a5a1dd3d6c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Mar 2015 18:58:27 -0700 Subject: [PATCH 523/817] [SPARK-6515] update OpenHashSet impl Though I don't see any bug in the existing code, the update in this PR makes it read better. rxin Author: Xiangrui Meng Closes #5176 from mengxr/SPARK-6515 and squashes the following commits: 134494d [Xiangrui Meng] update OpenHashSet impl --- .../spark/util/collection/OpenHashSet.scala | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index c80057f95e0b2..1501111a06655 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -122,7 +122,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( */ def addWithoutResize(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask - var i = 1 + var delta = 1 while (true) { if (!_bitset.get(pos)) { // This is a new key. @@ -134,14 +134,12 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( // Found an existing key. return pos } else { - val delta = i + // quadratic probing with values increase by 1, 2, 3, ... pos = (pos + delta) & _mask - i += 1 + delta += 1 } } - // Never reached here - assert(INVALID_POS != INVALID_POS) - INVALID_POS + throw new RuntimeException("Should never reach here.") } /** @@ -163,21 +161,19 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( */ def getPos(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask - var i = 1 - val maxProbe = _data.size - while (i < maxProbe) { + var delta = 1 + while (true) { if (!_bitset.get(pos)) { return INVALID_POS } else if (k == _data(pos)) { return pos } else { - val delta = i + // quadratic probing with values increase by 1, 2, 3, ... pos = (pos + delta) & _mask - i += 1 + delta += 1 } } - // Never reached here - INVALID_POS + throw new RuntimeException("Should never reach here.") } /** Return the value at the specified position. */ From c5cc41468e8709d09c09289bb55bc8edc99404b1 Mon Sep 17 00:00:00 2001 From: Bill Chambers Date: Tue, 24 Mar 2015 22:24:35 -0700 Subject: [PATCH 524/817] [DOCUMENTATION]Fixed Missing Type Import in Documentation Needed to import the types specifically, not the more general pyspark.sql Author: Bill Chambers Author: anabranch Closes #5179 from anabranch/master and squashes the following commits: 8fa67bf [anabranch] Corrected SqlContext Import 603b080 [Bill Chambers] [DOCUMENTATION]Fixed Missing Type Import in Documentation --- docs/sql-programming-guide.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6a333fdb562a7..c99a0b03442c4 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -624,7 +624,8 @@ tuples or lists in the RDD created in the step 1. For example: {% highlight python %} # Import SQLContext and data types -from pyspark.sql import * +from pyspark.sql import SQLContext +from pyspark.sql.types import * # sc is an existing SparkContext. sqlContext = SQLContext(sc) From 64262ed99912e780b51f240a14dc98fc3cdf916d Mon Sep 17 00:00:00 2001 From: zzcclp Date: Wed, 25 Mar 2015 19:11:04 +0800 Subject: [PATCH 525/817] [SPARK-6483][SQL]Improve ScalaUdf called performance. As issue [SPARK-6483](https://issues.apache.org/jira/browse/SPARK-6483) description, ScalaUdf is low performance because of calling *asInstanceOf* to convert per record. With this, the performance of ScalaUdf is the same as other case. thank lianhuiwang for telling me how to resolve this problem. Author: zzcclp Closes #5154 from zzcclp/SPARK-6483 and squashes the following commits: 5ac6e09 [zzcclp] Add a newline at the end of source file cc6868e [zzcclp] Fix for fail on unit test. 0a8cdc3 [zzcclp] indention issue b73836a [zzcclp] Access Seq[Expression] element by :: operator, and update the code gen script. 7763848 [zzcclp] rebase from master --- .../sql/catalyst/expressions/ScalaUdf.scala | 1016 +++++++++++------ 1 file changed, 661 insertions(+), 355 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 1fd5ce342b2ce..389dc4f745723 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -39,363 +39,669 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val evals = (0 to x - 1).map(x => s" ScalaReflection.convertToScala(children($x).eval(input), children($x).dataType)").reduce(_ + ",\n " + _) - - s""" - case $x => - function.asInstanceOf[($anys) => Any]( - $evals) - """ + val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) + val evals = (0 to x - 1).map(x => s"ScalaReflection.convertToScala(child$x.eval(input), child$x.dataType)").reduce(_ + ",\n " + _) + + s""" case $x => + val func = function.asInstanceOf[($anys) => Any] + $childs + (input: Row) => { + func( + $evals) + } + """ }.foreach(println) */ - - override def eval(input: Row): Any = { - val result = children.size match { - case 0 => function.asInstanceOf[() => Any]() - case 1 => - function.asInstanceOf[(Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType)) - - - case 2 => - function.asInstanceOf[(Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType)) - - - case 3 => - function.asInstanceOf[(Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType)) - - - case 4 => - function.asInstanceOf[(Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType)) - - - case 5 => - function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType)) - - - case 6 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType)) - - - case 7 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType)) - - - case 8 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType)) - - - case 9 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType)) - - - case 10 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType)) - - - case 11 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType)) - - - case 12 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType)) - - - case 13 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType)) - - - case 14 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType)) - - - case 15 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType)) - - - case 16 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType)) - - - case 17 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType)) - - - case 18 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType)) - - - case 19 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType)) - - - case 20 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType)) - - - case 21 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), - ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType)) - - - case 22 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), - ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType), - ScalaReflection.convertToScala(children(21).eval(input), children(21).dataType)) - - } - // scalastyle:on - - ScalaReflection.convertToCatalyst(result, dataType) + + val f = children.size match { + case 0 => + val func = function.asInstanceOf[() => Any] + (input: Row) => { + func() + } + + case 1 => + val func = function.asInstanceOf[(Any) => Any] + val child0 = children(0) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType)) + } + + case 2 => + val func = function.asInstanceOf[(Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType)) + } + + case 3 => + val func = function.asInstanceOf[(Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType)) + } + + case 4 => + val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType)) + } + + case 5 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType)) + } + + case 6 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType)) + } + + case 7 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType)) + } + + case 8 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType)) + } + + case 9 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType)) + } + + case 10 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType)) + } + + case 11 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType)) + } + + case 12 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType)) + } + + case 13 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType)) + } + + case 14 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType)) + } + + case 15 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType)) + } + + case 16 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType)) + } + + case 17 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType)) + } + + case 18 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType), + ScalaReflection.convertToScala(child17.eval(input), child17.dataType)) + } + + case 19 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType), + ScalaReflection.convertToScala(child17.eval(input), child17.dataType), + ScalaReflection.convertToScala(child18.eval(input), child18.dataType)) + } + + case 20 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType), + ScalaReflection.convertToScala(child17.eval(input), child17.dataType), + ScalaReflection.convertToScala(child18.eval(input), child18.dataType), + ScalaReflection.convertToScala(child19.eval(input), child19.dataType)) + } + + case 21 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + val child20 = children(20) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType), + ScalaReflection.convertToScala(child17.eval(input), child17.dataType), + ScalaReflection.convertToScala(child18.eval(input), child18.dataType), + ScalaReflection.convertToScala(child19.eval(input), child19.dataType), + ScalaReflection.convertToScala(child20.eval(input), child20.dataType)) + } + + case 22 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + val child20 = children(20) + val child21 = children(21) + (input: Row) => { + func( + ScalaReflection.convertToScala(child0.eval(input), child0.dataType), + ScalaReflection.convertToScala(child1.eval(input), child1.dataType), + ScalaReflection.convertToScala(child2.eval(input), child2.dataType), + ScalaReflection.convertToScala(child3.eval(input), child3.dataType), + ScalaReflection.convertToScala(child4.eval(input), child4.dataType), + ScalaReflection.convertToScala(child5.eval(input), child5.dataType), + ScalaReflection.convertToScala(child6.eval(input), child6.dataType), + ScalaReflection.convertToScala(child7.eval(input), child7.dataType), + ScalaReflection.convertToScala(child8.eval(input), child8.dataType), + ScalaReflection.convertToScala(child9.eval(input), child9.dataType), + ScalaReflection.convertToScala(child10.eval(input), child10.dataType), + ScalaReflection.convertToScala(child11.eval(input), child11.dataType), + ScalaReflection.convertToScala(child12.eval(input), child12.dataType), + ScalaReflection.convertToScala(child13.eval(input), child13.dataType), + ScalaReflection.convertToScala(child14.eval(input), child14.dataType), + ScalaReflection.convertToScala(child15.eval(input), child15.dataType), + ScalaReflection.convertToScala(child16.eval(input), child16.dataType), + ScalaReflection.convertToScala(child17.eval(input), child17.dataType), + ScalaReflection.convertToScala(child18.eval(input), child18.dataType), + ScalaReflection.convertToScala(child19.eval(input), child19.dataType), + ScalaReflection.convertToScala(child20.eval(input), child20.dataType), + ScalaReflection.convertToScala(child21.eval(input), child21.dataType)) + } } + + // scalastyle:on + + override def eval(input: Row): Any = ScalaReflection.convertToCatalyst(f(input), dataType) + } From 10c78607b2724f5a64b0cdb966e9c5805f23919b Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 25 Mar 2015 17:05:56 +0000 Subject: [PATCH 526/817] [SPARK-6496] [MLLIB] GeneralizedLinearAlgorithm.run(input, initialWeights) should initialize numFeatures In GeneralizedLinearAlgorithm ```numFeatures``` is default to -1, we need to update it to correct value when we call run() to train a model. ```LogisticRegressionWithLBFGS.run(input)``` works well, but when we call ```LogisticRegressionWithLBFGS.run(input, initialWeights)``` to train multiclass classification model, it will throw exception due to the numFeatures is not updated. In this PR, we just update numFeatures at the beginning of GeneralizedLinearAlgorithm.run(input, initialWeights) and add test case. Author: Yanbo Liang Closes #5167 from yanboliang/spark-6496 and squashes the following commits: 8131c48 [Yanbo Liang] LogisticRegressionWithLBFGS.run(input, initialWeights) should initialize numFeatures --- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 4 ++++ .../mllib/classification/LogisticRegressionSuite.scala | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 45b9ebb4cc0d6..9fd60ff7a0c79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -211,6 +211,10 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { + if (numFeatures < 0) { + numFeatures = input.map(_.features.size).first() + } + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index aaa81da9e273c..a26c52852c4d7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -425,6 +425,12 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M val model = lr.run(testRDD) + val numFeatures = testRDD.map(_.features.size).first() + val initialWeights = Vectors.dense(new Array[Double]((numFeatures + 1) * 2)) + val model2 = lr.run(testRDD, initialWeights) + + LogisticRegressionSuite.checkModelsEqual(model, model2) + /** * The following is the instruction to reproduce the model using R's glmnet package. * From 982952f4aebb474823dd886dd2b18f4277bd7c30 Mon Sep 17 00:00:00 2001 From: Augustin Borsu Date: Wed, 25 Mar 2015 10:16:39 -0700 Subject: [PATCH 527/817] [ML][FEATURE] SPARK-5566: RegEx Tokenizer Added a Regex based tokenizer for ml. Currently the regex is fixed but if I could add a regex type paramater to the paramMap, changing the tokenizer regex could be a parameter used in the crossValidation. Also I wonder what would be the best way to add a stop word list. Author: Augustin Borsu Author: Augustin Borsu Author: Augustin Borsu Author: Xiangrui Meng Closes #4504 from aborsu985/master and squashes the following commits: 716d257 [Augustin Borsu] Merge branch 'mengxr-SPARK-5566' cb07021 [Augustin Borsu] Merge branch 'SPARK-5566' of git://github.com/mengxr/spark into mengxr-SPARK-5566 5f09434 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' a164800 [Xiangrui Meng] remove tabs 556aa27 [Xiangrui Meng] Merge branch 'aborsu985-master' into SPARK-5566 9651aec [Xiangrui Meng] update test f96526d [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5566 2338da5 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' e88d7b8 [Xiangrui Meng] change pattern to a StringParameter; update tests 148126f [Augustin Borsu] Added return type to public functions 12dddb4 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' daf685e [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 6a85982 [Augustin Borsu] Style corrections 38b95a1 [Augustin Borsu] Added Java unit test for RegexTokenizer b66313f [Augustin Borsu] Modified the pattern Param so it is compiled when given to the Tokenizer e262bac [Augustin Borsu] Added unit tests in scala cd6642e [Augustin Borsu] Changed regex to pattern 132b00b [Augustin Borsu] Changed matching to gaps and removed case folding 201a107 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' cb9c9a7 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' d3ef6d3 [Augustin Borsu] Added doc to RegexTokenizer 9082fc3 [Augustin Borsu] Removed stopwords parameters and updated doc 19f9e53 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' f6a5002 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 7f930bb [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 77ff9ca [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 2e89719 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 196cd7a [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 11ca50f [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 9f8685a [Augustin Borsu] RegexTokenizer 9e07a78 [Augustin Borsu] Merge remote-tracking branch 'upstream/master' 9547e9d [Augustin Borsu] RegEx Tokenizer 01cd26f [Augustin Borsu] RegExTokenizer --- .../apache/spark/ml/feature/Tokenizer.scala | 66 +++++++++++++- .../spark/ml/feature/JavaTokenizerSuite.java | 71 ++++++++++++++++ .../spark/ml/feature/TokenizerSuite.scala | 85 +++++++++++++++++++ 3 files changed, 221 insertions(+), 1 deletion(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 0b1f90daa7d8e..68401e36950bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.{ParamMap, IntParam, BooleanParam, Param} import org.apache.spark.sql.types.{DataType, StringType, ArrayType} /** @@ -39,3 +39,67 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { override protected def outputDataType: DataType = new ArrayType(StringType, false) } + +/** + * :: AlphaComponent :: + * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + * or using it to split the text (set matching to false). Optional parameters also allow to fold + * the text to lowercase prior to it being tokenized and to filer tokens using a minimal length. + * It returns an array of strings that can be empty. + * The default parameters are regex = "\\p{L}+|[^\\p{L}\\s]+", matching = true, + * lowercase = false, minTokenLength = 1 + */ +@AlphaComponent +class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenizer] { + + /** + * param for minimum token length, default is one to avoid returning empty strings + * @group param + */ + val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length", Some(1)) + + /** @group setParam */ + def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) + + /** @group getParam */ + def getMinTokenLength: Int = get(minTokenLength) + + /** + * param sets regex as splitting on gaps (true) or matching tokens (false) + * @group param + */ + val gaps: BooleanParam = new BooleanParam( + this, "gaps", "Set regex to match gaps or tokens", Some(false)) + + /** @group setParam */ + def setGaps(value: Boolean): this.type = set(gaps, value) + + /** @group getParam */ + def getGaps: Boolean = get(gaps) + + /** + * param sets regex pattern used by tokenizer + * @group param + */ + val pattern: Param[String] = new Param( + this, "pattern", "regex pattern used for tokenizing", Some("\\p{L}+|[^\\p{L}\\s]+")) + + /** @group setParam */ + def setPattern(value: String): this.type = set(pattern, value) + + /** @group getParam */ + def getPattern: String = get(pattern) + + override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { str => + val re = paramMap(pattern).r + val tokens = if (paramMap(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq + val minLength = paramMap(minTokenLength) + tokens.filter(_.length >= minLength) + } + + override protected def validateInputType(inputType: DataType): Unit = { + require(inputType == StringType, s"Input type must be string type but got $inputType.") + } + + override protected def outputDataType: DataType = new ArrayType(StringType, false) +} diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java new file mode 100644 index 0000000000000..3806f650025b2 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -0,0 +1,71 @@ +/* + * 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.feature; + +import com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; + +public class JavaTokenizerSuite { + private transient JavaSparkContext jsc; + private transient SQLContext jsql; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaTokenizerSuite"); + jsql = new SQLContext(jsc); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void regexTokenizer() { + RegexTokenizer myRegExTokenizer = new RegexTokenizer() + .setInputCol("rawText") + .setOutputCol("tokens") + .setPattern("\\s") + .setGaps(true) + .setMinTokenLength(3); + + JavaRDD rdd = jsc.parallelize(Lists.newArrayList( + new TokenizerTestData("Test of tok.", new String[] {"Test", "tok."}), + new TokenizerTestData("Te,st. punct", new String[] {"Te,st.", "punct"}) + )); + DataFrame dataset = jsql.createDataFrame(rdd, TokenizerTestData.class); + + Row[] pairs = myRegExTokenizer.transform(dataset) + .select("tokens", "wantedTokens") + .collect(); + + for (Row r : pairs) { + Assert.assertEquals(r.get(0), r.get(1)); + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala new file mode 100644 index 0000000000000..bf862b912d326 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -0,0 +1,85 @@ +/* + * 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.feature + +import scala.beans.BeanInfo + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +@BeanInfo +case class TokenizerTestData(rawText: String, wantedTokens: Seq[String]) { + /** Constructor used in [[org.apache.spark.ml.feature.JavaTokenizerSuite]] */ + def this(rawText: String, wantedTokens: Array[String]) = this(rawText, wantedTokens.toSeq) +} + +class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { + import org.apache.spark.ml.feature.RegexTokenizerSuite._ + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("RegexTokenizer") { + val tokenizer = new RegexTokenizer() + .setInputCol("rawText") + .setOutputCol("tokens") + + val dataset0 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization", ".")), + TokenizerTestData("Te,st. punct", Seq("Te", ",", "st", ".", "punct")) + )) + testRegexTokenizer(tokenizer, dataset0) + + val dataset1 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization")), + TokenizerTestData("Te,st. punct", Seq("punct")) + )) + + tokenizer.setMinTokenLength(3) + testRegexTokenizer(tokenizer, dataset1) + + tokenizer + .setPattern("\\s") + .setGaps(true) + .setMinTokenLength(0) + val dataset2 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization.")), + TokenizerTestData("Te,st. punct", Seq("Te,st.", "", "punct")) + )) + testRegexTokenizer(tokenizer, dataset2) + } +} + +object RegexTokenizerSuite extends FunSuite { + + def testRegexTokenizer(t: RegexTokenizer, dataset: DataFrame): Unit = { + t.transform(dataset) + .select("tokens", "wantedTokens") + .collect() + .foreach { + case Row(tokens, wantedTokens) => + assert(tokens === wantedTokens) + } + } +} From 968408b345a0e26f7ee9105a6a0c3456cf10576a Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Wed, 25 Mar 2015 11:11:52 -0700 Subject: [PATCH 528/817] [SPARK-6409][SQL] It is not necessary that avoid old inteface of hive, because this will make some UDAF can not work. spark avoid old inteface of hive, then some udaf can not work like "org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage" Author: DoingDone9 <799203320@qq.com> Closes #5131 from DoingDone9/udaf and squashes the following commits: 9de08d0 [DoingDone9] Update HiveUdfSuite.scala 49c62dc [DoingDone9] Update hiveUdfs.scala 98b134f [DoingDone9] Merge pull request #5 from apache/master 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/sql/hive/hiveUdfs.scala | 3 +-- .../spark/sql/hive/execution/HiveUdfSuite.scala | 11 ++++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index bfe43373d9534..47305571e579e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -375,9 +375,8 @@ private[hive] case class HiveUdafFunction( private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - // Cast required to avoid type inference selecting a deprecated Hive API. private val buffer = - function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + function.getNewAggregationBuffer override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index cb405f56bf53d..d7c5d1a25a82b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -22,7 +22,7 @@ import java.util import java.util.Properties import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFAverage, GenericUDF} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} @@ -93,6 +93,15 @@ class HiveUdfSuite extends QueryTest { sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } + test("SPARK-6409 UDAFAverage test") { + sql(s"CREATE TEMPORARY FUNCTION test_avg AS '${classOf[GenericUDAFAverage].getName}'") + checkAnswer( + sql("SELECT test_avg(1), test_avg(substr(value,5)) FROM src"), + Seq(Row(1.0, 260.182))) + sql("DROP TEMPORARY FUNCTION IF EXISTS test_avg") + TestHive.reset() + } + test("SPARK-2693 udaf aggregates test") { checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"), sql("SELECT max(key) FROM src").collect().toSeq) From 883b7e9030e1a3948acee17608e51dcd9f4d55e1 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 25 Mar 2015 12:09:30 -0700 Subject: [PATCH 529/817] [SPARK-6076][Block Manager] Fix a potential OOM issue when StorageLevel is MEMORY_AND_DISK_SER In https://github.com/apache/spark/blob/dcd1e42d6b6ac08d2c0736bf61a15f515a1f222b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala#L538 , when StorageLevel is `MEMORY_AND_DISK_SER`, it will copy the content from file into memory, then put it into MemoryStore. ```scala val copyForMemory = ByteBuffer.allocate(bytes.limit) copyForMemory.put(bytes) memoryStore.putBytes(blockId, copyForMemory, level) bytes.rewind() ``` However, if the file is bigger than the free memory, OOM will happen. A better approach is testing if there is enough memory. If not, copyForMemory should not be created, since this is an optional operation. Author: zsxwing Closes #4827 from zsxwing/SPARK-6076 and squashes the following commits: 7d25545 [zsxwing] Add alias for tryToPut and dropFromMemory 1100a54 [zsxwing] Replace call-by-name with () => T 0cc0257 [zsxwing] Fix a potential OOM issue when StorageLevel is MEMORY_AND_DISK_SER --- .../apache/spark/storage/BlockManager.scala | 23 +++++++--- .../apache/spark/storage/MemoryStore.scala | 43 ++++++++++++++++--- .../spark/storage/BlockManagerSuite.scala | 34 +++++++++++++-- 3 files changed, 85 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 80d66e59132da..1dff09a75d038 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -535,9 +535,14 @@ private[spark] class BlockManager( /* We'll store the bytes in memory if the block's storage level includes * "memory serialized", or if it should be cached as objects in memory * but we only requested its serialized bytes. */ - val copyForMemory = ByteBuffer.allocate(bytes.limit) - copyForMemory.put(bytes) - memoryStore.putBytes(blockId, copyForMemory, level) + memoryStore.putBytes(blockId, bytes.limit, () => { + // https://issues.apache.org/jira/browse/SPARK-6076 + // If the file size is bigger than the free memory, OOM will happen. So if we cannot + // put it into MemoryStore, copyForMemory should not be created. That's why this + // action is put into a `() => ByteBuffer` and created lazily. + val copyForMemory = ByteBuffer.allocate(bytes.limit) + copyForMemory.put(bytes) + }) bytes.rewind() } if (!asBlockResult) { @@ -991,15 +996,23 @@ private[spark] class BlockManager( putIterator(blockId, Iterator(value), level, tellMaster) } + def dropFromMemory( + blockId: BlockId, + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { + dropFromMemory(blockId, () => data) + } + /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. * + * If `data` is not put on disk, it won't be created. + * * Return the block status if the given block has been updated, else None. */ def dropFromMemory( blockId: BlockId, - data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { + data: () => Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull @@ -1023,7 +1036,7 @@ private[spark] class BlockManager( // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { logInfo(s"Writing block $blockId to disk") - data match { + data() match { case Left(elements) => diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 1be860aea63d0..ed609772e6979 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -98,6 +98,26 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } + /** + * Use `size` to test if there is enough space in MemoryStore. If so, create the ByteBuffer and + * put it into MemoryStore. Otherwise, the ByteBuffer won't be created. + * + * The caller should guarantee that `size` is correct. + */ + def putBytes(blockId: BlockId, size: Long, _bytes: () => ByteBuffer): PutResult = { + // Work on a duplicate - since the original input might be used elsewhere. + lazy val bytes = _bytes().duplicate().rewind().asInstanceOf[ByteBuffer] + val putAttempt = tryToPut(blockId, () => bytes, size, deserialized = false) + val data = + if (putAttempt.success) { + assert(bytes.limit == size) + Right(bytes.duplicate()) + } else { + null + } + PutResult(size, data, putAttempt.droppedBlocks) + } + override def putArray( blockId: BlockId, values: Array[Any], @@ -312,11 +332,22 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId.asRDDId.map(_.rddId) } + private def tryToPut( + blockId: BlockId, + value: Any, + size: Long, + deserialized: Boolean): ResultWithDroppedBlocks = { + tryToPut(blockId, () => value, size, deserialized) + } + /** * Try to put in a set of values, if we can free up enough space. The value should either be * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size * must also be passed by the caller. * + * `value` will be lazily created. If it cannot be put into MemoryStore or disk, `value` won't be + * created to avoid OOM since it may be a big ByteBuffer. + * * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for @@ -326,7 +357,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) */ private def tryToPut( blockId: BlockId, - value: Any, + value: () => Any, size: Long, deserialized: Boolean): ResultWithDroppedBlocks = { @@ -345,7 +376,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks ++= freeSpaceResult.droppedBlocks if (enoughFreeSpace) { - val entry = new MemoryEntry(value, size, deserialized) + val entry = new MemoryEntry(value(), size, deserialized) entries.synchronized { entries.put(blockId, entry) currentMemory += size @@ -357,12 +388,12 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. - val data = if (deserialized) { - Left(value.asInstanceOf[Array[Any]]) + lazy val data = if (deserialized) { + Left(value().asInstanceOf[Array[Any]]) } else { - Right(value.asInstanceOf[ByteBuffer].duplicate()) + Right(value().asInstanceOf[ByteBuffer].duplicate()) } - val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + val droppedBlockStatus = blockManager.dropFromMemory(blockId, () => data) droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } // Release the unroll memory used because we no longer need the underlying Array diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 3fdbe99b5d02b..ecd1cba5b5abe 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -170,8 +170,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(master.getLocations("a3").size === 0, "master was told about a3") // Drop a1 and a2 from memory; this should be reported back to the master - store.dropFromMemory("a1", null) - store.dropFromMemory("a2", null) + store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) + store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) assert(store.getSingle("a1") === None, "a1 not removed from store") assert(store.getSingle("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") @@ -413,8 +413,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach t2.join() t3.join() - store.dropFromMemory("a1", null) - store.dropFromMemory("a2", null) + store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) + store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) store.waitForAsyncReregister() } } @@ -1223,4 +1223,30 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) } + + test("lazily create a big ByteBuffer to avoid OOM if it cannot be put into MemoryStore") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + val blockId = BlockId("rdd_3_10") + val result = memoryStore.putBytes(blockId, 13000, () => { + fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") + }) + assert(result.size === 13000) + assert(result.data === null) + assert(result.droppedBlocks === Nil) + } + + test("put a small ByteBuffer to MemoryStore") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + val blockId = BlockId("rdd_3_10") + var bytes: ByteBuffer = null + val result = memoryStore.putBytes(blockId, 10000, () => { + bytes = ByteBuffer.allocate(10000) + bytes + }) + assert(result.size === 10000) + assert(result.data === Right(bytes)) + assert(result.droppedBlocks === Nil) + } } From acef51defb991bcdc99b76cf2a126afd6d60ec70 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 25 Mar 2015 13:27:15 -0700 Subject: [PATCH 530/817] [SPARK-6537] UIWorkloadGenerator: The main thread should not stop SparkContext until all jobs finish The main thread of UIWorkloadGenerator spawn sub threads to launch jobs but the main thread stop SparkContext without waiting for finishing those threads. Author: Kousuke Saruta Closes #5187 from sarutak/SPARK-6537 and squashes the following commits: 4e9307a [Kousuke Saruta] Fixed UIWorkloadGenerator so that the main thread stop SparkContext after all jobs finish --- .../scala/org/apache/spark/ui/UIWorkloadGenerator.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 19ac7a826e306..5fbcd6bb8ad94 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui +import java.util.concurrent.Semaphore + import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} @@ -88,6 +90,8 @@ private[spark] object UIWorkloadGenerator { ("Job with delays", baseData.map(x => Thread.sleep(100)).count) ) + val barrier = new Semaphore(-nJobSet * jobs.size + 1) + (1 to nJobSet).foreach { _ => for ((desc, job) <- jobs) { new Thread { @@ -99,12 +103,17 @@ private[spark] object UIWorkloadGenerator { } catch { case e: Exception => println("Job Failed: " + desc) + } finally { + barrier.release() } } }.start Thread.sleep(INTER_JOB_WAIT_MS) } } + + // Waiting for threads. + barrier.acquire() sc.stop() } } From c1b74df6042b33b2b061cb07c2fbd82dba9074bb Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Mar 2015 13:28:32 -0700 Subject: [PATCH 531/817] [SPARK-5771] Master UI inconsistently displays application cores If the user calls `sc.stop()`, then the number of cores under "Completed Applications" will be 0. If the user does not call `sc.stop()`, then the number of cores will be however many cores were being used before the application exited. This PR makes both cases have the behavior of the latter. Note that there have been a series of PR that attempted to fix this. For the full discussion, please refer to #4841. The unregister event is necessary because of a subtle race condition explained in that PR. Tested this locally with and without calling `sc.stop()`. Author: Andrew Or Closes #5177 from andrewor14/master-ui-cores and squashes the following commits: 62449d1 [Andrew Or] Freeze application state before finishing it --- .../scala/org/apache/spark/deploy/DeployMessage.scala | 2 ++ .../org/apache/spark/deploy/client/AppClient.scala | 1 + .../apache/spark/deploy/master/ApplicationInfo.scala | 4 ++++ .../scala/org/apache/spark/deploy/master/Master.scala | 10 +++++++++- 4 files changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 0997507d016f5..9db6fd1ac4dbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -101,6 +101,8 @@ private[deploy] object DeployMessages { case class RegisterApplication(appDescription: ApplicationDescription) extends DeployMessage + case class UnregisterApplication(appId: String) + case class MasterChangeAcknowledged(appId: String) // Master to AppClient diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 3b729725257ef..4f06d7f96c46e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -157,6 +157,7 @@ private[spark] class AppClient( case StopAppClient => markDead("Application has been stopped.") + master ! UnregisterApplication(appId) sender ! true context.stop(self) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index f979ffa16641a..bc5b293379f2b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -111,6 +111,10 @@ private[deploy] class ApplicationInfo( endTime = System.currentTimeMillis() } + private[master] def isFinished: Boolean = { + state != ApplicationState.WAITING && state != ApplicationState.RUNNING + } + def duration: Long = { if (endTime != -1) { endTime - startTime diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 80506621f4d24..9a5d5877da86d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -339,7 +339,11 @@ private[master] class Master( if (ExecutorState.isFinished(state)) { // Remove this executor from the worker and app logInfo(s"Removing executor ${exec.fullId} because it is $state") - appInfo.removeExecutor(exec) + // If an application has already finished, preserve its + // state to display its information properly on the UI + if (!appInfo.isFinished) { + appInfo.removeExecutor(exec) + } exec.worker.removeExecutor(exec) val normalExit = exitStatus == Some(0) @@ -428,6 +432,10 @@ private[master] class Master( if (canCompleteRecovery) { completeRecovery() } } + case UnregisterApplication(applicationId) => + logInfo(s"Received unregister request from application $applicationId") + idToApp.get(applicationId).foreach(finishApplication) + case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was logInfo(s"$address got disassociated, removing it.") From 435337381f093f95248c8f0204e60c0b366edc81 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 25 Mar 2015 13:38:33 -0700 Subject: [PATCH 532/817] [SPARK-6256] [MLlib] MLlib Python API parity check for regression MLlib Python API parity check for Regression, major disparities need to be added for Python list following: ```scala LinearRegressionWithSGD setValidateData LassoWithSGD setIntercept setValidateData RidgeRegressionWithSGD setIntercept setValidateData ``` setFeatureScaling is mllib private function which is not needed to expose in pyspark. Author: Yanbo Liang Closes #4997 from yanboliang/spark-6256 and squashes the following commits: 102f498 [Yanbo Liang] fix intercept issue & add doc test 1fb7b4f [Yanbo Liang] change 'intercept' to 'addIntercept' de5ecbc [Yanbo Liang] MLlib Python API parity check for regression --- .../mllib/api/python/PythonMLLibAPI.scala | 16 +++++-- python/pyspark/mllib/regression.py | 43 ++++++++++++++++--- 2 files changed, 49 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 15ca2547d56a8..e39156734794c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -111,9 +111,11 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val lrAlg = new LinearRegressionWithSGD() lrAlg.setIntercept(intercept) + .setValidateData(validateData) lrAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -135,8 +137,12 @@ private[python] class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector): JList[Object] = { + initialWeights: Vector, + intercept: Boolean, + validateData: Boolean): JList[Object] = { val lassoAlg = new LassoWithSGD() + lassoAlg.setIntercept(intercept) + .setValidateData(validateData) lassoAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -157,8 +163,12 @@ private[python] class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector): JList[Object] = { + initialWeights: Vector, + intercept: Boolean, + validateData: Boolean): JList[Object] = { val ridgeAlg = new RidgeRegressionWithSGD() + ridgeAlg.setIntercept(intercept) + .setValidateData(validateData) ridgeAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 414a0ada80787..209f1ee473b5b 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -140,6 +140,13 @@ class LinearRegressionModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + ... miniBatchFraction=1.0, initialWeights=array([1.0]), regParam=0.1, regType="l2", + ... intercept=True, validateData=True) + >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True """ def save(self, sc, path): java_model = sc._jvm.org.apache.spark.mllib.regression.LinearRegressionModel( @@ -173,7 +180,8 @@ class LinearRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=0.0, regType=None, intercept=False): + initialWeights=None, regParam=0.0, regType=None, intercept=False, + validateData=True): """ Train a linear regression model on the given data. @@ -195,15 +203,18 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, (default: None) - @param intercept: Boolean parameter which indicates the use + :param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features are activated or not). (default: False) + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainLinearRegressionModelWithSGD", rdd, int(iterations), float(step), float(miniBatchFraction), i, float(regParam), - regType, bool(intercept)) + regType, bool(intercept), bool(validateData)) return _regression_train_wrapper(train, LinearRegressionModel, data, initialWeights) @@ -253,6 +264,13 @@ class LassoModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... validateData=True) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True """ def save(self, sc, path): java_model = sc._jvm.org.apache.spark.mllib.regression.LassoModel( @@ -273,11 +291,13 @@ class LassoWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None): + miniBatchFraction=1.0, initialWeights=None, intercept=False, + validateData=True): """Train a Lasso regression model on the given data.""" def train(rdd, i): return callMLlibFunc("trainLassoModelWithSGD", rdd, int(iterations), float(step), - float(regParam), float(miniBatchFraction), i) + float(regParam), float(miniBatchFraction), i, bool(intercept), + bool(validateData)) return _regression_train_wrapper(train, LassoModel, data, initialWeights) @@ -327,6 +347,13 @@ class RidgeRegressionModel(LinearRegressionModelBase): True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True + >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=100, step=1.0, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... validateData=True) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True """ def save(self, sc, path): java_model = sc._jvm.org.apache.spark.mllib.regression.RidgeRegressionModel( @@ -347,11 +374,13 @@ class RidgeRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None): + miniBatchFraction=1.0, initialWeights=None, intercept=False, + validateData=True): """Train a ridge regression model on the given data.""" def train(rdd, i): return callMLlibFunc("trainRidgeModelWithSGD", rdd, int(iterations), float(step), - float(regParam), float(miniBatchFraction), i) + float(regParam), float(miniBatchFraction), i, bool(intercept), + bool(validateData)) return _regression_train_wrapper(train, RidgeRegressionModel, data, initialWeights) From 4fc4d0369e8240defe0ee83252426402f1a28a36 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Wed, 25 Mar 2015 14:45:23 -0700 Subject: [PATCH 533/817] [SPARK-5987] [MLlib] Save/load for GaussianMixtureModels Should be self explanatory. Author: MechCoder Closes #4986 from MechCoder/spark-5987 and squashes the following commits: 7d2cd56 [MechCoder] Iterate over dataframe in a better way e7a14cb [MechCoder] Minor 33c84f9 [MechCoder] Store as Array[Data] instead of Data[Array] 505bd57 [MechCoder] Rebased over master and used MatrixUDT 7422bb4 [MechCoder] Store sigmas as Array[Double] instead of Array[Array[Double]] b9794e4 [MechCoder] Minor cb77095 [MechCoder] [SPARK-5987] Save/load for GaussianMixtureModels --- docs/mllib-clustering.md | 8 ++ .../clustering/GaussianMixtureModel.scala | 96 ++++++++++++++++++- .../clustering/GaussianMixtureSuite.scala | 52 +++++++--- 3 files changed, 136 insertions(+), 20 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 0b6db4fcb7b1f..f5aa15b7d9b79 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -173,6 +173,7 @@ to the algorithm. We then output the parameters of the mixture model. {% highlight scala %} import org.apache.spark.mllib.clustering.GaussianMixture +import org.apache.spark.mllib.clustering.GaussianMixtureModel import org.apache.spark.mllib.linalg.Vectors // Load and parse the data @@ -182,6 +183,10 @@ val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))) // Cluster the data into two classes using GaussianMixture val gmm = new GaussianMixture().setK(2).run(parsedData) +// Save and load model +gmm.save(sc, "myGMMModel") +val sameModel = GaussianMixtureModel.load(sc, "myGMMModel") + // output parameters of max-likelihood model for (i <- 0 until gmm.k) { println("weight=%f\nmu=%s\nsigma=\n%s\n" format @@ -231,6 +236,9 @@ public class GaussianMixtureExample { // Cluster the data into two classes using GaussianMixture GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd()); + // Save and load GaussianMixtureModel + gmm.save(sc, "myGMMModel") + GaussianMixtureModel sameModel = GaussianMixtureModel.load(sc, "myGMMModel") // Output the parameters of the mixture model for(int j=0; j BreezeVector} +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vector, Matrices, Matrix} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.{MLUtils, Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, Row} /** * :: Experimental :: @@ -41,10 +47,16 @@ import org.apache.spark.rdd.RDD @Experimental class GaussianMixtureModel( val weights: Array[Double], - val gaussians: Array[MultivariateGaussian]) extends Serializable { + val gaussians: Array[MultivariateGaussian]) extends Serializable with Saveable{ require(weights.length == gaussians.length, "Length of weight and Gaussian arrays must match") - + + override protected def formatVersion = "1.0" + + override def save(sc: SparkContext, path: String): Unit = { + GaussianMixtureModel.SaveLoadV1_0.save(sc, path, weights, gaussians) + } + /** Number of gaussians in mixture */ def k: Int = weights.length @@ -83,5 +95,79 @@ class GaussianMixtureModel( p(i) /= pSum } p - } + } +} + +@Experimental +object GaussianMixtureModel extends Loader[GaussianMixtureModel] { + + private object SaveLoadV1_0 { + + case class Data(weight: Double, mu: Vector, sigma: Matrix) + + val formatVersionV1_0 = "1.0" + + val classNameV1_0 = "org.apache.spark.mllib.clustering.GaussianMixtureModel" + + def save( + sc: SparkContext, + path: String, + weights: Array[Double], + gaussians: Array[MultivariateGaussian]): Unit = { + + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadata = compact(render + (("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ ("k" -> weights.length))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + // Create Parquet data. + val dataArray = Array.tabulate(weights.length) { i => + Data(weights(i), gaussians(i).mu, gaussians(i).sigma) + } + sc.parallelize(dataArray, 1).toDF().saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): GaussianMixtureModel = { + val dataPath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataFrame = sqlContext.parquetFile(dataPath) + val dataArray = dataFrame.select("weight", "mu", "sigma").collect() + + // Check schema explicitly since erasure makes it hard to use match-case for checking. + Loader.checkSchema[Data](dataFrame.schema) + + val (weights, gaussians) = dataArray.map { + case Row(weight: Double, mu: Vector, sigma: Matrix) => + (weight, new MultivariateGaussian(mu, sigma)) + }.unzip + + return new GaussianMixtureModel(weights.toArray, gaussians.toArray) + } + } + + override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + implicit val formats = DefaultFormats + val k = (metadata \ "k").extract[Int] + val classNameV1_0 = SaveLoadV1_0.classNameV1_0 + (loadedClassName, version) match { + case (classNameV1_0, "1.0") => { + val model = SaveLoadV1_0.load(sc, path) + require(model.weights.length == k, + s"GaussianMixtureModel requires weights of length $k " + + s"got weights of length ${model.weights.length}") + require(model.gaussians.length == k, + s"GaussianMixtureModel requires gaussians of length $k" + + s"got gaussians of length ${model.gaussians.length}") + model + } + case _ => throw new Exception( + s"GaussianMixtureModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index 1b46a4012d731..f356ffa3e3a26 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Matrices} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { test("single cluster") { @@ -48,13 +49,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { } test("two clusters") { - val data = sc.parallelize(Array( - Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), - Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), - Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), - Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), - Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) - )) + val data = sc.parallelize(GaussianTestData.data) // we set an initial gaussian to induce expected results val initialGmm = new GaussianMixtureModel( @@ -105,14 +100,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { } test("two clusters with sparse data") { - val data = sc.parallelize(Array( - Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), - Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), - Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), - Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), - Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) - )) - + val data = sc.parallelize(GaussianTestData.data) val sparseData = data.map(point => Vectors.sparse(1, Array(0), point.toArray)) // we set an initial gaussian to induce expected results val initialGmm = new GaussianMixtureModel( @@ -138,4 +126,38 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { assert(sparseGMM.gaussians(0).sigma ~== Esigma(0) absTol 1E-3) assert(sparseGMM.gaussians(1).sigma ~== Esigma(1) absTol 1E-3) } + + test("model save / load") { + val data = sc.parallelize(GaussianTestData.data) + + val gmm = new GaussianMixture().setK(2).setSeed(0).run(data) + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + try { + gmm.save(sc, path) + + // TODO: GaussianMixtureModel should implement equals/hashcode directly. + val sameModel = GaussianMixtureModel.load(sc, path) + assert(sameModel.k === gmm.k) + (0 until sameModel.k).foreach { i => + assert(sameModel.gaussians(i).mu === gmm.gaussians(i).mu) + assert(sameModel.gaussians(i).sigma === gmm.gaussians(i).sigma) + } + } finally { + Utils.deleteRecursively(tempDir) + } + } + + object GaussianTestData { + + val data = Array( + Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), + Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), + Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), + Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), + Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) + ) + + } } From d44a3362ed8cf3068f8ff233e13851a39da42219 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 25 Mar 2015 17:40:00 -0700 Subject: [PATCH 534/817] [SPARK-6079] Use index to speed up StatusTracker.getJobIdsForGroup() `StatusTracker.getJobIdsForGroup()` is implemented via a linear scan over a HashMap rather than using an index, which might be an expensive operation if there are many (e.g. thousands) of retained jobs. This patch adds a new map to `JobProgressListener` in order to speed up these lookups. Author: Josh Rosen Closes #4830 from JoshRosen/statustracker-job-group-indexing and squashes the following commits: e39c5c7 [Josh Rosen] Address review feedback 6709fb2 [Josh Rosen] Merge remote-tracking branch 'origin/master' into statustracker-job-group-indexing 2c49614 [Josh Rosen] getOrElse 97275a7 [Josh Rosen] Add jobGroup to jobId index to JobProgressListener --- .../org/apache/spark/SparkStatusTracker.scala | 3 +- .../spark/ui/jobs/JobProgressListener.scala | 23 ++++++++++++-- .../ui/jobs/JobProgressListenerSuite.scala | 31 +++++++++++++++++-- 3 files changed, 51 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index edbdda8a0bcb6..34ee3a48f8e74 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -45,8 +45,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { */ def getJobIdsForGroup(jobGroup: String): Array[Int] = { jobProgressListener.synchronized { - val jobData = jobProgressListener.jobIdToData.valuesIterator - jobData.filter(_.jobGroup.orNull == jobGroup).map(_.jobId).toArray + jobProgressListener.jobGroupToJobIds.getOrElse(jobGroup, Seq.empty).toArray } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 949e80d30f5eb..625596885faa1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -44,6 +44,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // These type aliases are public because they're used in the types of public fields: type JobId = Int + type JobGroupId = String type StageId = Int type StageAttemptId = Int type PoolName = String @@ -54,6 +55,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val completedJobs = ListBuffer[JobUIData]() val failedJobs = ListBuffer[JobUIData]() val jobIdToData = new HashMap[JobId, JobUIData] + val jobGroupToJobIds = new HashMap[JobGroupId, HashSet[JobId]] // Stages: val pendingStages = new HashMap[StageId, StageInfo] @@ -119,7 +121,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { Map( "jobIdToData" -> jobIdToData.size, "stageIdToData" -> stageIdToData.size, - "stageIdToStageInfo" -> stageIdToInfo.size + "stageIdToStageInfo" -> stageIdToInfo.size, + "jobGroupToJobIds" -> jobGroupToJobIds.values.map(_.size).sum, + // Since jobGroupToJobIds is map of sets, check that we don't leak keys with empty values: + "jobGroupToJobIds keySet" -> jobGroupToJobIds.keys.size ) } @@ -140,7 +145,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { if (jobs.size > retainedJobs) { val toRemove = math.max(retainedJobs / 10, 1) jobs.take(toRemove).foreach { job => - jobIdToData.remove(job.jobId) + // Remove the job's UI data, if it exists + jobIdToData.remove(job.jobId).foreach { removedJob => + // A null jobGroupId is used for jobs that are run without a job group + val jobGroupId = removedJob.jobGroup.orNull + // Remove the job group -> job mapping entry, if it exists + jobGroupToJobIds.get(jobGroupId).foreach { jobsInGroup => + jobsInGroup.remove(job.jobId) + // If this was the last job in this job group, remove the map entry for the job group + if (jobsInGroup.isEmpty) { + jobGroupToJobIds.remove(jobGroupId) + } + } + } } jobs.trimStart(toRemove) } @@ -158,6 +175,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageIds = jobStart.stageIds, jobGroup = jobGroup, status = JobExecutionStatus.RUNNING) + // A null jobGroupId is used for jobs that are run without a job group + jobGroupToJobIds.getOrElseUpdate(jobGroup.orNull, new HashSet[JobId]).add(jobStart.jobId) jobStart.stageInfos.foreach(x => pendingStages(x.stageId) = x) // Compute (a potential underestimate of) the number of tasks that will be run by this job. // This may be an underestimate because the job start event references all of the result diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 730a4b54f5aa1..c0c28cb60e21d 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui.jobs +import java.util.Properties + import org.scalatest.FunSuite import org.scalatest.Matchers @@ -44,11 +46,19 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc SparkListenerStageCompleted(stageInfo) } - private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = { + private def createJobStartEvent( + jobId: Int, + stageIds: Seq[Int], + jobGroup: Option[String] = None): SparkListenerJobStart = { val stageInfos = stageIds.map { stageId => new StageInfo(stageId, 0, stageId.toString, 0, null, "") } - SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos) + val properties: Option[Properties] = jobGroup.map { groupId => + val props = new Properties() + props.setProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + props + } + SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos, properties.orNull) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { @@ -110,6 +120,23 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc listener.stageIdToActiveJobIds.size should be (0) } + test("test clearing of jobGroupToJobIds") { + val conf = new SparkConf() + conf.set("spark.ui.retainedJobs", 5.toString) + val listener = new JobProgressListener(conf) + + // Run 50 jobs, each with one stage + for (jobId <- 0 to 50) { + listener.onJobStart(createJobStartEvent(jobId, Seq(0), jobGroup = Some(jobId.toString))) + listener.onStageSubmitted(createStageStartEvent(0)) + listener.onStageCompleted(createStageEndEvent(0, failed = false)) + listener.onJobEnd(createJobEndEvent(jobId, false)) + } + assertActiveJobsStateIsEmpty(listener) + // This collection won't become empty, but it should be bounded by spark.ui.retainedJobs + listener.jobGroupToJobIds.size should be (5) + } + test("test LRU eviction of jobs") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) From 8c3b0052f4792d97d23244ade335676e37cb1fae Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Mar 2015 17:40:19 -0700 Subject: [PATCH 535/817] [SPARK-6450] [SQL] Fixes metastore Parquet table conversion The `ParquetConversions` analysis rule generates a hash map, which maps from the original `MetastoreRelation` instances to the newly created `ParquetRelation2` instances. However, `MetastoreRelation.equals` doesn't compare output attributes. Thus, if a single metastore Parquet table appears multiple times in a query, only a single entry ends up in the hash map, and the conversion is not correctly performed. Proper fix for this issue should be overriding `equals` and `hashCode` for MetastoreRelation. Unfortunately, this breaks more tests than expected. It's possible that these tests are ill-formed from the very beginning. As 1.3.1 release is approaching, we'd like to make the change more surgical to avoid potential regressions. The proposed fix here is to make both the metastore relations and their output attributes as keys in the hash map used in ParquetConversions. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5183) Author: Cheng Lian Closes #5183 from liancheng/spark-6450 and squashes the following commits: 3536780 [Cheng Lian] Fixes metastore Parquet table conversion --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 34 ++++++++++--------- .../apache/spark/sql/hive/parquetSuites.scala | 25 ++++++++++++++ 2 files changed, 43 insertions(+), 16 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 4c5eb48661f7d..d1a99555e90c6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -459,7 +459,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation, parquetRelation, attributedRewrites) + (relation -> relation.output, parquetRelation, attributedRewrites) // Write path case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) @@ -470,7 +470,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation, parquetRelation, attributedRewrites) + (relation -> relation.output, parquetRelation, attributedRewrites) // Read path case p @ PhysicalOperation(_, _, relation: MetastoreRelation) @@ -479,33 +479,35 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation, parquetRelation, attributedRewrites) + (relation -> relation.output, parquetRelation, attributedRewrites) } + // Quick fix for SPARK-6450: Notice that we're using both the MetastoreRelation instances and + // their output attributes as the key of the map. This is because MetastoreRelation.equals + // doesn't take output attributes into account, thus multiple MetastoreRelation instances + // pointing to the same table get collapsed into a single entry in the map. A proper fix for + // this should be overriding equals & hashCode in MetastoreRelation. val relationMap = toBeReplaced.map(r => (r._1, r._2)).toMap val attributedRewrites = AttributeMap(toBeReplaced.map(_._3).fold(Nil)(_ ++: _)) // Replaces all `MetastoreRelation`s with corresponding `ParquetRelation2`s, and fixes // attribute IDs referenced in other nodes. plan.transformUp { - case r: MetastoreRelation if relationMap.contains(r) => { - val parquetRelation = relationMap(r) - val withAlias = - r.alias.map(a => Subquery(a, parquetRelation)).getOrElse( - Subquery(r.tableName, parquetRelation)) + case r: MetastoreRelation if relationMap.contains(r -> r.output) => + val parquetRelation = relationMap(r -> r.output) + val alias = r.alias.getOrElse(r.tableName) + Subquery(alias, parquetRelation) - withAlias - } case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r) => { - val parquetRelation = relationMap(r) + if relationMap.contains(r -> r.output) => + val parquetRelation = relationMap(r -> r.output) InsertIntoTable(parquetRelation, partition, child, overwrite) - } + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r) => { - val parquetRelation = relationMap(r) + if relationMap.contains(r -> r.output) => + val parquetRelation = relationMap(r -> r.output) InsertIntoTable(parquetRelation, partition, child, overwrite) - } + case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 8a31bd03092d1..432d65a874518 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -365,6 +365,31 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { sql("DROP TABLE IF EXISTS test_insert_parquet") } + + test("SPARK-6450 regression test") { + sql( + """CREATE TABLE IF NOT EXISTS ms_convert (key INT) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + // This shouldn't throw AnalysisException + val analyzed = sql( + """SELECT key FROM ms_convert + |UNION ALL + |SELECT key FROM ms_convert + """.stripMargin).queryExecution.analyzed + + assertResult(2) { + analyzed.collect { + case r @ LogicalRelation(_: ParquetRelation2) => r + }.size + } + + sql("DROP TABLE ms_convert") + } } class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { From e6d1406abd55bc24477eb8c6ee72c31e7110435e Mon Sep 17 00:00:00 2001 From: jeanlyn Date: Wed, 25 Mar 2015 17:47:45 -0700 Subject: [PATCH 536/817] [SPARK-5498][SQL]fix query exception when partition schema does not match table schema In hive,the schema of partition may be difference from the table schema.When we use spark-sql to query the data of partition which schema is difference from the table schema,we will get the exceptions as the description of the [jira](https://issues.apache.org/jira/browse/SPARK-5498) .For example: * We take a look of the schema for the partition and the table ```sql DESCRIBE partition_test PARTITION (dt='1'); id int None name string None dt string None # Partition Information # col_name data_type comment dt string None ``` ``` DESCRIBE partition_test; OK id bigint None name string None dt string None # Partition Information # col_name data_type comment dt string None ``` * run the sql ```sql SELECT * FROM partition_test where dt='1'; ``` we will get the cast exception `java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.MutableLong cannot be cast to org.apache.spark.sql.catalyst.expressions.MutableInt` Author: jeanlyn Closes #4289 from jeanlyn/schema and squashes the following commits: 9c8da74 [jeanlyn] fix style b41d6b9 [jeanlyn] fix compile errors 07d84b6 [jeanlyn] Merge branch 'master' into schema 535b0b6 [jeanlyn] reduce conflicts d6c93c5 [jeanlyn] fix bug 1e8b30c [jeanlyn] fix code style 0549759 [jeanlyn] fix code style c879aa1 [jeanlyn] clean the code 2a91a87 [jeanlyn] add more test case and clean the code 12d800d [jeanlyn] fix code style 63d170a [jeanlyn] fix compile problem 7470901 [jeanlyn] reduce conflicts afc7da5 [jeanlyn] make getConvertedOI compatible between 0.12.0 and 0.13.1 b1527d5 [jeanlyn] fix type mismatch 10744ca [jeanlyn] Insert a space after the start of the comment 3b27af3 [jeanlyn] SPARK-5498:fix bug when query the data when partition schema does not match table schema --- .../apache/spark/sql/hive/TableReader.scala | 37 +++++++++++----- .../sql/hive/InsertIntoHiveTableSuite.scala | 42 +++++++++++++++++++ .../org/apache/spark/sql/hive/Shim12.scala | 10 ++++- .../org/apache/spark/sql/hive/Shim13.scala | 9 +++- 4 files changed, 84 insertions(+), 14 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index af309c0c6ce2c..3563472c7ae81 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} @@ -116,7 +116,7 @@ class HadoopTableReader( val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) + HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } deserializedHadoopRDD @@ -189,9 +189,13 @@ class HadoopTableReader( val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) + // get the table deserializer + val tableSerDe = tableDesc.getDeserializerClass.newInstance() + tableSerDe.initialize(hconf, tableDesc.getProperties) // fill the non partition key attributes - HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, mutableRow) + HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, + mutableRow, tableSerDe) } }.toSeq @@ -261,25 +265,36 @@ private[hive] object HadoopTableReader extends HiveInspectors { * Transform all given raw `Writable`s into `Row`s. * * @param iterator Iterator of all `Writable`s to be transformed - * @param deserializer The `Deserializer` associated with the input `Writable` + * @param rawDeser The `Deserializer` associated with the input `Writable` * @param nonPartitionKeyAttrs Attributes that should be filled together with their corresponding * positions in the output schema * @param mutableRow A reusable `MutableRow` that should be filled + * @param tableDeser Table Deserializer * @return An `Iterator[Row]` transformed from `iterator` */ def fillObject( iterator: Iterator[Writable], - deserializer: Deserializer, + rawDeser: Deserializer, nonPartitionKeyAttrs: Seq[(Attribute, Int)], - mutableRow: MutableRow): Iterator[Row] = { + mutableRow: MutableRow, + tableDeser: Deserializer): Iterator[Row] = { + + val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) { + rawDeser.getObjectInspector.asInstanceOf[StructObjectInspector] + } else { + HiveShim.getConvertedOI( + rawDeser.getObjectInspector, + tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector] + } - val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => soi.getStructFieldRef(attr.name) -> ordinal }.unzip - // Builds specific unwrappers ahead of time according to object inspector types to avoid pattern - // matching and branching costs per row. + /** + * Builds specific unwrappers ahead of time according to object inspector + * types to avoid pattern matching and branching costs per row. + */ val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { _.getFieldObjectInspector match { case oi: BooleanObjectInspector => @@ -316,9 +331,11 @@ private[hive] object HadoopTableReader extends HiveInspectors { } } + val converter = ObjectInspectorConverters.getConverter(rawDeser.getObjectInspector, soi) + // Map each tuple to a row object iterator.map { value => - val raw = deserializer.deserialize(value) + val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 while (i < fieldRefs.length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 381cd2a29123e..aa6fb42de7f88 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -32,9 +32,12 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) +case class ThreeCloumntable(key: Int, value: String, key1: String) + class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ + val testData = TestHive.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() @@ -186,4 +189,43 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { sql("DROP TABLE hiveTableWithStructValue") } + + test("SPARK-5498:partition schema does not match table schema") { + val testData = TestHive.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") + + val testDatawithNull = TestHive.sparkContext.parallelize( + (1 to 10).map(i => ThreeCloumntable(i, i.toString,null))).toDF() + + val tmpDir = Files.createTempDir() + sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") + + // test schema the same between partition and table + sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") + checkAnswer(sql("select key,value from table_with_partition where ds='1' "), + testData.collect.toSeq + ) + + // test difference type of field + sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") + checkAnswer(sql("select key,value from table_with_partition where ds='1' "), + testData.collect.toSeq + ) + + // add column to table + sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)") + checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "), + testDatawithNull.collect.toSeq + ) + + // change column name to table + sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT") + checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "), + testData.collect.toSeq + ) + + sql("DROP TABLE table_with_partition") + } } diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 30646ddbc29d8..0ed93c2c5b1fa 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, ObjectInspector, PrimitiveObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} @@ -210,7 +210,7 @@ private[hive] object HiveShim { def getDataLocationPath(p: Partition) = p.getPartitionPath - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) def compatibilityBlackList = Seq( "decimal_.*", @@ -244,6 +244,12 @@ private[hive] object HiveShim { } } + def getConvertedOI( + inputOI: ObjectInspector, + outputOI: ObjectInspector): ObjectInspector = { + ObjectInspectorConverters.getConvertedOI(inputOI, outputOI, true) + } + def prepareWritable(w: Writable): Writable = { w } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index f9fcbdae15745..7577309900209 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive +import java.util import java.util.{ArrayList => JArrayList} import java.util.Properties import java.rmi.server.UID @@ -38,7 +39,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, PrimitiveObjectInspector, ObjectInspector} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable @@ -400,7 +401,11 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } - + + def getConvertedOI(inputOI: ObjectInspector, outputOI: ObjectInspector): ObjectInspector = { + ObjectInspectorConverters.getConvertedOI(inputOI, outputOI) + } + /* * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that * is needed to initialize before serialization. From 73d57754dd23d84331c10355338a4240b3ac5fee Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 25 Mar 2015 17:52:23 -0700 Subject: [PATCH 537/817] [SPARK-6326][SQL] Improve castStruct to be faster Current `castStruct` should be very slow. This pr slightly improves it. Author: Liang-Chi Hsieh Closes #5017 from viirya/faster_caststruct and squashes the following commits: 385d5b0 [Liang-Chi Hsieh] Further improved. 746fcfb [Liang-Chi Hsieh] Make castStruct faster. --- .../spark/sql/catalyst/expressions/Cast.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 9bde74ac22669..31f1a5fdc7e53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -394,10 +394,17 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w val casts = from.fields.zip(to.fields).map { case (fromField, toField) => cast(fromField.dataType, toField.dataType) } - // TODO: This is very slow! - buildCast[Row](_, row => Row(row.toSeq.zip(casts).map { - case (v, cast) => if (v == null) null else cast(v) - }: _*)) + // TODO: Could be faster? + val newRow = new GenericMutableRow(from.fields.size) + buildCast[Row](_, row => { + var i = 0 + while (i < row.length) { + val v = row(i) + newRow.update(i, if (v == null) null else casts(i)(v)) + i += 1 + } + newRow.copy() + }) } private[this] def cast(from: DataType, to: DataType): Any => Any = to match { From 328daf65f8e320d3bdfd12397a8f85891f1d14c7 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Wed, 25 Mar 2015 18:41:59 -0700 Subject: [PATCH 538/817] [SPARK-6271][SQL] Sort these tokens in alphabetic order to avoid further duplicate in HiveQl Author: DoingDone9 <799203320@qq.com> Closes #4973 from DoingDone9/sort_token and squashes the following commits: 855fa10 [DoingDone9] Update HiveQl.scala c7080b3 [DoingDone9] Sort these tokens in alphabetic order to avoid further duplicate in HiveQl c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../org/apache/spark/sql/hive/HiveQl.scala | 88 ++++++++++--------- 1 file changed, 46 insertions(+), 42 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 51775eb4cd6a0..c45c4ad70fae9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -55,37 +55,8 @@ private[hive] case object NativePlaceholder extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( - "TOK_DESCFUNCTION", - "TOK_DESCDATABASE", - "TOK_SHOW_CREATETABLE", - "TOK_SHOWCOLUMNS", - "TOK_SHOW_TABLESTATUS", - "TOK_SHOWDATABASES", - "TOK_SHOWFUNCTIONS", - "TOK_SHOWINDEXES", - "TOK_SHOWINDEXES", - "TOK_SHOWPARTITIONS", - "TOK_SHOW_TBLPROPERTIES", - - "TOK_LOCKTABLE", - "TOK_SHOWLOCKS", - "TOK_UNLOCKTABLE", - - "TOK_SHOW_ROLES", - "TOK_CREATEROLE", - "TOK_DROPROLE", - "TOK_GRANT", - "TOK_GRANT_ROLE", - "TOK_REVOKE", - "TOK_SHOW_GRANT", - "TOK_SHOW_ROLE_GRANT", - "TOK_SHOW_SET_ROLE", - - "TOK_CREATEFUNCTION", - "TOK_DROPFUNCTION", - - "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERDATABASE_OWNER", + "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", "TOK_ALTERTABLE_ADDCOLS", @@ -102,28 +73,61 @@ private[hive] object HiveQl { "TOK_ALTERTABLE_SKEWED", "TOK_ALTERTABLE_TOUCH", "TOK_ALTERTABLE_UNARCHIVE", - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", - "TOK_CREATEINDEX", - "TOK_DROPDATABASE", - "TOK_DROPINDEX", - "TOK_DROPTABLE_PROPERTIES", - "TOK_MSCK", - "TOK_ALTERVIEW_ADDPARTS", "TOK_ALTERVIEW_AS", "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", + + "TOK_CREATEDATABASE", + "TOK_CREATEFUNCTION", + "TOK_CREATEINDEX", + "TOK_CREATEROLE", "TOK_CREATEVIEW", - "TOK_DROPVIEW_PROPERTIES", + + "TOK_DESCDATABASE", + "TOK_DESCFUNCTION", + + "TOK_DROPDATABASE", + "TOK_DROPFUNCTION", + "TOK_DROPINDEX", + "TOK_DROPROLE", + "TOK_DROPTABLE_PROPERTIES", "TOK_DROPVIEW", - + "TOK_DROPVIEW_PROPERTIES", + "TOK_EXPORT", + + "TOK_GRANT", + "TOK_GRANT_ROLE", + "TOK_IMPORT", + "TOK_LOAD", - - "TOK_SWITCHDATABASE" + + "TOK_LOCKTABLE", + + "TOK_MSCK", + + "TOK_REVOKE", + + "TOK_SHOW_CREATETABLE", + "TOK_SHOW_GRANT", + "TOK_SHOW_ROLE_GRANT", + "TOK_SHOW_ROLES", + "TOK_SHOW_SET_ROLE", + "TOK_SHOW_TABLESTATUS", + "TOK_SHOW_TBLPROPERTIES", + "TOK_SHOWCOLUMNS", + "TOK_SHOWDATABASES", + "TOK_SHOWFUNCTIONS", + "TOK_SHOWINDEXES", + "TOK_SHOWLOCKS", + "TOK_SHOWPARTITIONS", + + "TOK_SWITCHDATABASE", + + "TOK_UNLOCKTABLE" ) // Commands that we do not need to explain. From 5ab6e9f0c06b763b06a0e1396bdb08a823146c32 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 25 Mar 2015 18:43:26 -0700 Subject: [PATCH 539/817] [SPARK-6202] [SQL] enable variable substitution on test framework Author: Daoyuan Wang Closes #4930 from adrian-wang/testvs and squashes the following commits: 2ce590f [Daoyuan Wang] add explicit function types b1d68bf [Daoyuan Wang] only substitute for parseSql 9c4a950 [Daoyuan Wang] add a comment explaining 18fb481 [Daoyuan Wang] enable variable substitute on test framework --- .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index dc61e9d2e3522..a3497eadd67f6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -23,6 +23,7 @@ import java.util.{Set => JavaSet} import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table +import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -153,8 +154,13 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { val describedTable = "DESCRIBE (\\w+)".r + val vs = new VariableSubstitution() + + // we should substitute variables in hql to pass the text to parseSql() as a parameter. + // Hive parser need substituted text. HiveContext.sql() does this but return a DataFrame, + // while we need a logicalPlan so we cannot reuse that. protected[hive] class HiveQLQueryExecution(hql: String) - extends this.QueryExecution(HiveQl.parseSql(hql)) { + extends this.QueryExecution(HiveQl.parseSql(vs.substitute(hiveconf, hql))) { def hiveExec(): Seq[String] = runSqlHive(hql) override def toString: String = hql + "\n" + super.toString } From e87bf3713e684fa83165a1036d76f7a84f043775 Mon Sep 17 00:00:00 2001 From: KaiXinXiaoLei Date: Wed, 25 Mar 2015 19:15:30 -0700 Subject: [PATCH 540/817] =?UTF-8?q?The=20UT=20test=20of=20spark=20is=20fai?= =?UTF-8?q?led.=20Because=20there=20is=20a=20test=20in=20SQLQuerySuite=20a?= =?UTF-8?q?bout=20creating=20table=20=E2=80=9Ctest=E2=80=9D?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If the tests in "sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala" are running before CachedTableSuite.scala, the test("Drop cached table") will failed. Because the table test is created in SQLQuerySuite.scala ,and this table not droped. So when running "drop cached table", table test already exists. There is error info: 01:18:35.738 ERROR hive.ql.exec.DDLTask: org.apache.hadoop.hive.ql.metadata.HiveException: AlreadyExistsException(message:Table test already exists) at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:616) at org.apache.hadoop.hive.ql.exec.DDLTask.createTable(DDLTask.java:4189) at org.apache.hadoop.hive.ql.exec.DDLTask.execute(DDLTask.java:281) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:153) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:85) at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1503) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1270) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1088) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:911) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:901)test” And the test about "create table test" in "sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala,is: test("SPARK-4825 save join to table") { val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF() sql("CREATE TABLE test1 (key INT, value STRING)") testData.insertInto("test1") sql("CREATE TABLE test2 (key INT, value STRING)") testData.insertInto("test2") testData.insertInto("test2") sql("CREATE TABLE test AS SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key") checkAnswer( table("test"), sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) } Author: KaiXinXiaoLei Closes #5150 from KaiXinXiaoLei/testFailed and squashes the following commits: 7534b02 [KaiXinXiaoLei] The UT test of spark is failed. --- .../org/apache/spark/sql/hive/CachedTableSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 44d24273e722a..221a0c263d36c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -92,12 +92,12 @@ class CachedTableSuite extends QueryTest { } test("Drop cached table") { - sql("CREATE TABLE test(a INT)") - cacheTable("test") - sql("SELECT * FROM test").collect() - sql("DROP TABLE test") + sql("CREATE TABLE cachedTableTest(a INT)") + cacheTable("cachedTableTest") + sql("SELECT * FROM cachedTableTest").collect() + sql("DROP TABLE cachedTableTest") intercept[AnalysisException] { - sql("SELECT * FROM test").collect() + sql("SELECT * FROM cachedTableTest").collect() } } From 276ef1c3cfd44b5fc082e1a495fff22fbaf6add3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 25 Mar 2015 19:21:54 -0700 Subject: [PATCH 541/817] [SPARK-6463][SQL] AttributeSet.equal should compare size Previously this could result in sets compare equals when in fact the right was a subset of the left. Based on #5133 by sisihj Author: sisihj Author: Michael Armbrust Closes #5194 from marmbrus/pr/5133 and squashes the following commits: 5ed4615 [Michael Armbrust] fix imports d4cbbc0 [Michael Armbrust] Add test cases 0a0834f [sisihj] AttributeSet.equal should compare size --- .../catalyst/expressions/AttributeSet.scala | 3 +- .../expressions/AttributeSetSuite.scala | 82 +++++++++++++++++++ 2 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index f9ae85a5cfc1b..11b4eb5c888be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -58,7 +58,8 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) /** Returns true if the members of this AttributeSet and other are the same. */ override def equals(other: Any): Boolean = other match { - case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + case otherSet: AttributeSet => + otherSet.size == baseSet.size && baseSet.map(_.a).forall(otherSet.contains) case _ => false } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala new file mode 100644 index 0000000000000..f2f3a84d19380 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.scalatest.FunSuite + +import org.apache.spark.sql.types.IntegerType + +class AttributeSetSuite extends FunSuite { + + val aUpper = AttributeReference("A", IntegerType)(exprId = ExprId(1)) + val aLower = AttributeReference("a", IntegerType)(exprId = ExprId(1)) + val fakeA = AttributeReference("a", IntegerType)(exprId = ExprId(3)) + val aSet = AttributeSet(aLower :: Nil) + + val bUpper = AttributeReference("B", IntegerType)(exprId = ExprId(2)) + val bLower = AttributeReference("b", IntegerType)(exprId = ExprId(2)) + val bSet = AttributeSet(bUpper :: Nil) + + val aAndBSet = AttributeSet(aUpper :: bUpper :: Nil) + + test("sanity check") { + assert(aUpper != aLower) + assert(bUpper != bLower) + } + + test("checks by id not name") { + assert(aSet.contains(aUpper) === true) + assert(aSet.contains(aLower) === true) + assert(aSet.contains(fakeA) === false) + + assert(aSet.contains(bUpper) === false) + assert(aSet.contains(bLower) === false) + } + + test("++ preserves AttributeSet") { + assert((aSet ++ bSet).contains(aUpper) === true) + assert((aSet ++ bSet).contains(aLower) === true) + } + + test("extracts all references references") { + val addSet = AttributeSet(Add(aUpper, Alias(bUpper, "test")()):: Nil) + assert(addSet.contains(aUpper)) + assert(addSet.contains(aLower)) + assert(addSet.contains(bUpper)) + assert(addSet.contains(bLower)) + } + + test("dedups attributes") { + assert(AttributeSet(aUpper :: aLower :: Nil).size === 1) + } + + test("subset") { + assert(aSet.subsetOf(aAndBSet) === true) + assert(aAndBSet.subsetOf(aSet) === false) + } + + test("equality") { + assert(aSet != aAndBSet) + assert(aAndBSet != aSet) + assert(aSet != bSet) + assert(bSet != aSet) + + assert(aSet == aSet) + assert(aSet == AttributeSet(aUpper :: Nil)) + } +} From f535802977c5a3ce45894d89fdf59f8723f023c8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 26 Mar 2015 00:01:24 -0700 Subject: [PATCH 542/817] [SPARK-6536] [PySpark] Column.inSet() in Python ``` >>> df[df.name.inSet("Bob", "Mike")].collect() [Row(age=5, name=u'Bob')] >>> df[df.age.inSet([1, 2, 3])].collect() [Row(age=2, name=u'Alice')] ``` Author: Davies Liu Closes #5190 from davies/in and squashes the following commits: 6b73a47 [Davies Liu] Column.inSet() in Python --- python/pyspark/sql/dataframe.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 5cb89da7a8ed5..bf7c47b7261a9 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -985,6 +985,23 @@ def substr(self, startPos, length): __getslice__ = substr + def inSet(self, *cols): + """ A boolean expression that is evaluated to true if the value of this + expression is contained by the evaluated values of the arguments. + + >>> df[df.name.inSet("Bob", "Mike")].collect() + [Row(age=5, name=u'Bob')] + >>> df[df.age.inSet([1, 2, 3])].collect() + [Row(age=2, name=u'Alice')] + """ + if len(cols) == 1 and isinstance(cols[0], (list, set)): + cols = cols[0] + cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols] + sc = SparkContext._active_spark_context + jcols = ListConverter().convert(cols, sc._gateway._gateway_client) + jc = getattr(self._jc, "in")(sc._jvm.PythonUtils.toSeq(jcols)) + return Column(jc) + # order asc = _unary_op("asc", "Returns a sort expression based on the" " ascending order of the given column name.") From 5bbcd1304cfebba31ec6857a80d3825a40d02e83 Mon Sep 17 00:00:00 2001 From: azagrebin Date: Thu, 26 Mar 2015 00:25:04 -0700 Subject: [PATCH 543/817] [SPARK-6117] [SQL] add describe function to DataFrame for summary statis... Please review my solution for SPARK-6117 Author: azagrebin Closes #5073 from azagrebin/SPARK-6117 and squashes the following commits: f9056ac [azagrebin] [SPARK-6117] [SQL] create one aggregation and split it locally into resulting DF, colocate test data with test case ddb3950 [azagrebin] [SPARK-6117] [SQL] simplify implementation, add test for DF without numeric columns 9daf31e [azagrebin] [SPARK-6117] [SQL] add describe function to DataFrame for summary statistics --- .../org/apache/spark/sql/DataFrame.scala | 53 ++++++++++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 45 ++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) 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 5aece166aad22..db561825e676b 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 @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.sql.types.{NumericType, StructType, StructField, StringType} import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils @@ -751,6 +751,57 @@ class DataFrame private[sql]( select(colNames :_*) } + /** + * Compute numerical statistics for given columns of this [[DataFrame]]: + * count, mean (avg), stddev (standard deviation), min, max. + * Each row of the resulting [[DataFrame]] contains column with statistic name + * and columns with statistic results for each given column. + * If no columns are given then computes for all numerical columns. + * + * {{{ + * df.describe("age", "height") + * + * // summary age height + * // count 10.0 10.0 + * // mean 53.3 178.05 + * // stddev 11.6 15.7 + * // min 18.0 163.0 + * // max 92.0 192.0 + * }}} + */ + @scala.annotation.varargs + def describe(cols: String*): DataFrame = { + + def stddevExpr(expr: Expression) = + Sqrt(Subtract(Average(Multiply(expr, expr)), Multiply(Average(expr), Average(expr)))) + + val statistics = List[(String, Expression => Expression)]( + "count" -> Count, + "mean" -> Average, + "stddev" -> stddevExpr, + "min" -> Min, + "max" -> Max) + + val aggCols = (if (cols.isEmpty) numericColumns.map(_.prettyString) else cols).toList + + val localAgg = if (aggCols.nonEmpty) { + val aggExprs = statistics.flatMap { case (_, colToAgg) => + aggCols.map(c => Column(colToAgg(Column(c).expr)).as(c)) + } + + agg(aggExprs.head, aggExprs.tail: _*).head().toSeq + .grouped(aggCols.size).toSeq.zip(statistics).map { case (aggregation, (statistic, _)) => + Row(statistic :: aggregation.toList: _*) + } + } else { + statistics.map { case (name, _) => Row(name) } + } + + val schema = StructType(("summary" :: aggCols).map(StructField(_, StringType))) + val rowRdd = sqlContext.sparkContext.parallelize(localAgg) + sqlContext.createDataFrame(rowRdd, schema) + } + /** * Returns the first `n` rows. * @group action diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index c30ed694a62f0..afbedd1e5825d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -443,6 +443,51 @@ class DataFrameSuite extends QueryTest { assert(df.schema.map(_.name).toSeq === Seq("key", "valueRenamed", "newCol")) } + test("describe") { + + val describeTestData = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + + val describeResult = Seq( + Row("count", 4, 4), + Row("mean", 33.0, 178.0), + Row("stddev", 16.583123951777, 10.0), + Row("min", 16, 164), + Row("max", 60, 192)) + + val emptyDescribeResult = Seq( + Row("count", 0, 0), + Row("mean", null, null), + Row("stddev", null, null), + Row("min", null, null), + Row("max", null, null)) + + def getSchemaAsSeq(df: DataFrame) = df.schema.map(_.name).toSeq + + val describeTwoCols = describeTestData.describe("age", "height") + assert(getSchemaAsSeq(describeTwoCols) === Seq("summary", "age", "height")) + checkAnswer(describeTwoCols, describeResult) + + val describeAllCols = describeTestData.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "age", "height")) + checkAnswer(describeAllCols, describeResult) + + val describeOneCol = describeTestData.describe("age") + assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) + checkAnswer(describeOneCol, describeResult.map { case Row(s, d, _) => Row(s, d)} ) + + val describeNoCol = describeTestData.select("name").describe() + assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _) => Row(s)} ) + + val emptyDescription = describeTestData.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "age", "height")) + checkAnswer(emptyDescription, emptyDescribeResult) + } + test("apply on query results (SPARK-5462)") { val df = testData.sqlContext.sql("select key from testData") checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) From 855cba8fe59ffe17b51ed00fbbb5d3d7cf17ade9 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Thu, 26 Mar 2015 17:04:19 +0800 Subject: [PATCH 544/817] [SPARK-6546][Build] Using the wrong code that will make spark compile failed!! wrong code : val tmpDir = Files.createTempDir() not Files should Utils Author: DoingDone9 <799203320@qq.com> Closes #5198 from DoingDone9/FilesBug and squashes the following commits: 6e0140d [DoingDone9] Update InsertIntoHiveTableSuite.scala e57d23f [DoingDone9] Update InsertIntoHiveTableSuite.scala 802261c [DoingDone9] Merge pull request #7 from apache/master d00303b [DoingDone9] Merge pull request #6 from apache/master 98b134f [DoingDone9] Merge pull request #5 from apache/master 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index aa6fb42de7f88..8011952e0d535 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -198,7 +198,7 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { val testDatawithNull = TestHive.sparkContext.parallelize( (1 to 10).map(i => ThreeCloumntable(i, i.toString,null))).toDF() - val tmpDir = Files.createTempDir() + val tmpDir = Utils.createTempDir() sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") From f88f51bbd461e0a42ad7021147268509b9c3c56e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 26 Mar 2015 18:46:57 +0800 Subject: [PATCH 545/817] [SPARK-6465][SQL] Fix serialization of GenericRowWithSchema using kryo Author: Michael Armbrust Closes #5191 from marmbrus/kryoRowsWithSchema and squashes the following commits: bb83522 [Michael Armbrust] Fix serialization of GenericRowWithSchema using kryo f914f16 [Michael Armbrust] Add no arg constructor to GenericRowWithSchema --- .../spark/sql/catalyst/expressions/rows.scala | 7 +++++-- .../org/apache/spark/sql/types/Metadata.scala | 3 +++ .../org/apache/spark/sql/types/dataTypes.scala | 18 ++++++++++++++++++ .../sql/execution/SparkSqlSerializer.scala | 4 +--- .../scala/org/apache/spark/sql/RowSuite.scala | 12 ++++++++++++ 5 files changed, 39 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 8bba26bc4cf7f..a8983df208318 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -66,7 +66,7 @@ object EmptyRow extends Row { */ class GenericRow(protected[sql] val values: Array[Any]) extends Row { /** No-arg constructor for serialization. */ - def this() = this(null) + protected def this() = this(null) def this(size: Int) = this(new Array[Any](size)) @@ -172,11 +172,14 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { class GenericRowWithSchema(values: Array[Any], override val schema: StructType) extends GenericRow(values) { + + /** No-arg constructor for serialization. */ + protected def this() = this(null, null) } class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { /** No-arg constructor for serialization. */ - def this() = this(null) + protected def this() = this(null) def this(size: Int) = this(new Array[Any](size)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index e50e9761431f5..6ee24ee0c1913 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -41,6 +41,9 @@ import org.apache.spark.annotation.DeveloperApi sealed class Metadata private[types] (private[types] val map: Map[String, Any]) extends Serializable { + /** No-arg constructor for kryo. */ + protected def this() = this(null) + /** Tests whether this Metadata contains a binding for a key. */ def contains(key: String): Boolean = map.contains(key) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index d973144de3468..952cf5c75688d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -670,6 +670,10 @@ case class PrecisionInfo(precision: Int, scale: Int) */ @DeveloperApi case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null) + private[sql] type JvmType = Decimal @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val numeric = Decimal.DecimalIsFractional @@ -819,6 +823,10 @@ object ArrayType { */ @DeveloperApi case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null, false) + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append( s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") @@ -857,6 +865,9 @@ case class StructField( nullable: Boolean = true, metadata: Metadata = Metadata.empty) { + /** No-arg constructor for kryo. */ + protected def this() = this(null, null) + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") DataType.buildFormattedString(dataType, s"$prefix |", builder) @@ -1003,6 +1014,9 @@ object StructType { @DeveloperApi case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { + /** No-arg constructor for kryo. */ + protected def this() = this(null) + /** Returns all field names in an array. */ def fieldNames: Array[String] = fields.map(_.name) @@ -1121,6 +1135,10 @@ case class MapType( keyType: DataType, valueType: DataType, valueContainsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + def this() = this(null, null, false) + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append(s"$prefix-- key: ${keyType.typeName}\n") builder.append(s"$prefix-- value: ${valueType.typeName} " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index c4534fd5f67e4..967bd76b302d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHa private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { - val kryo = new Kryo() + val kryo = super.newKryo() kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) @@ -57,8 +57,6 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[Decimal]) kryo.setReferences(false) - kryo.setClassLoader(Utils.getSparkClassLoader) - new AllScalaRegistrar().apply(kryo) kryo } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index f5b945f468dad..36465cc2fa11a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.SparkSqlSerializer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ class RowSuite extends FunSuite { @@ -50,4 +53,13 @@ class RowSuite extends FunSuite { row(0) = null assert(row.isNullAt(0)) } + + test("serialize w/ kryo") { + val row = Seq((1, Seq(1), Map(1 -> 1), BigDecimal(1))).toDF().first() + val serializer = new SparkSqlSerializer(TestSQLContext.sparkContext.getConf) + val instance = serializer.newInstance() + val ser = instance.serialize(row) + val de = instance.deserialize(ser).asInstanceOf[Row] + assert(de === row) + } } From 0c88ce5416d7687bc806a7655e17009ad5823d30 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 26 Mar 2015 12:54:48 +0000 Subject: [PATCH 546/817] [SPARK-6468][Block Manager] Fix the race condition of subDirs in DiskBlockManager There are two race conditions of `subDirs` in `DiskBlockManager`: 1. `getAllFiles` does not use correct locks to read the contents in `subDirs`. Although it's designed for testing, it's still worth to add correct locks to eliminate the race condition. 2. The double-check has a race condition in `getFile(filename: String)`. If a thread finds `subDirs(dirId)(subDirId)` is not null out of the `synchronized` block, it may not be able to see the correct content of the File instance pointed by `subDirs(dirId)(subDirId)` according to the Java memory model (there is no volatile variable here). This PR fixed the above race conditions. Author: zsxwing Closes #5136 from zsxwing/SPARK-6468 and squashes the following commits: cbb872b [zsxwing] Fix the race condition of subDirs in DiskBlockManager --- .../spark/storage/DiskBlockManager.scala | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 12cd8ea3bdf1f..2883137872600 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -47,6 +47,8 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } + // The content of subDirs is immutable but the content of subDirs(i) is mutable. And the content + // of subDirs(i) is protected by the lock of subDirs(i) private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private val shutdownHook = addShutdownHook() @@ -61,20 +63,17 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon val subDirId = (hash / localDirs.length) % subDirsPerLocalDir // Create the subdirectory if it doesn't already exist - var subDir = subDirs(dirId)(subDirId) - if (subDir == null) { - subDir = subDirs(dirId).synchronized { - val old = subDirs(dirId)(subDirId) - if (old != null) { - old - } else { - val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) - if (!newDir.exists() && !newDir.mkdir()) { - throw new IOException(s"Failed to create local dir in $newDir.") - } - subDirs(dirId)(subDirId) = newDir - newDir + val subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) + if (!newDir.exists() && !newDir.mkdir()) { + throw new IOException(s"Failed to create local dir in $newDir.") } + subDirs(dirId)(subDirId) = newDir + newDir } } @@ -91,7 +90,12 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** List all the files currently stored on disk by the disk manager. */ def getAllFiles(): Seq[File] = { // Get all the files inside the array of array of directories - subDirs.flatten.filter(_ != null).flatMap { dir => + subDirs.flatMap { dir => + dir.synchronized { + // Copy the content of dir because it may be modified in other threads + dir.clone() + } + }.filter(_ != null).flatMap { dir => val files = dir.listFiles() if (files != null) files else Seq.empty } From 1c05027a143d1b0bf3df192984e6cac752b1e926 Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Thu, 26 Mar 2015 21:13:38 +0800 Subject: [PATCH 547/817] [SQL][SPARK-6471]: Metastore schema should only be a subset of parquet schema to support dropping of columns using replace columns Currently in the parquet relation 2 implementation, error is thrown in case merged schema is not exactly the same as metastore schema. But to support cases like deletion of column using replace column command, we can relax the restriction so that even if metastore schema is a subset of merged parquet schema, the query will work. Author: Yash Datta Closes #5141 from saucam/replace_col and squashes the following commits: e858d5b [Yash Datta] SPARK-6471: Fix test cases, add a new test case for metastore schema to be subset of parquet schema 5f2f467 [Yash Datta] SPARK-6471: Metastore schema should only be a subset of parquet schema to support dropping of columns using replace columns --- .../apache/spark/sql/parquet/newParquet.scala | 5 +++-- .../spark/sql/parquet/ParquetSchemaSuite.scala | 18 ++++++++++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) 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 410600b0529d3..3516cfe680c61 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 @@ -758,12 +758,13 @@ private[sql] object ParquetRelation2 extends Logging { |${parquetSchema.prettyJson} """.stripMargin - assert(metastoreSchema.size == parquetSchema.size, schemaConflictMessage) + assert(metastoreSchema.size <= parquetSchema.size, schemaConflictMessage) val ordinalMap = metastoreSchema.zipWithIndex.map { case (field, index) => field.name.toLowerCase -> index }.toMap - val reorderedParquetSchema = parquetSchema.sortBy(f => ordinalMap(f.name.toLowerCase)) + val reorderedParquetSchema = parquetSchema.sortBy(f => + ordinalMap.getOrElse(f.name.toLowerCase, metastoreSchema.size + 1)) StructType(metastoreSchema.zip(reorderedParquetSchema).map { // Uses Parquet field names but retains Metastore data types. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 321832cd43211..8462f9bb2d620 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -212,8 +212,11 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { StructField("UPPERCase", IntegerType, nullable = true)))) } - // Conflicting field count - assert(intercept[Throwable] { + // MetaStore schema is subset of parquet schema + assertResult( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = false)))) { + ParquetRelation2.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false))), @@ -221,6 +224,17 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { StructType(Seq( StructField("lowerCase", BinaryType), StructField("UPPERCase", IntegerType, nullable = true)))) + } + + // Conflicting field count + assert(intercept[Throwable] { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("uppercase", DoubleType, nullable = false), + StructField("lowerCase", BinaryType))), + + StructType(Seq( + StructField("UPPERCase", IntegerType, nullable = true)))) }.getMessage.contains("detected conflicting schemas")) // Conflicting field names From 3ddb975faeddeb2674a7e7f7e80cf90dfbd4d6d2 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 26 Mar 2015 13:27:05 +0000 Subject: [PATCH 548/817] [MLlib]remove unused import minor thing. Let me know if jira is required. Author: Yuhao Yang Closes #5207 from hhbyyh/adjustImport and squashes the following commits: 2240121 [Yuhao Yang] remove unused import --- .../src/main/scala/org/apache/spark/mllib/clustering/LDA.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 5e17c8da61134..9d63a08e211bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import java.util.Random -import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} +import breeze.linalg.{DenseVector => BDV, normalize} import org.apache.spark.Logging import org.apache.spark.annotation.Experimental From fe15ea976073edd738c006af1eb8d31617a039fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 Mar 2015 15:00:23 +0000 Subject: [PATCH 549/817] SPARK-6480 [CORE] histogram() bucket function is wrong in some simple edge cases Fix fastBucketFunction for histogram() to handle edge conditions more correctly. Add a test, and fix existing one accordingly Author: Sean Owen Closes #5148 from srowen/SPARK-6480 and squashes the following commits: 974a0a0 [Sean Owen] Additional test of huge ranges, and a few more comments (and comment fixes) 23ec01e [Sean Owen] Fix fastBucketFunction for histogram() to handle edge conditions more correctly. Add a test, and fix existing one accordingly --- .../apache/spark/rdd/DoubleRDDFunctions.scala | 20 +++++++--------- .../org/apache/spark/rdd/DoubleRDDSuite.scala | 24 +++++++++++++++---- 2 files changed, 29 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 03afc289736bb..71e6e300fec5f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -191,25 +191,23 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } } // Determine the bucket function in constant time. Requires that buckets are evenly spaced - def fastBucketFunction(min: Double, increment: Double, count: Int)(e: Double): Option[Int] = { + def fastBucketFunction(min: Double, max: Double, count: Int)(e: Double): Option[Int] = { // If our input is not a number unless the increment is also NaN then we fail fast - if (e.isNaN()) { - return None - } - val bucketNumber = (e - min)/(increment) - // We do this rather than buckets.lengthCompare(bucketNumber) - // because Array[Double] fails to override it (for now). - if (bucketNumber > count || bucketNumber < 0) { + if (e.isNaN || e < min || e > max) { None } else { - Some(bucketNumber.toInt.min(count - 1)) + // Compute ratio of e's distance along range to total range first, for better precision + val bucketNumber = (((e - min) / (max - min)) * count).toInt + // should be less than count, but will equal count if e == max, in which case + // it's part of the last end-range-inclusive bucket, so return count-1 + Some(math.min(bucketNumber, count - 1)) } } // Decide which bucket function to pass to histogramPartition. We decide here - // rather than having a general function so that the decission need only be made + // rather than having a general function so that the decision need only be made // once rather than once per shard val bucketFunction = if (evenBuckets) { - fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _ + fastBucketFunction(buckets.head, buckets.last, buckets.length - 1) _ } else { basicBucketFunction _ } diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 4cd0f97368ca3..97079382c716f 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -235,6 +235,12 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithDoubleValuesAtMinMax") { + val rdd = sc.parallelize(Seq(1, 1, 1, 2, 3, 3)) + assert(Array(3, 0, 1, 2) === rdd.map(_.toDouble).histogram(4)._2) + assert(Array(3, 1, 2) === rdd.map(_.toDouble).histogram(3)._2) + } + test("WorksWithoutBucketsWithMoreRequestedThanElements") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2)) @@ -248,7 +254,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { } test("WorksWithoutBucketsForLargerDatasets") { - // Verify the case of slighly larger datasets + // Verify the case of slightly larger datasets val rdd = sc.parallelize(6 to 99) val (histogramBuckets, histogramResults) = rdd.histogram(8) val expectedHistogramResults = @@ -259,17 +265,27 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } - test("WorksWithoutBucketsWithIrrationalBucketEdges") { - // Verify the case of buckets with irrational edges. See #SPARK-2862. + test("WorksWithoutBucketsWithNonIntegralBucketEdges") { + // Verify the case of buckets with nonintegral edges. See #SPARK-2862. val rdd = sc.parallelize(6 to 99) val (histogramBuckets, histogramResults) = rdd.histogram(9) + // Buckets are 6.0, 16.333333333333336, 26.666666666666668, 37.0, 47.333333333333336 ... val expectedHistogramResults = - Array(11, 10, 11, 10, 10, 11, 10, 10, 11) + Array(11, 10, 10, 11, 10, 10, 11, 10, 11) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets(0) === 6.0) assert(histogramBuckets(9) === 99.0) } + test("WorksWithHugeRange") { + val rdd = sc.parallelize(Array(0, 1.0e24, 1.0e30)) + val histogramResults = rdd.histogram(1000000)._2 + assert(histogramResults(0) === 1) + assert(histogramResults(1) === 1) + assert(histogramResults.last === 1) + assert((2 to histogramResults.length - 2).forall(i => histogramResults(i) == 0)) + } + // Test the failure mode with an invalid RDD test("ThrowsExceptionOnInvalidRDDs") { // infinity From c3a52a08248db08eade29b265f02483144a282d6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 26 Mar 2015 10:52:31 -0700 Subject: [PATCH 550/817] SPARK-6532 [BUILD] LDAModel.scala fails scalastyle on Windows Use standard UTF-8 source / report encoding for scalastyle Author: Sean Owen Closes #5211 from srowen/SPARK-6532 and squashes the following commits: 16a33e5 [Sean Owen] Use standard UTF-8 source / report encoding for scalastyle --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 23bb16130b504..b3cecd1893a06 100644 --- a/pom.xml +++ b/pom.xml @@ -1452,7 +1452,8 @@ ${basedir}/src/test/scala scalastyle-config.xml scalastyle-output.xml - UTF-8 + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} From 784fcd532784fcfd9bf0a1db71c9f71c469ee716 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 Mar 2015 12:26:13 -0700 Subject: [PATCH 551/817] [SPARK-6117] [SQL] Improvements to DataFrame.describe() 1. Slightly modifications to the code to make it more readable. 2. Added Python implementation. 3. Updated the documentation to state that we don't guarantee the output schema for this function and it should only be used for exploratory data analysis. Author: Reynold Xin Closes #5201 from rxin/df-describe and squashes the following commits: 25a7834 [Reynold Xin] Reset run-tests. 6abdfee [Reynold Xin] [SPARK-6117] [SQL] Improvements to DataFrame.describe() --- python/pyspark/sql/dataframe.py | 19 ++++++++ .../org/apache/spark/sql/DataFrame.scala | 46 +++++++++++-------- .../org/apache/spark/sql/DataFrameSuite.scala | 3 +- 3 files changed, 48 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index bf7c47b7261a9..d51309f7ef5aa 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -520,6 +520,25 @@ def sort(self, *cols): orderBy = sort + def describe(self, *cols): + """Computes statistics for numeric columns. + + This include count, mean, stddev, min, and max. If no columns are + given, this function computes statistics for all numerical columns. + + >>> df.describe().show() + summary age + count 2 + mean 3.5 + stddev 1.5 + min 2 + max 5 + """ + cols = ListConverter().convert(cols, + self.sql_ctx._sc._gateway._gateway_client) + jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) + return DataFrame(jdf, self.sql_ctx) + def head(self, n=None): """ Return the first `n` rows or the first row if n is None. 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 db561825e676b..4c80359cf07af 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 @@ -33,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{expressions, ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.types.{NumericType, StructType, StructField, StringType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils @@ -752,15 +752,17 @@ class DataFrame private[sql]( } /** - * Compute numerical statistics for given columns of this [[DataFrame]]: - * count, mean (avg), stddev (standard deviation), min, max. - * Each row of the resulting [[DataFrame]] contains column with statistic name - * and columns with statistic results for each given column. - * If no columns are given then computes for all numerical columns. + * Computes statistics for numeric columns, including count, mean, stddev, min, and max. + * If no columns are given, this function computes statistics for all numerical columns. + * + * This function is meant for exploratory data analysis, as we make no guarantee about the + * backward compatibility of the schema of the resulting [[DataFrame]]. If you want to + * programmatically compute summary statistics, use the `agg` function instead. * * {{{ - * df.describe("age", "height") + * df.describe("age", "height").show() * + * // output: * // summary age height * // count 10.0 10.0 * // mean 53.3 178.05 @@ -768,13 +770,17 @@ class DataFrame private[sql]( * // min 18.0 163.0 * // max 92.0 192.0 * }}} + * + * @group action */ @scala.annotation.varargs def describe(cols: String*): DataFrame = { - def stddevExpr(expr: Expression) = + // TODO: Add stddev as an expression, and remove it from here. + def stddevExpr(expr: Expression): Expression = Sqrt(Subtract(Average(Multiply(expr, expr)), Multiply(Average(expr), Average(expr)))) + // The list of summary statistics to compute, in the form of expressions. val statistics = List[(String, Expression => Expression)]( "count" -> Count, "mean" -> Average, @@ -782,24 +788,28 @@ class DataFrame private[sql]( "min" -> Min, "max" -> Max) - val aggCols = (if (cols.isEmpty) numericColumns.map(_.prettyString) else cols).toList + val outputCols = (if (cols.isEmpty) numericColumns.map(_.prettyString) else cols).toList - val localAgg = if (aggCols.nonEmpty) { + val ret: Seq[Row] = if (outputCols.nonEmpty) { val aggExprs = statistics.flatMap { case (_, colToAgg) => - aggCols.map(c => Column(colToAgg(Column(c).expr)).as(c)) + outputCols.map(c => Column(colToAgg(Column(c).expr)).as(c)) } - agg(aggExprs.head, aggExprs.tail: _*).head().toSeq - .grouped(aggCols.size).toSeq.zip(statistics).map { case (aggregation, (statistic, _)) => - Row(statistic :: aggregation.toList: _*) + val row = agg(aggExprs.head, aggExprs.tail: _*).head().toSeq + + // Pivot the data so each summary is one row + row.grouped(outputCols.size).toSeq.zip(statistics).map { + case (aggregation, (statistic, _)) => Row(statistic :: aggregation.toList: _*) } } else { + // If there are no output columns, just output a single column that contains the stats. statistics.map { case (name, _) => Row(name) } } - val schema = StructType(("summary" :: aggCols).map(StructField(_, StringType))) - val rowRdd = sqlContext.sparkContext.parallelize(localAgg) - sqlContext.createDataFrame(rowRdd, schema) + // The first column is string type, and the rest are double type. + val schema = StructType( + StructField("summary", StringType) :: outputCols.map(StructField(_, DoubleType))).toAttributes + LocalRelation(schema, ret) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index afbedd1e5825d..fbc4065a9666c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -444,7 +444,6 @@ class DataFrameSuite extends QueryTest { } test("describe") { - val describeTestData = Seq( ("Bob", 16, 176), ("Alice", 32, 164), @@ -465,7 +464,7 @@ class DataFrameSuite extends QueryTest { Row("min", null, null), Row("max", null, null)) - def getSchemaAsSeq(df: DataFrame) = df.schema.map(_.name).toSeq + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) val describeTwoCols = describeTestData.describe("age", "height") assert(getSchemaAsSeq(describeTwoCols) === Seq("summary", "age", "height")) From 71a0d40ebd37c80d8020e184366778b57c762285 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 26 Mar 2015 13:11:37 -0700 Subject: [PATCH 552/817] [SPARK-6554] [SQL] Don't push down predicates which reference partition column(s) There are two cases for the new Parquet data source: 1. Partition columns exist in the Parquet data files We don't need to push-down these predicates since partition pruning already handles them. 1. Partition columns don't exist in the Parquet data files We can't push-down these predicates since they are considered as invalid columns by Parquet. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5210) Author: Cheng Lian Closes #5210 from liancheng/spark-6554 and squashes the following commits: 4f7ec03 [Cheng Lian] Adds comments e134ced [Cheng Lian] Don't push down predicates which reference partition column(s) --- .../apache/spark/sql/parquet/newParquet.scala | 17 ++++++++++++----- .../spark/sql/parquet/ParquetFilterSuite.scala | 17 +++++++++++++++++ 2 files changed, 29 insertions(+), 5 deletions(-) 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 3516cfe680c61..0d68810ec6043 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 @@ -435,11 +435,18 @@ private[sql] case class ParquetRelation2( // Push down filters when possible. Notice that not all filters can be converted to Parquet // filter predicate. Here we try to convert each individual predicate and only collect those // convertible ones. - predicates - .flatMap(ParquetFilters.createFilter) - .reduceOption(FilterApi.and) - .filter(_ => sqlContext.conf.parquetFilterPushDown) - .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) + if (sqlContext.conf.parquetFilterPushDown) { + predicates + // Don't push down predicates which reference partition columns + .filter { pred => + val partitionColNames = partitionColumns.map(_.name).toSet + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } + .flatMap(ParquetFilters.createFilter) + .reduceOption(FilterApi.and) + .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) + } if (isPartitioned) { logInfo { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index 4d32e84fc1115..6a2c2a7c4080a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -321,6 +321,23 @@ class ParquetDataSourceOnFilterSuite extends ParquetFilterSuiteBase with BeforeA override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6554: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + checkAnswer( + sqlContext.parquetFile(path).filter("part = 1"), + (1 to 3).map(i => Row(i, i.toString, 1))) + } + } + } } class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with BeforeAndAfterAll { From aad00322765d6041e817a6bd3fcff2187d212057 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 26 Mar 2015 14:51:46 -0700 Subject: [PATCH 553/817] [DOCS][SQL] Fix JDBC example Author: Michael Armbrust Closes #5192 from marmbrus/fixJDBCDocs and squashes the following commits: b48a33d [Michael Armbrust] [DOCS][SQL] Fix JDBC example --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c99a0b03442c4..4441d6a000a02 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1406,7 +1406,7 @@ DataFrame jdbcDF = sqlContext.load("jdbc", options) {% highlight python %} -df = sqlContext.load("jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") +df = sqlContext.load(source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") {% endhighlight %} From 39fb57968352549f2276ac4fcd2b92988ed6fe42 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Thu, 26 Mar 2015 19:08:09 -0700 Subject: [PATCH 554/817] [SPARK-6510][GraphX]: Add Graph#minus method to act as Set#difference Adds a `Graph#minus` method which will return only unique `VertexId`'s from the calling `VertexRDD`. To demonstrate a basic example with pseudocode: ``` Set((0L,0),(1L,1)).minus(Set((1L,1),(2L,2))) > Set((0L,0)) ``` Author: Brennon York Closes #5175 from brennonyork/SPARK-6510 and squashes the following commits: 248d5c8 [Brennon York] added minus(VertexRDD[VD]) method to avoid createUsingIndex and updated the mask operations to simplify with andNot call 3fb7cce [Brennon York] updated graphx doc to reflect the addition of minus method 6575d92 [Brennon York] updated mima exclude aaa030b [Brennon York] completed graph#minus functionality 7227c0f [Brennon York] beginning work on minus functionality --- docs/graphx-programming-guide.md | 2 ++ .../org/apache/spark/graphx/VertexRDD.scala | 16 +++++++++ .../graphx/impl/VertexPartitionBaseOps.scala | 15 +++++++++ .../spark/graphx/impl/VertexRDDImpl.scala | 25 ++++++++++++++ .../apache/spark/graphx/VertexRDDSuite.scala | 33 +++++++++++++++++-- project/MimaExcludes.scala | 3 ++ 6 files changed, 92 insertions(+), 2 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index c601d793a2e9a..3f10cb2dc3d2a 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -899,6 +899,8 @@ class VertexRDD[VD] extends RDD[(VertexID, VD)] { // Transform the values without changing the ids (preserves the internal index) def mapValues[VD2](map: VD => VD2): VertexRDD[VD2] def mapValues[VD2](map: (VertexId, VD) => VD2): VertexRDD[VD2] + // Show only vertices unique to this set based on their VertexId's + def minus(other: RDD[(VertexId, VD)]) // Remove vertices from this set that appear in the other set def diff(other: VertexRDD[VD]): VertexRDD[VD] // Join operators that take advantage of the internal indexing to accelerate joins (substantially) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index ad4bfe077293a..a9f04b559c3d1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -121,6 +121,22 @@ abstract class VertexRDD[VD]( */ def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other an RDD to run the set operation against + */ + def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] + + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other a VertexRDD to run the set operation against + */ + def minus(other: VertexRDD[VD]): VertexRDD[VD] + /** * For each vertex present in both `this` and `other`, `diff` returns only those vertices with * differing values; for values that are different, keeps the values from `other`. This is diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 4fd2548b7faf6..b90f9fa327052 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -88,6 +88,21 @@ private[graphx] abstract class VertexPartitionBaseOps this.withMask(newMask) } + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Minus operations on two VertexPartitions with different indexes is slow.") + minus(createUsingIndex(other.iterator)) + } else { + self.withMask(self.mask.andNot(other.mask)) + } + } + + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Iterator[(VertexId, VD)]): Self[VD] = { + minus(createUsingIndex(other)) + } + /** * Hides vertices that are the same between this and other. For vertices that are different, keeps * the values from `other`. The indices of `this` and `other` must be the same. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 125692ddaad83..349c8545bf201 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -103,6 +103,31 @@ class VertexRDDImpl[VD] private[graphx] ( override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + override def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + minus(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) + } + + override def minus (other: VertexRDD[VD]): VertexRDD[VD] = { + other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + this.withPartitionsRDD[VD]( + partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.minus(otherPart)) + }) + case _ => + this.withPartitionsRDD[VD]( + partitionsRDD.zipPartitions( + other.partitionBy(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.minus(msgs)) + } + ) + } + } + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { diff(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 4f7a442ab503d..c9443d11c76cf 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -47,6 +47,35 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("minus") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB = VertexRDD(sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1))).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + } + } + + test("minus with RDD[(VertexId, VD)]") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB: RDD[(VertexId, Int)] = + sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1)).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + } + } + + test("minus with non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 5).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(50 until 100, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect.toSet === (0 until 50).toSet) + } + } + test("diff") { withSpark { sc => val n = 100 @@ -71,7 +100,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } - test("diff vertices with the non-equal number of partitions") { + test("diff vertices with non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) @@ -96,7 +125,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } - test("leftJoin vertices with the non-equal number of partitions") { + test("leftJoin vertices with non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) val vertexB = VertexRDD( diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 56f5dbe53fad4..b9f40046e15a2 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -51,6 +51,9 @@ object MimaExcludes { "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"), ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast") + ) ++ Seq( + // SPARK-6510 Add a Graph#minus method acting as Set#difference + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") ) case v if v.startsWith("1.3") => From 49d2ec63eccec8a3a78b15b583c36f84310fc6f0 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 26 Mar 2015 22:48:42 -0700 Subject: [PATCH 555/817] [SPARK-6405] Limiting the maximum Kryo buffer size to be 2GB. Kryo buffers are backed by byte arrays, but primitive arrays can only be up to 2GB in size. It is misleading to allow users to set buffers past this size. Author: mcheah Closes #5218 from mccheah/feature/limit-kryo-buffer and squashes the following commits: 1d6d1be [mcheah] Fixing numeric typo e2e30ce [mcheah] Removing explicit int and double type to match style 09fd80b [mcheah] Should be >= not >. Slightly more consistent error message. 60634f9 [mcheah] [SPARK-6405] Limiting the maximum Kryo buffer size to be 2GB. --- .../apache/spark/serializer/KryoSerializer.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index f83bcaa5cc09e..579fb6624e692 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -49,10 +49,20 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSize = - (conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) * 1024 * 1024).toInt + private val bufferSizeMb = conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) + if (bufferSizeMb >= 2048) { + throw new IllegalArgumentException("spark.kryoserializer.buffer.mb must be less than " + + s"2048 mb, got: + $bufferSizeMb mb.") + } + private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt + + val maxBufferSizeMb = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) + if (maxBufferSizeMb >= 2048) { + throw new IllegalArgumentException("spark.kryoserializer.buffer.max.mb must be less than " + + s"2048 mb, got: + $maxBufferSizeMb mb.") + } + private val maxBufferSize = maxBufferSizeMb * 1024 * 1024 - private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) private val userRegistrator = conf.getOption("spark.kryo.registrator") From f43a61031fd7d9d4fab3d8ac584e7b4c7c5e1035 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Fri, 27 Mar 2015 00:15:02 -0700 Subject: [PATCH 556/817] [SPARK-6341][mllib] Upgrade breeze from 0.11.1 to 0.11.2 There are any bugs of breeze's SparseVector at 0.11.1. You know, Spark 1.3 depends on breeze 0.11.1. So I think we should upgrade it to 0.11.2. https://issues.apache.org/jira/browse/SPARK-6341 And thanks you for your great cooperation, David Hall(dlwh) Author: Yu ISHIKAWA Closes #5222 from yu-iskw/upgrade-breeze and squashes the following commits: ad8a688 [Yu ISHIKAWA] Upgrade breeze from 0.11.1 to 0.11.2 because of a bug of SparseVector. Thanks you for your great cooperation, David Hall(@dlwh) --- mllib/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 4c183543e3fa8..5dfab36c76907 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -64,7 +64,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.11.1 + 0.11.2 From da546b7ba03d84d7f6af97fe04471b12f5b3392f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 27 Mar 2015 12:31:06 +0000 Subject: [PATCH 557/817] [SPARK-6556][Core] Fix wrong parsing logic of executorTimeoutMs and checkTimeoutIntervalMs in HeartbeatReceiver The current reading logic of `executorTimeoutMs` is: ```Scala private val executorTimeoutMs = sc.conf.getLong("spark.network.timeout", sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120)) * 1000 ``` So if `spark.storage.blockManagerSlaveTimeoutMs` is 10000 and `spark.network.timeout` is not set, executorTimeoutMs will be 10000 * 1000. But the correct value should have been 10000. `checkTimeoutIntervalMs` has the same issue. This PR fixes them. Author: zsxwing Closes #5209 from zsxwing/SPARK-6556 and squashes the following commits: 6a0a411 [zsxwing] Fix docs c7d5422 [zsxwing] Add comments for executorTimeoutMs and checkTimeoutIntervalMs ccd5147 [zsxwing] Fix wrong parsing logic of executorTimeoutMs and checkTimeoutIntervalMs in HeartbeatReceiver --- .../org/apache/spark/HeartbeatReceiver.scala | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 715f292f03469..548dcb93c3358 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -49,12 +49,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] - - private val executorTimeoutMs = sc.conf.getLong("spark.network.timeout", - sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120)) * 1000 - - private val checkTimeoutIntervalMs = sc.conf.getLong("spark.network.timeoutInterval", - sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60)) * 1000 + + // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses + // "milliseconds" + private val executorTimeoutMs = sc.conf.getOption("spark.network.timeout").map(_.toLong * 1000). + getOrElse(sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120000)) + + // "spark.network.timeoutInterval" uses "seconds", while + // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" + private val checkTimeoutIntervalMs = + sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). + getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) private var timeoutCheckingTask: Cancellable = null From aa2b9917489f9bbb02c8acea5ff43335042e2705 Mon Sep 17 00:00:00 2001 From: Dean Chen Date: Fri, 27 Mar 2015 14:32:51 +0000 Subject: [PATCH 558/817] [SPARK-6544][build] Increment Avro version from 1.7.6 to 1.7.7 Fixes bug causing Kryo serialization to fail with Avro files in between stages. https://issues.apache.org/jira/browse/AVRO-1476?focusedCommentId=13999249&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13999249 Author: Dean Chen Closes #5193 from deanchen/SPARK-6544 and squashes the following commits: 813d4c5 [Dean Chen] [SPARK-6544][build] Increment Avro version from 1.7.6 to 1.7.7 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b3cecd1893a06..3eb3da2cd8af3 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 2.4.0 2.0.8 3.1.0 - 1.7.6 + 1.7.7 0.7.1 1.8.3 From 5d9c37c23d1edd91e6c5561780006b762cde5f66 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 27 Mar 2015 11:40:00 -0700 Subject: [PATCH 559/817] [SPARK-6550][SQL] Use analyzed plan in DataFrame This is based on bug and test case proposed by viirya. See #5203 for a excellent description of the problem. TLDR; The problem occurs because the function `groupBy(String)` calls `resolve`, which returns an `AttributeReference`. However, this `AttributeReference` is based on an analyzed plan which is thrown away. At execution time, we once again analyze the plan. However, in the case of self-joins, each call to analyze will produce a new tree for the left side of the join, rendering the previously returned `AttributeReference` invalid. As a fix, I propose we keep the analyzed plan instead of the unresolved plan inside of a `DataFrame`. Author: Michael Armbrust Closes #5217 from marmbrus/preanalyzer and squashes the following commits: 1f98e2d [Michael Armbrust] revert change dd4dec1 [Michael Armbrust] Use the analyzed plan in DataFrame 089c52e [Michael Armbrust] WIP --- sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala | 2 +- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 4c80359cf07af..423ef3912bc89 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 @@ -146,7 +146,7 @@ class DataFrame private[sql]( _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => - queryExecution.logical + queryExecution.analyzed } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index fbc4065a9666c..5f03805d70416 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -113,6 +113,10 @@ class DataFrameSuite extends QueryTest { checkAnswer( df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) } test("explode") { From 887e1b72dfa5965f8ab1aad212fb33bb365b0e1b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 27 Mar 2015 11:42:26 -0700 Subject: [PATCH 560/817] [SPARK-6574] [PySpark] fix sql example Fix the import in sql example. Author: Davies Liu Closes #5230 from davies/fix_sql_example and squashes the following commits: 7ecc5f4 [Davies Liu] fix sql example --- examples/src/main/python/sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 47202fde7510b..d89361f324917 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -19,7 +19,7 @@ from pyspark import SparkContext from pyspark.sql import SQLContext -from pyspark.sql import Row, StructField, StructType, StringType, IntegerType +from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": From d5497ab1343e4d1b2a1c336f2e3520d74c6674a1 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 27 Mar 2015 13:29:10 -0700 Subject: [PATCH 561/817] [SPARK-6526][ML] Add Normalizer transformer in ML package See [SPARK-6526](https://issues.apache.org/jira/browse/SPARK-6526). mengxr Should we add test suite for this transformer? There is no test suite for all feature transformers in ML package now. Author: Xusen Yin Closes #5181 from yinxusen/SPARK-6526 and squashes the following commits: 6faa7bf [Xusen Yin] fix style 8a462da [Xusen Yin] remove duplications ab35ab0 [Xusen Yin] add test suite bc8cd0f [Xusen Yin] fix comment 79774c9 [Xusen Yin] add Normalizer transformer in ML package --- .../apache/spark/ml/feature/Normalizer.scala | 53 +++++++++ .../spark/ml/feature/NormalizerSuite.scala | 109 ++++++++++++++++++ 2 files changed, 162 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala new file mode 100644 index 0000000000000..05f91dc9105fe --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.{DoubleParam, ParamMap} +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.linalg.{VectorUDT, Vector} +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Normalize a vector to have unit norm using the given p-norm. + */ +@AlphaComponent +class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { + + /** + * Normalization in L^p^ space, p = 2 by default. + * @group param + */ + val p = new DoubleParam(this, "p", "the p norm value", Some(2)) + + /** @group getParam */ + def getP: Double = get(p) + + /** @group setParam */ + def setP(value: Double): this.type = set(p, value) + + override protected def createTransformFunc(paramMap: ParamMap): Vector => Vector = { + val normalizer = new feature.Normalizer(paramMap(p)) + normalizer.transform + } + + override protected def outputDataType: DataType = new VectorUDT() +} + diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala new file mode 100644 index 0000000000000..a18c335952b96 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +private case class DataSet(features: Vector) + +class NormalizerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var data: Array[Vector] = _ + @transient var dataFrame: DataFrame = _ + @transient var normalizer: Normalizer = _ + @transient var l1Normalized: Array[Vector] = _ + @transient var l2Normalized: Array[Vector] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq((1, 0.91), (2, 3.2))), + Vectors.sparse(3, Seq((0, 5.7), (1, 0.72), (2, 2.7))), + Vectors.sparse(3, Seq()) + ) + l1Normalized = Array( + Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.12765957, -0.23404255, -0.63829787), + Vectors.sparse(3, Seq((1, 0.22141119), (2, 0.7785888))), + Vectors.dense(0.625, 0.07894737, 0.29605263), + Vectors.sparse(3, Seq()) + ) + l2Normalized = Array( + Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.184549876, -0.3383414, -0.922749378), + Vectors.sparse(3, Seq((1, 0.27352993), (2, 0.96186349))), + Vectors.dense(0.897906166, 0.113419726, 0.42532397), + Vectors.sparse(3, Seq()) + ) + + val sqlContext = new SQLContext(sc) + dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(DataSet)) + normalizer = new Normalizer() + .setInputCol("features") + .setOutputCol("normalized_features") + } + + def collectResult(result: DataFrame): Array[Vector] = { + result.select("normalized_features").collect().map { + case Row(features: Vector) => features + } + } + + def assertTypeOfVector(lhs: Array[Vector], rhs: Array[Vector]): Unit = { + assert((lhs, rhs).zipped.forall { + case (v1: DenseVector, v2: DenseVector) => true + case (v1: SparseVector, v2: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + } + + def assertValues(lhs: Array[Vector], rhs: Array[Vector]): Unit = { + assert((lhs, rhs).zipped.forall { (vector1, vector2) => + vector1 ~== vector2 absTol 1E-5 + }, "The vector value is not correct after normalization.") + } + + test("Normalization with default parameter") { + val result = collectResult(normalizer.transform(dataFrame)) + + assertTypeOfVector(data, result) + + assertValues(result, l2Normalized) + } + + test("Normalization with setter") { + normalizer.setP(1) + + val result = collectResult(normalizer.transform(dataFrame)) + + assertTypeOfVector(data, result) + + assertValues(result, l1Normalized) + } +} From 3af7334304341fba091aa39ce2efbdfd167c697b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 27 Mar 2015 14:56:57 -0700 Subject: [PATCH 562/817] [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 row, not 1 row Author: Reynold Xin Closes #5226 from rxin/empty-df and squashes the following commits: 1306d88 [Reynold Xin] Proper fix. e135bb9 [Reynold Xin] [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 rows, not 1 row. --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 5 ++++- .../sql/catalyst/optimizer/ExpressionOptimizationSuite.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala | 4 ++-- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- .../org/apache/spark/sql/execution/SparkStrategies.scala | 2 +- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 5 +++++ .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 4 ++-- 8 files changed, 18 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ea7d44a3723d1..b176f7e729a42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -139,7 +139,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { sortType.? ~ (LIMIT ~> expression).? ^^ { case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => - val base = r.getOrElse(NoRelation) + val base = r.getOrElse(OneRowRelation) val withFilter = f.map(Filter(_, base)).getOrElse(base) val withProjection = g .map(Aggregate(_, assignAliases(p), withFilter)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 4d9e41a2b5d85..190209238a4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -287,7 +287,10 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output } -case object NoRelation extends LeafNode { +/** + * A relation with one row. This is used in "SELECT ..." without a from clause. + */ +case object OneRowRelation extends LeafNode { override def output: Seq[Attribute] = Nil /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index ae99a3f9ba287..2f3704be59a9d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -29,7 +29,7 @@ class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) + val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation) val optimizedPlan = DefaultOptimizer(plan) super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 48884040bfce7..129d091ca03e3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{NoRelation, Filter, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Filter, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** @@ -55,6 +55,6 @@ class PlanTest extends FunSuite { /** Fails the test if the two expressions do not match */ protected def compareExpressions(e1: Expression, e2: Expression): Unit = { - comparePlans(Filter(e1, NoRelation), Filter(e2, NoRelation)) + comparePlans(Filter(e1, OneRowRelation), Filter(e2, OneRowRelation)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index e59cf9b9e037b..b8100782ec937 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -31,7 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, NoRelation} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, OneRowRelation} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.{ScalaReflection, expressions} import org.apache.spark.sql.execution.{Filter, _} @@ -177,7 +177,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental @transient - lazy val emptyDataFrame = DataFrame(this, NoRelation) + lazy val emptyDataFrame: DataFrame = createDataFrame(sparkContext.emptyRDD[Row], StructType(Nil)) /** * A collection of methods for registering user-defined functions (UDF). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 2b581152e5f77..f754fa770d1b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -296,7 +296,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil - case logical.NoRelation => + case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 5f03805d70416..6761d996fd975 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -84,6 +84,11 @@ class DataFrameSuite extends QueryTest { testData.collect().toSeq) } + test("empty data frame") { + assert(TestSQLContext.emptyDataFrame.columns.toSeq === Seq.empty[String]) + assert(TestSQLContext.emptyDataFrame.count() === 0) + } + test("head and take") { assert(testData.take(2) === testData.collect().take(2)) assert(testData.head(2) === testData.collect().take(2)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index c45c4ad70fae9..cd8e7c09eea5b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -479,7 +479,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => - ExplainCommand(NoRelation) + ExplainCommand(OneRowRelation) case Token("TOK_EXPLAIN", explainArgs) if "TOK_CREATETABLE" == explainArgs.head.getText => val Some(crtTbl) :: _ :: extended :: Nil = @@ -622,7 +622,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val relations = fromClause match { case Some(f) => nodeToRelation(f) - case None => NoRelation + case None => OneRowRelation } val withWhere = whereClause.map { whereNode => From 5909f0973de15f685836c2828e6d4c38f57d2c19 Mon Sep 17 00:00:00 2001 From: Adam Budde Date: Sat, 28 Mar 2015 09:14:09 +0800 Subject: [PATCH 563/817] [SPARK-6538][SQL] Add missing nullable Metastore fields when merging a Parquet schema Opening to replace #5188. When Spark SQL infers a schema for a DataFrame, it will take the union of all field types present in the structured source data (e.g. an RDD of JSON data). When the source data for a row doesn't define a particular field on the DataFrame's schema, a null value will simply be assumed for this field. This workflow makes it very easy to construct tables and query over a set of structured data with a nonuniform schema. However, this behavior is not consistent in some cases when dealing with Parquet files and an external table managed by an external Hive metastore. In our particular usecase, we use Spark Streaming to parse and transform our input data and then apply a window function to save an arbitrary-sized batch of data as a Parquet file, which itself will be added as a partition to an external Hive table via an *"ALTER TABLE... ADD PARTITION..."* statement. Since our input data is nonuniform, it is expected that not every partition batch will contain every field present in the table's schema obtained from the Hive metastore. As such, we expect that the schema of some of our Parquet files may not contain the same set fields present in the full metastore schema. In such cases, it seems natural that Spark SQL would simply assume null values for any missing fields in the partition's Parquet file, assuming these fields are specified as nullable by the metastore schema. This is not the case in the current implementation of ParquetRelation2. The **mergeMetastoreParquetSchema()** method used to reconcile differences between a Parquet file's schema and a schema retrieved from the Hive metastore will raise an exception if the Parquet file doesn't match the same set of fields specified by the metastore. This pull requests alters the behavior of **mergeMetastoreParquetSchema()** by having it first add any nullable fields from the metastore schema to the Parquet file schema if they aren't already present there. Author: Adam Budde Closes #5214 from budde/nullable-fields and squashes the following commits: a52d378 [Adam Budde] Refactor ParquetSchemaSuite.scala for cases now permitted by SPARK-6471 and SPARK-6538 9041bfa [Adam Budde] Add missing nullable Metastore fields when merging a Parquet schema --- .../apache/spark/sql/parquet/newParquet.scala | 32 ++++++++++++++- .../sql/parquet/ParquetSchemaSuite.scala | 40 +++++++++++++++++-- 2 files changed, 66 insertions(+), 6 deletions(-) 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 0d68810ec6043..53f765ee26a13 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 @@ -765,12 +765,14 @@ private[sql] object ParquetRelation2 extends Logging { |${parquetSchema.prettyJson} """.stripMargin - assert(metastoreSchema.size <= parquetSchema.size, schemaConflictMessage) + val mergedParquetSchema = mergeMissingNullableFields(metastoreSchema, parquetSchema) + + assert(metastoreSchema.size <= mergedParquetSchema.size, schemaConflictMessage) val ordinalMap = metastoreSchema.zipWithIndex.map { case (field, index) => field.name.toLowerCase -> index }.toMap - val reorderedParquetSchema = parquetSchema.sortBy(f => + val reorderedParquetSchema = mergedParquetSchema.sortBy(f => ordinalMap.getOrElse(f.name.toLowerCase, metastoreSchema.size + 1)) StructType(metastoreSchema.zip(reorderedParquetSchema).map { @@ -782,6 +784,32 @@ private[sql] object ParquetRelation2 extends Logging { }) } + /** + * Returns the original schema from the Parquet file with any missing nullable fields from the + * Hive Metastore schema merged in. + * + * When constructing a DataFrame from a collection of structured data, the resulting object has + * a schema corresponding to the union of the fields present in each element of the collection. + * Spark SQL simply assigns a null value to any field that isn't present for a particular row. + * In some cases, it is possible that a given table partition stored as a Parquet file doesn't + * contain a particular nullable field in its schema despite that field being present in the + * table schema obtained from the Hive Metastore. This method returns a schema representing the + * Parquet file schema along with any additional nullable fields from the Metastore schema + * merged in. + */ + private[parquet] def mergeMissingNullableFields( + metastoreSchema: StructType, + parquetSchema: StructType): StructType = { + val fieldMap = metastoreSchema.map(f => f.name.toLowerCase -> f).toMap + val missingFields = metastoreSchema + .map(_.name.toLowerCase) + .diff(parquetSchema.map(_.name.toLowerCase)) + .map(fieldMap(_)) + .filter(_.nullable) + StructType(parquetSchema ++ missingFields) + } + + // TODO Data source implementations shouldn't touch Catalyst types (`Literal`). // However, we are already using Catalyst expressions for partition pruning and predicate // push-down here... diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 8462f9bb2d620..61f1cf347ab0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -226,22 +226,54 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { StructField("UPPERCase", IntegerType, nullable = true)))) } - // Conflicting field count + // Metastore schema contains additional non-nullable fields. assert(intercept[Throwable] { ParquetRelation2.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false), - StructField("lowerCase", BinaryType))), + StructField("lowerCase", BinaryType, nullable = false))), StructType(Seq( StructField("UPPERCase", IntegerType, nullable = true)))) }.getMessage.contains("detected conflicting schemas")) - // Conflicting field names + // Conflicting non-nullable field names intercept[Throwable] { ParquetRelation2.mergeMetastoreParquetSchema( - StructType(Seq(StructField("lower", StringType))), + StructType(Seq(StructField("lower", StringType, nullable = false))), StructType(Seq(StructField("lowerCase", BinaryType)))) } } + + test("merge missing nullable fields from Metastore schema") { + // Standard case: Metastore schema contains additional nullable fields not present + // in the Parquet file schema. + assertResult( + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true)))) { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + } + + // Merge should fail if the Metastore contains any additional fields that are not + // nullable. + assert(intercept[Throwable] { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = false))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + }.getMessage.contains("detected conflicting schemas")) + } } From 99631438c0ec777d6a77974b148dbbd3e890260e Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Sat, 28 Mar 2015 12:32:35 +0000 Subject: [PATCH 564/817] [SPARK-6552][Deploy][Doc]expose start-slave.sh to user and update outdated doc https://issues.apache.org/jira/browse/SPARK-6552 /cc srowen Author: WangTaoTheTonic Closes #5205 from WangTaoTheTonic/SPARK-6552 and squashes the following commits: b02263c [WangTaoTheTonic] use less than rather than less equal f0fa408 [WangTaoTheTonic] expose start-slave.sh --- docs/spark-standalone.md | 3 ++- sbin/start-slave.sh | 10 ++++++++-- sbin/start-slaves.sh | 2 ++ 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 74d8653a8b845..0eed9adacf123 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -24,7 +24,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT + ./sbin/start-slave.sh Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). @@ -81,6 +81,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on. - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. +- `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. - `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 2fc35309f4ca5..c0155384f7395 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -17,8 +17,14 @@ # limitations under the License. # -# Usage: start-slave.sh -# where is like "spark://localhost:7077" +# Starts a slave on the machine this script is executed on. + +usage="Usage: start-slave.sh where is like "spark://localhost:7077" + +if [ $# -lt 2 ]; then + echo $usage + exit 1 +fi sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 76316a3067c93..4356c03657109 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,6 +17,8 @@ # limitations under the License. # +# Starts a slave instance on each machine specified in the conf/slaves file. + sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" From f75f633b21faaf911f04aeff847f25749b1ecd89 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 28 Mar 2015 15:08:05 -0700 Subject: [PATCH 565/817] [SPARK-6571][MLLIB] use wrapper in MatrixFactorizationModel.load This fixes `predictAll` after load. jkbradley Author: Xiangrui Meng Closes #5243 from mengxr/SPARK-6571 and squashes the following commits: 82dcaa7 [Xiangrui Meng] use wrapper in MatrixFactorizationModel.load --- .../MatrixFactorizationModelWrapper.scala | 40 +++++++++++++++++++ .../mllib/api/python/PythonMLLibAPI.scala | 18 --------- python/pyspark/mllib/recommendation.py | 8 ++++ 3 files changed, 48 insertions(+), 18 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala new file mode 100644 index 0000000000000..ecd3b16598438 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api.python + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.recommendation.{MatrixFactorizationModel, Rating} +import org.apache.spark.rdd.RDD + +/** + * A Wrapper of MatrixFactorizationModel to provide helper method for Python. + */ +private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) + extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { + + def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = + predict(SerDe.asTupleRDD(userAndProducts.rdd)) + + def getUserFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + } + + def getProductFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index e39156734794c..22fa684fd2895 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -58,7 +58,6 @@ import org.apache.spark.util.Utils */ private[python] class PythonMLLibAPI extends Serializable { - /** * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. * @param jsc Java SparkContext @@ -346,24 +345,7 @@ private[python] class PythonMLLibAPI extends Serializable { model.predictSoft(data) } - /** - * A Wrapper of MatrixFactorizationModel to provide helpfer method for Python - */ - private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) - extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { - def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = - predict(SerDe.asTupleRDD(userAndProducts.rdd)) - - def getUserFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) - } - - def getProductFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) - } - - } /** * Java stub for Python mllib ALS.train(). This stub returns a handle diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 1a4527b12cef2..b094e50856f70 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -90,6 +90,8 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> sameModel = MatrixFactorizationModel.load(sc, path) >>> sameModel.predict(2,2) 0.43... + >>> sameModel.predictAll(testset).collect() + [Rating(... >>> try: ... os.removedirs(path) ... except OSError: @@ -111,6 +113,12 @@ def userFeatures(self): def productFeatures(self): return self.call("getProductFeatures") + @classmethod + def load(cls, sc, path): + model = cls._load_java(sc, path) + wrapper = sc._jvm.MatrixFactorizationModelWrapper(model) + return MatrixFactorizationModel(wrapper) + class ALS(object): From 5eef00d0c6c7cc5448aca7b1c2a2e289a4c43eb0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 28 Mar 2015 23:59:27 -0700 Subject: [PATCH 566/817] [DOC] Improvements to Python docs. Author: Reynold Xin Closes #5238 from rxin/pyspark-docs and squashes the following commits: c285951 [Reynold Xin] Reset deprecation warning. 8c1031e [Reynold Xin] inferSchema dd91b1a [Reynold Xin] [DOC] Improvements to Python docs. --- python/docs/index.rst | 8 ++++++++ python/pyspark/sql/__init__.py | 14 ++++++++------ python/pyspark/sql/dataframe.py | 9 +-------- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/python/docs/index.rst b/python/docs/index.rst index d150de9d5c502..f7eede9c3c82a 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -29,6 +29,14 @@ Core classes: A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. + :class:`pyspark.sql.SQLContext` + + Main entry point for DataFrame and SQL functionality. + + :class:`pyspark.sql.DataFrame` + + A distributed collection of data grouped into named columns. + Indices and tables ================== diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index b9ffd6945ea7e..54a01631d8899 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -19,17 +19,19 @@ public classes of Spark SQL: - L{SQLContext} - Main entry point for SQL functionality. + Main entry point for :class:`DataFrame` and SQL functionality. - L{DataFrame} - A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In - addition to normal RDD operations, DataFrames also support SQL. + A distributed collection of data grouped into named columns. - L{GroupedData} + Aggregation methods, returned by :func:`DataFrame.groupBy`. - L{Column} - Column is a DataFrame with a single column. + A column expression in a :class:`DataFrame`. - L{Row} - A Row of data returned by a Spark SQL query. + A row of data in a :class:`DataFrame`. - L{HiveContext} - Main entry point for accessing data stored in Apache Hive.. + Main entry point for accessing data stored in Apache Hive. + - L{functions} + List of built-in functions available for :class:`DataFrame`. """ from pyspark.sql.context import SQLContext, HiveContext diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d51309f7ef5aa..23c0e63e77812 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -50,13 +50,6 @@ class DataFrame(object): ageCol = people.age - Note that the :class:`Column` type can also be manipulated - through its various functions:: - - # The following creates a new column that increases everybody's age by 10. - people.age + 10 - - A more concrete example:: # To create DataFrame using SQLContext @@ -77,7 +70,7 @@ def __init__(self, jdf, sql_ctx): @property def rdd(self): """ - Return the content of the :class:`DataFrame` as an :class:`RDD` + Return the content of the :class:`DataFrame` as an :class:`pyspark.RDD` of :class:`Row` s. """ if not hasattr(self, '_lazy_rdd'): From 55153f5c14fad10607b44fbb8eebd9636a6bc2e1 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Sun, 29 Mar 2015 12:37:53 +0100 Subject: [PATCH 567/817] [SPARK-4123][Project Infra]: Show new dependencies added in pull requests Starting work on this, but need to find a way to ensure that, after doing a checkout from `apache/master`, we can successfully return to the current checkout. I believe that `git rev-parse HEAD` will get me what I want, but pushing this PR up to test what the Jenkins boxes are seeing. Author: Brennon York Closes #5093 from brennonyork/SPARK-4123 and squashes the following commits: 42e243e [Brennon York] moved starting test output to before pr tests, fixed indentation, changed mvn call to build/mvn dadd941 [Brennon York] reverted assembly pom, put the regular test suite back in play 7aa1dee [Brennon York] set new dendencies into a block, removed the bash debugging flag 0074566 [Brennon York] fixed minor echo issue with quotes e229802 [Brennon York] updated to print the new dependency found 27bb9b5 [Brennon York] changed the assembly pom to test whether the pr test will pick up new deps 5375ad8 [Brennon York] git output to dev null 9bce980 [Brennon York] ensure both gate files exist 8f3c4b4 [Brennon York] updated to reflect the correct pushed in HEAD variable 2bc7b27 [Brennon York] added a pom gate check a18db71 [Brennon York] full test of new deps script ea170de [Brennon York] dont let mvn execute tests f70d8cd [Brennon York] testing mvn with package 62ffd65 [Brennon York] updated dependency output message and changed compile to package given the jenkins failure output 04747e4 [Brennon York] adding simple mvn statement to see if command executes and prints compile output 87f9bea [Brennon York] added -x flag with bash to get insight into what is executing and what isnt 9e87208 [Brennon York] added set blocks to catch any non-zero exit codes and updated output 6b3042b [Brennon York] removed excess git checkout print statements 4077d46 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4123 2bb5527 [Brennon York] added echo statement so jenkins logs which pr tests are running d027f8f [Brennon York] proper piping of unnecessary stderr and stdout 6e2890d [Brennon York] updated test output newlines d9f6f7f [Brennon York] removed echo bad9a3a [Brennon York] added back the new deps test e9e3ad1 [Brennon York] removed escapes for quotes 97e5cfb [Brennon York] commenting out new deps script 17379a5 [Brennon York] Merge remote-tracking branch 'upstream/master' into SPARK-4123 56f74a8 [Brennon York] updated the unop for ensuring a test is available f2abc8c [Brennon York] removed the git checkout 6912584 [Brennon York] added this_mssg echo output c610d42 [Brennon York] removed the error to dev/null b98f78c [Brennon York] added the removed deps and echo output for jenkins testing 291a8fe [Brennon York] updated location of maven binary 126ce61 [Brennon York] removing new deps test to isolate why jenkins isn't posting messages f8011d8 [Brennon York] minor updates and style changes 63a35c9 [Brennon York] updated new dependencies test dae7ba8 [Brennon York] Capturing output directly from dependency builds 94d3547 [Brennon York] adding the new dependencies script into the test mix 2bca3c3 [Brennon York] added a git checkout 'git rev-parse HEAD' to the end of each pr test ae83b90 [Brennon York] removed jenkins tests to grab some values from the jenkins box 4110993 [Brennon York] beginning work on pr test to add new dependencies --- dev/run-tests-jenkins | 41 ++++++----- dev/tests/pr_new_dependencies.sh | 117 +++++++++++++++++++++++++++++++ 2 files changed, 140 insertions(+), 18 deletions(-) create mode 100755 dev/tests/pr_new_dependencies.sh diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 3a937b637e003..f10aa6b59e1af 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -55,13 +55,14 @@ TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout # To write a PR test: #+ * the file must reside within the dev/tests directory #+ * be an executable bash script -#+ * accept two arguments on the command line, the first being the Github PR long commit -#+ hash and the second the Github SHA1 hash +#+ * accept three arguments on the command line, the first being the Github PR long commit +#+ hash, the second the Github SHA1 hash, and the final the current PR hash #+ * and, lastly, return string output to be included in the pr message output that will #+ be posted to Github PR_TESTS=( "pr_merge_ability" "pr_public_classes" + "pr_new_dependencies" ) function post_message () { @@ -146,34 +147,38 @@ function send_archived_logs () { fi } +# post start message +{ + start_message="\ + [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." + + post_message "$start_message" +} + # Environment variable to capture PR test output pr_message="" +# Ensure we save off the current HEAD to revert to +current_pr_head="`git rev-parse HEAD`" # Run pull request tests for t in "${PR_TESTS[@]}"; do this_test="${FWDIR}/dev/tests/${t}.sh" - # Ensure the test is a file and is executable - if [ -x "$this_test" ]; then - echo "ghprb: $ghprbActualCommit sha1: $sha1" - this_mssg="`bash \"${this_test}\" \"${ghprbActualCommit}\" \"${sha1}\" 2>/dev/null`" + # Ensure the test can be found and is a file + if [ -f "${this_test}" ]; then + echo "Running test: $t" + this_mssg="$(bash "${this_test}" "${ghprbActualCommit}" "${sha1}" "${current_pr_head}")" # Check if this is the merge test as we submit that note *before* and *after* # the tests run [ "$t" == "pr_merge_ability" ] && merge_note="${this_mssg}" pr_message="${pr_message}\n${this_mssg}" + # Ensure, after each test, that we're back on the current PR + git checkout -f "${current_pr_head}" &>/dev/null + else + echo "Cannot find test ${this_test}." fi done -# post start message -{ - start_message="\ - [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ - PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." - - start_message="${start_message}\n${merge_note}" - - post_message "$start_message" -} - # run tests { timeout "${TESTS_TIMEOUT}" ./dev/run-tests @@ -222,7 +227,7 @@ done PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" - result_message="${result_message}\n${pr_message}" + result_message="${result_message}${pr_message}" post_message "$result_message" } diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh new file mode 100755 index 0000000000000..115a5cd1354f0 --- /dev/null +++ b/dev/tests/pr_new_dependencies.sh @@ -0,0 +1,117 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# Arg3: Current PR Commit Hash +#+ the PR hash for the current commit +# + +ghprbActualCommit="$1" +sha1="$2" +current_pr_head="$3" + +MVN_BIN="build/mvn" +CURR_CP_FILE="my-classpath.txt" +MASTER_CP_FILE="master-classpath.txt" + +# First switch over to the master branch +git checkout master &>/dev/null +# Find and copy all pom.xml files into a *.gate file that we can check +# against through various `git` changes +find -name "pom.xml" -exec cp {} {}.gate \; +# Switch back to the current PR +git checkout "${current_pr_head}" &>/dev/null + +# Check if any *.pom files from the current branch are different from the master +difference_q="" +for p in $(find -name "pom.xml"); do + [[ -f "${p}" && -f "${p}.gate" ]] && \ + difference_q="${difference_q}$(diff $p.gate $p)" +done + +# If no pom files were changed we can easily say no new dependencies were added +if [ -z "${difference_q}" ]; then + echo " * This patch does not change any dependencies." +else + # Else we need to manually build spark to determine what, if any, dependencies + # were added into the Spark assembly jar + ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ + sed -n -e '/Building Spark Project Assembly/,$p' | \ + grep --context=1 -m 2 "Dependencies classpath:" | \ + head -n 3 | \ + tail -n 1 | \ + tr ":" "\n" | \ + rev | \ + cut -d "/" -f 1 | \ + rev | \ + sort > ${CURR_CP_FILE} + + # Checkout the master branch to compare against + git checkout master &>/dev/null + + ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ + sed -n -e '/Building Spark Project Assembly/,$p' | \ + grep --context=1 -m 2 "Dependencies classpath:" | \ + head -n 3 | \ + tail -n 1 | \ + tr ":" "\n" | \ + rev | \ + cut -d "/" -f 1 | \ + rev | \ + sort > ${MASTER_CP_FILE} + + DIFF_RESULTS="`diff my-classpath.txt master-classpath.txt`" + + if [ -z "${DIFF_RESULTS}" ]; then + echo " * This patch does not change any dependencies." + else + # Pretty print the new dependencies + added_deps=$(echo "${DIFF_RESULTS}" | grep "<" | cut -d' ' -f2 | awk '{print " * \`"$1"\`"}') + removed_deps=$(echo "${DIFF_RESULTS}" | grep ">" | cut -d' ' -f2 | awk '{print " * \`"$1"\`"}') + added_deps_text=" * This patch **adds the following new dependencies:**\n${added_deps}" + removed_deps_text=" * This patch **removes the following dependencies:**\n${removed_deps}" + + # Construct the final returned message with proper + return_mssg="" + [ -n "${added_deps}" ] && return_mssg="${added_deps_text}" + if [ -n "${removed_deps}" ]; then + if [ -n "${return_mssg}" ]; then + return_mssg="${return_mssg}\n${removed_deps_text}" + else + return_mssg="${removed_deps_text}" + fi + fi + echo "${return_mssg}" + fi + + # Remove the files we've left over + [ -f "${CURR_CP_FILE}" ] && rm -f "${CURR_CP_FILE}" + [ -f "${MASTER_CP_FILE}" ] && rm -f "${MASTER_CP_FILE}" + + # Clean up our mess from the Maven builds just in case + ${MVN_BIN} clean &>/dev/null +fi From e3eb393961051a48ed1cac756ac1928156aa161f Mon Sep 17 00:00:00 2001 From: Nishkam Ravi Date: Sun, 29 Mar 2015 12:40:37 +0100 Subject: [PATCH 568/817] [SPARK-6406] Launch Spark using assembly jar instead of a separate launcher jar Author: Nishkam Ravi Author: nishkamravi2 Author: nravi Closes #5085 from nishkamravi2/master_nravi and squashes the following commits: bad4349 [nishkamravi2] Update Main.java 36a6f87 [Nishkam Ravi] Minor changes and bug fixes b7f4ae7 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 4a45d6a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 458af39 [Nishkam Ravi] Locate the jar using getLocation, obviates the need to pass assembly path as an argument d9658d6 [Nishkam Ravi] Changes for SPARK-6406 ccdc334 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 3faa7a4 [Nishkam Ravi] Launcher library changes (SPARK-6406) 345206a [Nishkam Ravi] spark-class merge Merge branch 'master_nravi' of https://github.com/nishkamravi2/spark into master_nravi ac58975 [Nishkam Ravi] spark-class changes 06bfeb0 [nishkamravi2] Update spark-class 35af990 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 32c3ab3 [nishkamravi2] Update AbstractCommandBuilder.java 4bd4489 [nishkamravi2] Update AbstractCommandBuilder.java 746f35b [Nishkam Ravi] "hadoop" string in the assembly name should not be mandatory (everywhere else in spark we mandate spark-assembly*hadoop*.jar) bfe96e0 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi ee902fa [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi d453197 [nishkamravi2] Update NewHadoopRDD.scala 6f41a1d [nishkamravi2] Update NewHadoopRDD.scala 0ce2c32 [nishkamravi2] Update HadoopRDD.scala f7e33c2 [Nishkam Ravi] Merge branch 'master_nravi' of https://github.com/nishkamravi2/spark into master_nravi ba1eb8b [Nishkam Ravi] Try-catch block around the two occurrences of removeShutDownHook. Deletion of semi-redundant occurrences of expensive operation inShutDown. 71d0e17 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 494d8c0 [nishkamravi2] Update DiskBlockManager.scala 3c5ddba [nishkamravi2] Update DiskBlockManager.scala f0d12de [Nishkam Ravi] Workaround for IllegalStateException caused by recent changes to BlockManager.stop 79ea8b4 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi b446edc [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 5c9a4cb [nishkamravi2] Update TaskSetManagerSuite.scala 535295a [nishkamravi2] Update TaskSetManager.scala 3e1b616 [Nishkam Ravi] Modify test for maxResultSize 9f6583e [Nishkam Ravi] Changes to maxResultSize code (improve error message and add condition to check if maxResultSize > 0) 5f8f9ed [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi 636a9ff [nishkamravi2] Update YarnAllocator.scala 8f76c8b [Nishkam Ravi] Doc change for yarn memory overhead 35daa64 [Nishkam Ravi] Slight change in the doc for yarn memory overhead 5ac2ec1 [Nishkam Ravi] Remove out dac1047 [Nishkam Ravi] Additional documentation for yarn memory overhead issue 42c2c3d [Nishkam Ravi] Additional changes for yarn memory overhead issue 362da5e [Nishkam Ravi] Additional changes for yarn memory overhead c726bd9 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi f00fa31 [Nishkam Ravi] Improving logging for AM memoryOverhead 1cf2d1e [nishkamravi2] Update YarnAllocator.scala ebcde10 [Nishkam Ravi] Modify default YARN memory_overhead-- from an additive constant to a multiplier (redone to resolve merge conflicts) 2e69f11 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi efd688a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark 2b630f9 [nravi] Accept memory input as "30g", "512M" instead of an int value, to be consistent with rest of Spark 3bf8fad [nravi] Merge branch 'master' of https://github.com/apache/spark 5423a03 [nravi] Merge branch 'master' of https://github.com/apache/spark eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456) 6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed) 5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456) 681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles --- bin/spark-class | 61 +++++++----- bin/spark-class2.cmd | 33 +++---- .../launcher/AbstractCommandBuilder.java | 99 +++++-------------- make-distribution.sh | 1 - 4 files changed, 69 insertions(+), 125 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index e29b234afaf96..c03946d92e2e4 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -40,35 +40,46 @@ else fi fi -# Look for the launcher. In non-release mode, add the compiled classes directly to the classpath -# instead of looking for a jar file. -SPARK_LAUNCHER_CP= -if [ -f $SPARK_HOME/RELEASE ]; then - LAUNCHER_DIR="$SPARK_HOME/lib" - num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" - if [ "$num_jars" -eq "0" -a -z "$SPARK_LAUNCHER_CP" ]; then - echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 - exit 1 - fi +# Find assembly jar +SPARK_ASSEMBLY_JAR= +if [ -f "$SPARK_HOME/RELEASE" ]; then + ASSEMBLY_DIR="$SPARK_HOME/lib" +else + ASSEMBLY_DIR="$SPARK_HOME/assembly/target/scala-$SPARK_SCALA_VERSION" +fi - LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" || true)" - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 - echo "$LAUNCHER_JARS" 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 - fi +num_jars="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" | wc -l)" +if [ "$num_jars" -eq "0" -a -z "$SPARK_ASSEMBLY_JAR" ]; then + echo "Failed to find Spark assembly in $ASSEMBLY_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 +fi +ASSEMBLY_JARS="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" || true)" +if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark assembly jars in $ASSEMBLY_DIR:" 1>&2 + echo "$ASSEMBLY_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 + exit 1 +fi - SPARK_LAUNCHER_CP="${LAUNCHER_DIR}/${LAUNCHER_JARS}" +SPARK_ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}" + +# Verify that versions of java used to build the jars and run Spark are compatible +if [ -n "$JAVA_HOME" ]; then + JAR_CMD="$JAVA_HOME/bin/jar" else - LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" - if [ ! -d "$LAUNCHER_DIR/classes" ]; then - echo "Failed to find Spark launcher classes in $LAUNCHER_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 + JAR_CMD="jar" +fi + +if [ $(command -v "$JAR_CMD") ] ; then + jar_error_check=$("$JAR_CMD" -tf "$SPARK_ASSEMBLY_JAR" nonexistent/class/path 2>&1) + if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " 1>&2 + echo "This is likely because Spark was compiled with Java 7 and run " 1>&2 + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " 1>&2 + echo "or build Spark with Java 6." 1>&2 exit 1 fi - SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes" fi # The launcher library will print arguments separated by a NULL character, to allow arguments with @@ -77,7 +88,7 @@ fi CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$SPARK_ASSEMBLY_JAR" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 37d22215a0e7e..4ce727bc99128 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -29,31 +29,20 @@ if "x%1"=="x" ( exit /b 1 ) -set LAUNCHER_CP=0 -if exist %SPARK_HOME%\RELEASE goto find_release_launcher +rem Find assembly jar +set SPARK_ASSEMBLY_JAR=0 -rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use Scala so -rem it doesn't really matter which one is picked up. Add the compiled classes directly to the -rem classpath instead of looking for a jar file, since it's very common for people using sbt to use -rem the "assembly" target instead of "package". -set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.10\classes -if exist %LAUNCHER_CLASSES% ( - set LAUNCHER_CP=%LAUNCHER_CLASSES% +if exist "%SPARK_HOME%\RELEASE" ( + set ASSEMBLY_DIR=%SPARK_HOME%\lib +) else ( + set ASSEMBLY_DIR=%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION% ) -set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.11\classes -if exist %LAUNCHER_CLASSES% ( - set LAUNCHER_CP=%LAUNCHER_CLASSES% -) -goto check_launcher -:find_release_launcher -for %%d in (%SPARK_HOME%\lib\spark-launcher*.jar) do ( - set LAUNCHER_CP=%%d +for %%d in (%ASSEMBLY_DIR%\spark-assembly*hadoop*.jar) do ( + set SPARK_ASSEMBLY_JAR=%%d ) - -:check_launcher -if "%LAUNCHER_CP%"=="0" ( - echo Failed to find Spark launcher JAR. +if "%SPARK_ASSEMBLY_JAR%"=="0" ( + echo Failed to find Spark assembly JAR. echo You need to build Spark before running this program. exit /b 1 ) @@ -64,7 +53,7 @@ if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_CP% org.apache.spark.launcher.Main %*"') do ( +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %SPARK_ASSEMBLY_JAR% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) %SPARK_CMD% diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 2da5f7278729e..d8279145d8e90 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -86,10 +86,14 @@ public AbstractCommandBuilder() { */ List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList(); - if (javaHome == null) { - cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); - } else { + String envJavaHome; + + if (javaHome != null) { cmd.add(join(File.separator, javaHome, "bin", "java")); + } else if ((envJavaHome = System.getenv("JAVA_HOME")) != null) { + cmd.add(join(File.separator, envJavaHome, "bin", "java")); + } else { + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); } // Load extra JAVA_OPTS from conf/java-opts, if it exists. @@ -182,59 +186,25 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - String assembly = findAssembly(); + final String assembly = AbstractCommandBuilder.class.getProtectionDomain().getCodeSource(). + getLocation().getPath(); addToClassPath(cp, assembly); - // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus - // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt - // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built - // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark - // assembly is built for Hive, before actually populating the CLASSPATH with the jars. - // - // This block also serves as a check for SPARK-1703, when the assembly jar is built with - // Java 7 and ends up with too many files, causing issues with other JDK versions. - boolean needsDataNucleus = false; - JarFile assemblyJar = null; - try { - assemblyJar = new JarFile(assembly); - needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; - } catch (IOException ioe) { - if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { - System.err.println( - "Loading Spark jar failed.\n" + - "This is likely because Spark was compiled with Java 7 and run\n" + - "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + - "or build Spark with Java 6."); - System.exit(1); - } else { - throw ioe; - } - } finally { - if (assemblyJar != null) { - try { - assemblyJar.close(); - } catch (IOException e) { - // Ignore. - } - } + // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only + // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate + // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); } - if (needsDataNucleus) { - System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + - "in classpath."); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - } else { - libdir = new File(sparkHome, "lib_managed/jars"); - } - - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); - } + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); } } @@ -270,7 +240,6 @@ String getScalaVersion() { if (scala != null) { return scala; } - String sparkHome = getSparkHome(); File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); @@ -330,30 +299,6 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } - private String findAssembly() { - String sparkHome = getSparkHome(); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); - } - - final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); - FileFilter filter = new FileFilter() { - @Override - public boolean accept(File file) { - return file.isFile() && re.matcher(file.getName()).matches(); - } - }; - File[] assemblies = libdir.listFiles(filter); - checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); - checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); - return assemblies[0].getAbsolutePath(); - } - private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/make-distribution.sh b/make-distribution.sh index 9ed1abfe8c598..738a9c4d69601 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -199,7 +199,6 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" -cp "$SPARK_HOME"/launcher/target/spark-launcher_$SCALA_VERSION-$VERSION.jar "$DISTDIR/lib/" # This will fail if the -Pyarn profile is not provided # In this case, silence the error and ignore the return code of this command cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || : From 52ece26b8fb9769f6ed9167e3dffc8b1d7c61b02 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Sun, 29 Mar 2015 12:43:30 +0100 Subject: [PATCH 569/817] [SPARK-6558] Utils.getCurrentUserName returns the full principal name instead of login name Utils.getCurrentUserName returns UserGroupInformation.getCurrentUser().getUserName() when SPARK_USER isn't set. It should return UserGroupInformation.getCurrentUser().getShortUserName() getUserName() returns the users full principal name (ie user1CORP.COM). getShortUserName() returns just the users login name (user1). This just happens to work on YARN because the Client code sets: env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() Author: Thomas Graves Closes #5229 from tgravescs/SPARK-6558 and squashes the following commits: 24830bf [Thomas Graves] Utils.getCurrentUserName returns the full principal name instead of login name --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0b5a914e7dbbf..bb8bd1015668a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2055,7 +2055,7 @@ private[spark] object Utils extends Logging { */ def getCurrentUserName(): String = { Option(System.getenv("SPARK_USER")) - .getOrElse(UserGroupInformation.getCurrentUser().getUserName()) + .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } } From 0e2753ff14e0d3f2433272c13ce26f67dc89767f Mon Sep 17 00:00:00 2001 From: "June.He" Date: Sun, 29 Mar 2015 12:47:22 +0100 Subject: [PATCH 570/817] [SPARK-6585][Tests]Fix FileServerSuite testcase in some Env. Change FileServerSuite.test("HttpFileServer should not work with SSL when the server is untrusted") catch SSLException Author: June.He Closes #5239 from sisihj/SPARK-6585 and squashes the following commits: cb19ae3 [June.He] Change FileServerSuite.test("HttpFileServer should not work with SSL when the server is untrusted") catch SSLException --- core/src/test/scala/org/apache/spark/FileServerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 5fdf6bc2777e3..a69e9b761f9a7 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark import java.io._ import java.net.URI import java.util.jar.{JarEntry, JarOutputStream} -import javax.net.ssl.SSLHandshakeException +import javax.net.ssl.SSLException import com.google.common.io.ByteStreams import org.apache.commons.io.{FileUtils, IOUtils} @@ -228,7 +228,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { try { server.initialize() - intercept[SSLHandshakeException] { + intercept[SSLException] { fileTransferTest(server) } } finally { From a8d53afb4e119788fa0d9dd6b3e3ca94cea98581 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 29 Mar 2015 21:25:09 -0700 Subject: [PATCH 571/817] [SPARK-5124][Core] A standard RPC interface and an Akka implementation This PR added a standard internal RPC interface for Spark and an Akka implementation. See [the design document](https://issues.apache.org/jira/secure/attachment/12698710/Pluggable%20RPC%20-%20draft%202.pdf) for more details. I will split the whole work into multiple PRs to make it easier for code review. This is the first PR and avoid to touch too many files. Author: zsxwing Closes #4588 from zsxwing/rpc-part1 and squashes the following commits: fe3df4c [zsxwing] Move registerEndpoint and use actorSystem.dispatcher in asyncSetupEndpointRefByURI f6f3287 [zsxwing] Remove RpcEndpointRef.toURI 8bd1097 [zsxwing] Fix docs and the code style f459380 [zsxwing] Add RpcAddress.fromURI and rename urls to uris b221398 [zsxwing] Move send methods above ask methods 15cfd7b [zsxwing] Merge branch 'master' into rpc-part1 9ffa997 [zsxwing] Fix MiMa tests 78a1733 [zsxwing] Merge remote-tracking branch 'origin/master' into rpc-part1 385b9c3 [zsxwing] Fix the code style and add docs 2cc3f78 [zsxwing] Add an asynchronous version of setupEndpointRefByUrl e8dfec3 [zsxwing] Remove 'sendWithReply(message: Any, sender: RpcEndpointRef): Unit' 08564ae [zsxwing] Add RpcEnvFactory to create RpcEnv e5df4ca [zsxwing] Handle AkkaFailure(e) in Actor ec7c5b0 [zsxwing] Fix docs 7fc95e1 [zsxwing] Implement askWithReply in RpcEndpointRef 9288406 [zsxwing] Document thread-safety for setupThreadSafeEndpoint 3007c09 [zsxwing] Move setupDriverEndpointRef to RpcUtils and rename to makeDriverRef c425022 [zsxwing] Fix the code style 5f87700 [zsxwing] Move the logical of processing message to a private function 3e56123 [zsxwing] Use lazy to eliminate CountDownLatch 07f128f [zsxwing] Remove ActionScheduler.scala 4d34191 [zsxwing] Remove scheduler from RpcEnv 7cdd95e [zsxwing] Add docs for RpcEnv 51e6667 [zsxwing] Add 'sender' to RpcCallContext and rename the parameter of receiveAndReply to 'context' ffc1280 [zsxwing] Rename 'fail' to 'sendFailure' and other minor code style changes 28e6d0f [zsxwing] Add onXXX for network events and remove the companion objects of network events 3751c97 [zsxwing] Rename RpcResponse to RpcCallContext fe7d1ff [zsxwing] Add explicit reply in rpc 7b9e0c9 [zsxwing] Fix the indentation 04a106e [zsxwing] Remove NopCancellable and add a const NOP in object SettableCancellable 2a579f4 [zsxwing] Remove RpcEnv.systemName 155b987 [zsxwing] Change newURI to uriOf and add some comments 45b2317 [zsxwing] A standard RPC interface and An Akka implementation --- .../scala/org/apache/spark/SparkEnv.scala | 42 +- .../spark/deploy/worker/DriverWrapper.scala | 11 +- .../spark/deploy/worker/WorkerWatcher.scala | 59 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../scala/org/apache/spark/rpc/RpcEnv.scala | 429 ++++++++++++++ .../apache/spark/rpc/akka/AkkaRpcEnv.scala | 318 +++++++++++ .../scheduler/OutputCommitCoordinator.scala | 37 +- .../org/apache/spark/util/AkkaUtils.scala | 2 +- .../org/apache/spark/util/RpcUtils.scala | 35 ++ .../deploy/worker/WorkerWatcherSuite.scala | 38 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 525 ++++++++++++++++++ .../spark/rpc/akka/AkkaRpcEnvSuite.scala | 50 ++ project/MimaExcludes.scala | 4 +- 13 files changed, 1466 insertions(+), 86 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala create mode 100644 core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala create mode 100644 core/src/main/scala/org/apache/spark/util/RpcUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2a0c7e756dd3a..4a2ed82a40dec 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -34,12 +34,14 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.rpc.{RpcEndpointRef, RpcEndpoint, RpcEnv} +import org.apache.spark.rpc.akka.AkkaRpcEnv import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus} -import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor +import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} /** * :: DeveloperApi :: @@ -54,7 +56,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} @DeveloperApi class SparkEnv ( val executorId: String, - val actorSystem: ActorSystem, + private[spark] val rpcEnv: RpcEnv, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -71,6 +73,9 @@ class SparkEnv ( val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { + // TODO Remove actorSystem + val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem + private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -91,7 +96,8 @@ class SparkEnv ( blockManager.master.stop() metricsSystem.stop() outputCommitCoordinator.stop() - actorSystem.shutdown() + rpcEnv.shutdown() + // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. @@ -236,16 +242,15 @@ object SparkEnv extends Logging { val securityManager = new SecurityManager(conf) // Create the ActorSystem for Akka and get the port it binds to. - val (actorSystem, boundPort) = { - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) - } + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + val rpcEnv = RpcEnv.create(actorSystemName, hostname, port, conf, securityManager) + val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem // Figure out which port Akka actually bound to in case the original port is 0 or occupied. if (isDriver) { - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) } else { - conf.set("spark.executor.port", boundPort.toString) + conf.set("spark.executor.port", rpcEnv.address.port.toString) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -290,6 +295,15 @@ object SparkEnv extends Logging { } } + def registerOrLookupEndpoint(name: String, endpointCreator: => RpcEndpoint): RpcEndpointRef = { + if (isDriver) { + logInfo("Registering " + name) + rpcEnv.setupEndpoint(name, endpointCreator) + } else { + RpcUtils.makeDriverRef(name, conf, rpcEnv) + } + } + val mapOutputTracker = if (isDriver) { new MapOutputTrackerMaster(conf) } else { @@ -377,13 +391,13 @@ object SparkEnv extends Logging { val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { new OutputCommitCoordinator(conf) } - val outputCommitCoordinatorActor = registerOrLookup("OutputCommitCoordinator", - new OutputCommitCoordinatorActor(outputCommitCoordinator)) - outputCommitCoordinator.coordinatorActor = Some(outputCommitCoordinatorActor) + val outputCommitCoordinatorRef = registerOrLookupEndpoint("OutputCommitCoordinator", + new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator)) + outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef) val envInstance = new SparkEnv( executorId, - actorSystem, + rpcEnv, serializer, closureSerializer, cacheManager, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index deef6ef9043c6..d1a12b01e78f7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -19,10 +19,9 @@ package org.apache.spark.deploy.worker import java.io.File -import akka.actor._ - import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, Utils} +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} /** * Utility object for launching driver programs such that they share fate with the Worker process. @@ -39,9 +38,9 @@ object DriverWrapper { */ case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() - val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", + val rpcEnv = RpcEnv.create("Driver", Utils.localHostName(), 0, conf, new SecurityManager(conf)) - actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") + rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) val currentLoader = Thread.currentThread.getContextClassLoader val userJarUrl = new File(userJar).toURI().toURL() @@ -58,7 +57,7 @@ object DriverWrapper { val mainMethod = clazz.getMethod("main", classOf[Array[String]]) mainMethod.invoke(null, extraArgs.toArray[String]) - actorSystem.shutdown() + rpcEnv.shutdown() case _ => System.err.println("Usage: DriverWrapper [options]") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index e0790274d7d3e..83fb991891a41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -17,58 +17,63 @@ package org.apache.spark.deploy.worker -import akka.actor.{Actor, Address, AddressFromURIString} -import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, DisassociatedEvent, RemotingLifecycleEvent} - import org.apache.spark.Logging import org.apache.spark.deploy.DeployMessages.SendHeartbeat -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.rpc._ /** * Actor which connects to a worker process and terminates the JVM if the connection is severed. * Provides fate sharing between a worker and its associated child processes. */ -private[spark] class WorkerWatcher(workerUrl: String) - extends Actor with ActorLogReceive with Logging { - - override def preStart() { - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) +private[spark] class WorkerWatcher(override val rpcEnv: RpcEnv, workerUrl: String) + extends RpcEndpoint with Logging { + override def onStart() { logInfo(s"Connecting to worker $workerUrl") - val worker = context.actorSelection(workerUrl) - worker ! SendHeartbeat // need to send a message here to initiate connection + if (!isTesting) { + rpcEnv.asyncSetupEndpointRefByURI(workerUrl) + } } // Used to avoid shutting down JVM during tests + // In the normal case, exitNonZero will call `System.exit(-1)` to shutdown the JVM. In the unit + // test, the user should call `setTesting(true)` so that `exitNonZero` will set `isShutDown` to + // true rather than calling `System.exit`. The user can check `isShutDown` to know if + // `exitNonZero` is called. private[deploy] var isShutDown = false private[deploy] def setTesting(testing: Boolean) = isTesting = testing private var isTesting = false // Lets us filter events only from the worker's actor system - private val expectedHostPort = AddressFromURIString(workerUrl).hostPort - private def isWorker(address: Address) = address.hostPort == expectedHostPort + private val expectedAddress = RpcAddress.fromURIString(workerUrl) + private def isWorker(address: RpcAddress) = expectedAddress == address private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) - override def receiveWithLogging: PartialFunction[Any, Unit] = { - case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => - logInfo(s"Successfully connected to $workerUrl") + override def receive: PartialFunction[Any, Unit] = { + case e => logWarning(s"Received unexpected message: $e") + } - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) - if isWorker(remoteAddress) => - // These logs may not be seen if the worker (and associated pipe) has died - logError(s"Could not initialize connection to worker $workerUrl. Exiting.") - logError(s"Error was: $cause") - exitNonZero() + override def onConnected(remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { + logInfo(s"Successfully connected to $workerUrl") + } + } - case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { // This log message will never be seen logError(s"Lost connection to worker actor $workerUrl. Exiting.") exitNonZero() + } + } - case e: AssociationEvent => - // pass through association events relating to other remote actor systems - - case e => logWarning(s"Received unexpected actor system event: $e") + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { + // These logs may not be seen if the worker (and associated pipe) has died + logError(s"Could not initialize connection to worker $workerUrl. Exiting.") + logError(s"Error was: $cause") + exitNonZero() + } } } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b5205d4e997ae..900e678ee02ef 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -169,7 +169,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverUrl, executorId, sparkHostPort, cores, userClassPath, env), name = "Executor") workerUrl.foreach { url => - env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.actorSystem.awaitTermination() } diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala new file mode 100644 index 0000000000000..7985941d949c0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -0,0 +1,429 @@ +/* + * 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.rpc + +import java.net.URI + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.reflect.ClassTag + +import org.apache.spark.{Logging, SparkException, SecurityManager, SparkConf} +import org.apache.spark.util.{AkkaUtils, Utils} + +/** + * An RPC environment. [[RpcEndpoint]]s need to register itself with a name to [[RpcEnv]] to + * receives messages. Then [[RpcEnv]] will process messages sent from [[RpcEndpointRef]] or remote + * nodes, and deliver them to corresponding [[RpcEndpoint]]s. + * + * [[RpcEnv]] also provides some methods to retrieve [[RpcEndpointRef]]s given name or uri. + */ +private[spark] abstract class RpcEnv(conf: SparkConf) { + + private[spark] val defaultLookupTimeout = AkkaUtils.lookupTimeout(conf) + + /** + * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement + * [[RpcEndpoint.self]]. + * + * Note: This method won't return null. `IllegalArgumentException` will be thrown if calling this + * on a non-existent endpoint. + */ + private[rpc] def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Return the address that [[RpcEnv]] is listening to. + */ + def address: RpcAddress + + /** + * Register a [[RpcEndpoint]] with a name and return its [[RpcEndpointRef]]. [[RpcEnv]] does not + * guarantee thread-safety. + */ + def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Register a [[RpcEndpoint]] with a name and return its [[RpcEndpointRef]]. [[RpcEnv]] should + * make sure thread-safely sending messages to [[RpcEndpoint]]. + * + * Thread-safety means processing of one message happens before processing of the next message by + * the same [[RpcEndpoint]]. In the other words, changes to internal fields of a [[RpcEndpoint]] + * are visible when processing the next message, and fields in the [[RpcEndpoint]] need not be + * volatile or equivalent. + * + * However, there is no guarantee that the same thread will be executing the same [[RpcEndpoint]] + * for different messages. + */ + def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri` asynchronously. + */ + def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri`. This is a blocking action. + */ + def setupEndpointRefByURI(uri: String): RpcEndpointRef = { + Await.result(asyncSetupEndpointRefByURI(uri), defaultLookupTimeout) + } + + /** + * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName` + * asynchronously. + */ + def asyncSetupEndpointRef( + systemName: String, address: RpcAddress, endpointName: String): Future[RpcEndpointRef] = { + asyncSetupEndpointRefByURI(uriOf(systemName, address, endpointName)) + } + + /** + * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName`. + * This is a blocking action. + */ + def setupEndpointRef( + systemName: String, address: RpcAddress, endpointName: String): RpcEndpointRef = { + setupEndpointRefByURI(uriOf(systemName, address, endpointName)) + } + + /** + * Stop [[RpcEndpoint]] specified by `endpoint`. + */ + def stop(endpoint: RpcEndpointRef): Unit + + /** + * Shutdown this [[RpcEnv]] asynchronously. If need to make sure [[RpcEnv]] exits successfully, + * call [[awaitTermination()]] straight after [[shutdown()]]. + */ + def shutdown(): Unit + + /** + * Wait until [[RpcEnv]] exits. + * + * TODO do we need a timeout parameter? + */ + def awaitTermination(): Unit + + /** + * Create a URI used to create a [[RpcEndpointRef]]. Use this one to create the URI instead of + * creating it manually because different [[RpcEnv]] may have different formats. + */ + def uriOf(systemName: String, address: RpcAddress, endpointName: String): String +} + +private[spark] case class RpcEnvConfig( + conf: SparkConf, + name: String, + host: String, + port: Int, + securityManager: SecurityManager) + +/** + * A RpcEnv implementation must have a [[RpcEnvFactory]] implementation with an empty constructor + * so that it can be created via Reflection. + */ +private[spark] object RpcEnv { + + private def getRpcEnvFactory(conf: SparkConf): RpcEnvFactory = { + // Add more RpcEnv implementations here + val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory") + val rpcEnvName = conf.get("spark.rpc", "akka") + val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName) + Class.forName(rpcEnvFactoryClassName, true, Utils.getContextOrSparkClassLoader). + newInstance().asInstanceOf[RpcEnvFactory] + } + + def create( + name: String, + host: String, + port: Int, + conf: SparkConf, + securityManager: SecurityManager): RpcEnv = { + // Using Reflection to create the RpcEnv to avoid to depend on Akka directly + val config = RpcEnvConfig(conf, name, host, port, securityManager) + getRpcEnvFactory(conf).create(config) + } + +} + +/** + * A factory class to create the [[RpcEnv]]. It must have an empty constructor so that it can be + * created using Reflection. + */ +private[spark] trait RpcEnvFactory { + + def create(config: RpcEnvConfig): RpcEnv +} + +/** + * An end point for the RPC that defines what functions to trigger given a message. + * + * It is guaranteed that `onStart`, `receive` and `onStop` will be called in sequence. + * + * The lift-cycle will be: + * + * constructor onStart receive* onStop + * + * Note: `receive` can be called concurrently. If you want `receive` is thread-safe, please use + * [[RpcEnv.setupThreadSafeEndpoint]] + * + * If any error is thrown from one of [[RpcEndpoint]] methods except `onError`, `onError` will be + * invoked with the cause. If `onError` throws an error, [[RpcEnv]] will ignore it. + */ +private[spark] trait RpcEndpoint { + + /** + * The [[RpcEnv]] that this [[RpcEndpoint]] is registered to. + */ + val rpcEnv: RpcEnv + + /** + * The [[RpcEndpointRef]] of this [[RpcEndpoint]]. `self` will become valid when `onStart` is + * called. + * + * Note: Because before `onStart`, [[RpcEndpoint]] has not yet been registered and there is not + * valid [[RpcEndpointRef]] for it. So don't call `self` before `onStart` is called. + */ + final def self: RpcEndpointRef = { + require(rpcEnv != null, "rpcEnv has not been initialized") + rpcEnv.endpointRef(this) + } + + /** + * Process messages from [[RpcEndpointRef.send]] or [[RpcCallContext.reply)]]. If receiving a + * unmatched message, [[SparkException]] will be thrown and sent to `onError`. + */ + def receive: PartialFunction[Any, Unit] = { + case _ => throw new SparkException(self + " does not implement 'receive'") + } + + /** + * Process messages from [[RpcEndpointRef.sendWithReply]]. If receiving a unmatched message, + * [[SparkException]] will be thrown and sent to `onError`. + */ + def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case _ => context.sendFailure(new SparkException(self + " won't reply anything")) + } + + /** + * Call onError when any exception is thrown during handling messages. + * + * @param cause + */ + def onError(cause: Throwable): Unit = { + // By default, throw e and let RpcEnv handle it + throw cause + } + + /** + * Invoked before [[RpcEndpoint]] starts to handle any message. + */ + def onStart(): Unit = { + // By default, do nothing. + } + + /** + * Invoked when [[RpcEndpoint]] is stopping. + */ + def onStop(): Unit = { + // By default, do nothing. + } + + /** + * Invoked when `remoteAddress` is connected to the current node. + */ + def onConnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when `remoteAddress` is lost. + */ + def onDisconnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when some network error happens in the connection between the current node and + * `remoteAddress`. + */ + def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * A convenient method to stop [[RpcEndpoint]]. + */ + final def stop(): Unit = { + val _self = self + if (_self != null) { + rpcEnv.stop(self) + } + } +} + +/** + * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe. + */ +private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) + extends Serializable with Logging { + + private[this] val maxRetries = conf.getInt("spark.akka.num.retries", 3) + private[this] val retryWaitMs = conf.getLong("spark.akka.retry.wait", 3000) + private[this] val defaultTimeout = conf.getLong("spark.akka.lookupTimeout", 30) seconds + + /** + * return the address for the [[RpcEndpointRef]] + */ + def address: RpcAddress + + def name: String + + /** + * Sends a one-way asynchronous message. Fire-and-forget semantics. + */ + def send(message: Any): Unit + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a `Future` to + * receive the reply within a default timeout. + * + * This method only sends the message once and never retries. + */ + def sendWithReply[T: ClassTag](message: Any): Future[T] = sendWithReply(message, defaultTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a `Future` to + * receive the reply within the specified timeout. + * + * This method only sends the message once and never retries. + */ + def sendWithReply[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] + + /** + * Send a message to the corresponding [[RpcEndpoint]] and get its result within a default + * timeout, or throw a SparkException if this fails even after the default number of retries. + * The default `timeout` will be used in every trial of calling `sendWithReply`. Because this + * method retries, the message handling in the receiver side should be idempotent. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in an message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a + * specified timeout, throw a SparkException if this fails even after the specified number of + * retries. `timeout` will be used in every trial of calling `sendWithReply`. Because this method + * retries, the message handling in the receiver side should be idempotent. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in an message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @param timeout the timeout duration + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askWithReply[T: ClassTag](message: Any, timeout: FiniteDuration): T = { + // TODO: Consider removing multiple attempts + var attempts = 0 + var lastException: Exception = null + while (attempts < maxRetries) { + attempts += 1 + try { + val future = sendWithReply[T](message, timeout) + val result = Await.result(future, timeout) + if (result == null) { + throw new SparkException("Actor returned null") + } + return result + } catch { + case ie: InterruptedException => throw ie + case e: Exception => + lastException = e + logWarning(s"Error sending message [message = $message] in $attempts attempts", e) + } + Thread.sleep(retryWaitMs) + } + + throw new SparkException( + s"Error sending message [message = $message]", lastException) + } + +} + +/** + * Represent a host with a port + */ +private[spark] case class RpcAddress(host: String, port: Int) { + // TODO do we need to add the type of RpcEnv in the address? + + val hostPort: String = host + ":" + port + + override val toString: String = hostPort +} + +private[spark] object RpcAddress { + + /** + * Return the [[RpcAddress]] represented by `uri`. + */ + def fromURI(uri: URI): RpcAddress = { + RpcAddress(uri.getHost, uri.getPort) + } + + /** + * Return the [[RpcAddress]] represented by `uri`. + */ + def fromURIString(uri: String): RpcAddress = { + fromURI(new java.net.URI(uri)) + } + + def fromSparkURL(sparkUrl: String): RpcAddress = { + val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) + RpcAddress(host, port) + } +} + +/** + * A callback that [[RpcEndpoint]] can use it to send back a message or failure. + */ +private[spark] trait RpcCallContext { + + /** + * Reply a message to the sender. If the sender is [[RpcEndpoint]], its [[RpcEndpoint.receive]] + * will be called. + */ + def reply(response: Any): Unit + + /** + * Report a failure to the sender. + */ + def sendFailure(e: Throwable): Unit + + /** + * The sender of this message. + */ + def sender: RpcEndpointRef +} diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala new file mode 100644 index 0000000000000..769d59b7b3343 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala @@ -0,0 +1,318 @@ +/* + * 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.rpc.akka + +import java.net.URI +import java.util.concurrent.ConcurrentHashMap + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address} +import akka.pattern.{ask => akkaAsk} +import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent} +import org.apache.spark.{SparkException, Logging, SparkConf} +import org.apache.spark.rpc._ +import org.apache.spark.util.{ActorLogReceive, AkkaUtils} + +/** + * A RpcEnv implementation based on Akka. + * + * TODO Once we remove all usages of Akka in other place, we can move this file to a new project and + * remove Akka from the dependencies. + * + * @param actorSystem + * @param conf + * @param boundPort + */ +private[spark] class AkkaRpcEnv private[akka] ( + val actorSystem: ActorSystem, conf: SparkConf, boundPort: Int) + extends RpcEnv(conf) with Logging { + + private val defaultAddress: RpcAddress = { + val address = actorSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + // In some test case, ActorSystem doesn't bind to any address. + // So just use some default value since they are only some unit tests + RpcAddress(address.host.getOrElse("localhost"), address.port.getOrElse(boundPort)) + } + + override val address: RpcAddress = defaultAddress + + /** + * A lookup table to search a [[RpcEndpointRef]] for a [[RpcEndpoint]]. We need it to make + * [[RpcEndpoint.self]] work. + */ + private val endpointToRef = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]() + + /** + * Need this map to remove `RpcEndpoint` from `endpointToRef` via a `RpcEndpointRef` + */ + private val refToEndpoint = new ConcurrentHashMap[RpcEndpointRef, RpcEndpoint]() + + private def registerEndpoint(endpoint: RpcEndpoint, endpointRef: RpcEndpointRef): Unit = { + endpointToRef.put(endpoint, endpointRef) + refToEndpoint.put(endpointRef, endpoint) + } + + private def unregisterEndpoint(endpointRef: RpcEndpointRef): Unit = { + val endpoint = refToEndpoint.remove(endpointRef) + if (endpoint != null) { + endpointToRef.remove(endpoint) + } + } + + /** + * Retrieve the [[RpcEndpointRef]] of `endpoint`. + */ + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { + val endpointRef = endpointToRef.get(endpoint) + require(endpointRef != null, s"Cannot find RpcEndpointRef of ${endpoint} in ${this}") + endpointRef + } + + override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { + setupThreadSafeEndpoint(name, endpoint) + } + + override def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { + @volatile var endpointRef: AkkaRpcEndpointRef = null + // Use lazy because the Actor needs to use `endpointRef`. + // So `actorRef` should be created after assigning `endpointRef`. + lazy val actorRef = actorSystem.actorOf(Props(new Actor with ActorLogReceive with Logging { + + assert(endpointRef != null) + + override def preStart(): Unit = { + // Listen for remote client network events + context.system.eventStream.subscribe(self, classOf[AssociationEvent]) + safelyCall(endpoint) { + endpoint.onStart() + } + } + + override def receiveWithLogging: Receive = { + case AssociatedEvent(_, remoteAddress, _) => + safelyCall(endpoint) { + endpoint.onConnected(akkaAddressToRpcAddress(remoteAddress)) + } + + case DisassociatedEvent(_, remoteAddress, _) => + safelyCall(endpoint) { + endpoint.onDisconnected(akkaAddressToRpcAddress(remoteAddress)) + } + + case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) => + safelyCall(endpoint) { + endpoint.onNetworkError(cause, akkaAddressToRpcAddress(remoteAddress)) + } + + case e: AssociationEvent => + // TODO ignore? + + case m: AkkaMessage => + logDebug(s"Received RPC message: $m") + safelyCall(endpoint) { + processMessage(endpoint, m, sender) + } + + case AkkaFailure(e) => + safelyCall(endpoint) { + throw e + } + + case message: Any => { + logWarning(s"Unknown message: $message") + } + + } + + override def postStop(): Unit = { + unregisterEndpoint(endpoint.self) + safelyCall(endpoint) { + endpoint.onStop() + } + } + + }), name = name) + endpointRef = new AkkaRpcEndpointRef(defaultAddress, actorRef, conf, initInConstructor = false) + registerEndpoint(endpoint, endpointRef) + // Now actorRef can be created safely + endpointRef.init() + endpointRef + } + + private def processMessage(endpoint: RpcEndpoint, m: AkkaMessage, _sender: ActorRef): Unit = { + val message = m.message + val needReply = m.needReply + val pf: PartialFunction[Any, Unit] = + if (needReply) { + endpoint.receiveAndReply(new RpcCallContext { + override def sendFailure(e: Throwable): Unit = { + _sender ! AkkaFailure(e) + } + + override def reply(response: Any): Unit = { + _sender ! AkkaMessage(response, false) + } + + // Some RpcEndpoints need to know the sender's address + override val sender: RpcEndpointRef = + new AkkaRpcEndpointRef(defaultAddress, _sender, conf) + }) + } else { + endpoint.receive + } + try { + pf.applyOrElse[Any, Unit](message, { message => + throw new SparkException(s"Unmatched message $message from ${_sender}") + }) + } catch { + case NonFatal(e) => + if (needReply) { + // If the sender asks a reply, we should send the error back to the sender + _sender ! AkkaFailure(e) + } else { + throw e + } + } + } + + /** + * Run `action` safely to avoid to crash the thread. If any non-fatal exception happens, it will + * call `endpoint.onError`. If `endpoint.onError` throws any non-fatal exception, just log it. + */ + private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { + try { + action + } catch { + case NonFatal(e) => { + try { + endpoint.onError(e) + } catch { + case NonFatal(e) => logError(s"Ignore error: ${e.getMessage}", e) + } + } + } + } + + private def akkaAddressToRpcAddress(address: Address): RpcAddress = { + RpcAddress(address.host.getOrElse(defaultAddress.host), + address.port.getOrElse(defaultAddress.port)) + } + + override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { + import actorSystem.dispatcher + actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout). + map(new AkkaRpcEndpointRef(defaultAddress, _, conf)) + } + + override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String = { + AkkaUtils.address( + AkkaUtils.protocol(actorSystem), systemName, address.host, address.port, endpointName) + } + + override def shutdown(): Unit = { + actorSystem.shutdown() + } + + override def stop(endpoint: RpcEndpointRef): Unit = { + require(endpoint.isInstanceOf[AkkaRpcEndpointRef]) + actorSystem.stop(endpoint.asInstanceOf[AkkaRpcEndpointRef].actorRef) + } + + override def awaitTermination(): Unit = { + actorSystem.awaitTermination() + } + + override def toString: String = s"${getClass.getSimpleName}($actorSystem)" +} + +private[spark] class AkkaRpcEnvFactory extends RpcEnvFactory { + + def create(config: RpcEnvConfig): RpcEnv = { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + config.name, config.host, config.port, config.conf, config.securityManager) + new AkkaRpcEnv(actorSystem, config.conf, boundPort) + } +} + +private[akka] class AkkaRpcEndpointRef( + @transient defaultAddress: RpcAddress, + @transient _actorRef: => ActorRef, + @transient conf: SparkConf, + @transient initInConstructor: Boolean = true) + extends RpcEndpointRef(conf) with Logging { + + lazy val actorRef = _actorRef + + override lazy val address: RpcAddress = { + val akkaAddress = actorRef.path.address + RpcAddress(akkaAddress.host.getOrElse(defaultAddress.host), + akkaAddress.port.getOrElse(defaultAddress.port)) + } + + override lazy val name: String = actorRef.path.name + + private[akka] def init(): Unit = { + // Initialize the lazy vals + actorRef + address + name + } + + if (initInConstructor) { + init() + } + + override def send(message: Any): Unit = { + actorRef ! AkkaMessage(message, false) + } + + override def sendWithReply[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] = { + import scala.concurrent.ExecutionContext.Implicits.global + actorRef.ask(AkkaMessage(message, true))(timeout).flatMap { + case msg @ AkkaMessage(message, reply) => + if (reply) { + logError(s"Receive $msg but the sender cannot reply") + Future.failed(new SparkException(s"Receive $msg but the sender cannot reply")) + } else { + Future.successful(message) + } + case AkkaFailure(e) => + Future.failed(e) + }.mapTo[T] + } + + override def toString: String = s"${getClass.getSimpleName}($actorRef)" + +} + +/** + * A wrapper to `message` so that the receiver knows if the sender expects a reply. + * @param message + * @param needReply if the sender expects a reply message + */ +private[akka] case class AkkaMessage(message: Any, needReply: Boolean) + +/** + * A reply with the failure error from the receiver to the sender + */ +private[akka] case class AkkaFailure(e: Throwable) diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index a3caa9f000c89..f748f394d1347 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -19,10 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable -import akka.actor.{ActorRef, Actor} - import org.apache.spark._ -import org.apache.spark.util.{AkkaUtils, ActorLogReceive} +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, RpcEndpoint} private sealed trait OutputCommitCoordinationMessage extends Serializable @@ -34,8 +32,8 @@ private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttem * policy. * * OutputCommitCoordinator is instantiated in both the drivers and executors. On executors, it is - * configured with a reference to the driver's OutputCommitCoordinatorActor, so requests to commit - * output will be forwarded to the driver's OutputCommitCoordinator. + * configured with a reference to the driver's OutputCommitCoordinatorEndpoint, so requests to + * commit output will be forwarded to the driver's OutputCommitCoordinator. * * This class was introduced in SPARK-4879; see that JIRA issue (and the associated pull requests) * for an extensive design discussion. @@ -43,10 +41,7 @@ private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttem private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { // Initialized by SparkEnv - var coordinatorActor: Option[ActorRef] = None - private val timeout = AkkaUtils.askTimeout(conf) - private val maxAttempts = AkkaUtils.numRetries(conf) - private val retryInterval = AkkaUtils.retryWaitMs(conf) + var coordinatorRef: Option[RpcEndpointRef] = None private type StageId = Int private type PartitionId = Long @@ -81,9 +76,9 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { partition: PartitionId, attempt: TaskAttemptId): Boolean = { val msg = AskPermissionToCommitOutput(stage, partition, attempt) - coordinatorActor match { - case Some(actor) => - AkkaUtils.askWithReply[Boolean](msg, actor, maxAttempts, retryInterval, timeout) + coordinatorRef match { + case Some(endpointRef) => + endpointRef.askWithReply[Boolean](msg) case None => logError( "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?") @@ -125,8 +120,8 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { } def stop(): Unit = synchronized { - coordinatorActor.foreach(_ ! StopCoordinator) - coordinatorActor = None + coordinatorRef.foreach(_ send StopCoordinator) + coordinatorRef = None authorizedCommittersByStage.clear() } @@ -157,16 +152,18 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { private[spark] object OutputCommitCoordinator { // This actor is used only for RPC - class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator) - extends Actor with ActorLogReceive with Logging { + private[spark] class OutputCommitCoordinatorEndpoint( + override val rpcEnv: RpcEnv, outputCommitCoordinator: OutputCommitCoordinator) + extends RpcEndpoint with Logging { - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case AskPermissionToCommitOutput(stage, partition, taskAttempt) => - sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt) + context.reply( + outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt)) case StopCoordinator => logInfo("OutputCommitCoordinator stopped!") - context.stop(self) - sender ! true + context.reply(true) + stop() } } } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 48a6ede05e17b..6c2c5261306e7 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -179,7 +179,7 @@ private[spark] object AkkaUtils extends Logging { message: Any, actor: ActorRef, maxAttempts: Int, - retryInterval: Int, + retryInterval: Long, timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala new file mode 100644 index 0000000000000..6665b17c3d5df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.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.util + +import org.apache.spark.{SparkEnv, SparkConf} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} + +object RpcUtils { + + /** + * Retrieve a [[RpcEndpointRef]] which is located in the driver via its name. + */ + def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { + val driverActorSystemName = SparkEnv.driverActorSystemName + val driverHost: String = conf.get("spark.driver.host", "localhost") + val driverPort: Int = conf.getInt("spark.driver.port", 7077) + Utils.checkHost(driverHost, "Expected hostname") + rpcEnv.setupEndpointRef(driverActorSystemName, RpcAddress(driverHost, driverPort), name) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 5e538d6fab2a1..6a6f29dd613cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -17,32 +17,38 @@ package org.apache.spark.deploy.worker -import akka.actor.{ActorSystem, AddressFromURIString, Props} -import akka.testkit.TestActorRef -import akka.remote.DisassociatedEvent +import akka.actor.AddressFromURIString +import org.apache.spark.SparkConf +import org.apache.spark.SecurityManager +import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.scalatest.FunSuite class WorkerWatcherSuite extends FunSuite { test("WorkerWatcher shuts down on valid disassociation") { - val actorSystem = ActorSystem("test") - val targetWorkerUrl = "akka://1.2.3.4/user/Worker" + val conf = new SparkConf() + val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) + val targetWorkerUrl = "akka://test@1.2.3.4:1234/user/Worker" val targetWorkerAddress = AddressFromURIString(targetWorkerUrl) - val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) - val workerWatcher = actorRef.underlyingActor + val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl) workerWatcher.setTesting(testing = true) - actorRef.underlyingActor.receive(new DisassociatedEvent(null, targetWorkerAddress, false)) - assert(actorRef.underlyingActor.isShutDown) + rpcEnv.setupEndpoint("worker-watcher", workerWatcher) + workerWatcher.onDisconnected( + RpcAddress(targetWorkerAddress.host.get, targetWorkerAddress.port.get)) + assert(workerWatcher.isShutDown) + rpcEnv.shutdown() } test("WorkerWatcher stays alive on invalid disassociation") { - val actorSystem = ActorSystem("test") - val targetWorkerUrl = "akka://1.2.3.4/user/Worker" - val otherAkkaURL = "akka://4.3.2.1/user/OtherActor" + val conf = new SparkConf() + val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) + val targetWorkerUrl = "akka://test@1.2.3.4:1234/user/Worker" + val otherAkkaURL = "akka://test@4.3.2.1:1234/user/OtherActor" val otherAkkaAddress = AddressFromURIString(otherAkkaURL) - val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) - val workerWatcher = actorRef.underlyingActor + val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl) workerWatcher.setTesting(testing = true) - actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) - assert(!actorRef.underlyingActor.isShutDown) + rpcEnv.setupEndpoint("worker-watcher", workerWatcher) + workerWatcher.onDisconnected(RpcAddress(otherAkkaAddress.host.get, otherAkkaAddress.port.get)) + assert(!workerWatcher.isShutDown) + rpcEnv.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala new file mode 100644 index 0000000000000..e07bdb9637575 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -0,0 +1,525 @@ +/* + * 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.rpc + +import java.util.concurrent.{TimeUnit, CountDownLatch, TimeoutException} + +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{SparkException, SparkConf} + +/** + * Common tests for an RpcEnv implementation. + */ +abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { + + var env: RpcEnv = _ + + override def beforeAll(): Unit = { + val conf = new SparkConf() + env = createRpcEnv(conf, "local", 12345) + } + + override def afterAll(): Unit = { + if(env != null) { + env.shutdown() + } + } + + def createRpcEnv(conf: SparkConf, name: String, port: Int): RpcEnv + + test("send a message locally") { + @volatile var message: String = null + val rpcEndpointRef = env.setupEndpoint("send-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case msg: String => message = msg + } + }) + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(10 millis)) { + assert("hello" === message) + } + } + + test("send a message remotely") { + @volatile var message: String = null + // Set up a RpcEndpoint using env + env.setupEndpoint("send-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case msg: String => message = msg + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote" ,13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "send-remotely") + try { + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(10 millis)) { + assert("hello" === message) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("send a RpcEndpointRef") { + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case "Hello" => context.reply(self) + case "Echo" => context.reply("Echo") + } + } + val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) + + val newRpcEndpointRef = rpcEndpointRef.askWithReply[RpcEndpointRef]("Hello") + val reply = newRpcEndpointRef.askWithReply[String]("Echo") + assert("Echo" === reply) + } + + test("ask a message locally") { + val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case msg: String => { + context.reply(msg) + } + } + }) + val reply = rpcEndpointRef.askWithReply[String]("hello") + assert("hello" === reply) + } + + test("ask a message remotely") { + env.setupEndpoint("ask-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case msg: String => { + context.reply(msg) + } + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-remotely") + try { + val reply = rpcEndpointRef.askWithReply[String]("hello") + assert("hello" === reply) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("ask a message timeout") { + env.setupEndpoint("ask-timeout", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case msg: String => { + Thread.sleep(100) + context.reply(msg) + } + } + }) + + val conf = new SparkConf() + conf.set("spark.akka.retry.wait", "0") + conf.set("spark.akka.num.retries", "1") + val anotherEnv = createRpcEnv(conf, "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout") + try { + val e = intercept[Exception] { + rpcEndpointRef.askWithReply[String]("hello", 1 millis) + } + assert(e.isInstanceOf[TimeoutException] || e.getCause.isInstanceOf[TimeoutException]) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("onStart and onStop") { + val stopLatch = new CountDownLatch(1) + val calledMethods = mutable.ArrayBuffer[String]() + + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + calledMethods += "start" + } + + override def receive = { + case msg: String => + } + + override def onStop(): Unit = { + calledMethods += "stop" + stopLatch.countDown() + } + } + val rpcEndpointRef = env.setupEndpoint("start-stop-test", endpoint) + env.stop(rpcEndpointRef) + stopLatch.await(10, TimeUnit.SECONDS) + assert(List("start", "stop") === calledMethods) + } + + test("onError: error in onStart") { + @volatile var e: Throwable = null + env.setupEndpoint("onError-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + throw new RuntimeException("Oops!") + } + + override def receive = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in onStop") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + + override def onStop(): Unit = { + throw new RuntimeException("Oops!") + } + }) + + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in receive") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => throw new RuntimeException("Oops!") + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("self: call in onStart") { + @volatile var callSelfSuccessfully = false + + env.setupEndpoint("self-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + self + callSelfSuccessfully = true + } + + override def receive = { + case m => + } + }) + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `onStart` is fine + assert(callSelfSuccessfully === true) + } + } + + test("self: call in receive") { + @volatile var callSelfSuccessfully = false + + val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => { + self + callSelfSuccessfully = true + } + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `receive` is fine + assert(callSelfSuccessfully === true) + } + } + + test("self: call in onStop") { + @volatile var e: Throwable = null + + val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => + } + + override def onStop(): Unit = { + self + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `onStop` is invalid + assert(e != null) + assert(e.getMessage.contains("Cannot find RpcEndpointRef")) + } + } + + test("call receive in sequence") { + // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it + for(i <- 0 until 100) { + @volatile var result = 0 + val endpointRef = env.setupThreadSafeEndpoint(s"receive-in-sequence-$i", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => result += 1 + } + + }) + + (0 until 10) foreach { _ => + new Thread { + override def run() { + (0 until 100) foreach { _ => + endpointRef.send("Hello") + } + } + }.start() + } + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(result == 1000) + } + + env.stop(endpointRef) + } + } + + test("stop(RpcEndpointRef) reentrant") { + @volatile var onStopCount = 0 + val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case m => + } + + override def onStop(): Unit = { + onStopCount += 1 + } + }) + + env.stop(endpointRef) + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(5 millis)) { + // Calling stop twice should only trigger onStop once. + assert(onStopCount == 1) + } + } + + test("sendWithReply") { + val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case m => context.reply("ack") + } + }) + + val f = endpointRef.sendWithReply[String]("Hi") + val ack = Await.result(f, 5 seconds) + assert("ack" === ack) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely") { + env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case m => context.reply("ack") + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "sendWithReply-remotely") + try { + val f = rpcEndpointRef.sendWithReply[String]("hello") + val ack = Await.result(f, 5 seconds) + assert("ack" === ack) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("sendWithReply: error") { + val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case m => context.sendFailure(new SparkException("Oops")) + } + }) + + val f = endpointRef.sendWithReply[String]("Hi") + val e = intercept[SparkException] { + Await.result(f, 5 seconds) + } + assert("Oops" === e.getMessage) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely error") { + env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case msg: String => context.sendFailure(new SparkException("Oops")) + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "sendWithReply-remotely-error") + try { + val f = rpcEndpointRef.sendWithReply[String]("hello") + val e = intercept[SparkException] { + Await.result(f, 5 seconds) + } + assert("Oops" === e.getMessage) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("network events") { + val events = new mutable.ArrayBuffer[(Any, Any)] with mutable.SynchronizedBuffer[(Any, Any)] + env.setupThreadSafeEndpoint("network-events", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case "hello" => + case m => events += "receive" -> m + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + events += "onConnected" -> remoteAddress + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + events += "onDisconnected" -> remoteAddress + } + + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + events += "onNetworkError" -> remoteAddress + } + + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "network-events") + val remoteAddress = anotherEnv.address + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events === List(("onConnected", remoteAddress))) + } + + anotherEnv.shutdown() + anotherEnv.awaitTermination() + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events === List( + ("onConnected", remoteAddress), + ("onNetworkError", remoteAddress), + ("onDisconnected", remoteAddress))) + } + } +} + +case object Start + +case class Ping(id: Int) + +case class Pong(id: Int) diff --git a/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala new file mode 100644 index 0000000000000..58214c0637235 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rpc.akka + +import org.apache.spark.rpc._ +import org.apache.spark.{SecurityManager, SparkConf} + +class AkkaRpcEnvSuite extends RpcEnvSuite { + + override def createRpcEnv(conf: SparkConf, name: String, port: Int): RpcEnv = { + new AkkaRpcEnvFactory().create( + RpcEnvConfig(conf, name, "localhost", port, new SecurityManager(conf))) + } + + test("setupEndpointRef: systemName, address, endpointName") { + val ref = env.setupEndpoint("test_endpoint", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case _ => + } + }) + val conf = new SparkConf() + val newRpcEnv = new AkkaRpcEnvFactory().create( + RpcEnvConfig(conf, "test", "localhost", 12346, new SecurityManager(conf))) + try { + val newRef = newRpcEnv.setupEndpointRef("local", ref.address, "test_endpoint") + assert("akka.tcp://local@localhost:12345/user/test_endpoint" === + newRef.asInstanceOf[AkkaRpcEndpointRef].actorRef.path.toString) + } finally { + newRpcEnv.shutdown() + } + } + +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b9f40046e15a2..efd59a7e5470f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -50,7 +50,9 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"), ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast") + "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.scheduler.OutputCommitCoordinator$OutputCommitCoordinatorActor") ) ++ Seq( // SPARK-6510 Add a Graph#minus method acting as Set#difference ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") From 01dc9f50d1aae1f24021062291d73182a2622f2c Mon Sep 17 00:00:00 2001 From: Li Zhihui Date: Sun, 29 Mar 2015 21:30:37 -0700 Subject: [PATCH 572/817] Fix string interpolator error in HeartbeatReceiver Error log before fixed 15/03/29 10:07:25 ERROR YarnScheduler: Lost an executor 24 (already removed): Executor heartbeat timed out after ${now - lastSeenMs} ms Author: Li Zhihui Closes #5255 from li-zhihui/fixstringinterpolator and squashes the following commits: c93f2b7 [Li Zhihui] Fix string interpolator error in HeartbeatReceiver --- core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 548dcb93c3358..8435e1ea2611c 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -89,7 +89,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule logWarning(s"Removing executor $executorId with no recent heartbeats: " + s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms") scheduler.executorLost(executorId, SlaveLost("Executor heartbeat " + - "timed out after ${now - lastSeenMs} ms")) + s"timed out after ${now - lastSeenMs} ms")) if (sc.supportDynamicAllocation) { sc.killExecutor(executorId) } From 17b13c53ec9d8579a7fb801ab781bce43809db6a Mon Sep 17 00:00:00 2001 From: Eran Medan Date: Mon, 30 Mar 2015 00:02:52 -0700 Subject: [PATCH 573/817] [spark-sql] a better exception message than "scala.MatchError" for unsupported types in Schema creation Currently if trying to register an RDD (or DataFrame in 1.3) as a table that has types that have no supported Schema representation (e.g. type "Any") - it would throw a match error. e.g. scala.MatchError: Any (of class scala.reflect.internal.Types$ClassNoArgsTypeRef) This fix is just to have a nicer error message than a MatchError Author: Eran Medan Closes #5235 from eranation/patch-2 and squashes the following commits: af4b1a2 [Eran Medan] Line should be under 100 chars 0c69e9d [Eran Medan] Change from sys.error UnsupportedOperationException 524be86 [Eran Medan] better exception than scala.MatchError: Any --- .../scala/org/apache/spark/sql/catalyst/ScalaReflection.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d6126c24fc50d..2220970085462 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -179,6 +179,8 @@ trait ScalaReflection { case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + case other => + throw new UnsupportedOperationException(s"Schema for type $other is not supported") } } From de6733036e060e18b0d1f21f9365bda81132a1a2 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 30 Mar 2015 11:41:43 +0100 Subject: [PATCH 574/817] [SPARK-6596] fix the instruction on building scaladoc In README.md under docs/ directory, it says that > You can build just the Spark scaladoc by running build/sbt doc from the SPARK_PROJECT_ROOT directory. I guess the right approach is build/sbt unidoc Author: CodingCat Closes #5253 from CodingCat/SPARK-6596 and squashes the following commits: af379ed [CodingCat] fix the instruction on building scaladoc --- docs/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/README.md b/docs/README.md index 8a54724c4beae..3773ea25c8b67 100644 --- a/docs/README.md +++ b/docs/README.md @@ -60,7 +60,7 @@ We use Sphinx to generate Python API docs, so you will need to install it by run ## API Docs (Scaladoc and Sphinx) -You can build just the Spark scaladoc by running `build/sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `build/sbt unidoc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as @@ -68,7 +68,7 @@ public in `__init__.py`. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `build/sbt doc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `build/sbt unidoc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). From 4bdfb7bab3b9d20167571d9b6888a2a44d9d43fc Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 30 Mar 2015 11:52:02 +0100 Subject: [PATCH 575/817] [SPARK-5750][SPARK-3441][SPARK-5836][CORE] Added documentation explaining shuffle I've updated the Spark Programming Guide to add a section on the shuffle operation providing some background on what it does. I've also addressed some of its performance impacts. I've included documentation to address the following issues: https://issues.apache.org/jira/browse/SPARK-5836 https://issues.apache.org/jira/browse/SPARK-3441 https://issues.apache.org/jira/browse/SPARK-5750 https://issues.apache.org/jira/browse/SPARK-4227 is related but can be addressed in a separate PR since it involves updates to the Spark Configuration Guide. Author: Ilya Ganelin Author: Ilya Ganelin Closes #5074 from ilganeli/SPARK-5750 and squashes the following commits: 6178e24 [Ilya Ganelin] Update programming-guide.md 7a0b96f [Ilya Ganelin] Update programming-guide.md 2c5df08 [Ilya Ganelin] Merge branch 'SPARK-5750' of github.com:ilganeli/spark into SPARK-5750 dffbd2d [Ilya Ganelin] [SPARK-5750] Slight wording update 1ff4eb4 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5750 85f9c6e [Ilya Ganelin] Update programming-guide.md 349d1fa [Ilya Ganelin] Added cross linkf or configuration page eeb5a7a [Ilya Ganelin] [SPARK-5750] Added some minor fixes dd5cc9d [Ilya Ganelin] [SPARK-5750] Fixed some factual inaccuracies with regards to shuffle internals. a8adb57 [Ilya Ganelin] [SPARK-5750] Incoporated feedback from Sean Owen 9954bbe [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5750 159dd1c [Ilya Ganelin] [SPARK-5750] Style fixes from rxin. 75ef67b [Ilya Ganelin] [SPARK-5750][SPARK-3441][SPARK-5836] Added documentation explaining the shuffle operation and included errata from a number of other JIRAs --- docs/programming-guide.md | 83 +++++++++++++++++++++++++++++++++------ 1 file changed, 72 insertions(+), 11 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index f5b775da7930a..f4fabb0927b66 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -937,7 +937,7 @@ for details. Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item). - mapPartitions(func) + mapPartitions(func) Similar to map, but runs separately on each partition (block) of the RDD, so func must be of type Iterator<T> => Iterator<U> when running on an RDD of type T. @@ -964,7 +964,7 @@ for details. Return a new dataset that contains the distinct elements of the source dataset. - groupByKey([numTasks]) + groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
    Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will yield much better @@ -975,25 +975,25 @@ for details. - reduceByKey(func, [numTasks]) + reduceByKey(func, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, V) pairs where the values for each key are aggregated using the given reduce function func, which must be of type (V,V) => V. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument. - aggregateByKey(zeroValue)(seqOp, combOp, [numTasks]) + aggregateByKey(zeroValue)(seqOp, combOp, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, U) pairs where the values for each key are aggregated using the given combine functions and a neutral "zero" value. Allows an aggregated value type that is different than the input value type, while avoiding unnecessary allocations. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument. - sortByKey([ascending], [numTasks]) + sortByKey([ascending], [numTasks]) When called on a dataset of (K, V) pairs where K implements Ordered, returns a dataset of (K, V) pairs sorted by keys in ascending or descending order, as specified in the boolean ascending argument. - join(otherDataset, [numTasks]) + join(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin. - cogroup(otherDataset, [numTasks]) + cogroup(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith. @@ -1006,17 +1006,17 @@ for details. process's stdin and lines output to its stdout are returned as an RDD of strings. - coalesce(numPartitions) + coalesce(numPartitions) Decrease the number of partitions in the RDD to numPartitions. Useful for running operations more efficiently after filtering down a large dataset. repartition(numPartitions) Reshuffle the data in the RDD randomly to create either more or fewer partitions and balance it across them. - This always shuffles all data over the network. + This always shuffles all data over the network. - repartitionAndSortWithinPartitions(partitioner) + repartitionAndSortWithinPartitions(partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys. This is more efficient than calling repartition and then sorting within each partition because it can push the sorting down into the shuffle machinery. @@ -1080,7 +1080,7 @@ for details. SparkContext.objectFile(). - countByKey() + countByKey() Only available on RDDs of type (K, V). Returns a hashmap of (K, Int) pairs with the count of each key. @@ -1090,6 +1090,67 @@ for details. +### Shuffle operations + +Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's +mechanism for re-distributing data so that is grouped differently across partitions. This typically +involves copying data across executors and machines, making the shuffle a complex and +costly operation. + +#### Background + +To understand what happens during the shuffle we can consider the example of the +[`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all +values for a single key are combined into a tuple - the key and the result of executing a reduce +function against all values associated with that key. The challenge is that not all values for a +single key necessarily reside on the same partition, or even the same machine, but they must be +co-located to compute the result. + +In Spark, data is generally not distributed across partitions to be in the necessary place for a +specific operation. During computations, a single task will operate on a single partition - thus, to +organize all the data for a single `reduceByKey` reduce task to execute, Spark needs to perform an +all-to-all operation. It must read from all partitions to find all the values for all keys, +and then bring together values across partitions to compute the final result for each key - +this is called the **shuffle**. + +Although the set of elements in each partition of newly shuffled data will be deterministic, and so +is the ordering of partitions themselves, the ordering of these elements is not. If one desires predictably +ordered data following shuffle then it's possible to use: + +* `mapPartitions` to sort each partition using, for example, `.sorted` +* `repartitionAndSortWithinPartitions` to efficiently sort partitions while simultaneously repartitioning +* `sortBy` to make a globally ordered RDD + +Operations which can cause a shuffle include **repartition** operations like +[`repartition`](#RepartitionLink), and [`coalesce`](#CoalesceLink), **'ByKey** operations +(except for counting) like [`groupByKey`](#GroupByLink) and [`reduceByKey`](#ReduceByLink), and +**join** operations like [`cogroup`](#CogroupLink) and [`join`](#JoinLink). + +#### Performance Impact +The **Shuffle** is an expensive operation since it involves disk I/O, data serialization, and +network I/O. To organize data for the shuffle, Spark generates sets of tasks - *map* tasks to +organize the data, and a set of *reduce* tasks to aggregate it. This nomenclature comes from +MapReduce and does not directly relate to Spark's `map` and `reduce` operations. + +Internally, results from individual map tasks are kept in memory until they can't fit. Then, these +are sorted based on the target partition and written to a single file. On the reduce side, tasks +read the relevant sorted blocks. + +Certain shuffle operations can consume significant amounts of heap memory since they employ +in-memory data structures to organize records before or after transferring them. Specifically, +`reduceByKey` and `aggregateByKey` create these structures on the map side and `'ByKey` operations +generate these on the reduce side. When data does not fit in memory Spark will spill these tables +to disk, incurring the additional overhead of disk I/O and increased garbage collection. + +Shuffle also generates a large number of intermediate files on disk. As of Spark 1.3, these files +are not cleaned up from Spark's temporary storage until Spark is stopped, which means that +long-running Spark jobs may consume available disk space. This is done so the shuffle doesn't need +to be re-computed if the lineage is re-computed. The temporary storage directory is specified by the +`spark.local.dir` configuration parameter when configuring the Spark context. + +Shuffle behavior can be tuned by adjusting a variety of configuration parameters. See the +'Shuffle Behavior' section within the [Spark Configuration Guide](configuration.html). + ## RDD Persistence One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory From 19d4c392fa1738e5dd04418cb008abc8810b8122 Mon Sep 17 00:00:00 2001 From: Jose Manuel Gomez Date: Mon, 30 Mar 2015 14:59:08 +0100 Subject: [PATCH 576/817] [HOTFIX] Update start-slave.sh wihtout this change the below error happens when I execute sbin/start-all.sh localhost: /spark-1.3/sbin/start-slave.sh: line 32: unexpected EOF while looking for matching `"' localhost: /spark-1.3/sbin/start-slave.sh: line 33: syntax error: unexpected end of file my operating system is Linux Mint 17.1 Rebecca Author: Jose Manuel Gomez Closes #5262 from josegom/patch-2 and squashes the following commits: 453af8b [Jose Manuel Gomez] Update start-slave.sh 2c456bd [Jose Manuel Gomez] Update start-slave.sh --- sbin/start-slave.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index c0155384f7395..5a6de11afdd3d 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -19,7 +19,7 @@ # Starts a slave on the machine this script is executed on. -usage="Usage: start-slave.sh where is like "spark://localhost:7077" +usage="Usage: start-slave.sh where is like spark://localhost:7077" if [ $# -lt 2 ]; then echo $usage From fe81f6c779213a91369ec61cf5489ad5c66cc49c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 30 Mar 2015 22:24:12 +0800 Subject: [PATCH 577/817] [SPARK-6595][SQL] MetastoreRelation should be a MultiInstanceRelation Now that we have `DataFrame`s it is possible to have multiple copies in a single query plan. As such, it needs to inherit from `MultiInstanceRelation` or self joins will break. I also add better debugging errors when our self join handling fails in case there are future bugs. Author: Michael Armbrust Closes #5251 from marmbrus/multiMetaStore and squashes the following commits: 4272f6d [Michael Armbrust] [SPARK-6595][SQL] MetastoreRelation should be MuliInstanceRelation --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 10 +++++++++- .../sql/catalyst/analysis/MultiInstanceRelation.scala | 2 +- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 11 +++++++++-- .../spark/sql/hive/HiveMetastoreCatalogSuite.scala | 8 ++++++++ 4 files changed, 27 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 44eceb0b372e6..ba1ac141b9fab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -252,7 +252,15 @@ class Analyzer(catalog: Catalog, case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) - }.head // Only handle first case found, others will be fixed on the next pass. + }.headOption.getOrElse { // Only handle first case, others will be fixed on the next pass. + sys.error( + s""" + |Failure when resolving conflicting references in Join: + |$plan + | + |Conflicting attributes: ${conflictingAttributes.mkString(",")} + """.stripMargin) + } val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) val newRight = right transformUp { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 894c3500cf533..35b74024a4cab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -30,5 +30,5 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * of itself with globally unique expression ids. */ trait MultiInstanceRelation { - def newInstance(): this.type + def newInstance(): LogicalPlan } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d1a99555e90c6..203164ea84292 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical @@ -697,7 +697,7 @@ private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) (@transient sqlContext: SQLContext) - extends LeafNode { + extends LeafNode with MultiInstanceRelation { self: Product => @@ -778,6 +778,13 @@ private[hive] case class MetastoreRelation /** An attribute map for determining the ordinal for non-partition columns. */ val columnOrdinals = AttributeMap(attributes.zipWithIndex) + + override def newInstance() = { + val newCopy = MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) + // The project here is an ugly hack to work around the fact that MetastoreRelation's + // equals method is broken. Please remove this when SPARK-6555 is fixed. + Project(newCopy.output, newCopy) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index aad48ada52642..fa8e11ffec2b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.hive.test.TestHive import org.scalatest.FunSuite import org.apache.spark.sql.test.ExamplePointUDT @@ -36,4 +37,11 @@ class HiveMetastoreCatalogSuite extends FunSuite { assert(HiveMetastoreTypes.toMetastoreType(udt) === HiveMetastoreTypes.toMetastoreType(udt.sqlType)) } + + test("duplicated metastore relations") { + import TestHive.implicits._ + val df = TestHive.sql("SELECT * FROM src") + println(df.queryExecution) + df.as('a).join(df.as('b), $"a.key" === $"b.key") + } } From 32259c671ab419f4c8a6ba8e2f7d676c5dfd0f4f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 30 Mar 2015 11:54:44 -0700 Subject: [PATCH 578/817] [SPARK-6592][SQL] fix filter for scaladoc to generate API doc for Row class under catalyst dir https://issues.apache.org/jira/browse/SPARK-6592 The current impl in SparkBuild.scala filter all classes under catalyst directory, however, we have a corner case that Row class is a public API under that directory we need to include Row into the scaladoc while still excluding other classes of catalyst project Thanks for the help on this patch from rxin and liancheng Author: CodingCat Closes #5252 from CodingCat/SPARK-6592 and squashes the following commits: 02098a4 [CodingCat] ignore collection, enable types (except those protected classes) f7af2cb [CodingCat] commit 3ab4403 [CodingCat] fix filter for scaladoc to generate API doc for Row.scala under catalyst directory --- project/SparkBuild.scala | 16 ++++++++-------- .../spark/sql/types/DataTypeConversions.scala | 2 +- .../apache/spark/sql/types/DataTypeParser.scala | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ac37c605de4b6..d3faa551a4b14 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -360,15 +360,15 @@ object Unidoc { packages .map(_.filterNot(_.getName.contains("$"))) .map(_.filterNot(_.getCanonicalPath.contains("akka"))) - .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) - .map(_.filterNot(_.getCanonicalPath.contains("network"))) - .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) - .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/shuffle"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) - .map(_.filterNot(_.getCanonicalPath.contains("collection"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/execution"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/hive/test"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test"))) } lazy val settings = scalaJavaUnidocSettings ++ Seq ( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala index c243be07a91b6..a9d63e784963d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -protected[sql] object DataTypeConversions { +private[sql] object DataTypeConversions { def productToRow(product: Product, schema: StructType): Row = { val mutableRow = new GenericMutableRow(product.productArity) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala index 89278f7dbc806..34270d0ca7cd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala @@ -112,4 +112,4 @@ private[sql] object DataTypeParser { } /** The exception thrown from the [[DataTypeParser]]. */ -protected[sql] class DataTypeException(message: String) extends Exception(message) +private[sql] class DataTypeException(message: String) extends Exception(message) From df3550084c9975f999ed370dd9f7c495181a68ba Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 30 Mar 2015 12:48:26 -0700 Subject: [PATCH 579/817] [HOTFIX][SPARK-4123]: Updated to fix bug where multiple dependencies added breaks Github output Currently there is a bug whereby if a new patch introduces more than one new dependency (or removes more than one) it breaks the Github post output (see [this build](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29399/consoleFull)). This hotfix will remove `awk` print statements in place of `printf` so as not to automatically add the newline character which is then escaped and added directly at the end of the `awk` statement. This should take a failed build output such as: ```json data: {"body": " [Test build #29400 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29400/consoleFull) for PR 5266 at commit [`2aa4be0`](https://github.com/apache/spark/commit/2aa4be0e1d7ce052f8c901c6d9462c611c3a920a).\n * This patch **passes all tests**.\n * This patch merges cleanly.\n * This patch adds the following public classes _(experimental)_:\n * `class IDF extends Estimator[IDFModel] with IDFParams `\n * `class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] `\n\n * This patch **adds the following new dependencies:**\n * `avro-1.7.7.jar` * `breeze-macros_2.10-0.11.2.jar` * `breeze_2.10-0.11.2.jar`\n * This patch **removes the following dependencies:**\n * `avro-1.7.6.jar` * `breeze-macros_2.10-0.11.1.jar` * `breeze_2.10-0.11.1.jar`"} ``` and turn it into: ```json data: {"body": " [Test build #29400 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/29400/consoleFull) for PR 5266 at commit [`2aa4be0`](https://github.com/apache/spark/commit/2aa4be0e1d7ce052f8c901c6d9462c611c3a920a).\n * This patch **passes all tests**.\n * This patch merges cleanly.\n * This patch adds the following public classes _(experimental)_:\n * `class IDF extends Estimator[IDFModel] with IDFParams `\n * `class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] `\n\n * This patch **adds the following new dependencies:**\n * `avro-1.7.7.jar`\n * `breeze-macros_2.10-0.11.2.jar`\n * `breeze_2.10-0.11.2.jar`\n * This patch **removes the following dependencies:**\n * `avro-1.7.6.jar`\n * `breeze-macros_2.10-0.11.1.jar`\n * `breeze_2.10-0.11.1.jar`"} ``` I've tested this locally and all worked. /cc srowen pwendell nchammas Author: Brennon York Closes #5269 from brennonyork/HOTFIX-SPARK-4123 and squashes the following commits: a441068 [Brennon York] Updated awk to use printf and to manually insert newlines so that the JSON github string when posted is corrected --- dev/tests/pr_new_dependencies.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh index 115a5cd1354f0..370c7cc737bbd 100755 --- a/dev/tests/pr_new_dependencies.sh +++ b/dev/tests/pr_new_dependencies.sh @@ -90,8 +90,8 @@ else echo " * This patch does not change any dependencies." else # Pretty print the new dependencies - added_deps=$(echo "${DIFF_RESULTS}" | grep "<" | cut -d' ' -f2 | awk '{print " * \`"$1"\`"}') - removed_deps=$(echo "${DIFF_RESULTS}" | grep ">" | cut -d' ' -f2 | awk '{print " * \`"$1"\`"}') + added_deps=$(echo "${DIFF_RESULTS}" | grep "<" | cut -d' ' -f2 | awk '{printf " * \`"$1"\`\\n"}') + removed_deps=$(echo "${DIFF_RESULTS}" | grep ">" | cut -d' ' -f2 | awk '{printf " * \`"$1"\`\\n"}') added_deps_text=" * This patch **adds the following new dependencies:**\n${added_deps}" removed_deps_text=" * This patch **removes the following dependencies:**\n${removed_deps}" From f76d2e55b1a67bf5576e1aa001a0b872b9b3895a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 30 Mar 2015 15:47:00 -0700 Subject: [PATCH 580/817] [SPARK-6603] [PySpark] [SQL] add SQLContext.udf and deprecate inferSchema() and applySchema This PR create an alias for `registerFunction` as `udf.register`, to be consistent with Scala API. It also deprecated inferSchema() and applySchema(), show an warning for them. cc rxin Author: Davies Liu Closes #5273 from davies/udf and squashes the following commits: 476e947 [Davies Liu] address comments c096fdb [Davies Liu] add SQLContext.udf and deprecate inferSchema() and applySchema --- python/pyspark/sql/context.py | 87 ++++++++++++++++++++++++----------- 1 file changed, 60 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 795ef0dbc4c47..80939a1f8ab1e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -34,7 +34,7 @@ except ImportError: has_pandas = False -__all__ = ["SQLContext", "HiveContext"] +__all__ = ["SQLContext", "HiveContext", "UDFRegistration"] def _monkey_patch_RDD(sqlCtx): @@ -56,6 +56,31 @@ def toDF(self, schema=None, sampleRatio=None): RDD.toDF = toDF +class UDFRegistration(object): + """Wrapper for register UDF""" + + def __init__(self, sqlCtx): + self.sqlCtx = sqlCtx + + def register(self, name, f, returnType=StringType()): + """Registers a lambda function as a UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not given it default to a string and conversion will automatically + be done. For any other return type, the produced object must match the specified type. + + >>> sqlCtx.udf.register("stringLengthString", lambda x: len(x)) + >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + [Row(c0=u'4')] + + >>> from pyspark.sql.types import IntegerType + >>> sqlCtx.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + """ + return self.sqlCtx.registerFunction(name, f, returnType) + + class SQLContext(object): """Main entry point for Spark SQL functionality. @@ -118,6 +143,11 @@ def getConf(self, key, defaultValue): """ return self._ssql_ctx.getConf(key, defaultValue) + @property + def udf(self): + """Wrapper for register Python function as UDF """ + return UDFRegistration(self) + def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -198,14 +228,12 @@ def inferSchema(self, rdd, samplingRatio=None): >>> df.collect()[0] Row(field1=1, field2=u'row1') """ + warnings.warn("inferSchema is deprecated, please use createDataFrame instead") if isinstance(rdd, DataFrame): raise TypeError("Cannot apply schema to DataFrame") - schema = self._inferSchema(rdd, samplingRatio) - converter = _create_converter(schema) - rdd = rdd.map(converter) - return self.applySchema(rdd, schema) + return self.createDataFrame(rdd, None, samplingRatio) def applySchema(self, rdd, schema): """ @@ -230,6 +258,7 @@ def applySchema(self, rdd, schema): >>> df.collect() [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] """ + warnings.warn("applySchema is deprecated, please use createDataFrame instead") if isinstance(rdd, DataFrame): raise TypeError("Cannot apply schema to DataFrame") @@ -237,23 +266,7 @@ def applySchema(self, rdd, schema): if not isinstance(schema, StructType): raise TypeError("schema should be StructType, but got %s" % schema) - # take the first few rows to verify schema - rows = rdd.take(10) - # Row() cannot been deserialized by Pyrolite - if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': - rdd = rdd.map(tuple) - rows = rdd.take(10) - - for row in rows: - _verify_type(row, schema) - - # convert python objects to sql data - converter = _python_to_sql_converter(schema) - rdd = rdd.map(converter) - - jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) - df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) - return DataFrame(df, self) + return self.createDataFrame(rdd, schema) def createDataFrame(self, data, schema=None, samplingRatio=None): """ @@ -323,22 +336,42 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): if not isinstance(data, RDD): try: # data could be list, tuple, generator ... - data = self._sc.parallelize(data) + rdd = self._sc.parallelize(data) except Exception: raise ValueError("cannot create an RDD from type: %s" % type(data)) + else: + rdd = data if schema is None: - return self.inferSchema(data, samplingRatio) + schema = self._inferSchema(rdd, samplingRatio) + converter = _create_converter(schema) + rdd = rdd.map(converter) if isinstance(schema, (list, tuple)): - first = data.first() + first = rdd.first() if not isinstance(first, (list, tuple)): raise ValueError("each row in `rdd` should be list or tuple, " "but got %r" % type(first)) row_cls = Row(*schema) - schema = self._inferSchema(data.map(lambda r: row_cls(*r)), samplingRatio) + schema = self._inferSchema(rdd.map(lambda r: row_cls(*r)), samplingRatio) - return self.applySchema(data, schema) + # take the first few rows to verify schema + rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + + for row in rows: + _verify_type(row, schema) + + # convert python objects to sql data + converter = _python_to_sql_converter(schema) + rdd = rdd.map(converter) + + jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) + df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) + return DataFrame(df, self) def registerDataFrameAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. From fde6945417355ae57500b67d034c9cad4f20d240 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 31 Mar 2015 07:48:37 +0800 Subject: [PATCH 581/817] [SPARK-6369] [SQL] Uses commit coordinator to help committing Hive and Parquet tables This PR leverages the output commit coordinator introduced in #4066 to help committing Hive and Parquet tables. This PR extracts output commit code in `SparkHadoopWriter.commit` to `SparkHadoopMapRedUtil.commitTask`, and reuses it for committing Parquet and Hive tables on executor side. TODO - [ ] Add tests [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5139) Author: Cheng Lian Closes #5139 from liancheng/spark-6369 and squashes the following commits: 72eb628 [Cheng Lian] Fixes typo in javadoc 9a4b82b [Cheng Lian] Adds javadoc and addresses @aarondav's comments dfdf3ef [Cheng Lian] Uses commit coordinator to help committing Hive and Parquet tables --- .../org/apache/spark/SparkHadoopWriter.scala | 52 +---------- .../spark/mapred/SparkHadoopMapRedUtil.scala | 91 ++++++++++++++++++- .../sql/parquet/ParquetTableOperations.scala | 11 ++- .../apache/spark/sql/parquet/newParquet.scala | 4 +- .../hive/execution/InsertIntoHiveTable.scala | 1 - .../spark/sql/hive/hiveWriterContainers.scala | 17 +--- 6 files changed, 103 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 6eb4537d10477..2ec42d3aea169 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.executor.CommitDeniedException import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD @@ -104,55 +103,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } def commit() { - val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() - - // Called after we have decided to commit - def performCommit(): Unit = { - try { - cmtr.commitTask(taCtxt) - logInfo (s"$taID: Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e - } - } - - // First, check whether the task's output has already been committed by some other attempt - if (cmtr.needsTaskCommit(taCtxt)) { - // The task output needs to be committed, but we don't know whether some other task attempt - // might be racing to commit the same output partition. Therefore, coordinate with the driver - // in order to determine whether this attempt can commit (see SPARK-4879). - val shouldCoordinateWithDriver: Boolean = { - val sparkConf = SparkEnv.get.conf - // We only need to coordinate with the driver if there are multiple concurrent task - // attempts, which should only occur if speculation is enabled - val speculationEnabled = sparkConf.getBoolean("spark.speculation", false) - // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs - sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) - } - if (shouldCoordinateWithDriver) { - val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator - val canCommit = outputCommitCoordinator.canCommit(jobID, splitID, attemptID) - if (canCommit) { - performCommit() - } else { - val msg = s"$taID: Not committed because the driver did not authorize commit" - logInfo(msg) - // We need to abort the task so that the driver can reschedule new attempts, if necessary - cmtr.abortTask(taCtxt) - throw new CommitDeniedException(msg, jobID, splitID, attemptID) - } - } else { - // Speculation is disabled or a user has chosen to manually bypass the commit coordination - performCommit() - } - } else { - // Some other attempt committed the output, so we do nothing and signal success - logInfo(s"No need to commit output of task because needsTaskCommit=false: ${taID.value}") - } + SparkHadoopMapRedUtil.commitTask( + getOutputCommitter(), getTaskContext(), jobID, splitID, attemptID) } def commitJob() { diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 87c2aa481095d..818f7a4c8d422 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -17,9 +17,15 @@ package org.apache.spark.mapred +import java.io.IOException import java.lang.reflect.Modifier -import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{TaskAttemptContext => MapReduceTaskAttemptContext} +import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} + +import org.apache.spark.executor.CommitDeniedException +import org.apache.spark.{Logging, SparkEnv, TaskContext} private[spark] trait SparkHadoopMapRedUtil { @@ -65,3 +71,86 @@ trait SparkHadoopMapRedUtil { } } } + +object SparkHadoopMapRedUtil extends Logging { + /** + * Commits a task output. Before committing the task output, we need to know whether some other + * task attempt might be racing to commit the same output partition. Therefore, coordinate with + * the driver in order to determine whether this attempt can commit (please see SPARK-4879 for + * details). + * + * Output commit coordinator is only contacted when the following two configurations are both set + * to `true`: + * + * - `spark.speculation` + * - `spark.hadoop.outputCommitCoordination.enabled` + */ + def commitTask( + committer: MapReduceOutputCommitter, + mrTaskContext: MapReduceTaskAttemptContext, + jobId: Int, + splitId: Int, + attemptId: Int): Unit = { + + val mrTaskAttemptID = mrTaskContext.getTaskAttemptID + + // Called after we have decided to commit + def performCommit(): Unit = { + try { + committer.commitTask(mrTaskContext) + logInfo(s"$mrTaskAttemptID: Committed") + } catch { + case cause: IOException => + logError(s"Error committing the output of task: $mrTaskAttemptID", cause) + committer.abortTask(mrTaskContext) + throw cause + } + } + + // First, check whether the task's output has already been committed by some other attempt + if (committer.needsTaskCommit(mrTaskContext)) { + val shouldCoordinateWithDriver: Boolean = { + val sparkConf = SparkEnv.get.conf + // We only need to coordinate with the driver if there are multiple concurrent task + // attempts, which should only occur if speculation is enabled + val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false) + // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs + sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) + } + + if (shouldCoordinateWithDriver) { + val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator + val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, attemptId) + + if (canCommit) { + performCommit() + } else { + val message = + s"$mrTaskAttemptID: Not committed because the driver did not authorize commit" + logInfo(message) + // We need to abort the task so that the driver can reschedule new attempts, if necessary + committer.abortTask(mrTaskContext) + throw new CommitDeniedException(message, jobId, splitId, attemptId) + } + } else { + // Speculation is disabled or a user has chosen to manually bypass the commit coordination + performCommit() + } + } else { + // Some other attempt committed the output, so we do nothing and signal success + logInfo(s"No need to commit output of task because needsTaskCommit=false: $mrTaskAttemptID") + } + } + + def commitTask( + committer: MapReduceOutputCommitter, + mrTaskContext: MapReduceTaskAttemptContext, + sparkTaskContext: TaskContext): Unit = { + commitTask( + committer, + mrTaskContext, + sparkTaskContext.stageId(), + sparkTaskContext.partitionId(), + sparkTaskContext.attemptNumber()) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5130d8ad5e003..1c868da23e060 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.parquet import java.io.IOException import java.lang.{Long => JLong} -import java.text.SimpleDateFormat -import java.text.NumberFormat +import java.text.{NumberFormat, SimpleDateFormat} import java.util.concurrent.{Callable, TimeUnit} -import java.util.{ArrayList, Collections, Date, List => JList} +import java.util.{Date, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable @@ -43,12 +42,13 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.{Logging, SerializableWritable, TaskContext} /** @@ -356,7 +356,7 @@ private[sql] case class InsertIntoParquetTable( } finally { writer.close(hadoopContext) } - committer.commitTask(hadoopContext) + SparkHadoopMapRedUtil.commitTask(committer, hadoopContext, context) 1 } val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) @@ -512,6 +512,7 @@ private[parquet] class FilteringParquetRowInputFormat import parquet.filter2.compat.FilterCompat.Filter import parquet.filter2.compat.RowGroupFilter + import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.blockLocationCache val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) 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 53f765ee26a13..19800ad88c031 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 @@ -42,6 +42,7 @@ import parquet.hadoop.{ParquetInputFormat, _} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} import org.apache.spark.sql.catalyst.expressions @@ -669,7 +670,8 @@ private[sql] case class ParquetRelation2( } finally { writer.close(hadoopContext) } - committer.commitTask(hadoopContext) + + SparkHadoopMapRedUtil.commitTask(committer, hadoopContext, context) } val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) /* apparently we need a TaskAttemptID to construct an OutputCommitter; diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index da53d30354551..cdf012b5117be 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -72,7 +72,6 @@ case class InsertIntoHiveTable( val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName assert(outputFileFormatClassName != null, "Output format class not set") conf.value.set("mapred.output.format.class", outputFileFormatClassName) - conf.value.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( conf.value, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index ba2bf67aed684..8398da268174d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive -import java.io.IOException import java.text.NumberFormat import java.util.Date @@ -118,19 +117,7 @@ private[hive] class SparkHiveWriterContainer( } protected def commit() { - if (committer.needsTaskCommit(taskContext)) { - try { - committer.commitTask(taskContext) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - committer.abortTask(taskContext) - throw e - } - } else { - logInfo("No need to commit output of task: " + taID.value) - } + SparkHadoopMapRedUtil.commitTask(committer, taskContext, jobID, splitID, attemptID) } private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { @@ -213,7 +200,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( .zip(row.toSeq.takeRight(dynamicPartColNames.length)) .map { case (col, rawVal) => val string = if (rawVal == null) null else String.valueOf(rawVal) - val colString = + val colString = if (string == null || string.isEmpty) { defaultPartName } else { From b8ff2bc61c9835867f56afa1860ab5eb727c4a58 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 30 Mar 2015 20:47:10 -0700 Subject: [PATCH 582/817] [SPARK-6119][SQL] DataFrame support for missing data handling This pull request adds variants of DataFrame.na.drop and DataFrame.na.fill to the Scala/Java API, and DataFrame.fillna and DataFrame.dropna to the Python API. Author: Reynold Xin Closes #5274 from rxin/df-missing-value and squashes the following commits: 4ee1b98 [Reynold Xin] Improve error reporting in Python. 33a330c [Reynold Xin] Remove replace for now. bc4fdbb [Reynold Xin] Added documentation for replace. d56f5a5 [Reynold Xin] Added replace for Scala/Java. 2385d00 [Reynold Xin] Feedback from Xiangrui on "how". 914a374 [Reynold Xin] fill with map. 185c67e [Reynold Xin] Allow specifying column subsets in fill. 749eb47 [Reynold Xin] fillna 249b94e [Reynold Xin] Removing undefined functions. 6a73c68 [Reynold Xin] Missing file. 67d7003 [Reynold Xin] [SPARK-6119][SQL] DataFrame.na.drop (Scala/Java) and DataFrame.dropna (Python) --- python/pyspark/sql/dataframe.py | 86 +++++++ python/pyspark/sql/tests.py | 96 ++++++++ .../catalyst/expressions/nullFunctions.scala | 25 +- .../org/apache/spark/sql/DataFrame.scala | 15 +- .../spark/sql/DataFrameNaFunctions.scala | 228 ++++++++++++++++++ .../org/apache/spark/sql/GroupedData.scala | 5 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 157 ++++++++++++ 8 files changed, 606 insertions(+), 8 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 23c0e63e77812..4f174de811697 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -690,6 +690,86 @@ def subtract(self, other): """ return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) + def dropna(self, how='any', thresh=None, subset=None): + """Returns a new :class:`DataFrame` omitting rows with null values. + + :param how: 'any' or 'all'. + If 'any', drop a row if it contains any nulls. + If 'all', drop a row only if all its values are null. + :param thresh: int, default None + If specified, drop rows that have less than `thresh` non-null values. + This overwrites the `how` parameter. + :param subset: optional list of column names to consider. + + >>> df4.dropna().show() + age height name + 10 80 Alice + """ + if how is not None and how not in ['any', 'all']: + raise ValueError("how ('" + how + "') should be 'any' or 'all'") + + if subset is None: + subset = self.columns + elif isinstance(subset, basestring): + subset = [subset] + elif not isinstance(subset, (list, tuple)): + raise ValueError("subset should be a list or tuple of column names") + + if thresh is None: + thresh = len(subset) if how == 'any' else 1 + + cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) + cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) + return DataFrame(self._jdf.na().drop(thresh, cols), self.sql_ctx) + + def fillna(self, value, subset=None): + """Replace null values. + + :param value: int, long, float, string, or dict. + Value to replace null values with. + If the value is a dict, then `subset` is ignored and `value` must be a mapping + from column name (string) to replacement value. The replacement value must be + an int, long, float, or string. + :param subset: optional list of column names to consider. + Columns specified in subset that do not have matching data type are ignored. + For example, if `value` is a string, and subset contains a non-string column, + then the non-string column is simply ignored. + + >>> df4.fillna(50).show() + age height name + 10 80 Alice + 5 50 Bob + 50 50 Tom + 50 50 null + + >>> df4.fillna({'age': 50, 'name': 'unknown'}).show() + age height name + 10 80 Alice + 5 null Bob + 50 null Tom + 50 null unknown + """ + if not isinstance(value, (float, int, long, basestring, dict)): + raise ValueError("value should be a float, int, long, string, or dict") + + if isinstance(value, (int, long)): + value = float(value) + + if isinstance(value, dict): + value = MapConverter().convert(value, self.sql_ctx._sc._gateway._gateway_client) + return DataFrame(self._jdf.na().fill(value), self.sql_ctx) + elif subset is None: + return DataFrame(self._jdf.na().fill(value), self.sql_ctx) + else: + if isinstance(subset, basestring): + subset = [subset] + elif not isinstance(subset, (list, tuple)): + raise ValueError("subset should be a list or tuple of column names") + + cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) + cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) + return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + def withColumn(self, colName, col): """ Return a new :class:`DataFrame` by adding a column. @@ -1069,6 +1149,12 @@ def _test(): globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), Row(name='Bob', age=5, height=85)]).toDF() + + globs['df4'] = sc.parallelize([Row(name='Alice', age=10, height=80), + Row(name='Bob', age=5, height=None), + Row(name='Tom', age=None, height=None), + Row(name=None, age=None, height=None)]).toDF() + (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 2720439416682..258464b7f230d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -415,6 +415,102 @@ def test_infer_long_type(self): self.assertEqual(_infer_type(2**61), LongType()) self.assertEqual(_infer_type(2**71), LongType()) + def test_dropna(self): + schema = StructType([ + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("height", DoubleType(), True)]) + + # shouldn't drop a non-null row + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, 80.1)], schema).dropna().count(), + 1) + + # dropping rows with a single null value + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna().count(), + 0) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(how='any').count(), + 0) + + # if how = 'all', only drop rows if all values are null + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(how='all').count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(None, None, None)], schema).dropna(how='all').count(), + 0) + + # how and subset + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna(how='any', subset=['name', 'age']).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, None)], schema).dropna(how='any', subset=['name', 'age']).count(), + 0) + + # threshold + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(thresh=2).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, None)], schema).dropna(thresh=2).count(), + 0) + + # threshold and subset + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna(thresh=2, subset=['name', 'age']).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 180.9)], schema).dropna(thresh=2, subset=['name', 'age']).count(), + 0) + + # thresh should take precedence over how + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna( + how='any', thresh=2, subset=['name', 'age']).count(), + 1) + + def test_fillna(self): + schema = StructType([ + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("height", DoubleType(), True)]) + + # fillna shouldn't change non-null values + row = self.sqlCtx.createDataFrame([(u'Alice', 10, 80.1)], schema).fillna(50).first() + self.assertEqual(row.age, 10) + + # fillna with int + row = self.sqlCtx.createDataFrame([(u'Alice', None, None)], schema).fillna(50).first() + self.assertEqual(row.age, 50) + self.assertEqual(row.height, 50.0) + + # fillna with double + row = self.sqlCtx.createDataFrame([(u'Alice', None, None)], schema).fillna(50.1).first() + self.assertEqual(row.age, 50) + self.assertEqual(row.height, 50.1) + + # fillna with string + row = self.sqlCtx.createDataFrame([(None, None, None)], schema).fillna("hello").first() + self.assertEqual(row.name, u"hello") + self.assertEqual(row.age, None) + + # fillna with subset specified for numeric cols + row = self.sqlCtx.createDataFrame( + [(None, None, None)], schema).fillna(50, subset=['name', 'age']).first() + self.assertEqual(row.name, None) + self.assertEqual(row.age, 50) + self.assertEqual(row.height, None) + + # fillna with subset specified for numeric cols + row = self.sqlCtx.createDataFrame( + [(None, None, None)], schema).fillna("haha", subset=['name', 'age']).first() + self.assertEqual(row.name, "haha") + self.assertEqual(row.age, None) + self.assertEqual(row.height, None) + class HiveContextSQLTests(ReusedPySparkTestCase): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index d1f3d4f4ee9ee..f9161cf34f0c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -35,7 +35,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { override def toString: String = s"Coalesce(${children.mkString(",")})" - def dataType: DataType = if (resolved) { + override def dataType: DataType = if (resolved) { children.head.dataType } else { val childTypes = children.map(c => s"$c: ${c.dataType}").mkString(", ") @@ -74,3 +74,26 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E child.eval(input) != null } } + +/** + * A predicate that is evaluated to be true if there are at least `n` non-null values. + */ +case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate { + override def nullable: Boolean = false + override def foldable: Boolean = false + override def toString: String = s"AtLeastNNulls(n, ${children.mkString(",")})" + + private[this] val childrenArray = children.toArray + + override def eval(input: Row): Boolean = { + var numNonNulls = 0 + var i = 0 + while (i < childrenArray.length && numNonNulls < n) { + if (childrenArray(i).eval(input) != null) { + numNonNulls += 1 + } + i += 1 + } + numNonNulls >= n + } +} 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 423ef3912bc89..5cd0a18ff688c 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 @@ -237,8 +237,8 @@ class DataFrame private[sql]( def toDF(colNames: String*): DataFrame = { require(schema.size == colNames.size, "The number of columns doesn't match.\n" + - "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + colNames.mkString(", ")) + s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" + + s"New column names (${colNames.size}): " + colNames.mkString(", ")) val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => apply(oldName).as(newName) @@ -319,6 +319,17 @@ class DataFrame private[sql]( */ def show(): Unit = show(20) + /** + * Returns a [[DataFrameNaFunctions]] for working with missing data. + * {{{ + * // Dropping rows containing any null values. + * df.na.drop() + * }}} + * + * @group dfops + */ + def na: DataFrameNaFunctions = new DataFrameNaFunctions(this) + /** * Cartesian join with another [[DataFrame]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala new file mode 100644 index 0000000000000..3a3dc70f7285c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -0,0 +1,228 @@ +/* +* 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 + +import java.{lang => jl} + +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + + +/** + * Functionality for working with missing data in [[DataFrame]]s. + */ +final class DataFrameNaFunctions private[sql](df: DataFrame) { + + /** + * Returns a new [[DataFrame]] that drops rows containing any null values. + */ + def drop(): DataFrame = drop("any", df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing null values. + * + * If `how` is "any", then drop rows containing any null values. + * If `how` is "all", then drop rows only if every column is null for that row. + */ + def drop(how: String): DataFrame = drop(how, df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing any null values + * in the specified columns. + */ + def drop(cols: Array[String]): DataFrame = drop(cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame ]] that drops rows containing any null values + * in the specified columns. + */ + def drop(cols: Seq[String]): DataFrame = drop(cols.size, cols) + + /** + * Returns a new [[DataFrame]] that drops rows containing null values + * in the specified columns. + * + * If `how` is "any", then drop rows containing any null values in the specified columns. + * If `how` is "all", then drop rows only if every specified column is null for that row. + */ + def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that drops rows containing null values + * in the specified columns. + * + * If `how` is "any", then drop rows containing any null values in the specified columns. + * If `how` is "all", then drop rows only if every specified column is null for that row. + */ + def drop(how: String, cols: Seq[String]): DataFrame = { + how.toLowerCase match { + case "any" => drop(cols.size, cols) + case "all" => drop(1, cols) + case _ => throw new IllegalArgumentException(s"how ($how) must be 'any' or 'all'") + } + } + + /** + * Returns a new [[DataFrame]] that drops rows containing less than `minNonNulls` non-null values. + */ + def drop(minNonNulls: Int): DataFrame = drop(minNonNulls, df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing less than `minNonNulls` non-null + * values in the specified columns. + */ + def drop(minNonNulls: Int, cols: Array[String]): DataFrame = drop(minNonNulls, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that drops rows containing less than + * `minNonNulls` non-null values in the specified columns. + */ + def drop(minNonNulls: Int, cols: Seq[String]): DataFrame = { + // Filtering condition -- only keep the row if it has at least `minNonNulls` non-null values. + val predicate = AtLeastNNonNulls(minNonNulls, cols.map(name => df.resolve(name))) + df.filter(Column(predicate)) + } + + /** + * Returns a new [[DataFrame]] that replaces null values in numeric columns with `value`. + */ + def fill(value: Double): DataFrame = fill(value, df.columns) + + /** + * Returns a new [[DataFrame ]] that replaces null values in string columns with `value`. + */ + def fill(value: String): DataFrame = fill(value, df.columns) + + /** + * Returns a new [[DataFrame]] that replaces null values in specified numeric columns. + * If a specified column is not a numeric column, it is ignored. + */ + def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values in specified + * numeric columns. If a specified column is not a numeric column, it is ignored. + */ + def fill(value: Double, cols: Seq[String]): DataFrame = { + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + // Only fill if the column is part of the cols list. + if (f.dataType.isInstanceOf[NumericType] && cols.exists(col => columnEquals(f.name, col))) { + fillCol[Double](f, value) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + + /** + * Returns a new [[DataFrame]] that replaces null values in specified string columns. + * If a specified column is not a string column, it is ignored. + */ + def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values in + * specified string columns. If a specified column is not a string column, it is ignored. + */ + def fill(value: String, cols: Seq[String]): DataFrame = { + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + // Only fill if the column is part of the cols list. + if (f.dataType.isInstanceOf[StringType] && cols.exists(col => columnEquals(f.name, col))) { + fillCol[String](f, value) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + + /** + * Returns a new [[DataFrame]] that replaces null values. + * + * The key of the map is the column name, and the value of the map is the replacement value. + * The value must be of the following type: `Integer`, `Long`, `Float`, `Double`, `String`. + * + * For example, the following replaces null values in column "A" with string "unknown", and + * null values in column "B" with numeric value 1.0. + * {{{ + * import com.google.common.collect.ImmutableMap; + * df.na.fill(ImmutableMap.of("A", "unknown", "B", 1.0)); + * }}} + */ + def fill(valueMap: java.util.Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values. + * + * The key of the map is the column name, and the value of the map is the replacement value. + * The value must be of the following type: `Int`, `Long`, `Float`, `Double`, `String`. + * + * For example, the following replaces null values in column "A" with string "unknown", and + * null values in column "B" with numeric value 1.0. + * {{{ + * df.na.fill(Map( + * "A" -> "unknown", + * "B" -> 1.0 + * )) + * }}} + */ + def fill(valueMap: Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + + private def fill0(values: Seq[(String, Any)]): DataFrame = { + // Error handling + values.foreach { case (colName, replaceValue) => + // Check column name exists + df.resolve(colName) + + // Check data type + replaceValue match { + case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: String => + // This is good + case _ => throw new IllegalArgumentException( + s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).") + } + } + + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + values.find { case (k, _) => columnEquals(k, f.name) }.map { case (_, v) => + v match { + case v: jl.Float => fillCol[Double](f, v.toDouble) + case v: jl.Double => fillCol[Double](f, v) + case v: jl.Long => fillCol[Double](f, v.toDouble) + case v: jl.Integer => fillCol[Double](f, v.toDouble) + case v: String => fillCol[String](f, v) + } + }.getOrElse(df.col(f.name)) + } + df.select(projections : _*) + } + + /** + * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. + */ + private def fillCol[T](col: StructField, replacement: T): Column = { + coalesce(df.col(col.name), lit(replacement).cast(col.dataType)).as(col.name) + } +} 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 45a63ae26ed71..a5e6b638d2150 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 @@ -127,10 +127,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department * import com.google.common.collect.ImmutableMap; - * df.groupBy("department").agg(ImmutableMap.builder() - * .put("age", "max") - * .put("expense", "sum") - * .build()); + * df.groupBy("department").agg(ImmutableMap.of("age", "max", "expense", "sum")); * }}} */ def agg(exprs: java.util.Map[String, String]): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 2b0358c4e2a1e..0b770f2251943 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -49,7 +49,7 @@ private[sql] object JsonRDD extends Logging { val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = if (schemaData.isEmpty()) { - Set.empty[(String,DataType)] + Set.empty[(String, DataType)] } else { parseJson(schemaData, columnNameOfCorruptRecords).map(allKeysWithValueTypes).reduce(_ ++ _) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala new file mode 100644 index 0000000000000..0896f175c056f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -0,0 +1,157 @@ +/* + * 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 + +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.test.TestSQLContext.implicits._ + + +class DataFrameNaFunctionsSuite extends QueryTest { + + def createDF(): DataFrame = { + Seq[(String, java.lang.Integer, java.lang.Double)]( + ("Bob", 16, 176.5), + ("Alice", null, 164.3), + ("David", 60, null), + ("Amy", null, null), + (null, null, null)).toDF("name", "age", "height") + } + + test("drop") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop("name" :: Nil), + rows(0) :: rows(1) :: rows(2) :: rows(3) :: Nil) + + checkAnswer( + input.na.drop("age" :: Nil), + rows(0) :: rows(2) :: Nil) + + checkAnswer( + input.na.drop("age" :: "height" :: Nil), + rows(0) :: Nil) + + checkAnswer( + input.na.drop(), + rows(0)) + + // dropna on an a dataframe with no column should return an empty data frame. + val empty = input.sqlContext.emptyDataFrame.select() + assert(empty.na.drop().count() === 0L) + + // Make sure the columns are properly named. + assert(input.na.drop().columns.toSeq === input.columns.toSeq) + } + + test("drop with how") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop("all"), + rows(0) :: rows(1) :: rows(2) :: rows(3) :: Nil) + + checkAnswer( + input.na.drop("any"), + rows(0) :: Nil) + + checkAnswer( + input.na.drop("any", Seq("age", "height")), + rows(0) :: Nil) + + checkAnswer( + input.na.drop("all", Seq("age", "height")), + rows(0) :: rows(1) :: rows(2) :: Nil) + } + + test("drop with threshold") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop(2, Seq("age", "height")), + rows(0) :: Nil) + + checkAnswer( + input.na.drop(3, Seq("name", "age", "height")), + rows(0)) + + // Make sure the columns are properly named. + assert(input.na.drop(2, Seq("age", "height")).columns.toSeq === input.columns.toSeq) + } + + test("fill") { + val input = createDF() + + val fillNumeric = input.na.fill(50.6) + checkAnswer( + fillNumeric, + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, 50.6) :: + Row("Amy", 50, 50.6) :: + Row(null, 50, 50.6) :: Nil) + + // Make sure the columns are properly named. + assert(fillNumeric.columns.toSeq === input.columns.toSeq) + + // string + checkAnswer( + input.na.fill("unknown").select("name"), + Row("Bob") :: Row("Alice") :: Row("David") :: Row("Amy") :: Row("unknown") :: Nil) + assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) + + // fill double with subset columns + checkAnswer( + input.na.fill(50.6, "age" :: Nil), + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, null) :: + Row("Amy", 50, null) :: + Row(null, 50, null) :: Nil) + + // fill string with subset columns + checkAnswer( + Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), + Row("test", null)) + } + + test("fill with map") { + val df = Seq[(String, String, java.lang.Long, java.lang.Double)]( + (null, null, null, null)).toDF("a", "b", "c", "d") + checkAnswer( + df.na.fill(Map( + "a" -> "test", + "c" -> 1, + "d" -> 2.2 + )), + Row("test", null, 1, 2.2)) + + // Test Java version + checkAnswer( + df.na.fill(mapAsJavaMap(Map( + "a" -> "test", + "c" -> 1, + "d" -> 2.2 + ))), + Row("test", null, 1, 2.2)) + } +} From 56775571cb938c819e5f7c3d49c5dd416ed034cb Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 30 Mar 2015 22:10:49 -0700 Subject: [PATCH 583/817] [SPARK-5124][Core] Move StopCoordinator to the receive method since it does not require a reply Hotfix for #4588 cc rxin Author: zsxwing Closes #5283 from zsxwing/hotfix and squashes the following commits: cf3e5a7 [zsxwing] Move StopCoordinator to the receive method since it does not require a reply --- .../spark/scheduler/OutputCommitCoordinator.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index f748f394d1347..17055e2f22d0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -156,14 +156,16 @@ private[spark] object OutputCommitCoordinator { override val rpcEnv: RpcEnv, outputCommitCoordinator: OutputCommitCoordinator) extends RpcEndpoint with Logging { + override def receive: PartialFunction[Any, Unit] = { + case StopCoordinator => + logInfo("OutputCommitCoordinator stopped!") + stop() + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case AskPermissionToCommitOutput(stage, partition, taskAttempt) => context.reply( outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt)) - case StopCoordinator => - logInfo("OutputCommitCoordinator stopped!") - context.reply(true) - stop() } } } From f07e714062f02feadff10a45f9b9061444bb8ec5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 31 Mar 2015 00:19:51 -0700 Subject: [PATCH 584/817] [SPARK-6625][SQL] Add common string filters to data sources. Filters such as startsWith, endsWith, contains will be very useful for data sources that provide search functionality, e.g. Succinct, Elastic Search, Solr. I also took this chance to improve documentation for the data source filters. Author: Reynold Xin Closes #5285 from rxin/ds-string-filters and squashes the following commits: f021727 [Reynold Xin] Fixed grammar. 7695a52 [Reynold Xin] [SPARK-6625][SQL] Add common string filters to data sources. --- .../sql/sources/DataSourceStrategy.scala | 10 +++ .../apache/spark/sql/sources/filters.scala | 69 ++++++++++++++++++ .../apache/spark/sql/sources/interfaces.scala | 3 + .../spark/sql/sources/FilteredScanSuite.scala | 73 +++++++++++++------ 4 files changed, 133 insertions(+), 22 deletions(-) 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 67f3507c61ab6..83b603a4bb245 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.StringType import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -166,6 +167,15 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.Not(child) => translate(child).map(sources.Not) + case expressions.StartsWith(a: Attribute, Literal(v: String, StringType)) => + Some(sources.StringStartsWith(a.name, v)) + + case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => + Some(sources.StringEndsWith(a.name, v)) + + case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => + Some(sources.StringContains(a.name, v)) + case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index 1e4505e36d2f0..791046e0079d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,16 +17,85 @@ package org.apache.spark.sql.sources +/** + * A filter predicate for data sources. + */ abstract class Filter +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * equal to `value`. + */ case class EqualTo(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * greater than `value`. + */ case class GreaterThan(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * greater than or equal to `value`. + */ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * less than `value`. + */ case class LessThan(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * less than or equal to `value`. + */ case class LessThanOrEqual(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to one of the values in the array. + */ case class In(attribute: String, values: Array[Any]) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to null. + */ case class IsNull(attribute: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a non-null value. + */ case class IsNotNull(attribute: String) extends Filter + +/** + * A filter that evaluates to `true` iff both `left` or `right` evaluate to `true`. + */ case class And(left: Filter, right: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff at least one of `left` or `right` evaluates to `true`. + */ case class Or(left: Filter, right: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff `child` is evaluated to `false`. + */ case class Not(child: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that starts with `value`. + */ +case class StringStartsWith(attribute: String, value: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that starts with `value`. + */ +case class StringEndsWith(attribute: String, value: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that contains the string `value`. + */ +case class StringContains(attribute: String, value: String) extends Filter 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 a046a48c1733d..8f9946a5a801e 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 @@ -152,6 +152,9 @@ trait PrunedScan { * A BaseRelation that can eliminate unneeded columns and filter using selected * predicates before producing an RDD containing all matching tuples as Row objects. * + * The actual filter should be the conjunction of all `filters`, + * i.e. they should be "and" together. + * * 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. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index ffeccf0b69394..72ddc0ea2c8cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -35,20 +35,23 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL extends BaseRelation with PrunedFilteredScan { - override def schema = + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: - StructField("b", IntegerType, nullable = false) :: Nil) + StructField("b", IntegerType, nullable = false) :: + StructField("c", StringType, nullable = false) :: Nil) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) + case "c" => (i: Int) => Seq((i - 1 + 'a').toChar.toString * 10) } FiltersPushed.list = filters - def translateFilter(filter: Filter): Int => Boolean = filter match { + // Predicate test on integer column + def translateFilterOnA(filter: Filter): Int => Boolean = filter match { case EqualTo("a", v) => (a: Int) => a == v case LessThan("a", v: Int) => (a: Int) => a < v case LessThanOrEqual("a", v: Int) => (a: Int) => a <= v @@ -57,13 +60,27 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL case In("a", values) => (a: Int) => values.map(_.asInstanceOf[Int]).toSet.contains(a) case IsNull("a") => (a: Int) => false // Int can't be null case IsNotNull("a") => (a: Int) => true - case Not(pred) => (a: Int) => !translateFilter(pred)(a) - case And(left, right) => (a: Int) => translateFilter(left)(a) && translateFilter(right)(a) - case Or(left, right) => (a: Int) => translateFilter(left)(a) || translateFilter(right)(a) + case Not(pred) => (a: Int) => !translateFilterOnA(pred)(a) + case And(left, right) => (a: Int) => + translateFilterOnA(left)(a) && translateFilterOnA(right)(a) + case Or(left, right) => (a: Int) => + translateFilterOnA(left)(a) || translateFilterOnA(right)(a) case _ => (a: Int) => true } - def eval(a: Int) = !filters.map(translateFilter(_)(a)).contains(false) + // Predicate test on string column + def translateFilterOnC(filter: Filter): String => Boolean = filter match { + case StringStartsWith("c", v) => _.startsWith(v) + case StringEndsWith("c", v) => _.endsWith(v) + case StringContains("c", v) => _.contains(v) + case _ => (c: String) => true + } + + def eval(a: Int) = { + val c = (a - 1 + 'a').toChar.toString * 10 + !filters.map(translateFilterOnA(_)(a)).contains(false) && + !filters.map(translateFilterOnC(_)(c)).contains(false) + } sqlContext.sparkContext.parallelize(from to to).filter(eval).map(i => Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) @@ -93,7 +110,7 @@ class FilteredScanSuite extends DataSourceTest { sqlTest( "SELECT * FROM oneToTenFiltered", - (1 to 10).map(i => Row(i, i * 2)).toSeq) + (1 to 10).map(i => Row(i, i * 2, (i - 1 + 'a').toChar.toString * 10)).toSeq) sqlTest( "SELECT a, b FROM oneToTenFiltered", @@ -128,41 +145,53 @@ class FilteredScanSuite extends DataSourceTest { (2 to 10 by 2).map(i => Row(i, i)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a = 1", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a = 1", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IN (1,3,5)", - Seq(1,3,5).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a IN (1,3,5)", + Seq(1,3,5).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE A = 1", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE A = 1", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE b = 2", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE b = 2", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IS NULL", + "SELECT a, b FROM oneToTenFiltered WHERE a IS NULL", Seq.empty[Row]) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IS NOT NULL", + "SELECT a, b FROM oneToTenFiltered WHERE a IS NOT NULL", (1 to 10).map(i => Row(i, i * 2)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a < 5 AND a > 1", + "SELECT a, b FROM oneToTenFiltered WHERE a < 5 AND a > 1", (2 to 4).map(i => Row(i, i * 2)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", - Seq(1, 2, 9, 10).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a < 3 OR a > 8", + Seq(1, 2, 9, 10).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", + "SELECT a, b FROM oneToTenFiltered WHERE NOT (a < 6)", (6 to 10).map(i => Row(i, i * 2)).toSeq) + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like 'c%'", + Seq(Row(3, 3 * 2, "c" * 10))) + + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like 'd%'", + Seq(Row(4, 4 * 2, "d" * 10))) + + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%e%'", + Seq(Row(5, 5 * 2, "e" * 10))) + testPushDown("SELECT * FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT a FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT b FROM oneToTenFiltered WHERE A = 1", 1) From b80a030e90d790e27e89b26f536565c582dbf3d5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 31 Mar 2015 00:25:23 -0700 Subject: [PATCH 585/817] [SPARK-6623][SQL] Alias DataFrame.na.drop and DataFrame.na.fill in Python. To maintain consistency with the Scala API. Author: Reynold Xin Closes #5284 from rxin/df-na-alias and squashes the following commits: 19f46b7 [Reynold Xin] Show DataFrameNaFunctions in docs. 6618118 [Reynold Xin] [SPARK-6623][SQL] Alias DataFrame.na.drop and DataFrame.na.fill in Python. --- python/pyspark/sql/__init__.py | 10 ++++---- python/pyspark/sql/dataframe.py | 41 +++++++++++++++++++++++++++++++-- 2 files changed, 45 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 54a01631d8899..9d39e5d9c2449 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -22,22 +22,24 @@ Main entry point for :class:`DataFrame` and SQL functionality. - L{DataFrame} A distributed collection of data grouped into named columns. - - L{GroupedData} - Aggregation methods, returned by :func:`DataFrame.groupBy`. - L{Column} A column expression in a :class:`DataFrame`. - L{Row} A row of data in a :class:`DataFrame`. - L{HiveContext} Main entry point for accessing data stored in Apache Hive. + - L{GroupedData} + Aggregation methods, returned by :func:`DataFrame.groupBy`. + - L{DataFrameNaFunctions} + Methods for handling missing data (null values). - L{functions} List of built-in functions available for :class:`DataFrame`. """ from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row -from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD +from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD, DataFrameNaFunctions __all__ = [ - 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', + 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', 'DataFrameNaFunctions' ] diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 4f174de811697..15508023326cc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -31,7 +31,7 @@ from pyspark.sql.types import _create_cls, _parse_datatype_json_string -__all__ = ["DataFrame", "GroupedData", "Column", "SchemaRDD"] +__all__ = ["DataFrame", "GroupedData", "Column", "SchemaRDD", "DataFrameNaFunctions"] class DataFrame(object): @@ -86,6 +86,12 @@ def applySchema(it): return self._lazy_rdd + @property + def na(self): + """Returns a :class:`DataFrameNaFunctions` for handling missing values. + """ + return DataFrameNaFunctions(self) + def toJSON(self, use_unicode=False): """Convert a :class:`DataFrame` into a MappedRDD of JSON documents; one document per row. @@ -693,6 +699,8 @@ def subtract(self, other): def dropna(self, how='any', thresh=None, subset=None): """Returns a new :class:`DataFrame` omitting rows with null values. + This is an alias for `na.drop`. + :param how: 'any' or 'all'. If 'any', drop a row if it contains any nulls. If 'all', drop a row only if all its values are null. @@ -704,6 +712,10 @@ def dropna(self, how='any', thresh=None, subset=None): >>> df4.dropna().show() age height name 10 80 Alice + + >>> df4.na.drop().show() + age height name + 10 80 Alice """ if how is not None and how not in ['any', 'all']: raise ValueError("how ('" + how + "') should be 'any' or 'all'") @@ -723,7 +735,7 @@ def dropna(self, how='any', thresh=None, subset=None): return DataFrame(self._jdf.na().drop(thresh, cols), self.sql_ctx) def fillna(self, value, subset=None): - """Replace null values. + """Replace null values, alias for `na.fill`. :param value: int, long, float, string, or dict. Value to replace null values with. @@ -748,6 +760,13 @@ def fillna(self, value, subset=None): 5 null Bob 50 null Tom 50 null unknown + + >>> df4.na.fill({'age': 50, 'name': 'unknown'}).show() + age height name + 10 80 Alice + 5 null Bob + 50 null Tom + 50 null unknown """ if not isinstance(value, (float, int, long, basestring, dict)): raise ValueError("value should be a float, int, long, string, or dict") @@ -1134,6 +1153,24 @@ def __repr__(self): return 'Column<%s>' % self._jc.toString().encode('utf8') +class DataFrameNaFunctions(object): + """Functionality for working with missing data in :class:`DataFrame`. + """ + + def __init__(self, df): + self.df = df + + def drop(self, how='any', thresh=None, subset=None): + return self.df.dropna(how=how, thresh=thresh, subset=subset) + + drop.__doc__ = DataFrame.dropna.__doc__ + + def fill(self, value, subset=None): + return self.df.fillna(value=value, subset=subset) + + fill.__doc__ = DataFrame.fillna.__doc__ + + def _test(): import doctest from pyspark.context import SparkContext From 314afd0e2f08dd8d3333d3143712c2c79fa40d1e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 31 Mar 2015 16:28:40 +0800 Subject: [PATCH 586/817] [SPARK-6618][SQL] HiveMetastoreCatalog.lookupRelation should use fine-grained lock JIRA: https://issues.apache.org/jira/browse/SPARK-6618 Author: Yin Huai Closes #5281 from yhuai/lookupRelationLock and squashes the following commits: 591b4be [Yin Huai] A test? b3a9625 [Yin Huai] Just protect client. --- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++++++++--- .../spark/sql/hive/execution/SQLQuerySuite.scala | 11 +++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 203164ea84292..6a01a23124d95 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -172,12 +172,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with def lookupRelation( tableIdentifier: Seq[String], - alias: Option[String]): LogicalPlan = synchronized { + alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - val table = try client.getTable(databaseName, tblName) catch { + val table = try { + synchronized { + client.getTable(databaseName, tblName) + } + } catch { case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => throw new NoSuchTableException } @@ -199,7 +203,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } else { val partitions: Seq[Partition] = if (table.isPartitioned) { - HiveShim.getAllPartitionsOf(client, table).toSeq + synchronized { + HiveShim.getAllPartitionsOf(client, table).toSeq + } } else { Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 1187228f4c3db..2f50a33448462 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -433,4 +433,15 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") setConf("spark.sql.hive.convertCTAS", originalConf) } + + test("sanity test for SPARK-6618") { + (1 to 100).par.map { i => + val tableName = s"SPARK_6618_table_$i" + sql(s"CREATE TABLE $tableName (col1 string)") + catalog.lookupRelation(Seq(tableName)) + table(tableName) + tables() + sql(s"DROP TABLE $tableName") + } + } } From a05835b89fe2086e460f0b80f7c22e284c0c32d0 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 31 Mar 2015 17:05:23 +0800 Subject: [PATCH 587/817] [SPARK-6542][SQL] add CreateStruct Similar to `CreateArray`, we can add `CreateStruct` to create nested columns. marmbrus Author: Xiangrui Meng Closes #5195 from mengxr/SPARK-6542 and squashes the following commits: 3795c57 [Xiangrui Meng] update error message ae7ac3e [Xiangrui Meng] move unit test to a separate suite 85dd559 [Xiangrui Meng] use NamedExpr c78e31a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-6542 85f3106 [Xiangrui Meng] add CreateStruct --- .../sql/catalyst/analysis/Analyzer.scala | 6 ++ .../catalyst/expressions/complexTypes.scala | 29 ++++++++- .../ExpressionEvaluationSuite.scala | 61 ++++++++++++------- 3 files changed, 73 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ba1ac141b9fab..dc14f49e6ee99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -212,6 +212,12 @@ class Analyzer(catalog: Catalog, case o => o :: Nil } Alias(c.copy(children = expandedArgs), name)() :: Nil + case Alias(c @ CreateStruct(args), name) if containsStar(args) => + val expandedArgs = args.flatMap { + case s: Star => s.expand(child.output, resolver) + case o => o :: Nil + } + Alias(c.copy(children = expandedArgs), name)() :: Nil case o => o :: Nil }, child) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 3fd78db297462..3b2b9211268a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -120,7 +120,7 @@ case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, co case class CreateArray(children: Seq[Expression]) extends Expression { override type EvaluatedType = Any - override def foldable: Boolean = !children.exists(!_.foldable) + override def foldable: Boolean = children.forall(_.foldable) lazy val childTypes = children.map(_.dataType).distinct @@ -142,3 +142,30 @@ case class CreateArray(children: Seq[Expression]) extends Expression { override def toString: String = s"Array(${children.mkString(",")})" } + +/** + * Returns a Row containing the evaluation of all children expressions. + * TODO: [[CreateStruct]] does not support codegen. + */ +case class CreateStruct(children: Seq[NamedExpression]) extends Expression { + override type EvaluatedType = Row + + override def foldable: Boolean = children.forall(_.foldable) + + override lazy val resolved: Boolean = childrenResolved + + override lazy val dataType: StructType = { + assert(resolved, + s"CreateStruct contains unresolvable children: ${children.filterNot(_.resolved)}.") + val fields = children.map { child => + StructField(child.name, child.dataType, child.nullable, child.metadata) + } + StructType(fields) + } + + override def nullable: Boolean = false + + override def eval(input: Row): EvaluatedType = { + Row(children.map(_.eval(input)): _*) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index dcfd8b28cb02a..1183a0d899dda 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -30,7 +30,34 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField import org.apache.spark.sql.types._ -class ExpressionEvaluationSuite extends FunSuite { +class ExpressionEvaluationBaseSuite extends FunSuite { + + def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { + expression.eval(inputRow) + } + + def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + def checkDoubleEvaluation( + expression: Expression, + expected: Spread[Double], + inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + actual.asInstanceOf[Double] shouldBe expected + } +} + +class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("literals") { checkEvaluation(Literal(1), 1) @@ -134,27 +161,6 @@ class ExpressionEvaluationSuite extends FunSuite { } } - def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.eval(inputRow) - } - - def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - if(actual != expected) { - val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - - def checkDoubleEvaluation(expression: Expression, expected: Spread[Double], inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - actual.asInstanceOf[Double] shouldBe expected - } - test("IN") { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) @@ -1081,3 +1087,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(~c1, -2, row) } } + +// TODO: Make the tests work with codegen. +class ExpressionEvaluationWithoutCodeGenSuite extends ExpressionEvaluationBaseSuite { + + test("CreateStruct") { + val row = Row(1, 2, 3) + val c1 = 'a.int.at(0).as("a") + val c3 = 'c.int.at(2).as("c") + checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row) + } +} From d01a6d8c33fc5c8325b0cc4b51395dba5eb3462c Mon Sep 17 00:00:00 2001 From: leahmcguire Date: Tue, 31 Mar 2015 11:16:55 -0700 Subject: [PATCH 588/817] [SPARK-4894][mllib] Added Bernoulli option to NaiveBayes model in mllib Added optional model type parameter for NaiveBayes training. Can be either Multinomial or Bernoulli. When Bernoulli is given the Bernoulli smoothing is used for fitting and for prediction as per: http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html. Default for model is original Multinomial fit and predict. Added additional testing for Bernoulli and Multinomial models. Author: leahmcguire Author: Joseph K. Bradley Author: Leah McGuire Closes #4087 from leahmcguire/master and squashes the following commits: f3c8994 [leahmcguire] changed checks on model type to requires acb69af [leahmcguire] removed enum type and replaces all modelType parameters with strings 2224b15 [Leah McGuire] Merge pull request #2 from jkbradley/leahmcguire-master 9ad89ca [Joseph K. Bradley] removed old code 6a8f383 [Joseph K. Bradley] Added new model save/load format 2.0 for NaiveBayesModel after modelType parameter was added. Updated tests. Also updated ModelType enum-like type. 852a727 [leahmcguire] merged with upstream master a22d670 [leahmcguire] changed NaiveBayesModel modelType parameter back to NaiveBayes.ModelType, made NaiveBayes.ModelType serializable, fixed getter method in NavieBayes 18f3219 [leahmcguire] removed private from naive bayes constructor for lambda only bea62af [leahmcguire] put back in constructor for NaiveBayes 01baad7 [leahmcguire] made fixes from code review fb0a5c7 [leahmcguire] removed typo e2d925e [leahmcguire] fixed nonserializable error that was causing naivebayes test failures 2d0c1ba [leahmcguire] fixed typo in NaiveBayes c298e78 [leahmcguire] fixed scala style errors b85b0c9 [leahmcguire] Merge remote-tracking branch 'upstream/master' 900b586 [leahmcguire] fixed model call so that uses type argument ea09b28 [leahmcguire] Merge remote-tracking branch 'upstream/master' e016569 [leahmcguire] updated test suite with model type fix 85f298f [leahmcguire] Merge remote-tracking branch 'upstream/master' dc65374 [leahmcguire] integrated model type fix 7622b0c [leahmcguire] added comments and fixed style as per rb b93aaf6 [Leah McGuire] Merge pull request #1 from jkbradley/nb-model-type 3730572 [Joseph K. Bradley] modified NB model type to be more Java-friendly b61b5e2 [leahmcguire] added back compatable constructor to NaiveBayesModel to fix MIMA test failure 5a4a534 [leahmcguire] fixed scala style error in NaiveBayes 3891bf2 [leahmcguire] synced with apache spark and resolved merge conflict d9477ed [leahmcguire] removed old inaccurate comment from test suite for mllib naive bayes 76e5b0f [leahmcguire] removed unnecessary sort from test 0313c0c [leahmcguire] fixed style error in NaiveBayes.scala 4a3676d [leahmcguire] Updated changes re-comments. Got rid of verbose populateMatrix method. Public api now has string instead of enumeration. Docs are updated." ce73c63 [leahmcguire] added Bernoulli option to niave bayes model in mllib, added optional model type parameter for training. When Bernoulli is given the Bernoulli smoothing is used for fitting and for prediction http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html --- docs/mllib-naive-bayes.md | 17 +- .../mllib/classification/NaiveBayes.scala | 225 ++++++++++++++---- .../classification/JavaNaiveBayesSuite.java | 23 +- .../classification/NaiveBayesSuite.scala | 148 +++++++++--- 4 files changed, 322 insertions(+), 91 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index a83472f5be52e..9780ea52c4994 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -13,12 +13,15 @@ compute the conditional probability distribution of label given an observation and use it for prediction. MLlib supports [multinomial naive -Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), -which is typically used for [document -classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +and [Bernoulli naive Bayes] (http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). +These models are typically used for [document classification] +(http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each -feature represents a term whose value is the frequency of the term. -Feature values must be nonnegative to represent term frequencies. +feature represents a term whose value is the frequency of the term (in multinomial naive Bayes) or +a zero or one indicating whether the term was found in the document (in Bernoulli naive Bayes). +Feature values must be nonnegative. The model type is selected with an optional parameter +"Multinomial" or "Bernoulli" with "Multinomial" as the default. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of @@ -32,7 +35,7 @@ sparsity. Since the training data is only used once, it is not necessary to cach [NaiveBayes](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements multinomial naive Bayes. It takes an RDD of [LabeledPoint](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional -smoothing parameter `lambda` as input, and output a +smoothing parameter `lambda` as input, an optional model type parameter (default is Multinomial), and outputs a [NaiveBayesModel](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which can be used for evaluation and prediction. @@ -51,7 +54,7 @@ val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) val training = splits(0) val test = splits(1) -val model = NaiveBayes.train(training, lambda = 1.0) +val model = NaiveBayes.train(training, lambda = 1.0, model = "Multinomial") val predictionAndLabel = test.map(p => (model.predict(p.features), p.label)) val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() 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 d60e82c410979..c9b3ff0172e2e 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 @@ -21,9 +21,12 @@ import java.lang.{Iterable => JIterable} import scala.collection.JavaConverters._ -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum, Axis} +import breeze.numerics.{exp => brzExp, log => brzLog} + import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.json4s.{DefaultFormats, JValue} import org.apache.spark.{Logging, SparkContext, SparkException} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector} @@ -32,6 +35,7 @@ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} + /** * Model for Naive Bayes Classifiers. * @@ -39,11 +43,17 @@ import org.apache.spark.sql.{DataFrame, SQLContext} * @param pi log of class priors, whose dimension is C, number of labels * @param theta log of class conditional probabilities, whose dimension is C-by-D, * where D is number of features + * @param modelType The type of NB model to fit can be "Multinomial" or "Bernoulli" */ class NaiveBayesModel private[mllib] ( val labels: Array[Double], val pi: Array[Double], - val theta: Array[Array[Double]]) extends ClassificationModel with Serializable with Saveable { + val theta: Array[Array[Double]], + val modelType: String) + extends ClassificationModel with Serializable with Saveable { + + private[mllib] def this(labels: Array[Double], pi: Array[Double], theta: Array[Array[Double]]) = + this(labels, pi, theta, "Multinomial") /** A Java-friendly constructor that takes three Iterable parameters. */ private[mllib] def this( @@ -53,19 +63,19 @@ class NaiveBayesModel private[mllib] ( this(labels.asScala.toArray, pi.asScala.toArray, theta.asScala.toArray.map(_.asScala.toArray)) private val brzPi = new BDV[Double](pi) - private val brzTheta = new BDM[Double](theta.length, theta(0).length) - - { - // Need to put an extra pair of braces to prevent Scala treating `i` as a member. - var i = 0 - while (i < theta.length) { - var j = 0 - while (j < theta(i).length) { - brzTheta(i, j) = theta(i)(j) - j += 1 - } - i += 1 - } + private val brzTheta = new BDM(theta(0).length, theta.length, theta.flatten).t + + // Bernoulli scoring requires log(condprob) if 1, log(1-condprob) if 0. + // This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra + // application of this condition (in predict function). + private val (brzNegTheta, brzNegThetaSum) = modelType match { + case "Multinomial" => (None, None) + case "Bernoulli" => + val negTheta = brzLog((brzExp(brzTheta.copy) :*= (-1.0)) :+= 1.0) // log(1.0 - exp(x)) + (Option(negTheta), Option(brzSum(negTheta, Axis._1))) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayesModel was created with an unknown ModelType: $modelType") } override def predict(testData: RDD[Vector]): RDD[Double] = { @@ -77,22 +87,78 @@ class NaiveBayesModel private[mllib] ( } override def predict(testData: Vector): Double = { - labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) + modelType match { + case "Multinomial" => + labels (brzArgmax (brzPi + brzTheta * testData.toBreeze) ) + case "Bernoulli" => + labels (brzArgmax (brzPi + + (brzTheta - brzNegTheta.get) * testData.toBreeze + brzNegThetaSum.get)) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayesModel was created with an unknown ModelType: $modelType") + } } override def save(sc: SparkContext, path: String): Unit = { - val data = NaiveBayesModel.SaveLoadV1_0.Data(labels, pi, theta) - NaiveBayesModel.SaveLoadV1_0.save(sc, path, data) + val data = NaiveBayesModel.SaveLoadV2_0.Data(labels, pi, theta, modelType) + NaiveBayesModel.SaveLoadV2_0.save(sc, path, data) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = "2.0" } object NaiveBayesModel extends Loader[NaiveBayesModel] { import org.apache.spark.mllib.util.Loader._ - private object SaveLoadV1_0 { + private[mllib] object SaveLoadV2_0 { + + def thisFormatVersion: String = "2.0" + + /** Hard-code class name string in case it changes in the future */ + def thisClassName: String = "org.apache.spark.mllib.classification.NaiveBayesModel" + + /** Model data for model import/export */ + case class Data( + labels: Array[Double], + pi: Array[Double], + theta: Array[Array[Double]], + modelType: String) + + def save(sc: SparkContext, path: String, data: Data): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("numFeatures" -> data.theta(0).length) ~ ("numClasses" -> data.pi.length))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) + + // Create Parquet data. + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() + dataRDD.saveAsParquetFile(dataPath(path)) + } + + def load(sc: SparkContext, path: String): NaiveBayesModel = { + val sqlContext = new SQLContext(sc) + // Load Parquet data. + val dataRDD = sqlContext.parquetFile(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) + assert(dataArray.size == 1, s"Unable to load NaiveBayesModel data from: ${dataPath(path)}") + val data = dataArray(0) + val labels = data.getAs[Seq[Double]](0).toArray + val pi = data.getAs[Seq[Double]](1).toArray + val theta = data.getAs[Seq[Seq[Double]]](2).map(_.toArray).toArray + val modelType = data.getString(3) + new NaiveBayesModel(labels, pi, theta, modelType) + } + + } + + private[mllib] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" @@ -100,7 +166,10 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { def thisClassName: String = "org.apache.spark.mllib.classification.NaiveBayesModel" /** Model data for model import/export */ - case class Data(labels: Array[Double], pi: Array[Double], theta: Array[Array[Double]]) + case class Data( + labels: Array[Double], + pi: Array[Double], + theta: Array[Array[Double]]) def save(sc: SparkContext, path: String, data: Data): Unit = { val sqlContext = new SQLContext(sc) @@ -136,26 +205,32 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { override def load(sc: SparkContext, path: String): NaiveBayesModel = { val (loadedClassName, version, metadata) = loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName - (loadedClassName, version) match { + val classNameV2_0 = SaveLoadV2_0.thisClassName + val (model, numFeatures, numClasses) = (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) val model = SaveLoadV1_0.load(sc, path) - assert(model.pi.size == numClasses, - s"NaiveBayesModel.load expected $numClasses classes," + - s" but class priors vector pi had ${model.pi.size} elements") - assert(model.theta.size == numClasses, - s"NaiveBayesModel.load expected $numClasses classes," + - s" but class conditionals array theta had ${model.theta.size} elements") - assert(model.theta.forall(_.size == numFeatures), - s"NaiveBayesModel.load expected $numFeatures features," + - s" but class conditionals array theta had elements of size:" + - s" ${model.theta.map(_.size).mkString(",")}") - model + (model, numFeatures, numClasses) + case (className, "2.0") if className == classNameV2_0 => + val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) + val model = SaveLoadV2_0.load(sc, path) + (model, numFeatures, numClasses) case _ => throw new Exception( s"NaiveBayesModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $version). Supported:\n" + s" ($classNameV1_0, 1.0)") } + assert(model.pi.size == numClasses, + s"NaiveBayesModel.load expected $numClasses classes," + + s" but class priors vector pi had ${model.pi.size} elements") + assert(model.theta.size == numClasses, + s"NaiveBayesModel.load expected $numClasses classes," + + s" but class conditionals array theta had ${model.theta.size} elements") + assert(model.theta.forall(_.size == numFeatures), + s"NaiveBayesModel.load expected $numFeatures features," + + s" but class conditionals array theta had elements of size:" + + s" ${model.theta.map(_.size).mkString(",")}") + model } } @@ -167,9 +242,14 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. */ -class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { - def this() = this(1.0) +class NaiveBayes private ( + private var lambda: Double, + private var modelType: String) extends Serializable with Logging { + + def this(lambda: Double) = this(lambda, "Multinomial") + + def this() = this(1.0, "Multinomial") /** Set the smoothing parameter. Default: 1.0. */ def setLambda(lambda: Double): NaiveBayes = { @@ -177,9 +257,24 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with this } - /** Get the smoothing parameter. Default: 1.0. */ + /** Get the smoothing parameter. */ def getLambda: Double = lambda + /** + * Set the model type using a string (case-sensitive). + * Supported options: "Multinomial" and "Bernoulli". + * (default: Multinomial) + */ + def setModelType(modelType:String): NaiveBayes = { + require(NaiveBayes.supportedModelTypes.contains(modelType), + s"NaiveBayes was created with an unknown ModelType: $modelType") + this.modelType = modelType + this + } + + /** Get the model type. */ + def getModelType: String = this.modelType + /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * @@ -213,21 +308,30 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => (c1._1 + c2._1, c1._2 += c2._2) ).collect() + val numLabels = aggregated.length var numDocuments = 0L aggregated.foreach { case (_, (n, _)) => numDocuments += n } val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } + val labels = new Array[Double](numLabels) val pi = new Array[Double](numLabels) val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) + val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 aggregated.foreach { case (label, (n, sumTermFreqs)) => labels(i) = label - val thetaLogDenom = math.log(brzSum(sumTermFreqs) + numFeatures * lambda) pi(i) = math.log(n + lambda) - piLogDenom + val thetaLogDenom = modelType match { + case "Multinomial" => math.log(brzSum(sumTermFreqs) + numFeatures * lambda) + case "Bernoulli" => math.log(n + 2.0 * lambda) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayes was created with an unknown ModelType: $modelType") + } var j = 0 while (j < numFeatures) { theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom @@ -236,7 +340,7 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with i += 1 } - new NaiveBayesModel(labels, pi, theta) + new NaiveBayesModel(labels, pi, theta, modelType) } } @@ -244,13 +348,16 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with * Top-level methods for calling naive Bayes. */ object NaiveBayes { + + /* Set of modelTypes that NaiveBayes supports */ + private[mllib] val supportedModelTypes = Set("Multinomial", "Bernoulli") + /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of - * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for - * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all + * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it + * can be used for document classification. * * This version of the method uses a default smoothing parameter of 1.0. * @@ -264,16 +371,40 @@ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of - * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for - * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all + * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it + * can be used for document classification. * * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency * vector or a count vector. * @param lambda The smoothing parameter */ def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { - new NaiveBayes(lambda).run(input) + new NaiveBayes(lambda, "Multinomial").run(input) + } + + /** + * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. + * + * The model type can be set to either Multinomial NB ([[http://tinyurl.com/lsdw6p]]) + * or Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The Multinomial NB can handle + * discrete count data and can be called by setting the model type to "multinomial". + * For example, it can be used with word counts or TF_IDF vectors of documents. + * The Bernoulli model fits presence or absence (0-1) counts. By making every vector a + * 0-1 vector and setting the model type to "bernoulli", the fits and predicts as + * Bernoulli NB. + * + * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency + * vector or a count vector. + * @param lambda The smoothing parameter + * + * @param modelType The type of NB model to fit from the enumeration NaiveBayesModels, can be + * multinomial or bernoulli + */ + def train(input: RDD[LabeledPoint], lambda: Double, modelType: String): NaiveBayesModel = { + require(supportedModelTypes.contains(modelType), + s"NaiveBayes was created with an unknown ModelType: $modelType") + new NaiveBayes(lambda, modelType).run(input) } + } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 1c90522a0714a..71fb7f13c39c2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -17,20 +17,22 @@ package org.apache.spark.mllib.classification; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; public class JavaNaiveBayesSuite implements Serializable { private transient JavaSparkContext sc; @@ -102,4 +104,11 @@ public Vector call(LabeledPoint v) throws Exception { // Should be able to get the first prediction. predictions.first(); } + + @Test + public void testModelTypeSetters() { + NaiveBayes nb = new NaiveBayes() + .setModelType("Bernoulli") + .setModelType("Multinomial"); + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 5a27c7d2309c5..f9fe3e006ccb8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.mllib.classification import scala.util.Random +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum, Axis} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} + import org.scalatest.FunSuite import org.apache.spark.SparkException @@ -41,37 +44,48 @@ object NaiveBayesSuite { // Generate input of the form Y = (theta * x).argmax() def generateNaiveBayesInput( - pi: Array[Double], // 1XC - theta: Array[Array[Double]], // CXD - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int, + modelType: String = "Multinomial", + sample: Int = 10): Seq[LabeledPoint] = { val D = theta(0).length val rnd = new Random(seed) - val _pi = pi.map(math.pow(math.E, _)) val _theta = theta.map(row => row.map(math.pow(math.E, _))) for (i <- 0 until nPoints) yield { val y = calcLabel(rnd.nextDouble(), _pi) - val xi = Array.tabulate[Double](D) { j => - if (rnd.nextDouble() < _theta(y)(j)) 1 else 0 + val xi = modelType match { + case "Bernoulli" => Array.tabulate[Double] (D) { j => + if (rnd.nextDouble () < _theta(y)(j) ) 1 else 0 + } + case "Multinomial" => + val mult = BrzMultinomial(BDV(_theta(y))) + val emptyMap = (0 until D).map(x => (x, 0.0)).toMap + val counts = emptyMap ++ mult.sample(sample).groupBy(x => x).map { + case (index, reps) => (index, reps.size.toDouble) + } + counts.toArray.sortBy(_._1).map(_._2) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayesSuite found unknown ModelType: $modelType") } LabeledPoint(y, Vectors.dense(xi)) } } - private val smallPi = Array(0.5, 0.3, 0.2).map(math.log) + /** Bernoulli NaiveBayes with binary labels, 3 features */ + private val binaryBernoulliModel = new NaiveBayesModel(labels = Array(0.0, 1.0), + pi = Array(0.2, 0.8), theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4)), + "Bernoulli") - private val smallTheta = Array( - Array(0.91, 0.03, 0.03, 0.03), // label 0 - Array(0.03, 0.91, 0.03, 0.03), // label 1 - Array(0.03, 0.03, 0.91, 0.03) // label 2 - ).map(_.map(math.log)) - - /** Binary labels, 3 features */ - private val binaryModel = new NaiveBayesModel(labels = Array(0.0, 1.0), pi = Array(0.2, 0.8), - theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4))) + /** Multinomial NaiveBayes with binary labels, 3 features */ + private val binaryMultinomialModel = new NaiveBayesModel(labels = Array(0.0, 1.0), + pi = Array(0.2, 0.8), theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4)), + "Multinomial") } class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { @@ -85,6 +99,24 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(numOfPredictions < input.length / 5) } + def validateModelFit( + piData: Array[Double], + thetaData: Array[Array[Double]], + model: NaiveBayesModel) = { + def closeFit(d1: Double, d2: Double, precision: Double): Boolean = { + (d1 - d2).abs <= precision + } + val modelIndex = (0 until piData.length).zip(model.labels.map(_.toInt)) + for (i <- modelIndex) { + assert(closeFit(math.exp(piData(i._2)), math.exp(model.pi(i._1)), 0.05)) + } + for (i <- modelIndex) { + for (j <- 0 until thetaData(i._2).length) { + assert(closeFit(math.exp(thetaData(i._2)(j)), math.exp(model.theta(i._1)(j)), 0.05)) + } + } + } + test("get, set params") { val nb = new NaiveBayes() nb.setLambda(2.0) @@ -93,19 +125,53 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(nb.getLambda === 3.0) } - test("Naive Bayes") { - val nPoints = 10000 + test("Naive Bayes Multinomial") { + val nPoints = 1000 + val pi = Array(0.5, 0.1, 0.4).map(math.log) + val theta = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0 + Array(0.10, 0.70, 0.10, 0.10), // label 1 + Array(0.10, 0.10, 0.70, 0.10) // label 2 + ).map(_.map(math.log)) + + val testData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 42, "Multinomial") + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val model = NaiveBayes.train(testRDD, 1.0, "Multinomial") + validateModelFit(pi, theta, model) + + val validationData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 17, "Multinomial") + val validationRDD = sc.parallelize(validationData, 2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) - val pi = NaiveBayesSuite.smallPi - val theta = NaiveBayesSuite.smallTheta + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } - val testData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 42) + test("Naive Bayes Bernoulli") { + val nPoints = 10000 + val pi = Array(0.5, 0.3, 0.2).map(math.log) + val theta = Array( + Array(0.50, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.40), // label 0 + Array(0.02, 0.70, 0.10, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02), // label 1 + Array(0.02, 0.02, 0.60, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.30) // label 2 + ).map(_.map(math.log)) + + val testData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 45, "Bernoulli") val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val model = NaiveBayes.train(testRDD) + val model = NaiveBayes.train(testRDD, 1.0, "Bernoulli") + validateModelFit(pi, theta, model) - val validationData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 17) + val validationData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 20, "Bernoulli") val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. @@ -142,19 +208,41 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { } } - test("model save/load") { - val model = NaiveBayesSuite.binaryModel + test("model save/load: 2.0 to 2.0") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + Seq(NaiveBayesSuite.binaryBernoulliModel, NaiveBayesSuite.binaryMultinomialModel).map { + model => + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = NaiveBayesModel.load(sc, path) + assert(model.labels === sameModel.labels) + assert(model.pi === sameModel.pi) + assert(model.theta === sameModel.theta) + assert(model.modelType === sameModel.modelType) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } + + test("model save/load: 1.0 to 2.0") { + val model = NaiveBayesSuite.binaryMultinomialModel val tempDir = Utils.createTempDir() val path = tempDir.toURI.toString - // Save model, load it back, and compare. + // Save model as version 1.0, load it back, and compare. try { - model.save(sc, path) + val data = NaiveBayesModel.SaveLoadV1_0.Data(model.labels, model.pi, model.theta) + NaiveBayesModel.SaveLoadV1_0.save(sc, path, data) val sameModel = NaiveBayesModel.load(sc, path) assert(model.labels === sameModel.labels) assert(model.pi === sameModel.pi) assert(model.theta === sameModel.theta) + assert(model.modelType === "Multinomial") } finally { Utils.deleteRecursively(tempDir) } @@ -172,8 +260,8 @@ class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) } } - // If we serialize data directly in the task closure, the size of the serialized task would be - // greater than 1MB and hence Spark would throw an error. + // If we serialize data directly in the task closure, the size of the serialized task + // would be greater than 1MB and hence Spark would throw an error. val model = NaiveBayes.train(examples) val predictions = model.predict(examples.map(_.features)) } From a7992ffaf1e8adc9d2c225a986fa3162e8e130eb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 31 Mar 2015 11:18:25 -0700 Subject: [PATCH 589/817] [SPARK-6555] [SQL] Overrides equals() and hashCode() for MetastoreRelation Also removes temporary workarounds made in #5183 and #5251. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5289) Author: Cheng Lian Closes #5289 from liancheng/spark-6555 and squashes the following commits: d0095ac [Cheng Lian] Removes unused imports cfafeeb [Cheng Lian] Removes outdated comment 75a2746 [Cheng Lian] Overrides equals() and hashCode() for MetastoreRelation --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 42 +++++++++++-------- .../sql/hive/execution/HivePlanTest.scala | 6 ++- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 6a01a23124d95..f20f0ad99f865 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} +import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.metastore.{TableType, Warehouse} @@ -465,7 +466,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation -> relation.output, parquetRelation, attributedRewrites) + (relation, parquetRelation, attributedRewrites) // Write path case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) @@ -476,7 +477,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation -> relation.output, parquetRelation, attributedRewrites) + (relation, parquetRelation, attributedRewrites) // Read path case p @ PhysicalOperation(_, _, relation: MetastoreRelation) @@ -485,33 +486,28 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) - (relation -> relation.output, parquetRelation, attributedRewrites) + (relation, parquetRelation, attributedRewrites) } - // Quick fix for SPARK-6450: Notice that we're using both the MetastoreRelation instances and - // their output attributes as the key of the map. This is because MetastoreRelation.equals - // doesn't take output attributes into account, thus multiple MetastoreRelation instances - // pointing to the same table get collapsed into a single entry in the map. A proper fix for - // this should be overriding equals & hashCode in MetastoreRelation. val relationMap = toBeReplaced.map(r => (r._1, r._2)).toMap val attributedRewrites = AttributeMap(toBeReplaced.map(_._3).fold(Nil)(_ ++: _)) // Replaces all `MetastoreRelation`s with corresponding `ParquetRelation2`s, and fixes // attribute IDs referenced in other nodes. plan.transformUp { - case r: MetastoreRelation if relationMap.contains(r -> r.output) => - val parquetRelation = relationMap(r -> r.output) + case r: MetastoreRelation if relationMap.contains(r) => + val parquetRelation = relationMap(r) val alias = r.alias.getOrElse(r.tableName) Subquery(alias, parquetRelation) case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r -> r.output) => - val parquetRelation = relationMap(r -> r.output) + if relationMap.contains(r) => + val parquetRelation = relationMap(r) InsertIntoTable(parquetRelation, partition, child, overwrite) case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r -> r.output) => - val parquetRelation = relationMap(r -> r.output) + if relationMap.contains(r) => + val parquetRelation = relationMap(r) InsertIntoTable(parquetRelation, partition, child, overwrite) case other => other.transformExpressions { @@ -707,6 +703,19 @@ private[hive] case class MetastoreRelation self: Product => + override def equals(other: scala.Any): Boolean = other match { + case relation: MetastoreRelation => + databaseName == relation.databaseName && + tableName == relation.tableName && + alias == relation.alias && + output == relation.output + case _ => false + } + + override def hashCode(): Int = { + Objects.hashCode(databaseName, tableName, alias, output) + } + // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. // Right now, using org.apache.hadoop.hive.ql.metadata.Table and @@ -786,10 +795,7 @@ private[hive] case class MetastoreRelation val columnOrdinals = AttributeMap(attributes.zipWithIndex) override def newInstance() = { - val newCopy = MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) - // The project here is an ugly hack to work around the fact that MetastoreRelation's - // equals method is broken. Please remove this when SPARK-6555 is fixed. - Project(newCopy.output, newCopy) + MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index c939e6e99d28a..bdb53ddf59c19 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -22,10 +22,12 @@ import org.apache.spark.sql.hive.test.TestHive class HivePlanTest extends QueryTest { import TestHive._ + import TestHive.implicits._ test("udf constant folding") { - val optimized = sql("SELECT cos(null) FROM src").queryExecution.optimizedPlan - val correctAnswer = sql("SELECT cast(null as double) FROM src").queryExecution.optimizedPlan + Seq.empty[Tuple1[Int]].toDF("a").registerTempTable("t") + val optimized = sql("SELECT cos(null) FROM t").queryExecution.optimizedPlan + val correctAnswer = sql("SELECT cast(null as double) FROM t").queryExecution.optimizedPlan comparePlans(optimized, correctAnswer) } From 81020144708773ba3af4932288ffa09ef901269e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 31 Mar 2015 11:21:15 -0700 Subject: [PATCH 590/817] [SPARK-6575] [SQL] Adds configuration to disable schema merging while converting metastore Parquet tables Consider a metastore Parquet table that 1. doesn't have schema evolution issue 2. has lots of data files and/or partitions In this case, driver schema merging can be both slow and unnecessary. Would be good to have a configuration to let the use disable schema merging when converting such a metastore Parquet table. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5231) Author: Cheng Lian Closes #5231 from liancheng/spark-6575 and squashes the following commits: cd96159 [Cheng Lian] Adds configuration to disable schema merging while converting metastore Parquet tables --- .../org/apache/spark/sql/hive/HiveContext.scala | 9 +++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 16 ++++++---------- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index c06c2e396bbc1..6bb1c47dba920 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -57,6 +57,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertMetastoreParquet: Boolean = getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true" + /** + * When true, also tries to merge possibly different but compatible Parquet schemas in different + * Parquet data files. + * + * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. + */ + protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean = + getConf("spark.sql.hive.convertMetastoreParquet.mergeSchema", "false") == "true" + /** * When true, a table created by a Hive CTAS statement (no USING clause) will be * converted to a data source table, using the data source set by spark.sql.sources.default. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f20f0ad99f865..2b5d031741a63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -218,6 +218,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with private def convertToParquetRelation(metastoreRelation: MetastoreRelation): LogicalRelation = { val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) + val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging + val parquetOptions = Map( + ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json, + ParquetRelation2.MERGE_SCHEMA -> mergeSchema.toString) // NOTE: Instead of passing Metastore schema directly to `ParquetRelation2`, we have to // serialize the Metastore schema to JSON and pass it as a data source option because of the @@ -234,18 +238,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } val partitionSpec = PartitionSpec(partitionSchema, partitions) val paths = partitions.map(_.path) - LogicalRelation( - ParquetRelation2( - paths, - Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json), - None, - Some(partitionSpec))(hive)) + LogicalRelation(ParquetRelation2(paths, parquetOptions, None, Some(partitionSpec))(hive)) } else { val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) - LogicalRelation( - ParquetRelation2( - paths, - Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json))(hive)) + LogicalRelation(ParquetRelation2(paths, parquetOptions)(hive)) } } From cd48ca50129e8952f487051796244e7569275416 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Mar 2015 11:23:18 -0700 Subject: [PATCH 591/817] [SPARK-6145][SQL] fix ORDER BY on nested fields This PR is based on work by cloud-fan in #4904, but with two differences: - We isolate the logic for Sort's special handling into `ResolveSortReferences` - We avoid creating UnresolvedGetField expressions during resolution. Instead we either resolve GetField or we return None. This avoids us going down the wrong path early on. Author: Michael Armbrust Closes #5189 from marmbrus/nestedOrderBy and squashes the following commits: b8cae45 [Michael Armbrust] fix another test 0f36a11 [Michael Armbrust] WIP 91820cd [Michael Armbrust] Fix bug. --- .../sql/catalyst/analysis/Analyzer.scala | 76 ++++++++++++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 12 ++- .../catalyst/expressions/AttributeSet.scala | 2 +- .../catalyst/plans/logical/LogicalPlan.scala | 76 +++++++++++++++---- .../sql/catalyst/analysis/AnalysisSuite.scala | 39 +++++++++- .../org/apache/spark/sql/SQLContext.scala | 14 ++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 19 +++-- .../spark/sql/sources/DataSourceTest.scala | 4 + 8 files changed, 185 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index dc14f49e6ee99..c578d084a45b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -37,11 +37,12 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and * a [[FunctionRegistry]]. */ -class Analyzer(catalog: Catalog, - registry: FunctionRegistry, - caseSensitive: Boolean, - maxIterations: Int = 100) - extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { +class Analyzer( + catalog: Catalog, + registry: FunctionRegistry, + caseSensitive: Boolean, + maxIterations: Int = 100) + extends RuleExecutor[LogicalPlan] with HiveTypeCoercion with CheckAnalysis { val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution @@ -354,19 +355,16 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case s @ Sort(ordering, global, p @ Project(projectList, child)) if !s.resolved && p.resolved => - val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) - val resolved = unresolved.flatMap(child.resolve(_, resolver)) - val requiredAttributes = - AttributeSet(resolved.flatMap(_.collect { case a: Attribute => a })) + val (resolvedOrdering, missing) = resolveAndFindMissing(ordering, p, child) - val missingInProject = requiredAttributes -- p.output - if (missingInProject.nonEmpty) { + // If this rule was not a no-op, return the transformed plan, otherwise return the original. + if (missing.nonEmpty) { // Add missing attributes and then project them away after the sort. - Project(projectList.map(_.toAttribute), - Sort(ordering, global, - Project(projectList ++ missingInProject, child))) + Project(p.output, + Sort(resolvedOrdering, global, + Project(projectList ++ missing, child))) } else { - logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") + logDebug(s"Failed to find $missing in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } case s @ Sort(ordering, global, a @ Aggregate(grouping, aggs, child)) @@ -378,18 +376,54 @@ class Analyzer(catalog: Catalog, grouping.collect { case ne: NamedExpression => ne.toAttribute } ) - logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve(_, resolver)) - val missingInAggs = resolved.filterNot(a.outputSet.contains) - logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") - if (missingInAggs.nonEmpty) { + val (resolvedOrdering, missing) = resolveAndFindMissing(ordering, a, groupingRelation) + + if (missing.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. Project(a.output, - Sort(ordering, global, Aggregate(grouping, aggs ++ missingInAggs, child))) + Sort(resolvedOrdering, global, + Aggregate(grouping, aggs ++ missing, child))) } else { s // Nothing we can do here. Return original plan. } } + + /** + * Given a child and a grandchild that are present beneath a sort operator, returns + * a resolved sort ordering and a list of attributes that are missing from the child + * but are present in the grandchild. + */ + def resolveAndFindMissing( + ordering: Seq[SortOrder], + child: LogicalPlan, + grandchild: LogicalPlan): (Seq[SortOrder], Seq[Attribute]) = { + // Find any attributes that remain unresolved in the sort. + val unresolved: Seq[String] = + ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + + // Create a map from name, to resolved attributes, when the desired name can be found + // prior to the projection. + val resolved: Map[String, NamedExpression] = + unresolved.flatMap(u => grandchild.resolve(u, resolver).map(a => u -> a)).toMap + + // Construct a set that contains all of the attributes that we need to evaluate the + // ordering. + val requiredAttributes = AttributeSet(resolved.values) + + // Figure out which ones are missing from the projection, so that we can add them and + // remove them after the sort. + val missingInProject = requiredAttributes -- child.output + + // Now that we have all the attributes we need, reconstruct a resolved ordering. + // It is important to do it here, instead of waiting for the standard resolved as adding + // attributes to the project below can actually introduce ambiquity that was not present + // before. + val resolvedOrdering = ordering.map(_ transform { + case u @ UnresolvedAttribute(name) => resolved.getOrElse(name, u) + }).asInstanceOf[Seq[SortOrder]] + + (resolvedOrdering, missingInProject.toSeq) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 40472a1cbb3b4..fa02111385c06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.types._ /** * Throws user facing errors when passed invalid queries that fail to analyze. */ -class CheckAnalysis { +trait CheckAnalysis { + self: Analyzer => /** * Override to provide additional checks for correct analysis. @@ -33,17 +34,22 @@ class CheckAnalysis { */ val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil - def failAnalysis(msg: String): Nothing = { + protected def failAnalysis(msg: String): Nothing = { throw new AnalysisException(msg) } - def apply(plan: LogicalPlan): Unit = { + def checkAnalysis(plan: LogicalPlan): Unit = { // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. plan.foreachUp { case operator: LogicalPlan => operator transformExpressionsUp { case a: Attribute if !a.resolved => + if (operator.childrenResolved) { + // Throw errors for specific problems with get field. + operator.resolveChildren(a.name, resolver, throwErrors = true) + } + val from = operator.inputSet.map(_.name).mkString(", ") a.failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 11b4eb5c888be..5345696570b41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -34,7 +34,7 @@ object AttributeSet { def apply(a: Attribute): AttributeSet = new AttributeSet(Set(new AttributeEquals(a))) /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ - def apply(baseSet: Seq[Expression]): AttributeSet = { + def apply(baseSet: Iterable[Expression]): AttributeSet = { new AttributeSet( baseSet .flatMap(_.references) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index b01a61d7bf8d6..2e9f3aa4ec4ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.types.{ArrayType, StructType, StructField} abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { @@ -109,16 +110,22 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolveChildren(name: String, resolver: Resolver): Option[NamedExpression] = - resolve(name, children.flatMap(_.output), resolver) + def resolveChildren( + name: String, + resolver: Resolver, + throwErrors: Boolean = false): Option[NamedExpression] = + resolve(name, children.flatMap(_.output), resolver, throwErrors) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this * LogicalPlan. The attribute is expressed as string in the following form: * `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String, resolver: Resolver): Option[NamedExpression] = - resolve(name, output, resolver) + def resolve( + name: String, + resolver: Resolver, + throwErrors: Boolean = false): Option[NamedExpression] = + resolve(name, output, resolver, throwErrors) /** * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. @@ -162,7 +169,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { protected def resolve( name: String, input: Seq[Attribute], - resolver: Resolver): Option[NamedExpression] = { + resolver: Resolver, + throwErrors: Boolean): Option[NamedExpression] = { val parts = name.split("\\.") @@ -196,14 +204,19 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - // The foldLeft adds UnresolvedGetField for every remaining parts of the name, - // and aliased it with the last part of the name. - // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. - // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias - // the final expression as "c". - val fieldExprs = nestedFields.foldLeft(a: Expression)(UnresolvedGetField) - val aliasName = nestedFields.last - Some(Alias(fieldExprs, aliasName)()) + try { + + // The foldLeft adds UnresolvedGetField for every remaining parts of the name, + // and aliased it with the last part of the name. + // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. + // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias + // the final expression as "c". + val fieldExprs = nestedFields.foldLeft(a: Expression)(resolveGetField(_, _, resolver)) + val aliasName = nestedFields.last + Some(Alias(fieldExprs, aliasName)()) + } catch { + case a: AnalysisException if !throwErrors => None + } // No matches. case Seq() => @@ -212,11 +225,46 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // More than one match. case ambiguousReferences => - val referenceNames = ambiguousReferences.map(_._1.qualifiedName).mkString(", ") + val referenceNames = ambiguousReferences.map(_._1).mkString(", ") throw new AnalysisException( s"Reference '$name' is ambiguous, could be: $referenceNames.") } } + + /** + * Returns the resolved `GetField`, and report error if no desired field or over one + * desired fields are found. + * + * TODO: this code is duplicated from Analyzer and should be refactored to avoid this. + */ + protected def resolveGetField( + expr: Expression, + fieldName: String, + resolver: Resolver): Expression = { + def findField(fields: Array[StructField]): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + throw new AnalysisException( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } + expr.dataType match { + case StructType(fields) => + val ordinal = findField(fields) + StructGetField(expr, fields(ordinal), ordinal) + case ArrayType(StructType(fields), containsNull) => + val ordinal = findField(fields) + ArrayGetField(expr, fields(ordinal), ordinal, containsNull) + case otherType => + throw new AnalysisException(s"GetField is not valid on fields of type $otherType") + } + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 756cd36f05c8c..ee7b14c7a157c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -40,14 +40,12 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { override val extendedResolutionRules = EliminateSubQueries :: Nil } - val checkAnalysis = new CheckAnalysis - def caseSensitiveAnalyze(plan: LogicalPlan) = - checkAnalysis(caseSensitiveAnalyzer(plan)) + caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer(plan)) def caseInsensitiveAnalyze(plan: LogicalPlan) = - checkAnalysis(caseInsensitiveAnalyzer(plan)) + caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer(plan)) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) val testRelation2 = LocalRelation( @@ -57,6 +55,21 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) + val nestedRelation = LocalRelation( + AttributeReference("top", StructType( + StructField("duplicateField", StringType) :: + StructField("duplicateField", StringType) :: + StructField("differentCase", StringType) :: + StructField("differentcase", StringType) :: Nil + ))()) + + val nestedRelation2 = LocalRelation( + AttributeReference("top", StructType( + StructField("aField", StringType) :: + StructField("bField", StringType) :: + StructField("cField", StringType) :: Nil + ))()) + before { caseSensitiveCatalog.registerTable(Seq("TaBlE"), testRelation) caseInsensitiveCatalog.registerTable(Seq("TaBlE"), testRelation) @@ -169,6 +182,24 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { "'b'" :: "group by" :: Nil ) + errorTest( + "ambiguous field", + nestedRelation.select($"top.duplicateField"), + "Ambiguous reference to fields" :: "duplicateField" :: Nil, + caseSensitive = false) + + errorTest( + "ambiguous field due to case insensitivity", + nestedRelation.select($"top.differentCase"), + "Ambiguous reference to fields" :: "differentCase" :: "differentcase" :: Nil, + caseSensitive = false) + + errorTest( + "missing field", + nestedRelation2.select($"top.c"), + "No such struct field" :: "aField" :: "bField" :: "cField" :: Nil, + caseSensitive = false) + case class UnresolvedTestPlan() extends LeafNode { override lazy val resolved = false override def output = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index b8100782ec937..1794936a52c6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -120,6 +120,10 @@ class SQLContext(@transient val sparkContext: SparkContext) ExtractPythonUdfs :: sources.PreInsertCastAndRename :: Nil + + override val extendedCheckRules = Seq( + sources.PreWriteCheck(catalog) + ) } @transient @@ -1065,14 +1069,6 @@ class SQLContext(@transient val sparkContext: SparkContext) Batch("Add exchange", Once, AddExchange(self)) :: Nil } - @transient - protected[sql] lazy val checkAnalysis = new CheckAnalysis { - override val extendedCheckRules = Seq( - sources.PreWriteCheck(catalog) - ) - } - - protected[sql] def openSession(): SQLSession = { detachSession() val session = createSession() @@ -1105,7 +1101,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @DeveloperApi protected[sql] class QueryExecution(val logical: LogicalPlan) { - def assertAnalyzed(): Unit = checkAnalysis(analyzed) + def assertAnalyzed(): Unit = analyzer.checkAnalysis(analyzed) lazy val analyzed: LogicalPlan = analyzer(logical) lazy val withCachedData: LogicalPlan = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a3c0076e16d6c..87e7cf8c8af9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1084,10 +1084,19 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("SPARK-6145: ORDER BY test for nested fields") { jsonRDD(sparkContext.makeRDD( """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder") - // These should be successfully analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY a.b").queryExecution.analyzed - sql("SELECT a.b FROM nestedOrder ORDER BY a.b").queryExecution.analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a").queryExecution.analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d").queryExecution.analyzed + + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) + checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a"), Row(1)) + checkAnswer(sql("SELECT a.a.a FROM nestedOrder ORDER BY a.a.a"), Row(1)) + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d"), Row(1)) + checkAnswer(sql("SELECT c[0].d FROM nestedOrder ORDER BY c[0].d"), Row(1)) + } + + test("SPARK-6145: special cases") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"b": [1]}, "b": [{"a": 1}], "c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY c0.a"), Row(1)) + checkAnswer(sql("SELECT b[0].a FROM t ORDER BY c0.a"), Row(1)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 91c6367371f15..33c67355967dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -32,6 +32,10 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { override val extendedResolutionRules = PreInsertCastAndRename :: Nil + + override val extendedCheckRules = Seq( + sources.PreWriteCheck(catalog) + ) } } } From 46de6c05e0619250346f0988e296849f8f93d2b1 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Tue, 31 Mar 2015 11:25:21 -0700 Subject: [PATCH 592/817] [SPARK-6598][MLLIB] Python API for IDFModel This is the sub-task of SPARK-6254. Wrapping IDFModel `idf` member function for pyspark. Author: lewuathe Closes #5264 from Lewuathe/SPARK-6598 and squashes the following commits: 1dc522c [lewuathe] [SPARK-6598] Python API for IDFModel --- python/pyspark/mllib/feature.py | 6 ++++++ python/pyspark/mllib/tests.py | 14 ++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 0ffe092a07365..4bfe3014ef748 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -244,6 +244,12 @@ def transform(self, x): x = _convert_to_vector(x) return JavaVectorTransformer.transform(self, x) + def idf(self): + """ + Returns the current IDF vector. + """ + return self.call('idf') + class IDF(object): """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 155019638f806..3bb0f0ca68128 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -41,6 +41,7 @@ from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics +from pyspark.mllib.feature import IDF from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase @@ -620,6 +621,19 @@ def test_right_number_of_results(self): self.assertEqual(len(chi), num_cols) self.assertIsNotNone(chi[1000]) + +class FeatureTest(PySparkTestCase): + def test_idf_model(self): + data = [ + Vectors.dense([1, 2, 6, 0, 2, 3, 1, 1, 0, 0, 3]), + Vectors.dense([1, 3, 0, 1, 3, 0, 0, 2, 0, 0, 1]), + Vectors.dense([1, 4, 1, 0, 0, 4, 9, 0, 1, 2, 0]), + Vectors.dense([2, 1, 0, 3, 0, 0, 5, 0, 2, 3, 9]) + ] + model = IDF().fit(self.sc.parallelize(data, 2)) + idf = model.idf() + self.assertEqual(len(idf), 11) + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" From b5bd75d90a761199c3f9cb583c1fe48c8fda7780 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 31 Mar 2015 11:32:14 -0700 Subject: [PATCH 593/817] [SPARK-6255] [MLLIB] Support multiclass classification in Python API Python API parity check for classification and multiclass classification support, major disparities need to be added for Python: ```scala LogisticRegressionWithLBFGS setNumClasses setValidateData LogisticRegressionModel getThreshold numClasses numFeatures SVMWithSGD setValidateData SVMModel getThreshold ``` For users the greatest benefit in this PR is multiclass classification was supported by Python API. Users can train multiclass classification model and use it to predict in pyspark. Author: Yanbo Liang Closes #5137 from yanboliang/spark-6255 and squashes the following commits: 0bd531e [Yanbo Liang] address comments 444d5e2 [Yanbo Liang] LogisticRegressionModel.predict() optimization fc7990b [Yanbo Liang] address comments b0d9c63 [Yanbo Liang] Support Mulinomial LR model predict in Python API ded847c [Yanbo Liang] Python API parity check for classification (support multiclass classification) --- .../mllib/api/python/PythonMLLibAPI.scala | 22 ++- python/pyspark/mllib/classification.py | 134 ++++++++++++++---- python/pyspark/mllib/regression.py | 10 +- 3 files changed, 134 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 22fa684fd2895..662ec5fbed453 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -77,7 +77,13 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector): JList[Object] = { try { val model = learner.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), initialWeights) - List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + if (model.isInstanceOf[LogisticRegressionModel]) { + val lrModel = model.asInstanceOf[LogisticRegressionModel] + List(lrModel.weights, lrModel.intercept, lrModel.numFeatures, lrModel.numClasses) + .map(_.asInstanceOf[Object]).asJava + } else { + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + } } finally { data.rdd.unpersist(blocking = false) } @@ -190,9 +196,11 @@ private[python] class PythonMLLibAPI extends Serializable { miniBatchFraction: Double, initialWeights: Vector, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val SVMAlg = new SVMWithSGD() SVMAlg.setIntercept(intercept) + .setValidateData(validateData) SVMAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -216,9 +224,11 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val LogRegAlg = new LogisticRegressionWithSGD() LogRegAlg.setIntercept(intercept) + .setValidateData(validateData) LogRegAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -242,9 +252,13 @@ private[python] class PythonMLLibAPI extends Serializable { regType: String, intercept: Boolean, corrections: Int, - tolerance: Double): JList[Object] = { + tolerance: Double, + validateData: Boolean, + numClasses: Int): JList[Object] = { val LogRegAlg = new LogisticRegressionWithLBFGS() LogRegAlg.setIntercept(intercept) + .setValidateData(validateData) + .setNumClasses(numClasses) LogRegAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 6766f3ebb8894..2466e8ac43458 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -22,7 +22,7 @@ from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper from pyspark.mllib.util import Saveable, Loader, inherit_doc @@ -31,13 +31,13 @@ 'SVMModel', 'SVMWithSGD', 'NaiveBayesModel', 'NaiveBayes'] -class LinearBinaryClassificationModel(LinearModel): +class LinearClassificationModel(LinearModel): """ - Represents a linear binary classification model that predicts to whether an - example is positive (1.0) or negative (0.0). + A private abstract class representing a multiclass classification model. + The categories are represented by int values: 0, 1, 2, etc. """ def __init__(self, weights, intercept): - super(LinearBinaryClassificationModel, self).__init__(weights, intercept) + super(LinearClassificationModel, self).__init__(weights, intercept) self._threshold = None def setThreshold(self, value): @@ -47,14 +47,26 @@ def setThreshold(self, value): Sets the threshold that separates positive predictions from negative predictions. An example with prediction score greater than or equal to this threshold is identified as an positive, and negative otherwise. + It is used for binary classification only. """ self._threshold = value + @property + def threshold(self): + """ + .. note:: Experimental + + Returns the threshold (if any) used for converting raw prediction scores + into 0/1 predictions. It is used for binary classification only. + """ + return self._threshold + def clearThreshold(self): """ .. note:: Experimental Clears the threshold so that `predict` will output raw prediction scores. + It is used for binary classification only. """ self._threshold = None @@ -66,7 +78,7 @@ def predict(self, test): raise NotImplementedError -class LogisticRegressionModel(LinearBinaryClassificationModel): +class LogisticRegressionModel(LinearClassificationModel): """A linear binary classification model derived from logistic regression. @@ -112,10 +124,39 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): ... os.removedirs(path) ... except: ... pass + >>> multi_class_data = [ + ... LabeledPoint(0.0, [0.0, 1.0, 0.0]), + ... LabeledPoint(1.0, [1.0, 0.0, 0.0]), + ... LabeledPoint(2.0, [0.0, 0.0, 1.0]) + ... ] + >>> mcm = LogisticRegressionWithLBFGS.train(data=sc.parallelize(multi_class_data), numClasses=3) + >>> mcm.predict([0.0, 0.5, 0.0]) + 0 + >>> mcm.predict([0.8, 0.0, 0.0]) + 1 + >>> mcm.predict([0.0, 0.0, 0.3]) + 2 """ - def __init__(self, weights, intercept): + def __init__(self, weights, intercept, numFeatures, numClasses): super(LogisticRegressionModel, self).__init__(weights, intercept) + self._numFeatures = int(numFeatures) + self._numClasses = int(numClasses) self._threshold = 0.5 + if self._numClasses == 2: + self._dataWithBiasSize = None + self._weightsMatrix = None + else: + self._dataWithBiasSize = self._coeff.size / (self._numClasses - 1) + self._weightsMatrix = self._coeff.toArray().reshape(self._numClasses - 1, + self._dataWithBiasSize) + + @property + def numFeatures(self): + return self._numFeatures + + @property + def numClasses(self): + return self._numClasses def predict(self, x): """ @@ -126,20 +167,38 @@ def predict(self, x): return x.map(lambda v: self.predict(v)) x = _convert_to_vector(x) - margin = self.weights.dot(x) + self._intercept - if margin > 0: - prob = 1 / (1 + exp(-margin)) + if self.numClasses == 2: + margin = self.weights.dot(x) + self._intercept + if margin > 0: + prob = 1 / (1 + exp(-margin)) + else: + exp_margin = exp(margin) + prob = exp_margin / (1 + exp_margin) + if self._threshold is None: + return prob + else: + return 1 if prob > self._threshold else 0 else: - exp_margin = exp(margin) - prob = exp_margin / (1 + exp_margin) - if self._threshold is None: - return prob - else: - return 1 if prob > self._threshold else 0 + best_class = 0 + max_margin = 0.0 + if x.size + 1 == self._dataWithBiasSize: + for i in range(0, self._numClasses - 1): + margin = x.dot(self._weightsMatrix[i][0:x.size]) + \ + self._weightsMatrix[i][x.size] + if margin > max_margin: + max_margin = margin + best_class = i + 1 + else: + for i in range(0, self._numClasses - 1): + margin = x.dot(self._weightsMatrix[i]) + if margin > max_margin: + max_margin = margin + best_class = i + 1 + return best_class def save(self, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.LogisticRegressionModel( - _py2java(sc, self._coeff), self.intercept) + _py2java(sc, self._coeff), self.intercept, self.numFeatures, self.numClasses) java_model.save(sc._jsc.sc(), path) @classmethod @@ -148,8 +207,10 @@ def load(cls, sc, path): sc._jsc.sc(), path) weights = _java2py(sc, java_model.weights()) intercept = java_model.intercept() + numFeatures = java_model.numFeatures() + numClasses = java_model.numClasses() threshold = java_model.getThreshold().get() - model = LogisticRegressionModel(weights, intercept) + model = LogisticRegressionModel(weights, intercept, numFeatures, numClasses) model.setThreshold(threshold) return model @@ -158,7 +219,8 @@ class LogisticRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=0.01, regType="l2", intercept=False): + initialWeights=None, regParam=0.01, regType="l2", intercept=False, + validateData=True): """ Train a logistic regression model on the given data. @@ -184,11 +246,14 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, or not of the augmented representation for training data (i.e. whether bias features are activated or not). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainLogisticRegressionModelWithSGD", rdd, int(iterations), float(step), float(miniBatchFraction), i, float(regParam), regType, - bool(intercept)) + bool(intercept), bool(validateData)) return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) @@ -197,7 +262,7 @@ class LogisticRegressionWithLBFGS(object): @classmethod def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType="l2", - intercept=False, corrections=10, tolerance=1e-4): + intercept=False, corrections=10, tolerance=1e-4, validateData=True, numClasses=2): """ Train a logistic regression model on the given data. @@ -223,6 +288,11 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType update (default: 10). :param tolerance: The convergence tolerance of iterations for L-BFGS (default: 1e-4). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) + :param numClasses: The number of classes (i.e., outcomes) a label can take + in Multinomial Logistic Regression (default: 2). >>> data = [ ... LabeledPoint(0.0, [0.0, 1.0]), @@ -237,12 +307,20 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType def train(rdd, i): return callMLlibFunc("trainLogisticRegressionModelWithLBFGS", rdd, int(iterations), i, float(regParam), regType, bool(intercept), int(corrections), - float(tolerance)) - + float(tolerance), bool(validateData), int(numClasses)) + + if initialWeights is None: + if numClasses == 2: + initialWeights = [0.0] * len(data.first().features) + else: + if intercept: + initialWeights = [0.0] * (len(data.first().features) + 1) * (numClasses - 1) + else: + initialWeights = [0.0] * len(data.first().features) * (numClasses - 1) return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) -class SVMModel(LinearBinaryClassificationModel): +class SVMModel(LinearClassificationModel): """A support vector machine. @@ -325,7 +403,8 @@ class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None, regType="l2", intercept=False): + miniBatchFraction=1.0, initialWeights=None, regType="l2", + intercept=False, validateData=True): """ Train a support vector machine on the given data. @@ -351,11 +430,14 @@ def train(cls, data, iterations=100, step=1.0, regParam=0.01, or not of the augmented representation for training data (i.e. whether bias features are activated or not). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainSVMModelWithSGD", rdd, int(iterations), float(step), float(regParam), float(miniBatchFraction), i, regType, - bool(intercept)) + bool(intercept), bool(validateData)) return _regression_train_wrapper(train, SVMModel, data, initialWeights) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 209f1ee473b5b..cd7310a64f4ae 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -167,13 +167,19 @@ def load(cls, sc, path): # return the result of a call to the appropriate JVM stub. # _regression_train_wrapper is responsible for setup and error checking. def _regression_train_wrapper(train_func, modelClass, data, initial_weights): + from pyspark.mllib.classification import LogisticRegressionModel first = data.first() if not isinstance(first, LabeledPoint): raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) if initial_weights is None: initial_weights = [0.0] * len(data.first().features) - weights, intercept = train_func(data, _convert_to_vector(initial_weights)) - return modelClass(weights, intercept) + if (modelClass == LogisticRegressionModel): + weights, intercept, numFeatures, numClasses = train_func( + data, _convert_to_vector(initial_weights)) + return modelClass(weights, intercept, numFeatures, numClasses) + else: + weights, intercept = train_func(data, _convert_to_vector(initial_weights)) + return modelClass(weights, intercept) class LinearRegressionWithSGD(object): From beebb7ffc21c66ae3e4c615555194d1e19ede1bb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 31 Mar 2015 11:34:29 -0700 Subject: [PATCH 594/817] [SPARK-5371][SQL] Propagate types after function conversion, before futher resolution Before it was possible for a query to flip back and forth from a resolved state, allowing resolution to propagate up before coercion had stabilized. The issue was that `ResolvedReferences` would run after `FunctionArgumentConversion`, but before `PropagateTypes` had run. This PR ensures we correctly `PropagateTypes` after any coercion has applied. Author: Michael Armbrust Closes #5278 from marmbrus/unionNull and squashes the following commits: dc3581a [Michael Armbrust] [SPARK-5371][SQL] Propogate types after function conversion / before futher resolution --- .../catalyst/analysis/HiveTypeCoercion.scala | 1 + .../plans/logical/basicOperators.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 26 ++++++++++++++++++- 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 34ef7d28cc7f2..3c7b46e0702a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -78,6 +78,7 @@ trait HiveTypeCoercion { FunctionArgumentConversion :: CaseWhenCoercion :: Division :: + PropagateTypes :: Nil /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 190209238a4a5..8633e06093cf3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -80,7 +80,7 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved: Boolean = childrenResolved && - !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } + left.output.zip(right.output).forall { case (l,r) => l.dataType == r.dataType } override def statistics: Statistics = { val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 2f50a33448462..2065f0d60d92f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -41,8 +41,32 @@ case class NestedArray1(a: NestedArray2) */ class SQLQuerySuite extends QueryTest { + test("SPARK-5371: union with null and sum") { + val df = Seq((1, 1)).toDF("c1", "c2") + df.registerTempTable("table1") + + val query = sql( + """ + |SELECT + | MIN(c1), + | MIN(c2) + |FROM ( + | SELECT + | SUM(c1) c1, + | NULL c2 + | FROM table1 + | UNION ALL + | SELECT + | NULL c1, + | SUM(c2) c2 + | FROM table1 + |) a + """.stripMargin) + checkAnswer(query, Row(1, 1) :: Nil) + } + test("explode nested Field") { - Seq(NestedArray1(NestedArray2(Seq(1,2,3)))).toDF.registerTempTable("nestedArray") + Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.registerTempTable("nestedArray") checkAnswer( sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), Row(1) :: Row(2) :: Row(3) :: Nil) From 2036bc5993022da550f0cb1c0485ae92ec3e6fb0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 31 Mar 2015 13:18:07 -0700 Subject: [PATCH 595/817] [SPARK-6633][SQL] Should be "Contains" instead of "EndsWith" when constructing sources.StringContains Author: Liang-Chi Hsieh Closes #5299 from viirya/stringcontains and squashes the following commits: c1ece4c [Liang-Chi Hsieh] Should be Contains instead of EndsWith. --- .../scala/org/apache/spark/sql/sources/DataSourceStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 83b603a4bb245..e13759b7feb7b 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 @@ -173,7 +173,7 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => Some(sources.StringEndsWith(a.name, v)) - case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => + case expressions.Contains(a: Attribute, Literal(v: String, StringType)) => Some(sources.StringContains(a.name, v)) case _ => None From 0e00f12d33d28d064c166262b14e012a1aeaa7b0 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Tue, 31 Mar 2015 16:01:08 -0700 Subject: [PATCH 596/817] [SPARK-5692] [MLlib] Word2Vec save/load Word2Vec model now supports saving and loading. a] The Metadata stored in JSON format consists of "version", "classname", "vectorSize" and "numWords" b] The data stored in Parquet file format consists of an Array of rows with each row consisting of 2 columns, first being the word: String and the second, an Array of Floats. Author: MechCoder Closes #5291 from MechCoder/spark-5692 and squashes the following commits: 1142f3a [MechCoder] Add numWords to metaData bfe4c39 [MechCoder] [SPARK-5692] Word2Vec save/load --- .../apache/spark/mllib/feature/Word2Vec.scala | 87 ++++++++++++++++++- .../spark/mllib/feature/Word2VecSuite.scala | 26 ++++++ 2 files changed, 110 insertions(+), 3 deletions(-) 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 59a79e5c6a4ac..9ee7e4a66b535 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 @@ -25,14 +25,21 @@ import scala.collection.mutable.ArrayBuilder import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.Logging +import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.sql.{SQLContext, Row} /** * Entry in vocabulary @@ -422,7 +429,7 @@ class Word2Vec extends Serializable with Logging { */ @Experimental class Word2VecModel private[mllib] ( - private val model: Map[String, Array[Float]]) extends Serializable { + private val model: Map[String, Array[Float]]) extends Serializable with Saveable { private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { require(v1.length == v2.length, "Vectors should have the same length") @@ -432,7 +439,13 @@ class Word2VecModel private[mllib] ( if (norm1 == 0 || norm2 == 0) return 0.0 blas.sdot(n, v1, 1, v2,1) / norm1 / norm2 } - + + override protected def formatVersion = "1.0" + + def save(sc: SparkContext, path: String): Unit = { + Word2VecModel.SaveLoadV1_0.save(sc, path, model) + } + /** * Transforms a word to its vector representation * @param word a word @@ -475,7 +488,7 @@ class Word2VecModel private[mllib] ( .tail .toArray } - + /** * Returns a map of words to their vector representations. */ @@ -483,3 +496,71 @@ class Word2VecModel private[mllib] ( model } } + +@Experimental +object Word2VecModel extends Loader[Word2VecModel] { + + private object SaveLoadV1_0 { + + val formatVersionV1_0 = "1.0" + + val classNameV1_0 = "org.apache.spark.mllib.feature.Word2VecModel" + + case class Data(word: String, vector: Array[Float]) + + def load(sc: SparkContext, path: String): Word2VecModel = { + val dataPath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataFrame = sqlContext.parquetFile(dataPath) + + val dataArray = dataFrame.select("word", "vector").collect() + + // Check schema explicitly since erasure makes it hard to use match-case for checking. + Loader.checkSchema[Data](dataFrame.schema) + + val word2VecMap = dataArray.map(i => (i.getString(0), i.getSeq[Float](1).toArray)).toMap + new Word2VecModel(word2VecMap) + } + + def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]) = { + + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val vectorSize = model.values.head.size + val numWords = model.size + val metadata = compact(render + (("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ + ("vectorSize" -> vectorSize) ~ ("numWords" -> numWords))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val dataArray = model.toSeq.map { case (w, v) => Data(w, v) } + sc.parallelize(dataArray.toSeq, 1).toDF().saveAsParquetFile(Loader.dataPath(path)) + } + } + + override def load(sc: SparkContext, path: String): Word2VecModel = { + + val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) + implicit val formats = DefaultFormats + val expectedVectorSize = (metadata \ "vectorSize").extract[Int] + val expectedNumWords = (metadata \ "numWords").extract[Int] + val classNameV1_0 = SaveLoadV1_0.classNameV1_0 + (loadedClassName, loadedVersion) match { + case (classNameV1_0, "1.0") => + val model = SaveLoadV1_0.load(sc, path) + val vectorSize = model.getVectors.values.head.size + val numWords = model.getVectors.size + require(expectedVectorSize == vectorSize, + s"Word2VecModel requires each word to be mapped to a vector of size " + + s"$expectedVectorSize, got vector of size $vectorSize") + require(expectedNumWords == numWords, + s"Word2VecModel requires $expectedNumWords words, but got $numWords") + model + case _ => throw new Exception( + s"Word2VecModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $loadedVersion). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index 52278690dbd89..98a98a7599bcb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -21,6 +21,9 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils + class Word2VecSuite extends FunSuite with MLlibTestSparkContext { // TODO: add more tests @@ -51,4 +54,27 @@ class Word2VecSuite extends FunSuite with MLlibTestSparkContext { assert(syms(0)._1 == "taiwan") assert(syms(1)._1 == "japan") } + + test("model load / save") { + + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + try { + model.save(sc, path) + val sameModel = Word2VecModel.load(sc, path) + assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + } finally { + Utils.deleteRecursively(tempDir) + } + + } } From 37326079d818fdb140415a65653767d997613dac Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 31 Mar 2015 16:18:39 -0700 Subject: [PATCH 597/817] [SPARK-6614] OutputCommitCoordinator should clear authorized committer only after authorized committer fails, not after any failure In OutputCommitCoordinator, there is some logic to clear the authorized committer's lock on committing in case that task fails. However, it looks like the current code also clears this lock if other non-authorized tasks fail, which is an obvious bug. In theory, it's possible that this could allow a new committer to start, run to completion, and commit output before the authorized committer finished, but it's unlikely that this race occurs often in practice due to the complex combination of failure and timing conditions that would be required to expose it. This patch addresses this issue and adds a regression test. Thanks to aarondav for spotting this issue. Author: Josh Rosen Closes #5276 from JoshRosen/SPARK-6614 and squashes the following commits: d532ba7 [Josh Rosen] Check whether failed task was authorized committer cbb3784 [Josh Rosen] Add regression test for SPARK-6614 --- .../scheduler/OutputCommitCoordinator.scala | 8 +++--- .../OutputCommitCoordinatorSuite.scala | 25 +++++++++++++++++++ 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 17055e2f22d0d..9e29fd13821dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -113,9 +113,11 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { logInfo( s"Task was denied committing, stage: $stage, partition: $partition, attempt: $attempt") case otherReason => - logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" + - s" clearing lock") - authorizedCommitters.remove(partition) + if (authorizedCommitters.get(partition).exists(_ == attempt)) { + logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" + + s" clearing lock") + authorizedCommitters.remove(partition) + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index c8c957856247a..cf97707946706 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -161,6 +161,31 @@ class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { } assert(tempDir.list().size === 0) } + + test("Only authorized committer failures can clear the authorized committer lock (SPARK-6614)") { + val stage: Int = 1 + val partition: Long = 2 + val authorizedCommitter: Long = 3 + val nonAuthorizedCommitter: Long = 100 + outputCommitCoordinator.stageStart(stage) + assert(outputCommitCoordinator.canCommit(stage, partition, attempt = authorizedCommitter)) + assert(!outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter)) + // The non-authorized committer fails + outputCommitCoordinator.taskCompleted( + stage, partition, attempt = nonAuthorizedCommitter, reason = TaskKilled) + // New tasks should still not be able to commit because the authorized committer has not failed + assert( + !outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 1)) + // The authorized committer now fails, clearing the lock + outputCommitCoordinator.taskCompleted( + stage, partition, attempt = authorizedCommitter, reason = TaskKilled) + // A new task should now be allowed to become the authorized committer + assert( + outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 2)) + // There can only be one authorized committer + assert( + !outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 3)) + } } /** From 305abe1e57450f49e3ec4dffb073c5adf17cadef Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 31 Mar 2015 18:31:36 -0700 Subject: [PATCH 598/817] [Doc] Improve Python DataFrame documentation Author: Reynold Xin Closes #5287 from rxin/pyspark-df-doc-cleanup-context and squashes the following commits: 1841b60 [Reynold Xin] Lint. f2007f1 [Reynold Xin] functions and types. bc3b72b [Reynold Xin] More improvements to DataFrame Python doc. ac1d4c0 [Reynold Xin] Bug fix. b163365 [Reynold Xin] Python fix. Added Experimental flag to DataFrameNaFunctions. 608422d [Reynold Xin] [Doc] Cleanup context.py Python docs. --- python/pyspark/sql/__init__.py | 4 +- python/pyspark/sql/context.py | 227 ++++++---------- python/pyspark/sql/dataframe.py | 249 +++++++++--------- python/pyspark/sql/functions.py | 6 +- python/pyspark/sql/types.py | 154 +++-------- .../spark/sql/DataFrameNaFunctions.scala | 3 + 6 files changed, 253 insertions(+), 390 deletions(-) diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 9d39e5d9c2449..65abb24eed823 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -16,7 +16,7 @@ # """ -public classes of Spark SQL: +Important classes of Spark SQL and DataFrames: - L{SQLContext} Main entry point for :class:`DataFrame` and SQL functionality. @@ -34,6 +34,8 @@ Methods for handling missing data (null values). - L{functions} List of built-in functions available for :class:`DataFrame`. + - L{types} + List of data types available. """ from pyspark.sql.context import SQLContext, HiveContext diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 80939a1f8ab1e..c2d81ba804110 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -40,9 +40,9 @@ def _monkey_patch_RDD(sqlCtx): def toDF(self, schema=None, sampleRatio=None): """ - Convert current :class:`RDD` into a :class:`DataFrame` + Converts current :class:`RDD` into a :class:`DataFrame` - This is a shorthand for `sqlCtx.createDataFrame(rdd, schema, sampleRatio)` + This is a shorthand for ``sqlCtx.createDataFrame(rdd, schema, sampleRatio)`` :param schema: a StructType or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring @@ -56,49 +56,23 @@ def toDF(self, schema=None, sampleRatio=None): RDD.toDF = toDF -class UDFRegistration(object): - """Wrapper for register UDF""" - - def __init__(self, sqlCtx): - self.sqlCtx = sqlCtx - - def register(self, name, f, returnType=StringType()): - """Registers a lambda function as a UDF so it can be used in SQL statements. - - In addition to a name and the function itself, the return type can be optionally specified. - When the return type is not given it default to a string and conversion will automatically - be done. For any other return type, the produced object must match the specified type. - - >>> sqlCtx.udf.register("stringLengthString", lambda x: len(x)) - >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() - [Row(c0=u'4')] - - >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() - [Row(c0=4)] - """ - return self.sqlCtx.registerFunction(name, f, returnType) - - class SQLContext(object): - """Main entry point for Spark SQL functionality. - A SQLContext can be used create L{DataFrame}, register L{DataFrame} as + A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. - """ - def __init__(self, sparkContext, sqlContext=None): - """Create a new SQLContext. - - It will add a method called `toDF` to :class:`RDD`, which could be - used to convert an RDD into a DataFrame, it's a shorthand for - :func:`SQLContext.createDataFrame`. + When created, :class:`SQLContext` adds a method called ``toDF`` to :class:`RDD`, + which could be used to convert an RDD into a DataFrame, it's a shorthand for + :func:`SQLContext.createDataFrame`. - :param sparkContext: The SparkContext to wrap. - :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + :param sparkContext: The :class:`SparkContext` backing this SQLContext. + :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instantiate a new SQLContext in the JVM, instead we make all calls to this object. + """ + + def __init__(self, sparkContext, sqlContext=None): + """Creates a new SQLContext. >>> from datetime import datetime >>> sqlCtx = SQLContext(sc) @@ -145,7 +119,7 @@ def getConf(self, key, defaultValue): @property def udf(self): - """Wrapper for register Python function as UDF """ + """Returns a :class:`UDFRegistration` for UDF registration.""" return UDFRegistration(self) def registerFunction(self, name, f, returnType=StringType()): @@ -155,6 +129,10 @@ def registerFunction(self, name, f, returnType=StringType()): When the return type is not given it default to a string and conversion will automatically be done. For any other return type, the produced object must match the specified type. + :param name: name of the UDF + :param samplingRatio: lambda function + :param returnType: a :class:`DataType` object + >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() [Row(c0=u'4')] @@ -163,6 +141,11 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] + + >>> from pyspark.sql.types import IntegerType + >>> sqlCtx.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) @@ -203,30 +186,7 @@ def _inferSchema(self, rdd, samplingRatio=None): return schema def inferSchema(self, rdd, samplingRatio=None): - """Infer and apply a schema to an RDD of L{Row}. - - ::note: - Deprecated in 1.3, use :func:`createDataFrame` instead - - When samplingRatio is specified, the schema is inferred by looking - at the types of each row in the sampled dataset. Otherwise, the - first 100 rows of the RDD are inspected. Nested collections are - supported, which can include array, dict, list, Row, tuple, - namedtuple, or object. - - Each row could be L{pyspark.sql.Row} object or namedtuple or objects. - Using top level dicts is deprecated, as dict is used to represent Maps. - - If a single column has multiple distinct inferred types, it may cause - runtime exceptions. - - >>> rdd = sc.parallelize( - ... [Row(field1=1, field2="row1"), - ... Row(field1=2, field2="row2"), - ... Row(field1=3, field2="row3")]) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.collect()[0] - Row(field1=1, field2=u'row1') + """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ warnings.warn("inferSchema is deprecated, please use createDataFrame instead") @@ -236,27 +196,7 @@ def inferSchema(self, rdd, samplingRatio=None): return self.createDataFrame(rdd, None, samplingRatio) def applySchema(self, rdd, schema): - """ - Applies the given schema to the given RDD of L{tuple} or L{list}. - - ::note: - Deprecated in 1.3, use :func:`createDataFrame` instead - - These tuples or lists can contain complex nested structures like - lists, maps or nested rows. - - The schema should be a StructType. - - It is important that the schema matches the types of the objects - in each row or exceptions could be thrown at runtime. - - >>> from pyspark.sql.types import * - >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) - >>> schema = StructType([StructField("field1", IntegerType(), False), - ... StructField("field2", StringType(), False)]) - >>> df = sqlCtx.applySchema(rdd2, schema) - >>> df.collect() - [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] + """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ warnings.warn("applySchema is deprecated, please use createDataFrame instead") @@ -270,25 +210,23 @@ def applySchema(self, rdd, schema): def createDataFrame(self, data, schema=None, samplingRatio=None): """ - Create a DataFrame from an RDD of tuple/list, list or pandas.DataFrame. + Creates a :class:`DataFrame` from an :class:`RDD` of :class:`tuple`/:class:`list`, + list or :class:`pandas.DataFrame`. - `schema` could be :class:`StructType` or a list of column names. + When ``schema`` is a list of column names, the type of each column + will be inferred from ``data``. - When `schema` is a list of column names, the type of each column - will be inferred from `rdd`. + When ``schema`` is ``None``, it will try to infer the schema (column names and types) + from ``data``, which should be an RDD of :class:`Row`, + or :class:`namedtuple`, or :class:`dict`. - When `schema` is None, it will try to infer the column name and type - from `rdd`, which should be an RDD of :class:`Row`, or namedtuple, - or dict. + If schema inference is needed, ``samplingRatio`` is used to determined the ratio of + rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. - If referring needed, `samplingRatio` is used to determined how many - rows will be used to do referring. The first row will be used if - `samplingRatio` is None. - - :param data: an RDD of Row/tuple/list/dict, list, or pandas.DataFrame - :param schema: a StructType or list of names of columns + :param data: an RDD of :class:`Row`/:class:`tuple`/:class:`list`/:class:`dict`, + :class:`list`, or :class:`pandas.DataFrame`. + :param schema: a :class:`StructType` or list of column names. default None. :param samplingRatio: the sample ratio of rows used for inferring - :return: a DataFrame >>> l = [('Alice', 1)] >>> sqlCtx.createDataFrame(l).collect() @@ -373,22 +311,20 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) return DataFrame(df, self) - def registerDataFrameAsTable(self, rdd, tableName): - """Registers the given RDD as a temporary table in the catalog. + def registerDataFrameAsTable(self, df, tableName): + """Registers the given :class:`DataFrame` as a temporary table in the catalog. - Temporary tables exist only during the lifetime of this instance of - SQLContext. + Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`. >>> sqlCtx.registerDataFrameAsTable(df, "table1") """ - if (rdd.__class__ is DataFrame): - df = rdd._jdf - self._ssql_ctx.registerDataFrameAsTable(df, tableName) + if (df.__class__ is DataFrame): + self._ssql_ctx.registerDataFrameAsTable(df._jdf, tableName) else: raise ValueError("Can only register DataFrame as table") def parquetFile(self, *paths): - """Loads a Parquet file, returning the result as a L{DataFrame}. + """Loads a Parquet file, returning the result as a :class:`DataFrame`. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() @@ -406,15 +342,10 @@ def parquetFile(self, *paths): return DataFrame(jdf, self) def jsonFile(self, path, schema=None, samplingRatio=1.0): - """ - Loads a text file storing one JSON object per line as a - L{DataFrame}. + """Loads a text file storing one JSON object per line as a :class:`DataFrame`. - If the schema is provided, applies the given schema to this - JSON dataset. - - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -450,13 +381,10 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): return DataFrame(df, self) def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): - """Loads an RDD storing one JSON object per string as a L{DataFrame}. - - If the schema is provided, applies the given schema to this - JSON dataset. + """Loads an RDD storing one JSON object per string as a :class:`DataFrame`. - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. >>> df1 = sqlCtx.jsonRDD(json) >>> df1.first() @@ -475,7 +403,6 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): >>> df3 = sqlCtx.jsonRDD(json, schema) >>> df3.first() Row(field2=u'row1', field3=Row(field5=None)) - """ def func(iterator): @@ -496,11 +423,11 @@ def func(iterator): return DataFrame(df, self) def load(self, path=None, source=None, schema=None, **options): - """Returns the dataset in a data source as a DataFrame. + """Returns the dataset in a data source as a :class:`DataFrame`. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Optionally, a schema can be provided as the schema of the returned DataFrame. """ @@ -526,11 +453,11 @@ def createExternalTable(self, tableName, path=None, source=None, It returns the DataFrame associated with the external table. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. - Optionally, a schema can be provided as the schema of the returned DataFrame and + Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and created external table. """ if path is not None: @@ -551,7 +478,7 @@ def createExternalTable(self, tableName, path=None, source=None, return DataFrame(df, self) def sql(self, sqlQuery): - """Return a L{DataFrame} representing the result of the given query. + """Returns a :class:`DataFrame` representing the result of the given query. >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") @@ -561,7 +488,7 @@ def sql(self, sqlQuery): return DataFrame(self._ssql_ctx.sql(sqlQuery), self) def table(self, tableName): - """Returns the specified table as a L{DataFrame}. + """Returns the specified table as a :class:`DataFrame`. >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.table("table1") @@ -571,12 +498,12 @@ def table(self, tableName): return DataFrame(self._ssql_ctx.table(tableName), self) def tables(self, dbName=None): - """Returns a DataFrame containing names of tables in the given database. + """Returns a :class:`DataFrame` containing names of tables in the given database. - If `dbName` is not specified, the current database will be used. + If ``dbName`` is not specified, the current database will be used. - The returned DataFrame has two columns, tableName and isTemporary - (a column with BooleanType indicating if a table is a temporary one or not). + The returned DataFrame has two columns: ``tableName`` and ``isTemporary`` + (a column with :class:`BooleanType` indicating if a table is a temporary one or not). >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> df2 = sqlCtx.tables() @@ -589,9 +516,9 @@ def tables(self, dbName=None): return DataFrame(self._ssql_ctx.tables(dbName), self) def tableNames(self, dbName=None): - """Returns a list of names of tables in the database `dbName`. + """Returns a list of names of tables in the database ``dbName``. - If `dbName` is not specified, the current database will be used. + If ``dbName`` is not specified, the current database will be used. >>> sqlCtx.registerDataFrameAsTable(df, "table1") >>> "table1" in sqlCtx.tableNames() @@ -618,22 +545,18 @@ def clearCache(self): class HiveContext(SQLContext): - """A variant of Spark SQL that integrates with data stored in Hive. - Configuration for Hive is read from hive-site.xml on the classpath. + Configuration for Hive is read from ``hive-site.xml`` on the classpath. It supports running both SQL and HiveQL commands. + + :param sparkContext: The SparkContext to wrap. + :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new + :class:`HiveContext` in the JVM, instead we make all calls to this object. """ def __init__(self, sparkContext, hiveContext=None): - """Create a new HiveContext. - - :param sparkContext: The SparkContext to wrap. - :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new - HiveContext in the JVM, instead we make all calls to this object. - """ SQLContext.__init__(self, sparkContext) - if hiveContext: self._scala_HiveContext = hiveContext @@ -652,6 +575,18 @@ def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) +class UDFRegistration(object): + """Wrapper for user-defined function registration.""" + + def __init__(self, sqlCtx): + self.sqlCtx = sqlCtx + + def register(self, name, f, returnType=StringType()): + return self.sqlCtx.registerFunction(name, f, returnType) + + register.__doc__ = SQLContext.registerFunction.__doc__ + + def _test(): import doctest from pyspark.context import SparkContext diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 15508023326cc..c30326ebd133e 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -35,8 +35,7 @@ class DataFrame(object): - - """A collection of rows that have the same columns. + """A distributed collection of data grouped into named columns. A :class:`DataFrame` is equivalent to a relational table in Spark SQL, and can be created using various functions in :class:`SQLContext`:: @@ -69,9 +68,7 @@ def __init__(self, jdf, sql_ctx): @property def rdd(self): - """ - Return the content of the :class:`DataFrame` as an :class:`pyspark.RDD` - of :class:`Row` s. + """Returns the content as an :class:`pyspark.RDD` of :class:`Row`. """ if not hasattr(self, '_lazy_rdd'): jrdd = self._jdf.javaToPython() @@ -93,7 +90,9 @@ def na(self): return DataFrameNaFunctions(self) def toJSON(self, use_unicode=False): - """Convert a :class:`DataFrame` into a MappedRDD of JSON documents; one document per row. + """Converts a :class:`DataFrame` into a :class:`RDD` of string. + + Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() '{"age":2,"name":"Alice"}' @@ -102,10 +101,10 @@ def toJSON(self, use_unicode=False): return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) def saveAsParquetFile(self, path): - """Save the contents as a Parquet file, preserving the schema. + """Saves the contents as a Parquet file, preserving the schema. Files that are written out using this method can be read back in as - a :class:`DataFrame` using the L{SQLContext.parquetFile} method. + a :class:`DataFrame` using :func:`SQLContext.parquetFile`. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() @@ -120,8 +119,8 @@ def saveAsParquetFile(self, path): def registerTempTable(self, name): """Registers this RDD as a temporary table using the given name. - The lifetime of this temporary table is tied to the L{SQLContext} - that was used to create this DataFrame. + The lifetime of this temporary table is tied to the :class:`SQLContext` + that was used to create this :class:`DataFrame`. >>> df.registerTempTable("people") >>> df2 = sqlCtx.sql("select * from people") @@ -131,7 +130,7 @@ def registerTempTable(self, name): self._jdf.registerTempTable(name) def registerAsTable(self, name): - """DEPRECATED: use registerTempTable() instead""" + """DEPRECATED: use :func:`registerTempTable` instead""" warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) self.registerTempTable(name) @@ -162,22 +161,19 @@ def _java_save_mode(self, mode): return jmode def saveAsTable(self, tableName, source=None, mode="error", **options): - """Saves the contents of the :class:`DataFrame` to a data source as a table. + """Saves the contents of this :class:`DataFrame` to a data source as a table. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Additionally, mode is used to specify the behavior of the saveAsTable operation when table already exists in the data source. There are four modes: - * append: Contents of this :class:`DataFrame` are expected to be appended \ - to existing table. - * overwrite: Data in the existing table is expected to be overwritten by \ - the contents of this DataFrame. - * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of the \ - :class:`DataFrame` and to not change the existing table. + * `append`: Append contents of this :class:`DataFrame` to existing data. + * `overwrite`: Overwrite existing data. + * `error`: Throw an exception if data already exists. + * `ignore`: Silently ignore this operation if data already exists. """ if source is None: source = self.sql_ctx.getConf("spark.sql.sources.default", @@ -190,18 +186,17 @@ def saveAsTable(self, tableName, source=None, mode="error", **options): def save(self, path=None, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Additionally, mode is used to specify the behavior of the save operation when data already exists in the data source. There are four modes: - * append: Contents of this :class:`DataFrame` are expected to be appended to existing data. - * overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. - * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of \ - the :class:`DataFrame` and to not change the existing data. + * `append`: Append contents of this :class:`DataFrame` to existing data. + * `overwrite`: Overwrite existing data. + * `error`: Throw an exception if data already exists. + * `ignore`: Silently ignore this operation if data already exists. """ if path is not None: options["path"] = path @@ -215,8 +210,7 @@ def save(self, path=None, source=None, mode="error", **options): @property def schema(self): - """Returns the schema of this :class:`DataFrame` (represented by - a L{StructType}). + """Returns the schema of this :class:`DataFrame` as a :class:`types.StructType`. >>> df.schema StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) @@ -237,11 +231,9 @@ def printSchema(self): print (self._jdf.schema().treeString()) def explain(self, extended=False): - """ - Prints the plans (logical and physical) to the console for - debugging purpose. + """Prints the (logical and physical) plans to the console for debugging purpose. - If extended is False, only prints the physical plan. + :param extended: boolean, default ``False``. If ``False``, prints only the physical plan. >>> df.explain() PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at SQLContext.scala:... @@ -263,15 +255,13 @@ def explain(self, extended=False): print self._jdf.queryExecution().executedPlan().toString() def isLocal(self): - """ - Returns True if the `collect` and `take` methods can be run locally + """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally (without any Spark executors). """ return self._jdf.isLocal() def show(self, n=20): - """ - Print the first n rows. + """Prints the first ``n`` rows to the console. >>> df DataFrame[age: int, name: string] @@ -286,11 +276,7 @@ def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) def count(self): - """Return the number of elements in this RDD. - - Unlike the base RDD implementation of count, this implementation - leverages the query optimizer to compute the count on the DataFrame, - which supports features such as filter pushdown. + """Returns the number of rows in this :class:`DataFrame`. >>> df.count() 2L @@ -298,10 +284,7 @@ def count(self): return self._jdf.count() def collect(self): - """Return a list that contains all of the rows. - - Each object in the list is a Row, the fields can be accessed as - attributes. + """Returns all the records as a list of :class:`Row`. >>> df.collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -313,7 +296,7 @@ def collect(self): return [cls(r) for r in rs] def limit(self, num): - """Limit the result count to the number specified. + """Limits the result count to the number specified. >>> df.limit(1).collect() [Row(age=2, name=u'Alice')] @@ -324,10 +307,7 @@ def limit(self, num): return DataFrame(jdf, self.sql_ctx) def take(self, num): - """Take the first num rows of the RDD. - - Each object in the list is a Row, the fields can be accessed as - attributes. + """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. >>> df.take(2) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -335,9 +315,9 @@ def take(self, num): return self.limit(num).collect() def map(self, f): - """ Return a new RDD by applying a function to each Row + """ Returns a new :class:`RDD` by applying a the ``f`` function to each :class:`Row`. - It's a shorthand for df.rdd.map() + This is a shorthand for ``df.rdd.map()``. >>> df.map(lambda p: p.name).collect() [u'Alice', u'Bob'] @@ -345,10 +325,10 @@ def map(self, f): return self.rdd.map(f) def flatMap(self, f): - """ Return a new RDD by first applying a function to all elements of this, + """ Returns a new :class:`RDD` by first applying the ``f`` function to each :class:`Row`, and then flattening the results. - It's a shorthand for df.rdd.flatMap() + This is a shorthand for ``df.rdd.flatMap()``. >>> df.flatMap(lambda p: p.name).collect() [u'A', u'l', u'i', u'c', u'e', u'B', u'o', u'b'] @@ -356,10 +336,9 @@ def flatMap(self, f): return self.rdd.flatMap(f) def mapPartitions(self, f, preservesPartitioning=False): - """ - Return a new RDD by applying a function to each partition. + """Returns a new :class:`RDD` by applying the ``f`` function to each partition. - It's a shorthand for df.rdd.mapPartitions() + This is a shorthand for ``df.rdd.mapPartitions()``. >>> rdd = sc.parallelize([1, 2, 3, 4], 4) >>> def f(iterator): yield 1 @@ -369,10 +348,9 @@ def mapPartitions(self, f, preservesPartitioning=False): return self.rdd.mapPartitions(f, preservesPartitioning) def foreach(self, f): - """ - Applies a function to all rows of this DataFrame. + """Applies the ``f`` function to all :class:`Row` of this :class:`DataFrame`. - It's a shorthand for df.rdd.foreach() + This is a shorthand for ``df.rdd.foreach()``. >>> def f(person): ... print person.name @@ -381,10 +359,9 @@ def foreach(self, f): return self.rdd.foreach(f) def foreachPartition(self, f): - """ - Applies a function to each partition of this DataFrame. + """Applies the ``f`` function to each partition of this :class:`DataFrame`. - It's a shorthand for df.rdd.foreachPartition() + This a shorthand for ``df.rdd.foreachPartition()``. >>> def f(people): ... for person in people: @@ -394,14 +371,14 @@ def foreachPartition(self, f): return self.rdd.foreachPartition(f) def cache(self): - """ Persist with the default storage level (C{MEMORY_ONLY_SER}). + """ Persists with the default storage level (C{MEMORY_ONLY_SER}). """ self.is_cached = True self._jdf.cache() return self def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): - """ Set the storage level to persist its values across operations + """Sets the storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). @@ -412,7 +389,7 @@ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): return self def unpersist(self, blocking=True): - """ Mark it as non-persistent, and remove all blocks for it from + """Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from memory and disk. """ self.is_cached = False @@ -424,8 +401,7 @@ def unpersist(self, blocking=True): # return DataFrame(rdd, self.sql_ctx) def repartition(self, numPartitions): - """ Return a new :class:`DataFrame` that has exactly `numPartitions` - partitions. + """Returns a new :class:`DataFrame` that has exactly ``numPartitions`` partitions. >>> df.repartition(10).rdd.getNumPartitions() 10 @@ -433,8 +409,7 @@ def repartition(self, numPartitions): return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx) def distinct(self): - """ - Return a new :class:`DataFrame` containing the distinct rows in this DataFrame. + """Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`. >>> df.distinct().count() 2L @@ -442,8 +417,7 @@ def distinct(self): return DataFrame(self._jdf.distinct(), self.sql_ctx) def sample(self, withReplacement, fraction, seed=None): - """ - Return a sampled subset of this DataFrame. + """Returns a sampled subset of this :class:`DataFrame`. >>> df.sample(False, 0.5, 97).count() 1L @@ -455,7 +429,7 @@ def sample(self, withReplacement, fraction, seed=None): @property def dtypes(self): - """Return all column names and their data types as a list. + """Returns all column names and their data types as a list. >>> df.dtypes [('age', 'int'), ('name', 'string')] @@ -464,7 +438,7 @@ def dtypes(self): @property def columns(self): - """ Return all column names as a list. + """Returns all column names as a list. >>> df.columns [u'age', u'name'] @@ -472,13 +446,14 @@ def columns(self): return [f.name for f in self.schema.fields] def join(self, other, joinExprs=None, joinType=None): - """ - Join with another :class:`DataFrame`, using the given join expression. - The following performs a full outer join between `df1` and `df2`. + """Joins with another :class:`DataFrame`, using the given join expression. + + The following performs a full outer join between ``df1`` and ``df2``. :param other: Right side of the join :param joinExprs: Join expression - :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. + :param joinType: str, default 'inner'. + One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] @@ -496,9 +471,9 @@ def join(self, other, joinExprs=None, joinType=None): return DataFrame(jdf, self.sql_ctx) def sort(self, *cols): - """ Return a new :class:`DataFrame` sorted by the specified column(s). + """Returns a new :class:`DataFrame` sorted by the specified column(s). - :param cols: The columns or expressions used for sorting + :param cols: list of :class:`Column` to sort by. >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] @@ -539,7 +514,9 @@ def describe(self, *cols): return DataFrame(jdf, self.sql_ctx) def head(self, n=None): - """ Return the first `n` rows or the first row if n is None. + """ + Returns the first ``n`` rows as a list of :class:`Row`, + or the first :class:`Row` if ``n`` is ``None.`` >>> df.head() Row(age=2, name=u'Alice') @@ -552,7 +529,7 @@ def head(self, n=None): return self.take(n) def first(self): - """ Return the first row. + """Returns the first row as a :class:`Row`. >>> df.first() Row(age=2, name=u'Alice') @@ -560,7 +537,7 @@ def first(self): return self.head() def __getitem__(self, item): - """ Return the column by given name + """Returns the column as a :class:`Column`. >>> df.select(df['age']).collect() [Row(age=2), Row(age=5)] @@ -580,7 +557,7 @@ def __getitem__(self, item): raise IndexError("unexpected index: %s" % item) def __getattr__(self, name): - """ Return the column by given name + """Returns the :class:`Column` denoted by ``name``. >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] @@ -591,7 +568,11 @@ def __getattr__(self, name): return Column(jc) def select(self, *cols): - """ Selecting a set of expressions. + """Projects a set of expressions and returns a new :class:`DataFrame`. + + :param cols: list of column names (string) or expressions (:class:`Column`). + If one of the column names is '*', that column is expanded to include all columns + in the current DataFrame. >>> df.select('*').collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -606,9 +587,9 @@ def select(self, *cols): return DataFrame(jdf, self.sql_ctx) def selectExpr(self, *expr): - """ - Selects a set of SQL expressions. This is a variant of - `select` that accepts SQL expressions. + """Projects a set of SQL expressions and returns a new :class:`DataFrame`. + + This is a variant of :func:`select` that accepts SQL expressions. >>> df.selectExpr("age * 2", "abs(age)").collect() [Row((age * 2)=4, Abs(age)=2), Row((age * 2)=10, Abs(age)=5)] @@ -618,10 +599,12 @@ def selectExpr(self, *expr): return DataFrame(jdf, self.sql_ctx) def filter(self, condition): - """ Filtering rows using the given condition, which could be - :class:`Column` expression or string of SQL expression. + """Filters rows using the given condition. + + :func:`where` is an alias for :func:`filter`. - where() is an alias for filter(). + :param condition: a :class:`Column` of :class:`types.BooleanType` + or a string of SQL expression. >>> df.filter(df.age > 3).collect() [Row(age=5, name=u'Bob')] @@ -644,10 +627,13 @@ def filter(self, condition): where = filter def groupBy(self, *cols): - """ Group the :class:`DataFrame` using the specified columns, + """Groups the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedData` for all the available aggregate functions. + :param cols: list of columns to group by. + Each element should be a column name (string) or an expression (:class:`Column`). + >>> df.groupBy().avg().collect() [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() @@ -662,7 +648,7 @@ def groupBy(self, *cols): def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups - (shorthand for df.groupBy.agg()). + (shorthand for ``df.groupBy.agg()``). >>> df.agg({"age": "max"}).collect() [Row(MAX(age)=5)] @@ -699,7 +685,7 @@ def subtract(self, other): def dropna(self, how='any', thresh=None, subset=None): """Returns a new :class:`DataFrame` omitting rows with null values. - This is an alias for `na.drop`. + This is an alias for ``na.drop()``. :param how: 'any' or 'all'. If 'any', drop a row if it contains any nulls. @@ -735,7 +721,7 @@ def dropna(self, how='any', thresh=None, subset=None): return DataFrame(self._jdf.na().drop(thresh, cols), self.sql_ctx) def fillna(self, value, subset=None): - """Replace null values, alias for `na.fill`. + """Replace null values, alias for ``na.fill()``. :param value: int, long, float, string, or dict. Value to replace null values with. @@ -790,7 +776,10 @@ def fillna(self, value, subset=None): return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) def withColumn(self, colName, col): - """ Return a new :class:`DataFrame` by adding a column. + """Returns a new :class:`DataFrame` by adding a column. + + :param colName: string, name of the new column. + :param col: a :class:`Column` expression for the new column. >>> df.withColumn('age2', df.age + 2).collect() [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] @@ -798,7 +787,10 @@ def withColumn(self, colName, col): return self.select('*', col.alias(colName)) def withColumnRenamed(self, existing, new): - """ Rename an existing column to a new name + """REturns a new :class:`DataFrame` by renaming an existing column. + + :param existing: string, name of the existing column to rename. + :param col: string, new name of the column. >>> df.withColumnRenamed('age', 'age2').collect() [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] @@ -809,8 +801,9 @@ def withColumnRenamed(self, existing, new): return self.select(*cols) def toPandas(self): - """ - Collect all the rows and return a `pandas.DataFrame`. + """Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``. + + This is only available if Pandas is installed and available. >>> df.toPandas() # doctest: +SKIP age name @@ -823,8 +816,7 @@ def toPandas(self): # Having SchemaRDD for backward compatibility (for docs) class SchemaRDD(DataFrame): - """ - SchemaRDD is deprecated, please use DataFrame + """SchemaRDD is deprecated, please use :class:`DataFrame`. """ @@ -851,10 +843,9 @@ def _api(self, *args): class GroupedData(object): - """ A set of methods for aggregations on a :class:`DataFrame`, - created by DataFrame.groupBy(). + created by :func:`DataFrame.groupBy`. """ def __init__(self, jdf, sql_ctx): @@ -862,14 +853,17 @@ def __init__(self, jdf, sql_ctx): self.sql_ctx = sql_ctx def agg(self, *exprs): - """ Compute aggregates by specifying a map from column name - to aggregate methods. + """Compute aggregates and returns the result as a :class:`DataFrame`. + + The available aggregate functions are `avg`, `max`, `min`, `sum`, `count`. + + If ``exprs`` is a single :class:`dict` mapping from string to string, then the key + is the column to perform aggregation on, and the value is the aggregate function. - The available aggregate methods are `avg`, `max`, `min`, - `sum`, `count`. + Alternatively, ``exprs`` can also be a list of aggregate :class:`Column` expressions. - :param exprs: list or aggregate columns or a map from column - name to aggregate methods. + :param exprs: a dict mapping from column name (string) to aggregate functions (string), + or a list of :class:`Column`. >>> gdf = df.groupBy(df.name) >>> gdf.agg({"*": "count"}).collect() @@ -894,7 +888,7 @@ def agg(self, *exprs): @dfapi def count(self): - """ Count the number of rows for each group. + """Counts the number of records for each group. >>> df.groupBy(df.age).count().collect() [Row(age=2, count=1), Row(age=5, count=1)] @@ -902,8 +896,11 @@ def count(self): @df_varargs_api def mean(self, *cols): - """Compute the average value for each numeric columns - for each group. This is an alias for `avg`. + """Computes average values for each numeric columns for each group. + + :func:`mean` is an alias for :func:`avg`. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().mean('age').collect() [Row(AVG(age)=3.5)] @@ -913,8 +910,11 @@ def mean(self, *cols): @df_varargs_api def avg(self, *cols): - """Compute the average value for each numeric columns - for each group. + """Computes average values for each numeric columns for each group. + + :func:`mean` is an alias for :func:`avg`. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().avg('age').collect() [Row(AVG(age)=3.5)] @@ -924,8 +924,7 @@ def avg(self, *cols): @df_varargs_api def max(self, *cols): - """Compute the max value for each numeric columns for - each group. + """Computes the max value for each numeric columns for each group. >>> df.groupBy().max('age').collect() [Row(MAX(age)=5)] @@ -935,8 +934,9 @@ def max(self, *cols): @df_varargs_api def min(self, *cols): - """Compute the min value for each numeric column for - each group. + """Computes the min value for each numeric column for each group. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().min('age').collect() [Row(MIN(age)=2)] @@ -946,8 +946,9 @@ def min(self, *cols): @df_varargs_api def sum(self, *cols): - """Compute the sum for each numeric columns for each - group. + """Compute the sum for each numeric columns for each group. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().sum('age').collect() [Row(SUM(age)=7)] diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5873f09ae3275..8a478fddf0e95 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -76,7 +76,7 @@ def _(col): def countDistinct(col, *cols): - """ Return a new Column for distinct count of `col` or `cols` + """Returns a new :class:`Column` for distinct count of ``col`` or ``cols``. >>> df.agg(countDistinct(df.age, df.name).alias('c')).collect() [Row(c=2)] @@ -91,7 +91,7 @@ def countDistinct(col, *cols): def approxCountDistinct(col, rsd=None): - """ Return a new Column for approximate distinct count of `col` + """Returns a new :class:`Column` for approximate distinct count of ``col``. >>> df.agg(approxCountDistinct(df.age).alias('c')).collect() [Row(c=2)] @@ -142,7 +142,7 @@ def __call__(self, *cols): def udf(f, returnType=StringType()): - """Create a user defined function (UDF) + """Creates a :class:`Column` expression representing a user defined function (UDF). >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 0169028ccc4eb..45eb8b945dcb0 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -33,8 +33,7 @@ class DataType(object): - - """Spark SQL DataType""" + """Base class for data types.""" def __repr__(self): return self.__class__.__name__ @@ -67,7 +66,6 @@ def json(self): # This singleton pattern does not work with pickle, you will get # another object after pickle and unpickle class PrimitiveTypeSingleton(type): - """Metaclass for PrimitiveType""" _instances = {} @@ -79,66 +77,45 @@ def __call__(cls): class PrimitiveType(DataType): - """Spark SQL PrimitiveType""" __metaclass__ = PrimitiveTypeSingleton class NullType(PrimitiveType): + """Null type. - """Spark SQL NullType - - The data type representing None, used for the types which has not - been inferred. + The data type representing None, used for the types that cannot be inferred. """ class StringType(PrimitiveType): - - """Spark SQL StringType - - The data type representing string values. + """String data type. """ class BinaryType(PrimitiveType): - - """Spark SQL BinaryType - - The data type representing bytearray values. + """Binary (byte array) data type. """ class BooleanType(PrimitiveType): - - """Spark SQL BooleanType - - The data type representing bool values. + """Boolean data type. """ class DateType(PrimitiveType): - - """Spark SQL DateType - - The data type representing datetime.date values. + """Date (datetime.date) data type. """ class TimestampType(PrimitiveType): - - """Spark SQL TimestampType - - The data type representing datetime.datetime values. + """Timestamp (datetime.datetime) data type. """ class DecimalType(DataType): - - """Spark SQL DecimalType - - The data type representing decimal.Decimal values. + """Decimal (decimal.Decimal) data type. """ def __init__(self, precision=None, scale=None): @@ -166,80 +143,55 @@ def __repr__(self): class DoubleType(PrimitiveType): - - """Spark SQL DoubleType - - The data type representing float values. + """Double data type, representing double precision floats. """ class FloatType(PrimitiveType): - - """Spark SQL FloatType - - The data type representing single precision floating-point values. + """Float data type, representing single precision floats. """ class ByteType(PrimitiveType): - - """Spark SQL ByteType - - The data type representing int values with 1 singed byte. + """Byte data type, i.e. a signed integer in a single byte. """ def simpleString(self): return 'tinyint' class IntegerType(PrimitiveType): - - """Spark SQL IntegerType - - The data type representing int values. + """Int data type, i.e. a signed 32-bit integer. """ def simpleString(self): return 'int' class LongType(PrimitiveType): + """Long data type, i.e. a signed 64-bit integer. - """Spark SQL LongType - - The data type representing long values. If the any value is - beyond the range of [-9223372036854775808, 9223372036854775807], - please use DecimalType. + If the values are beyond the range of [-9223372036854775808, 9223372036854775807], + please use :class:`DecimalType`. """ def simpleString(self): return 'bigint' class ShortType(PrimitiveType): - - """Spark SQL ShortType - - The data type representing int values with 2 signed bytes. + """Short data type, i.e. a signed 16-bit integer. """ def simpleString(self): return 'smallint' class ArrayType(DataType): + """Array data type. - """Spark SQL ArrayType - - The data type representing list values. An ArrayType object - comprises two fields, elementType (a DataType) and containsNull (a bool). - The field of elementType is used to specify the type of array elements. - The field of containsNull is used to specify if the array has None values. - + :param elementType: :class:`DataType` of each element in the array. + :param containsNull: boolean, whether the array can contain null (None) values. """ def __init__(self, elementType, containsNull=True): - """Creates an ArrayType - - :param elementType: the data type of elements. - :param containsNull: indicates whether the list contains None values. - + """ >>> ArrayType(StringType()) == ArrayType(StringType(), True) True >>> ArrayType(StringType(), False) == ArrayType(StringType()) @@ -268,29 +220,17 @@ def fromJson(cls, json): class MapType(DataType): + """Map data type. - """Spark SQL MapType - - The data type representing dict values. A MapType object comprises - three fields, keyType (a DataType), valueType (a DataType) and - valueContainsNull (a bool). - - The field of keyType is used to specify the type of keys in the map. - The field of valueType is used to specify the type of values in the map. - The field of valueContainsNull is used to specify if values of this - map has None values. - - For values of a MapType column, keys are not allowed to have None values. + :param keyType: :class:`DataType` of the keys in the map. + :param valueType: :class:`DataType` of the values in the map. + :param valueContainsNull: indicates whether values can contain null (None) values. + Keys in a map data type are not allowed to be null (None). """ def __init__(self, keyType, valueType, valueContainsNull=True): - """Creates a MapType - :param keyType: the data type of keys. - :param valueType: the data type of values. - :param valueContainsNull: indicates whether values contains - null values. - + """ >>> (MapType(StringType(), IntegerType()) ... == MapType(StringType(), IntegerType(), True)) True @@ -325,30 +265,16 @@ def fromJson(cls, json): class StructField(DataType): + """A field in :class:`StructType`. - """Spark SQL StructField - - Represents a field in a StructType. - A StructField object comprises three fields, name (a string), - dataType (a DataType) and nullable (a bool). The field of name - is the name of a StructField. The field of dataType specifies - the data type of a StructField. - - The field of nullable specifies if values of a StructField can - contain None values. - + :param name: string, name of the field. + :param dataType: :class:`DataType` of the field. + :param nullable: boolean, whether the field can be null (None) or not. + :param metadata: a dict from string to simple type that can be serialized to JSON automatically """ def __init__(self, name, dataType, nullable=True, metadata=None): - """Creates a StructField - :param name: the name of this field. - :param dataType: the data type of this field. - :param nullable: indicates whether values of this field - can be null. - :param metadata: metadata of this field, which is a map from string - to simple type that can be serialized to JSON - automatically - + """ >>> (StructField("f1", StringType(), True) ... == StructField("f1", StringType(), True)) True @@ -384,17 +310,13 @@ def fromJson(cls, json): class StructType(DataType): + """Struct type, consisting of a list of :class:`StructField`. - """Spark SQL StructType - - The data type representing rows. - A StructType object comprises a list of L{StructField}. - + This is the data type representing a :class:`Row`. """ def __init__(self, fields): - """Creates a StructType - + """ >>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 @@ -425,9 +347,9 @@ def fromJson(cls, json): class UserDefinedType(DataType): - """ + """User-defined type (UDT). + .. note:: WARN: Spark Internal Use Only - SQL User-Defined Type (UDT). """ @classmethod diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 3a3dc70f7285c..bf3c3fe876873 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -21,14 +21,17 @@ import java.{lang => jl} import scala.collection.JavaConversions._ +import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** + * :: Experimental :: * Functionality for working with missing data in [[DataFrame]]s. */ +@Experimental final class DataFrameNaFunctions private[sql](df: DataFrame) { /** From ff1915e12edc4d23e0b4e88933429c2d3470f3d9 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 1 Apr 2015 11:09:00 +0100 Subject: [PATCH 599/817] [SPARK-4655][Core] Split Stage into ShuffleMapStage and ResultStage subclasses Hi all - this patch changes the Stage class to an abstract class and introduces two new classes that extend it: ShuffleMapStage and ResultStage - with the goal of increasing readability of the DAGScheduler class. Their usage is updated within DAGScheduler. Author: Ilya Ganelin Author: Ilya Ganelin Closes #4708 from ilganeli/SPARK-4655 and squashes the following commits: c248924 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655 d930385 [Ilya Ganelin] Fixed merge conflict from a9a765f [Ilya Ganelin] Update DAGScheduler.scala c03563c [Ilya Ganelin] Minor fixeS c39e971 [Ilya Ganelin] Added return typing for public methods 845bc87 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655 e8031d8 [Ilya Ganelin] Minor string fixes 4ec53ac [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655 c004f62 [Ilya Ganelin] Update DAGScheduler.scala a2cb03f [Ilya Ganelin] [SPARK-4655] Replaced usages of Nil and eliminated some code reuse 3d5cf20 [Ilya Ganelin] [SPARK-4655] Moved mima exclude to 1.4 6912c55 [Ilya Ganelin] Resolved merge conflict 4bff208 [Ilya Ganelin] Minor stylistic fixes c6fffbb [Ilya Ganelin] newline 41402ad [Ilya Ganelin] Style fixes 02c6981 [Ilya Ganelin] Merge branch 'SPARK-4655' of github.com:ilganeli/spark into SPARK-4655 c755a09 [Ilya Ganelin] Some more stylistic updates and minor refactoring b6257a0 [Ilya Ganelin] Update MimaExcludes.scala 0f0c624 [Ilya Ganelin] Fixed merge conflict 2eba262 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655 6b43d7b [Ilya Ganelin] Got rid of some spaces 6f1a5db [Ilya Ganelin] Revert "More minor formatting and refactoring" 1b3471b [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-4655 c9288e2 [Ilya Ganelin] More minor formatting and refactoring d548caf [Ilya Ganelin] Formatting fix c3ae5c2 [Ilya Ganelin] Explicit typing 0dacaf3 [Ilya Ganelin] Got rid of stale import 6da3a71 [Ilya Ganelin] Trailing whitespace b85c5fe [Ilya Ganelin] Added minor fixes a57dfcd [Ilya Ganelin] Added MiMA exclusion to get around binary compatibility check 83ed849 [Ilya Ganelin] moved braces for consistency 96dd161 [Ilya Ganelin] Fixed minor style error cfd6f10 [Ilya Ganelin] Updated DAGScheduler to use new ResultStage and ShuffleMapStage classes 83494e9 [Ilya Ganelin] Added new Stage classes --- .../apache/spark/scheduler/ActiveJob.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 297 ++++++++++-------- .../apache/spark/scheduler/ResultStage.scala | 40 +++ .../spark/scheduler/ShuffleMapStage.scala | 84 +++++ .../org/apache/spark/scheduler/Stage.scala | 65 +--- project/MimaExcludes.scala | 6 +- 6 files changed, 298 insertions(+), 196 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index b755d8fb15757..50a69379412d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.CallSite */ private[spark] class ActiveJob( val jobId: Int, - val finalStage: Stage, + val finalStage: ResultStage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], val callSite: CallSite, diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b405bd3338e7c..d35b4f9dbaf88 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -83,7 +83,7 @@ class DAGScheduler( private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] + private[scheduler] val shuffleToMapStage = new HashMap[Int, ShuffleMapStage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] // Stages we need to run whose parents aren't done @@ -150,7 +150,7 @@ class DAGScheduler( result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { + taskMetrics: TaskMetrics): Unit = { eventProcessLoop.post( CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) } @@ -173,18 +173,18 @@ class DAGScheduler( } // Called by TaskScheduler when an executor fails. - def executorLost(execId: String) { + def executorLost(execId: String): Unit = { eventProcessLoop.post(ExecutorLost(execId)) } // Called by TaskScheduler when a host is added - def executorAdded(execId: String, host: String) { + def executorAdded(execId: String, host: String): Unit = { eventProcessLoop.post(ExecutorAdded(execId, host)) } // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or // cancellation of the job itself. - def taskSetFailed(taskSet: TaskSet, reason: String) { + def taskSetFailed(taskSet: TaskSet, reason: String): Unit = { eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } @@ -210,40 +210,65 @@ class DAGScheduler( * The jobId value passed in will be used if the stage doesn't already exist with * a lower jobId (jobId always increases across jobs.) */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): Stage = { + private def getShuffleMapStage( + shuffleDep: ShuffleDependency[_, _, _], + jobId: Int): ShuffleMapStage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => // We are going to register ancestor shuffle dependencies registerShuffleDependencies(shuffleDep, jobId) // Then register current shuffleDep - val stage = - newOrUsedStage( - shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, - shuffleDep.rdd.creationSite) + val stage = newOrUsedShuffleStage(shuffleDep, jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage - + stage } } /** - * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation - * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided - * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage - * directly. + * Helper function to eliminate some code re-use when creating new stages. */ - private def newStage( + private def getParentStagesAndId(rdd: RDD[_], jobId: Int): (List[Stage], Int) = { + val parentStages = getParentStages(rdd, jobId) + val id = nextStageId.getAndIncrement() + (parentStages, id) + } + + /** + * Create a ShuffleMapStage as part of the (re)-creation of a shuffle map stage in + * newOrUsedShuffleStage. The stage will be associated with the provided jobId. + * Production of shuffle map stages should always use newOrUsedShuffleStage, not + * newShuffleMapStage directly. + */ + private def newShuffleMapStage( rdd: RDD[_], numTasks: Int, - shuffleDep: Option[ShuffleDependency[_, _, _]], + shuffleDep: ShuffleDependency[_, _, _], jobId: Int, - callSite: CallSite) - : Stage = - { - val parentStages = getParentStages(rdd, jobId) - val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, numTasks, shuffleDep, parentStages, jobId, callSite) + callSite: CallSite): ShuffleMapStage = { + val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, jobId) + val stage: ShuffleMapStage = new ShuffleMapStage(id, rdd, numTasks, parentStages, + jobId, callSite, shuffleDep) + + stageIdToStage(id) = stage + updateJobIdStageIdMaps(jobId, stage) + stage + } + + /** + * Create a ResultStage -- either directly for use as a result stage, or as part of the + * (re)-creation of a shuffle map stage in newOrUsedShuffleStage. The stage will be associated + * with the provided jobId. + */ + private def newResultStage( + rdd: RDD[_], + numTasks: Int, + jobId: Int, + callSite: CallSite): ResultStage = { + val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, jobId) + val stage: ResultStage = new ResultStage(id, rdd, numTasks, parentStages, jobId, callSite) + stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage @@ -255,20 +280,17 @@ class DAGScheduler( * present in the MapOutputTracker, then the number and location of available outputs are * recovered from the MapOutputTracker */ - private def newOrUsedStage( - rdd: RDD[_], - numTasks: Int, + private def newOrUsedShuffleStage( shuffleDep: ShuffleDependency[_, _, _], - jobId: Int, - callSite: CallSite) - : Stage = - { - val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) + jobId: Int): ShuffleMapStage = { + val rdd = shuffleDep.rdd + val numTasks = rdd.partitions.size + val stage = newShuffleMapStage(rdd, numTasks, shuffleDep, jobId, rdd.creationSite) if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) for (i <- 0 until locs.size) { - stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing + stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing } stage.numAvailableOutputs = locs.count(_ != null) } else { @@ -306,26 +328,23 @@ class DAGScheduler( } } waitingForVisit.push(rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } parents.toList } - // Find ancestor missing shuffle dependencies and register into shuffleToMapStage - private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) = { + /** Find ancestor missing shuffle dependencies and register into shuffleToMapStage */ + private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) { val parentsWithNoMapStage = getAncestorShuffleDependencies(shuffleDep.rdd) - while (!parentsWithNoMapStage.isEmpty) { + while (parentsWithNoMapStage.nonEmpty) { val currentShufDep = parentsWithNoMapStage.pop() - val stage = - newOrUsedStage( - currentShufDep.rdd, currentShufDep.rdd.partitions.size, currentShufDep, jobId, - currentShufDep.rdd.creationSite) + val stage = newOrUsedShuffleStage(currentShufDep, jobId) shuffleToMapStage(currentShufDep.shuffleId) = stage } } - // Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet + /** Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet */ private def getAncestorShuffleDependencies(rdd: RDD[_]): Stack[ShuffleDependency[_, _, _]] = { val parents = new Stack[ShuffleDependency[_, _, _]] val visited = new HashSet[RDD[_]] @@ -351,7 +370,7 @@ class DAGScheduler( } waitingForVisit.push(rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } parents @@ -382,7 +401,7 @@ class DAGScheduler( } } waitingForVisit.push(stage.rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } missing.toList @@ -392,7 +411,7 @@ class DAGScheduler( * Registers the given jobId among the jobs that need the given stage and * all of that stage's ancestors. */ - private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { + private def updateJobIdStageIdMaps(jobId: Int, stage: Stage): Unit = { def updateJobIdStageIdMapsList(stages: List[Stage]) { if (stages.nonEmpty) { val s = stages.head @@ -412,7 +431,7 @@ class DAGScheduler( * * @param job The job whose state to cleanup. */ - private def cleanupStateForJobAndIndependentStages(job: ActiveJob) { + private def cleanupStateForJobAndIndependentStages(job: ActiveJob): Unit = { val registeredStages = jobIdToStageIds.get(job.jobId) if (registeredStages.isEmpty || registeredStages.get.isEmpty) { logError("No stages registered for job " + job.jobId) @@ -474,8 +493,7 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null): JobWaiter[U] = - { + properties: Properties = null): JobWaiter[U] = { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length partitions.find(p => p >= maxPartitions || p < 0).foreach { p => @@ -504,15 +522,13 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null) - { + properties: Properties = null): Unit = { val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { - case JobSucceeded => { + case JobSucceeded => logInfo("Job %d finished: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) - } case JobFailed(exception: Exception) => logInfo("Job %d failed: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) @@ -526,9 +542,7 @@ class DAGScheduler( evaluator: ApproximateEvaluator[U, R], callSite: CallSite, timeout: Long, - properties: Properties = null) - : PartialResult[R] = - { + properties: Properties = null): PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray @@ -541,12 +555,12 @@ class DAGScheduler( /** * Cancel a job that is running or waiting in the queue. */ - def cancelJob(jobId: Int) { + def cancelJob(jobId: Int): Unit = { logInfo("Asked to cancel job " + jobId) eventProcessLoop.post(JobCancelled(jobId)) } - def cancelJobGroup(groupId: String) { + def cancelJobGroup(groupId: String): Unit = { logInfo("Asked to cancel job group " + groupId) eventProcessLoop.post(JobGroupCancelled(groupId)) } @@ -554,7 +568,7 @@ class DAGScheduler( /** * Cancel all jobs that are running or waiting in the queue. */ - def cancelAllJobs() { + def cancelAllJobs(): Unit = { eventProcessLoop.post(AllJobsCancelled) } @@ -722,13 +736,12 @@ class DAGScheduler( allowLocal: Boolean, callSite: CallSite, listener: JobListener, - properties: Properties = null) - { - var finalStage: Stage = null + properties: Properties = null) { + var finalStage: ResultStage = null try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = newStage(finalRDD, partitions.size, None, jobId, callSite) + finalStage = newResultStage(finalRDD, partitions.size, jobId, callSite) } catch { case e: Exception => logWarning("Creating new stage failed due to exception - job: " + jobId, e) @@ -773,7 +786,7 @@ class DAGScheduler( if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) - if (missing == Nil) { + if (missing.isEmpty) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) } else { @@ -794,13 +807,15 @@ class DAGScheduler( // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() + // First figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = { - if (stage.isShuffleMap) { - (0 until stage.numPartitions).filter(id => stage.outputLocs(id) == Nil) - } else { - val job = stage.resultOfJob.get - (0 until job.numPartitions).filter(id => !job.finished(id)) + stage match { + case stage: ShuffleMapStage => + (0 until stage.numPartitions).filter(id => stage.outputLocs(id).isEmpty) + case stage: ResultStage => + val job = stage.resultOfJob.get + (0 until job.numPartitions).filter(id => !job.finished(id)) } } @@ -830,18 +845,21 @@ class DAGScheduler( try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = - if (stage.isShuffleMap) { - closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() - } else { - closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() - } + val taskBinaryBytes: Array[Byte] = stage match { + case stage: ShuffleMapStage => + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef).array() + case stage: ResultStage => + closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func): AnyRef).array() + } + taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage + + // Abort execution return case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") @@ -849,20 +867,22 @@ class DAGScheduler( return } - val tasks: Seq[Task[_]] = if (stage.isShuffleMap) { - partitionsToCompute.map { id => - val locs = getPreferredLocs(stage.rdd, id) - val part = stage.rdd.partitions(id) - new ShuffleMapTask(stage.id, taskBinary, part, locs) - } - } else { - val job = stage.resultOfJob.get - partitionsToCompute.map { id => - val p: Int = job.partitions(id) - val part = stage.rdd.partitions(p) - val locs = getPreferredLocs(stage.rdd, p) - new ResultTask(stage.id, taskBinary, part, locs, id) - } + val tasks: Seq[Task[_]] = stage match { + case stage: ShuffleMapStage => + partitionsToCompute.map { id => + val locs = getPreferredLocs(stage.rdd, id) + val part = stage.rdd.partitions(id) + new ShuffleMapTask(stage.id, taskBinary, part, locs) + } + + case stage: ResultStage => + val job = stage.resultOfJob.get + partitionsToCompute.map { id => + val p: Int = job.partitions(id) + val part = stage.rdd.partitions(p) + val locs = getPreferredLocs(stage.rdd, p) + new ResultTask(stage.id, taskBinary, part, locs, id) + } } if (tasks.size > 0) { @@ -877,8 +897,17 @@ class DAGScheduler( // SparkListenerStageCompleted here in case there are no tasks to run. outputCommitCoordinator.stageEnd(stage.id) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) - logDebug("Stage " + stage + " is actually done; %b %d %d".format( - stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) + + val debugString = stage match { + case stage: ShuffleMapStage => + s"Stage ${stage} is actually done; " + + s"(available: ${stage.isAvailable}," + + s"available outputs: ${stage.numAvailableOutputs}," + + s"partitions: ${stage.numPartitions})" + case stage : ResultStage => + s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" + } + logDebug(debugString) runningStages -= stage } } @@ -968,7 +997,10 @@ class DAGScheduler( stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => - stage.resultOfJob match { + // Cast to ResultStage here because it's part of the ResultTask + // TODO Refactor this out to a function that accepts a ResultStage + val resultStage = stage.asInstanceOf[ResultStage] + resultStage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { updateAccumulators(event) @@ -976,7 +1008,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - markStageAsFinished(stage) + markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) listenerBus.post( SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) @@ -988,7 +1020,7 @@ class DAGScheduler( job.listener.taskSucceeded(rt.outputId, event.result) } catch { case e: Exception => - // TODO: Perhaps we want to mark the stage as failed? + // TODO: Perhaps we want to mark the resultStage as failed? job.listener.jobFailed(new SparkDriverExecutionException(e)) } } @@ -997,6 +1029,7 @@ class DAGScheduler( } case smt: ShuffleMapTask => + val shuffleStage = stage.asInstanceOf[ShuffleMapStage] updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId @@ -1004,50 +1037,54 @@ class DAGScheduler( if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { - stage.addOutputLoc(smt.partitionId, status) + shuffleStage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(stage) && stage.pendingTasks.isEmpty) { - markStageAsFinished(stage) + if (runningStages.contains(shuffleStage) && shuffleStage.pendingTasks.isEmpty) { + markStageAsFinished(shuffleStage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) logInfo("waiting: " + waitingStages) logInfo("failed: " + failedStages) - if (stage.shuffleDep.isDefined) { - // We supply true to increment the epoch number here in case this is a - // recomputation of the map outputs. In that case, some nodes may have cached - // locations with holes (from when we detected the error) and will need the - // epoch incremented to refetch them. - // TODO: Only increment the epoch number if this is not the first time - // we registered these map outputs. - mapOutputTracker.registerMapOutputs( - stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true) - } + + // We supply true to increment the epoch number here in case this is a + // recomputation of the map outputs. In that case, some nodes may have cached + // locations with holes (from when we detected the error) and will need the + // epoch incremented to refetch them. + // TODO: Only increment the epoch number if this is not the first time + // we registered these map outputs. + mapOutputTracker.registerMapOutputs( + shuffleStage.shuffleDep.shuffleId, + shuffleStage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = true) + clearCacheLocs() - if (stage.outputLocs.exists(_ == Nil)) { - // Some tasks had failed; let's resubmit this stage + if (shuffleStage.outputLocs.contains(Nil)) { + // Some tasks had failed; let's resubmit this shuffleStage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " (" + stage.name + + logInfo("Resubmitting " + shuffleStage + " (" + shuffleStage.name + ") because some of its tasks had failed: " + - stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) - submitStage(stage) + shuffleStage.outputLocs.zipWithIndex.filter(_._1.isEmpty) + .map(_._2).mkString(", ")) + submitStage(shuffleStage) } else { val newlyRunnable = new ArrayBuffer[Stage] - for (stage <- waitingStages) { - logInfo("Missing parents for " + stage + ": " + getMissingParentStages(stage)) + for (shuffleStage <- waitingStages) { + logInfo("Missing parents for " + shuffleStage + ": " + + getMissingParentStages(shuffleStage)) } - for (stage <- waitingStages if getMissingParentStages(stage) == Nil) { - newlyRunnable += stage + for (shuffleStage <- waitingStages if getMissingParentStages(shuffleStage).isEmpty) + { + newlyRunnable += shuffleStage } waitingStages --= newlyRunnable runningStages ++= newlyRunnable for { - stage <- newlyRunnable.sortBy(_.id) - jobId <- activeJobForStage(stage) + shuffleStage <- newlyRunnable.sortBy(_.id) + jobId <- activeJobForStage(shuffleStage) } { - logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable") - submitMissingTasks(stage, jobId) + logInfo("Submitting " + shuffleStage + " (" + + shuffleStage.rdd + "), which is now runnable") + submitMissingTasks(shuffleStage, jobId) } } } @@ -1204,9 +1241,7 @@ class DAGScheduler( } } - /** - * Fails a job and all stages that are only used by that job, and cleans up relevant state. - */ + /** Fails a job and all stages that are only used by that job, and cleans up relevant state. */ private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { val error = new SparkException(failureReason) var ableToCancelStages = true @@ -1254,9 +1289,7 @@ class DAGScheduler( } } - /** - * Return true if one of stage's ancestors is target. - */ + /** Return true if one of stage's ancestors is target. */ private def stageDependsOn(stage: Stage, target: Stage): Boolean = { if (stage == target) { return true @@ -1282,7 +1315,7 @@ class DAGScheduler( } } waitingForVisit.push(stage.rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } visitedRdds.contains(target.rdd) @@ -1312,9 +1345,7 @@ class DAGScheduler( private def getPreferredLocsInternal( rdd: RDD[_], partition: Int, - visited: HashSet[(RDD[_],Int)]) - : Seq[TaskLocation] = - { + visited: HashSet[(RDD[_],Int)]): Seq[TaskLocation] = { // If the partition has already been visited, no need to re-visit. // This avoids exponential path exploration. SPARK-695 if (!visited.add((rdd,partition))) { @@ -1323,12 +1354,12 @@ class DAGScheduler( } // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) - if (!cached.isEmpty) { + if (cached.nonEmpty) { return cached } // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList - if (!rddPrefs.isEmpty) { + if (rddPrefs.nonEmpty) { return rddPrefs.map(TaskLocation(_)) } // If the RDD has narrow dependencies, pick the first partition of the first narrow dep @@ -1412,7 +1443,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler dagScheduler.sc.stop() } - override def onStop() { + override def onStop(): Unit = { // Cancel any active jobs in postStop hook dagScheduler.cleanUpAfterSchedulerStop() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala new file mode 100644 index 0000000000000..c0f3d5a13d623 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.rdd.RDD +import org.apache.spark.util.CallSite + +/** + * The ResultStage represents the final stage in a job. + */ +private[spark] class ResultStage( + id: Int, + rdd: RDD[_], + numTasks: Int, + parents: List[Stage], + jobId: Int, + callSite: CallSite) + extends Stage(id, rdd, numTasks, parents, jobId, callSite) { + + // The active job for this result stage. Will be empty if the job has already finished + // (e.g., because the job was cancelled). + var resultOfJob: Option[ActiveJob] = None + + override def toString: String = "ResultStage " + id +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala new file mode 100644 index 0000000000000..d02210743484c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.ShuffleDependency +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.CallSite + +/** + * The ShuffleMapStage represents the intermediate stages in a job. + */ +private[spark] class ShuffleMapStage( + id: Int, + rdd: RDD[_], + numTasks: Int, + parents: List[Stage], + jobId: Int, + callSite: CallSite, + val shuffleDep: ShuffleDependency[_, _, _]) + extends Stage(id, rdd, numTasks, parents, jobId, callSite) { + + override def toString: String = "ShuffleMapStage " + id + + var numAvailableOutputs: Long = 0 + + def isAvailable: Boolean = numAvailableOutputs == numPartitions + + val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) + + def addOutputLoc(partition: Int, status: MapStatus): Unit = { + val prevList = outputLocs(partition) + outputLocs(partition) = status :: prevList + if (prevList == Nil) { + numAvailableOutputs += 1 + } + } + + def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { + val prevList = outputLocs(partition) + val newList = prevList.filterNot(_.location == bmAddress) + outputLocs(partition) = newList + if (prevList != Nil && newList == Nil) { + numAvailableOutputs -= 1 + } + } + + /** + * Removes all shuffle outputs associated with this executor. Note that this will also remove + * outputs which are served by an external shuffle server (if one exists), as they are still + * registered with this execId. + */ + def removeOutputsOnExecutor(execId: String): Unit = { + var becameUnavailable = false + for (partition <- 0 until numPartitions) { + val prevList = outputLocs(partition) + val newList = prevList.filterNot(_.location.executorId == execId) + outputLocs(partition) = newList + if (prevList != Nil && newList == Nil) { + becameUnavailable = true + numAvailableOutputs -= 1 + } + } + if (becameUnavailable) { + logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( + this, execId, numAvailableOutputs, numPartitions, isAvailable)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 4cbc6e84a6bdd..5d0ddb8377c33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -21,7 +21,6 @@ import scala.collection.mutable.HashSet import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.CallSite /** @@ -47,29 +46,23 @@ import org.apache.spark.util.CallSite * be updated for each attempt. * */ -private[spark] class Stage( +private[spark] abstract class Stage( val id: Int, val rdd: RDD[_], val numTasks: Int, - val shuffleDep: Option[ShuffleDependency[_, _, _]], // Output shuffle if stage is a map stage val parents: List[Stage], val jobId: Int, val callSite: CallSite) extends Logging { - val isShuffleMap = shuffleDep.isDefined val numPartitions = rdd.partitions.size - val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) - var numAvailableOutputs = 0 /** Set of jobs that this stage belongs to. */ val jobIds = new HashSet[Int] - /** For stages that are the final (consists of only ResultTasks), link to the ActiveJob. */ - var resultOfJob: Option[ActiveJob] = None var pendingTasks = new HashSet[Task[_]] - private var nextAttemptId = 0 + private var nextAttemptId: Int = 0 val name = callSite.shortForm val details = callSite.longForm @@ -77,53 +70,6 @@ private[spark] class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) - def isAvailable: Boolean = { - if (!isShuffleMap) { - true - } else { - numAvailableOutputs == numPartitions - } - } - - def addOutputLoc(partition: Int, status: MapStatus) { - val prevList = outputLocs(partition) - outputLocs(partition) = status :: prevList - if (prevList == Nil) { - numAvailableOutputs += 1 - } - } - - def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location == bmAddress) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - numAvailableOutputs -= 1 - } - } - - /** - * Removes all shuffle outputs associated with this executor. Note that this will also remove - * outputs which are served by an external shuffle server (if one exists), as they are still - * registered with this execId. - */ - def removeOutputsOnExecutor(execId: String) { - var becameUnavailable = false - for (partition <- 0 until numPartitions) { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location.executorId == execId) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - becameUnavailable = true - numAvailableOutputs -= 1 - } - } - if (becameUnavailable) { - logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( - this, execId, numAvailableOutputs, numPartitions, isAvailable)) - } - } - /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { val id = nextAttemptId @@ -133,11 +79,8 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - override def toString: String = "Stage " + id - - override def hashCode(): Int = id - - override def equals(other: Any): Boolean = other match { + override final def hashCode(): Int = id + override final def equals(other: Any): Boolean = other match { case stage: Stage => stage != null && stage.id == id case _ => false } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index efd59a7e5470f..54500f7c2701f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,7 +54,11 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.scheduler.OutputCommitCoordinator$OutputCommitCoordinatorActor") ) ++ Seq( - // SPARK-6510 Add a Graph#minus method acting as Set#difference + // SPARK-4655 - Making Stage an Abstract class broke binary compatility even though + // the stage class is defined as private[spark] + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.scheduler.Stage") + ) ++ Seq( + // SPARK-6510 Add a Graph#minus method acting as Set#difference ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") ) From 412262346f6f48e641bd6899c703efa31aeaba1e Mon Sep 17 00:00:00 2001 From: Florian Verhein Date: Wed, 1 Apr 2015 11:10:43 +0100 Subject: [PATCH 600/817] [EC2] [SPARK-6600] Open ports in ec2/spark_ec2.py to allow HDFS NFS gateway Authorizes incoming access to master on the ports required to use the hadoop hdfs nfs gateway from outside the cluster. Author: Florian Verhein Closes #5257 from florianverhein/master and squashes the following commits: 72a586a [Florian Verhein] [EC2] [SPARK-6600] initial impl --- ec2/spark_ec2.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index c467cd08ed742..5507a9c5a4733 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -456,6 +456,13 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50070, 50070, authorized_address) master_group.authorize('tcp', 60070, 60070, authorized_address) master_group.authorize('tcp', 4040, 4045, authorized_address) + # HDFS NFS gateway requires 111,2049,4242 for tcp & udp + master_group.authorize('tcp', 111, 111, authorized_address) + master_group.authorize('udp', 111, 111, authorized_address) + master_group.authorize('tcp', 2049, 2049, authorized_address) + master_group.authorize('udp', 2049, 2049, authorized_address) + master_group.authorize('tcp', 4242, 4242, authorized_address) + master_group.authorize('udp', 4242, 4242, authorized_address) if opts.ganglia: master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created From d824c11c9fe8af1ca1d7c694b2fb81289eb83f97 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 1 Apr 2015 11:11:56 +0100 Subject: [PATCH 601/817] [SPARK-6597][Minor] Replace `input:checkbox` with `input[type="checkbox"]` in additional-metrics.js In additional-metrics.js, there are some selector notation like `input:checkbox` but JQuery's official document says `input[type="checkbox"]` is better. https://api.jquery.com/checkbox-selector/ Author: Kousuke Saruta Closes #5254 from sarutak/SPARK-6597 and squashes the following commits: a253bc4 [Kousuke Saruta] Replaced input:checkbox with input[type="checkbox"] --- .../org/apache/spark/ui/static/additional-metrics.js | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index 14ba37d7c9bd9..013db8df9b363 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -30,7 +30,7 @@ $(function() { stripeSummaryTable(); - $("input:checkbox").click(function() { + $('input[type="checkbox"]').click(function() { var column = "table ." + $(this).attr("name"); $(column).toggle(); stripeSummaryTable(); @@ -39,15 +39,15 @@ $(function() { $("#select-all-metrics").click(function() { if (this.checked) { // Toggle all un-checked options. - $('input:checkbox:not(:checked)').trigger('click'); + $('input[type="checkbox"]:not(:checked)').trigger('click'); } else { // Toggle all checked options. - $('input:checkbox:checked').trigger('click'); + $('input[type="checkbox"]:checked').trigger('click'); } }); // Trigger a click on the checkbox if a user clicks the label next to it. $("span.additional-metric-title").click(function() { - $(this).parent().find('input:checkbox').trigger('click'); + $(this).parent().find('input[type="checkbox"]').trigger('click'); }); }); From 0358b08db85b3ee4ae70834626e7a42311bcc635 Mon Sep 17 00:00:00 2001 From: jayson Date: Wed, 1 Apr 2015 11:12:55 +0100 Subject: [PATCH 602/817] SPARK-6626 [DOCS]: Corrected Scala:TwitterUtils parameters Per Sean Owen's request, here is the update call for TwitterUtils using Scala :) Author: jayson Closes #5295 from JaysonSunshine/master and squashes the following commits: df1d056 [jayson] Corrected Scala:TwitterUtils parameters --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6d6229625f3f9..262512a639046 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -704,7 +704,7 @@ create a DStream using data from Twitter's stream of tweets, you have to do the {% highlight scala %} import org.apache.spark.streaming.twitter._ -TwitterUtils.createStream(ssc) +TwitterUtils.createStream(ssc, None) {% endhighlight %}
    From d36c5fca7b9227c4c6e1b0c1455269b5fd8d4852 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 1 Apr 2015 21:34:45 +0800 Subject: [PATCH 603/817] [SPARK-6608] [SQL] Makes DataFrame.rdd a lazy val Before 1.3.0, `SchemaRDD.id` works as a unique identifier of each `SchemaRDD`. In 1.3.0, unlike `SchemaRDD`, `DataFrame` is no longer an RDD, and `DataFrame.rdd` is actually a function which always returns a new RDD instance. Making `DataFrame.rdd` a lazy val should bring the unique identifier back. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5265) Author: Cheng Lian Closes #5265 from liancheng/spark-6608 and squashes the following commits: 7500968 [Cheng Lian] Updates javadoc 7f37d21 [Cheng Lian] Makes DataFrame.rdd a lazy val --- .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 5cd0a18ff688c..19cfa15f27b09 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 @@ -952,10 +952,12 @@ class DataFrame private[sql]( ///////////////////////////////////////////////////////////////////////////// /** - * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. + * Represents the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. Note that the RDD is + * memoized. Once called, it won't change even if you change any query planning related Spark SQL + * configurations (e.g. `spark.sql.shuffle.partitions`). * @group rdd */ - def rdd: RDD[Row] = { + lazy val rdd: RDD[Row] = { // use a local variable to make sure the map closure doesn't capture the whole DataFrame val schema = this.schema queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) From ee11be258251adf900680927ba200bf46512cc04 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 1 Apr 2015 16:26:54 +0100 Subject: [PATCH 604/817] SPARK-6433 hive tests to import spark-sql test JAR for QueryTest access 1. Test JARs are built & published 1. log4j.resources is explicitly excluded. Without this, downstream test run logging depends on the order the JARs are listed/loaded 1. sql/hive pulls in spark-sql &...spark-catalyst for its test runs 1. The copied in test classes were rm'd, and a test edited to remove its now duplicate assert method 1. Spark streaming is now build with the same plugin/phase as the rest, but its shade plugin declaration is kept in (so different from the rest of the test plugins). Due to (#2), this means the test JAR no longer includes its log4j file. Outstanding issues: * should the JARs be shaded? `spark-streaming-test.jar` does, but given these are test jars for developers only, especially in the same spark source tree, it's hard to justify. * `maven-jar-plugin` v 2.6 was explicitly selected; without this the apache-1.4 parent template JAR version (2.4) chosen. * Are there any other resources to exclude? Author: Steve Loughran Closes #5119 from steveloughran/stevel/patches/SPARK-6433-test-jars and squashes the following commits: 81ceb01 [Steve Loughran] SPARK-6433 add a clearer comment explaining what the plugin is doing & why a6dca33 [Steve Loughran] SPARK-6433 : pull configuration section form archive plugin c2b5f89 [Steve Loughran] SPARK-6433 omit "jar" goal from jar plugin fdac51b [Steve Loughran] SPARK-6433 -002; indentation & delegate plugin version to parent 650f442 [Steve Loughran] SPARK-6433 patch 001: test JARs are built; sql/hive pulls in spark-sql & spark-catalyst for its test runs --- pom.xml | 20 +++ sql/hive/pom.xml | 14 ++ .../org/apache/spark/sql/QueryTest.scala | 140 ------------------ .../spark/sql/catalyst/plans/PlanTest.scala | 57 ------- .../spark/sql/hive/CachedTableSuite.scala | 15 -- streaming/pom.xml | 28 ---- 6 files changed, 34 insertions(+), 240 deletions(-) delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala diff --git a/pom.xml b/pom.xml index 3eb3da2cd8af3..42bd926a2fcb8 100644 --- a/pom.xml +++ b/pom.xml @@ -1265,6 +1265,7 @@ create-source-jar jar-no-fork + test-jar-no-fork @@ -1473,6 +1474,25 @@ org.scalatest scalatest-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + prepare-package + + test-jar + + + + log4j.properties + + + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a9816f6c38cd2..04440076a26a3 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -89,6 +89,20 @@ junit test + + org.apache.spark + spark-sql_${scala.binary.version} + test-jar + ${project.version} + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + ${project.version} + test + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala deleted file mode 100644 index 0270e63557963..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ /dev/null @@ -1,140 +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 - -import scala.collection.JavaConversions._ - -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.util._ - - -/** - * *** DUPLICATED FROM sql/core. *** - * - * It is hard to have maven allow one subproject depend on another subprojects test code. - * So, we duplicate this code here. - */ -class QueryTest extends PlanTest { - - /** - * Runs the plan and makes sure the answer contains all of the keywords, or the - * none of keywords are listed in the answer - * @param rdd the [[DataFrame]] to be executed - * @param exists true for make sure the keywords are listed in the output, otherwise - * to make sure none of the keyword are not listed in the output - * @param keywords keyword in string array - */ - def checkExistence(rdd: DataFrame, exists: Boolean, keywords: String*) { - val outputs = rdd.collect().map(_.mkString).mkString - for (key <- keywords) { - if (exists) { - assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") - } else { - assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") - } - } - } - - /** - * Runs the plan and makes sure the answer matches the expected result. - * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. - */ - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { - QueryTest.checkAnswer(rdd, expectedAnswer) match { - case Some(errorMessage) => fail(errorMessage) - case None => - } - } - - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { - checkAnswer(rdd, Seq(expectedAnswer)) - } - - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { - test(sqlString) { - checkAnswer(sqlContext.sql(sqlString), expectedAnswer) - } - } -} - -object QueryTest { - /** - * Runs the plan and makes sure the answer matches the expected result. - * If there was exception during the execution or the contents of the DataFrame does not - * match the expected result, an error message will be returned. Otherwise, a [[None]] will - * be returned. - * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. - */ - def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { - val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty - def prepareAnswer(answer: Seq[Row]): Seq[Row] = { - // Converts data to types that we can do equality comparison using Scala collections. - // For BigDecimal type, the Scala type has a better definition of equality test (similar to - // Java's java.math.BigDecimal.compareTo). - val converted: Seq[Row] = answer.map { s => - Row.fromSeq(s.toSeq.map { - case d: java.math.BigDecimal => BigDecimal(d) - case o => o - }) - } - if (!isSorted) converted.sortBy(_.toString) else converted - } - val sparkAnswer = try rdd.collect().toSeq catch { - case e: Exception => - val errorMessage = - s""" - |Exception thrown while executing query: - |${rdd.queryExecution} - |== Exception == - |$e - |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} - """.stripMargin - return Some(errorMessage) - } - - if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - val errorMessage = - s""" - |Results do not match for query: - |${rdd.logicalPlan} - |== Analyzed Plan == - |${rdd.queryExecution.analyzed} - |== Physical Plan == - |${rdd.queryExecution.executedPlan} - |== Results == - |${sideBySide( - s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} - """.stripMargin - return Some(errorMessage) - } - - return None - } - - def checkAnswer(rdd: DataFrame, expectedAnswer: java.util.List[Row]): String = { - checkAnswer(rdd, expectedAnswer.toSeq) match { - case Some(errorMessage) => errorMessage - case None => null - } - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala deleted file mode 100644 index 98f1c0e69e29d..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ /dev/null @@ -1,57 +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.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util._ -import org.scalatest.FunSuite - -/** - * *** DUPLICATED FROM sql/catalyst/plans. *** - * - * It is hard to have maven allow one subproject depend on another subprojects test code. - * So, we duplicate this code here. - */ -class PlanTest extends FunSuite { - - /** - * Since attribute references are given globally unique ids during analysis, - * we must normalize them to check if two different queries are identical. - */ - protected def normalizeExprIds(plan: LogicalPlan) = { - plan transformAllExpressions { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) - case a: Alias => - Alias(a.child, a.name)(exprId = ExprId(0)) - } - } - - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) - fail( - s""" - |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 221a0c263d36c..c188264072a84 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -24,21 +24,6 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} import org.apache.spark.storage.RDDBlockId class CachedTableSuite extends QueryTest { - /** - * Throws a test failed exception when the number of cached tables differs from the expected - * number. - */ - def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = { - val planWithCaching = query.queryExecution.withCachedData - val cachedData = planWithCaching collect { - case cached: InMemoryRelation => cached - } - - assert( - cachedData.size == numCachedTables, - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + - planWithCaching) - } def rddIdOf(tableName: String): Int = { val executedPlan = table(tableName).queryExecution.executedPlan diff --git a/streaming/pom.xml b/streaming/pom.xml index 23a8358d45c2a..5ca55a4f680bb 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -97,34 +97,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - test-jar-on-test-compile - test-compile - - test-jar - - - - - org.apache.maven.plugins maven-shade-plugin From 2275acce7ba5fac83c58554d7ee9f4c7f3e866cf Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 1 Apr 2015 13:29:04 -0700 Subject: [PATCH 605/817] [SPARK-6651][MLLIB] delegate dense vector arithmetics to the underlying numpy array Users should be able to use numpy operators directly on dense vectors. davies atalwalkar Author: Xiangrui Meng Closes #5312 from mengxr/SPARK-6651 and squashes the following commits: e665c5c [Xiangrui Meng] wrap the result in a dense vector 23dfca3 [Xiangrui Meng] delegate dense vector arithmetics to the underlying numpy array --- python/pyspark/mllib/linalg.py | 38 +++++++++++++++++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index f5aad28afda0f..8b791ff6a7877 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -173,7 +173,24 @@ def toArray(self): class DenseVector(Vector): """ - A dense vector represented by a value array. + A dense vector represented by a value array. We use numpy array for + storage and arithmetics will be delegated to the underlying numpy + array. + + >>> v = Vectors.dense([1.0, 2.0]) + >>> u = Vectors.dense([3.0, 4.0]) + >>> v + u + DenseVector([4.0, 6.0]) + >>> 2 - v + DenseVector([1.0, 0.0]) + >>> v / 2 + DenseVector([0.5, 1.0]) + >>> v * u + DenseVector([3.0, 8.0]) + >>> u / v + DenseVector([3.0, 2.0]) + >>> u % 2 + DenseVector([1.0, 0.0]) """ def __init__(self, ar): if isinstance(ar, basestring): @@ -292,6 +309,25 @@ def __ne__(self, other): def __getattr__(self, item): return getattr(self.array, item) + def _delegate(op): + def func(self, other): + if isinstance(other, DenseVector): + other = other.array + return DenseVector(getattr(self.array, op)(other)) + return func + + __neg__ = _delegate("__neg__") + __add__ = _delegate("__add__") + __sub__ = _delegate("__sub__") + __mul__ = _delegate("__mul__") + __div__ = _delegate("__div__") + __mod__ = _delegate("__mod__") + __radd__ = _delegate("__radd__") + __rsub__ = _delegate("__rsub__") + __rmul__ = _delegate("__rmul__") + __rdiv__ = _delegate("__rdiv__") + __rmod__ = _delegate("__rmod__") + class SparseVector(Vector): """ From fb25e8c7f45b4f96561e3f7434a0f4dfce8ddefe Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 1 Apr 2015 15:15:47 -0700 Subject: [PATCH 606/817] [SPARK-6657] [Python] [Docs] fixed python doc build warnings fixed python doc build warnings CC whomever wants to review: rxin mengxr davies Author: Joseph K. Bradley Closes #5317 from jkbradley/python-doc-warnings and squashes the following commits: 4cd43c2 [Joseph K. Bradley] fixed python doc build warnings --- python/docs/pyspark.streaming.rst | 2 +- python/pyspark/mllib/tree.py | 26 ++++++++++---------------- 2 files changed, 11 insertions(+), 17 deletions(-) diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index 7890d9dcaac21..50822c93faba1 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -10,7 +10,7 @@ Module contents :show-inheritance: pyspark.streaming.kafka module ----------------------------- +------------------------------ .. automodule:: pyspark.streaming.kafka :members: :undoc-members: diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index bf288d76447bd..a7a4d2aaf855b 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -286,21 +286,18 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, :param numTrees: Number of trees in the random forest. :param featureSubsetStrategy: Number of features to consider for splits at each node. - Supported: "auto" (default), "all", "sqrt", "log2", - "onethird". - If "auto" is set, this parameter is set based on - numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "sqrt". - :param impurity: Criterion used for information gain - calculation. + Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + If "auto" is set, this parameter is set based on numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "sqrt". + :param impurity: Criterion used for information gain calculation. Supported values: "gini" (recommended) or "entropy". :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. (default: 4) :param maxBins: maximum number of bins used for splitting features - (default: 100) + (default: 100) :param seed: Random seed for bootstrapping and choosing feature subsets. :return: RandomForestModel that can be used for prediction @@ -365,13 +362,10 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt :param numTrees: Number of trees in the random forest. :param featureSubsetStrategy: Number of features to consider for splits at each node. - Supported: "auto" (default), "all", "sqrt", "log2", - "onethird". - If "auto" is set, this parameter is set based on - numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "onethird" for - regression. + Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + If "auto" is set, this parameter is set based on numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "onethird" for regression. :param impurity: Criterion used for information gain calculation. Supported values: "variance". From f084c5de14eb10a6aba82a39e03e7877926ebb9e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 1 Apr 2015 16:06:11 -0700 Subject: [PATCH 607/817] [SPARK-6578] [core] Fix thread-safety issue in outbound path of network library. While the inbound path of a netty pipeline is thread-safe, the outbound path is not. That means that multiple threads can compete to write messages to the next stage of the pipeline. The network library sometimes breaks a single RPC message into multiple buffers internally to avoid copying data (see MessageEncoder). This can result in the following scenario (where "FxBy" means "frame x, buffer y"): T1 F1B1 F1B2 \ \ \ \ socket F1B1 F2B1 F1B2 F2B2 / / / / T2 F2B1 F2B2 And the frames now cannot be rebuilt on the receiving side because the different messages have been mixed up on the wire. The fix wraps these multi-buffer messages into a `FileRegion` object so that these messages are written "atomically" to the next pipeline handler. Author: Marcelo Vanzin Closes #5234 from vanzin/SPARK-6578 and squashes the following commits: 16b2d70 [Marcelo Vanzin] Forgot to update a type. c9c2e4e [Marcelo Vanzin] Review comments: simplify some code. 9c888ac [Marcelo Vanzin] Small style nits. 8474bab [Marcelo Vanzin] Fix multiple calls to MessageWithHeader.transferTo(). e26509f [Marcelo Vanzin] Merge branch 'master' into SPARK-6578 c503f6c [Marcelo Vanzin] Implement a custom FileRegion instead of using locks. 84aa7ce [Marcelo Vanzin] Rename handler to the correct name. 432f3bd [Marcelo Vanzin] Remove unneeded method. 8d70e60 [Marcelo Vanzin] Fix thread-safety issue in outbound path of network library. --- network/common/pom.xml | 5 + .../network/protocol/MessageEncoder.java | 6 +- .../network/protocol/MessageWithHeader.java | 106 ++++++++++++++ .../network/ByteArrayWritableChannel.java | 55 ++++++++ .../apache/spark/network/ProtocolSuite.java | 46 +++++-- .../protocol/MessageWithHeaderSuite.java | 129 ++++++++++++++++++ .../src/test/resources/log4j.properties | 27 ++++ 7 files changed, 364 insertions(+), 10 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java create mode 100644 network/common/src/test/java/org/apache/spark/network/ByteArrayWritableChannel.java create mode 100644 network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java create mode 100644 network/common/src/test/resources/log4j.properties diff --git a/network/common/pom.xml b/network/common/pom.xml index 7b51845206f4a..22c738bde6d42 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -80,6 +80,11 @@ mockito-all test + + org.slf4j + slf4j-log4j12 + test + diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 91d1e8a538a77..0f999f5dfe8d8 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -72,9 +72,11 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) { in.encode(header); assert header.writableBytes() == 0; - out.add(header); if (body != null && bodyLength > 0) { - out.add(body); + out.add(new MessageWithHeader(header, body, bodyLength)); + } else { + out.add(header); } } + } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java new file mode 100644 index 0000000000000..215a8517e8608 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -0,0 +1,106 @@ +/* + * 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.network.protocol; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import io.netty.buffer.ByteBuf; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import io.netty.util.ReferenceCountUtil; + +/** + * A wrapper message that holds two separate pieces (a header and a body) to avoid + * copying the body's content. + */ +class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { + + private final ByteBuf header; + private final int headerLength; + private final Object body; + private final long bodyLength; + private long totalBytesTransferred; + + MessageWithHeader(ByteBuf header, Object body, long bodyLength) { + Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, + "Body must be a ByteBuf or a FileRegion."); + this.header = header; + this.headerLength = header.readableBytes(); + this.body = body; + this.bodyLength = bodyLength; + } + + @Override + public long count() { + return headerLength + bodyLength; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return totalBytesTransferred; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); + long written = 0; + + if (position < headerLength) { + written += copyByteBuf(header, target); + if (header.readableBytes() > 0) { + totalBytesTransferred += written; + return written; + } + } + + if (body instanceof FileRegion) { + // Adjust the position. If the write is happening as part of the same call where the header + // (or some part of it) is written, `position` will be less than the header size, so we want + // to start from position 0 in the FileRegion object. Otherwise, we start from the position + // requested by the caller. + long bodyPos = position > headerLength ? position - headerLength : 0; + written += ((FileRegion)body).transferTo(target, bodyPos); + } else if (body instanceof ByteBuf) { + written += copyByteBuf((ByteBuf) body, target); + } + + totalBytesTransferred += written; + return written; + } + + @Override + protected void deallocate() { + header.release(); + ReferenceCountUtil.release(body); + } + + private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { + int written = target.write(buf.nioBuffer()); + buf.skipBytes(written); + return written; + } + +} diff --git a/network/common/src/test/java/org/apache/spark/network/ByteArrayWritableChannel.java b/network/common/src/test/java/org/apache/spark/network/ByteArrayWritableChannel.java new file mode 100644 index 0000000000000..b525ed69fc9fb --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/ByteArrayWritableChannel.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class ByteArrayWritableChannel implements WritableByteChannel { + + private final byte[] data; + private int offset; + + public ByteArrayWritableChannel(int size) { + this.data = new byte[size]; + this.offset = 0; + } + + public byte[] getData() { + return data; + } + + @Override + public int write(ByteBuffer src) { + int available = src.remaining(); + src.get(data, offset, available); + offset += available; + return available; + } + + @Override + public void close() { + + } + + @Override + public boolean isOpen() { + return true; + } + +} diff --git a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java index 43dc0cf8c7194..860dd6d9b3915 100644 --- a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java @@ -17,26 +17,34 @@ package org.apache.spark.network; +import java.util.List; + +import com.google.common.primitives.Ints; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.FileRegion; import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.MessageToMessageEncoder; import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.network.protocol.Message; -import org.apache.spark.network.protocol.StreamChunkId; -import org.apache.spark.network.protocol.ChunkFetchRequest; import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchRequest; import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.RpcRequest; -import org.apache.spark.network.protocol.RpcFailure; -import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.Message; import org.apache.spark.network.protocol.MessageDecoder; import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; import org.apache.spark.network.util.NettyUtils; public class ProtocolSuite { private void testServerToClient(Message msg) { - EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder()); + EmbeddedChannel serverChannel = new EmbeddedChannel(new FileRegionEncoder(), + new MessageEncoder()); serverChannel.writeOutbound(msg); EmbeddedChannel clientChannel = new EmbeddedChannel( @@ -51,7 +59,8 @@ private void testServerToClient(Message msg) { } private void testClientToServer(Message msg) { - EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder()); + EmbeddedChannel clientChannel = new EmbeddedChannel(new FileRegionEncoder(), + new MessageEncoder()); clientChannel.writeOutbound(msg); EmbeddedChannel serverChannel = new EmbeddedChannel( @@ -83,4 +92,25 @@ public void responses() { testServerToClient(new RpcFailure(0, "this is an error")); testServerToClient(new RpcFailure(0, "")); } + + /** + * Handler to transform a FileRegion into a byte buffer. EmbeddedChannel doesn't actually transfer + * bytes, but messages, so this is needed so that the frame decoder on the receiving side can + * understand what MessageWithHeader actually contains. + */ + private static class FileRegionEncoder extends MessageToMessageEncoder { + + @Override + public void encode(ChannelHandlerContext ctx, FileRegion in, List out) + throws Exception { + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(Ints.checkedCast(in.count())); + while (in.transfered() < in.count()) { + in.transferTo(channel, in.transfered()); + } + out.add(Unpooled.wrappedBuffer(channel.getData())); + } + + } + } diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java new file mode 100644 index 0000000000000..ff985096d72d5 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -0,0 +1,129 @@ +/* + * 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.network.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.ByteArrayWritableChannel; + +public class MessageWithHeaderSuite { + + @Test + public void testSingleWrite() throws Exception { + testFileRegionBody(8, 8); + } + + @Test + public void testShortWrite() throws Exception { + testFileRegionBody(8, 1); + } + + @Test + public void testByteBufBody() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ByteBuf body = Unpooled.copyLong(84); + MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); + + ByteBuf result = doWrite(msg, 1); + assertEquals(msg.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + assertEquals(84, result.readLong()); + } + + private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { + ByteBuf header = Unpooled.copyLong(42); + int headerLength = header.readableBytes(); + TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); + MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); + + ByteBuf result = doWrite(msg, totalWrites / writesPerCall); + assertEquals(headerLength + region.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + for (long i = 0; i < 8; i++) { + assertEquals(i, result.readLong()); + } + } + + private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { + int writes = 0; + ByteArrayWritableChannel channel = new ByteArrayWritableChannel((int) msg.count()); + while (msg.transfered() < msg.count()) { + msg.transferTo(channel, msg.transfered()); + writes++; + } + assertTrue("Not enough writes!", minExpectedWrites <= writes); + return Unpooled.wrappedBuffer(channel.getData()); + } + + private static class TestFileRegion extends AbstractReferenceCounted implements FileRegion { + + private final int writeCount; + private final int writesPerCall; + private int written; + + TestFileRegion(int totalWrites, int writesPerCall) { + this.writeCount = totalWrites; + this.writesPerCall = writesPerCall; + } + + @Override + public long count() { + return 8 * writeCount; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return 8 * written; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + for (int i = 0; i < writesPerCall; i++) { + ByteBuf buf = Unpooled.copyLong((position / 8) + i); + ByteBuffer nio = buf.nioBuffer(); + while (nio.remaining() > 0) { + target.write(nio); + } + buf.release(); + written++; + } + return 8 * writesPerCall; + } + + @Override + protected void deallocate() { + } + + } + +} diff --git a/network/common/src/test/resources/log4j.properties b/network/common/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e8da774f7ca9e --- /dev/null +++ b/network/common/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO From ccafd757eda478913f783f3127be715bf6413740 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 1 Apr 2015 16:47:18 -0700 Subject: [PATCH 608/817] [SPARK-6642][MLLIB] use 1.2 lambda scaling and remove addImplicit from NormalEquation This PR changes lambda scaling from number of users/items to number of explicit ratings. The latter is the behavior in 1.2. Slight refactor of NormalEquation to make it independent of ALS models. srowen codexiang Author: Xiangrui Meng Closes #5314 from mengxr/SPARK-6642 and squashes the following commits: dc655a1 [Xiangrui Meng] relax python tests f410df2 [Xiangrui Meng] use 1.2 scaling and remove addImplicit from NormalEquation --- .../apache/spark/ml/recommendation/ALS.scala | 67 +++++++++-------- .../spark/ml/recommendation/ALSSuite.scala | 71 +++++++------------ python/pyspark/mllib/recommendation.py | 6 +- 3 files changed, 60 insertions(+), 84 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 514b4ef98dc5b..52c9e95d6012f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -320,7 +320,7 @@ object ALS extends Logging { /** Trait for least squares solvers applied to the normal equation. */ private[recommendation] trait LeastSquaresNESolver extends Serializable { - /** Solves a least squares problem (possibly with other constraints). */ + /** Solves a least squares problem with regularization (possibly with other constraints). */ def solve(ne: NormalEquation, lambda: Double): Array[Float] } @@ -332,20 +332,19 @@ object ALS extends Logging { /** * Solves a least squares problem with L2 regularization: * - * min norm(A x - b)^2^ + lambda * n * norm(x)^2^ + * min norm(A x - b)^2^ + lambda * norm(x)^2^ * * @param ne a [[NormalEquation]] instance that contains AtA, Atb, and n (number of instances) - * @param lambda regularization constant, which will be scaled by n + * @param lambda regularization constant * @return the solution x */ override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { val k = ne.k // Add scaled lambda to the diagonals of AtA. - val scaledlambda = lambda * ne.n var i = 0 var j = 2 while (i < ne.triK) { - ne.ata(i) += scaledlambda + ne.ata(i) += lambda i += j j += 1 } @@ -391,7 +390,7 @@ object ALS extends Logging { override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { val rank = ne.k initialize(rank) - fillAtA(ne.ata, lambda * ne.n) + fillAtA(ne.ata, lambda) val x = NNLS.solve(ata, ne.atb, workspace) ne.reset() x.map(x => x.toFloat) @@ -420,7 +419,15 @@ object ALS extends Logging { } } - /** Representing a normal equation (ALS' subproblem). */ + /** + * Representing a normal equation to solve the following weighted least squares problem: + * + * minimize \sum,,i,, c,,i,, (a,,i,,^T^ x - b,,i,,)^2^ + lambda * x^T^ x. + * + * Its normal equation is given by + * + * \sum,,i,, c,,i,, (a,,i,, a,,i,,^T^ x - b,,i,, a,,i,,) + lambda * x = 0. + */ private[recommendation] class NormalEquation(val k: Int) extends Serializable { /** Number of entries in the upper triangular part of a k-by-k matrix. */ @@ -429,8 +436,6 @@ object ALS extends Logging { val ata = new Array[Double](triK) /** A^T^ * b */ val atb = new Array[Double](k) - /** Number of observations. */ - var n = 0 private val da = new Array[Double](k) private val upper = "U" @@ -444,28 +449,13 @@ object ALS extends Logging { } /** Adds an observation. */ - def add(a: Array[Float], b: Float): this.type = { - require(a.length == k) - copyToDouble(a) - blas.dspr(upper, k, 1.0, da, 1, ata) - blas.daxpy(k, b.toDouble, da, 1, atb, 1) - n += 1 - this - } - - /** - * Adds an observation with implicit feedback. Note that this does not increment the counter. - */ - def addImplicit(a: Array[Float], b: Float, alpha: Double): this.type = { + def add(a: Array[Float], b: Double, c: Double = 1.0): this.type = { + require(c >= 0.0) require(a.length == k) - // Extension to the original paper to handle b < 0. confidence is a function of |b| instead - // so that it is never negative. - val confidence = 1.0 + alpha * math.abs(b) copyToDouble(a) - blas.dspr(upper, k, confidence - 1.0, da, 1, ata) - // For b <= 0, the corresponding preference is 0. So the term below is only added for b > 0. - if (b > 0) { - blas.daxpy(k, confidence, da, 1, atb, 1) + blas.dspr(upper, k, c, da, 1, ata) + if (b != 0.0) { + blas.daxpy(k, c * b, da, 1, atb, 1) } this } @@ -475,7 +465,6 @@ object ALS extends Logging { require(other.k == k) blas.daxpy(ata.length, 1.0, other.ata, 1, ata, 1) blas.daxpy(atb.length, 1.0, other.atb, 1, atb, 1) - n += other.n this } @@ -483,7 +472,6 @@ object ALS extends Logging { def reset(): Unit = { ju.Arrays.fill(ata, 0.0) ju.Arrays.fill(atb, 0.0) - n = 0 } } @@ -1114,6 +1102,7 @@ object ALS extends Logging { ls.merge(YtY.get) } var i = srcPtrs(j) + var numExplicits = 0 while (i < srcPtrs(j + 1)) { val encoded = srcEncodedIndices(i) val blockId = srcEncoder.blockId(encoded) @@ -1121,13 +1110,23 @@ object ALS extends Logging { val srcFactor = sortedSrcFactors(blockId)(localIndex) val rating = ratings(i) if (implicitPrefs) { - ls.addImplicit(srcFactor, rating, alpha) + // Extension to the original paper to handle b < 0. confidence is a function of |b| + // instead so that it is never negative. c1 is confidence - 1.0. + val c1 = alpha * math.abs(rating) + // For rating <= 0, the corresponding preference is 0. So the term below is only added + // for rating > 0. Because YtY is already added, we need to adjust the scaling here. + if (rating > 0) { + numExplicits += 1 + ls.add(srcFactor, (c1 + 1.0) / c1, c1) + } } else { ls.add(srcFactor, rating) + numExplicits += 1 } i += 1 } - dstFactors(j) = solver.solve(ls, regParam) + // Weight lambda by the number of explicit ratings based on the ALS-WR paper. + dstFactors(j) = solver.solve(ls, numExplicits * regParam) j += 1 } dstFactors @@ -1141,7 +1140,7 @@ object ALS extends Logging { private def computeYtY(factorBlocks: RDD[(Int, FactorBlock)], rank: Int): NormalEquation = { factorBlocks.values.aggregate(new NormalEquation(rank))( seqOp = (ne, factors) => { - factors.foreach(ne.add(_, 0.0f)) + factors.foreach(ne.add(_, 0.0)) ne }, combOp = (ne1, ne2) => ne1.merge(ne2)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 0bb06e9e8ac9c..29d4ec5f85c1e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -68,39 +68,42 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { } } - test("normal equation construction with explict feedback") { + test("normal equation construction") { val k = 2 val ne0 = new NormalEquation(k) - .add(Array(1.0f, 2.0f), 3.0f) - .add(Array(4.0f, 5.0f), 6.0f) + .add(Array(1.0f, 2.0f), 3.0) + .add(Array(4.0f, 5.0f), 6.0, 2.0) // weighted assert(ne0.k === k) assert(ne0.triK === k * (k + 1) / 2) - assert(ne0.n === 2) // NumPy code that computes the expected values: // A = np.matrix("1 2; 4 5") // b = np.matrix("3; 6") - // ata = A.transpose() * A - // atb = A.transpose() * b - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(17.0, 22.0, 29.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(27.0, 36.0) relTol 1e-8) + // C = np.matrix(np.diag([1, 2])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(33.0, 42.0, 54.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(51.0, 66.0) relTol 1e-8) val ne1 = new NormalEquation(2) - .add(Array(7.0f, 8.0f), 9.0f) + .add(Array(7.0f, 8.0f), 9.0) ne0.merge(ne1) - assert(ne0.n === 3) // NumPy code that computes the expected values: // A = np.matrix("1 2; 4 5; 7 8") // b = np.matrix("3; 6; 9") - // ata = A.transpose() * A - // atb = A.transpose() * b - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(66.0, 78.0, 93.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(90.0, 108.0) relTol 1e-8) + // C = np.matrix(np.diag([1, 2, 1])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(82.0, 98.0, 118.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(114.0, 138.0) relTol 1e-8) intercept[IllegalArgumentException] { - ne0.add(Array(1.0f), 2.0f) + ne0.add(Array(1.0f), 2.0) } intercept[IllegalArgumentException] { - ne0.add(Array(1.0f, 2.0f, 3.0f), 4.0f) + ne0.add(Array(1.0f, 2.0f, 3.0f), 4.0) + } + intercept[IllegalArgumentException] { + ne0.add(Array(1.0f, 2.0f), 0.0, -1.0) } intercept[IllegalArgumentException] { val ne2 = new NormalEquation(3) @@ -108,41 +111,16 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { } ne0.reset() - assert(ne0.n === 0) assert(ne0.ata.forall(_ == 0.0)) assert(ne0.atb.forall(_ == 0.0)) } - test("normal equation construction with implicit feedback") { - val k = 2 - val alpha = 0.5 - val ne0 = new NormalEquation(k) - .addImplicit(Array(-5.0f, -4.0f), -3.0f, alpha) - .addImplicit(Array(-2.0f, -1.0f), 0.0f, alpha) - .addImplicit(Array(1.0f, 2.0f), 3.0f, alpha) - assert(ne0.k === k) - assert(ne0.triK === k * (k + 1) / 2) - assert(ne0.n === 0) // addImplicit doesn't increase the count. - // NumPy code that computes the expected values: - // alpha = 0.5 - // A = np.matrix("-5 -4; -2 -1; 1 2") - // b = np.matrix("-3; 0; 3") - // b1 = b > 0 - // c = 1.0 + alpha * np.abs(b) - // C = np.diag(c.A1) - // I = np.eye(3) - // ata = A.transpose() * (C - I) * A - // atb = A.transpose() * C * b1 - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(39.0, 33.0, 30.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(2.5, 5.0) relTol 1e-8) - } - test("CholeskySolver") { val k = 2 val ne0 = new NormalEquation(k) - .add(Array(1.0f, 2.0f), 4.0f) - .add(Array(1.0f, 3.0f), 9.0f) - .add(Array(1.0f, 4.0f), 16.0f) + .add(Array(1.0f, 2.0f), 4.0) + .add(Array(1.0f, 3.0f), 9.0) + .add(Array(1.0f, 4.0f), 16.0) val ne1 = new NormalEquation(k) .merge(ne0) @@ -154,13 +132,12 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { // x0 = np.linalg.lstsq(A, b)[0] assert(Vectors.dense(x0) ~== Vectors.dense(-8.333333, 6.0) relTol 1e-6) - assert(ne0.n === 0) assert(ne0.ata.forall(_ == 0.0)) assert(ne0.atb.forall(_ == 0.0)) - val x1 = chol.solve(ne1, 0.5).map(_.toDouble) + val x1 = chol.solve(ne1, 1.5).map(_.toDouble) // NumPy code that computes the expected solution, where lambda is scaled by n: - // x0 = np.linalg.solve(A.transpose() * A + 0.5 * 3 * np.eye(2), A.transpose() * b) + // x0 = np.linalg.solve(A.transpose() * A + 1.5 * np.eye(2), A.transpose() * b) assert(Vectors.dense(x1) ~== Vectors.dense(-0.1155556, 3.28) relTol 1e-6) } diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index b094e50856f70..c5c4c13dae105 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -52,7 +52,7 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> ratings = sc.parallelize([r1, r2, r3]) >>> model = ALS.trainImplicit(ratings, 1, seed=10) >>> model.predict(2, 2) - 0.43... + 0.4... >>> testset = sc.parallelize([(1, 2), (1, 1)]) >>> model = ALS.train(ratings, 2, seed=0) @@ -82,14 +82,14 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) >>> model.predict(2,2) - 0.43... + 0.4... >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) >>> sameModel = MatrixFactorizationModel.load(sc, path) >>> sameModel.predict(2,2) - 0.43... + 0.4... >>> sameModel.predictAll(testset).collect() [Rating(... >>> try: From 2fa3b47dbf38aae58514473932c69bbd35de4e4c Mon Sep 17 00:00:00 2001 From: MechCoder Date: Wed, 1 Apr 2015 17:03:39 -0700 Subject: [PATCH 609/817] [SPARK-6576] [MLlib] [PySpark] DenseMatrix in PySpark should support indexing Support indexing in DenseMatrices in PySpark Author: MechCoder Closes #5232 from MechCoder/SPARK-6576 and squashes the following commits: a735078 [MechCoder] Change bounds a062025 [MechCoder] Matrices are stored in column order 7917bc1 [MechCoder] [SPARK-6576] DenseMatrix in PySpark should support indexing --- python/pyspark/mllib/linalg.py | 10 ++++++++++ python/pyspark/mllib/tests.py | 7 +++++++ 2 files changed, 17 insertions(+) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 8b791ff6a7877..51c1490b1618d 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -670,6 +670,16 @@ def toArray(self): """ return self.values.reshape((self.numRows, self.numCols), order='F') + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j >= self.numCols or j < 0: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + return self.values[i + j * self.numRows] + def __eq__(self, other): return (isinstance(other, DenseMatrix) and self.numRows == other.numRows and diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 3bb0f0ca68128..893fc6f491ab3 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -135,6 +135,13 @@ def test_sparse_vector_indexing(self): for ind in [4, -5, 7.8]: self.assertRaises(ValueError, sv.__getitem__, ind) + def test_matrix_indexing(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + expected = [[0, 6], [1, 8], [4, 10]] + for i in range(3): + for j in range(2): + self.assertEquals(mat[i, j], expected[i][j]) + class ListTests(PySparkTestCase): From 86b43993517104e6d5ad0785704ceec6db8acc20 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 1 Apr 2015 17:19:36 -0700 Subject: [PATCH 610/817] [SPARK-6580] [MLLIB] Optimize LogisticRegressionModel.predictPoint https://issues.apache.org/jira/browse/SPARK-6580 Author: Yanbo Liang Closes #5249 from yanboliang/spark-6580 and squashes the following commits: 6f47f21 [Yanbo Liang] address comments 4e0bd0f [Yanbo Liang] fix typos 04e2e2a [Yanbo Liang] trigger jenkins cad5bcd [Yanbo Liang] Optimize LogisticRegressionModel.predictPoint --- .../classification/LogisticRegression.scala | 55 +++++++++---------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index e7c3599ff619c..057b628c6a586 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -62,6 +62,15 @@ class LogisticRegressionModel ( s" but was given weights of length ${weights.size}") } + private val dataWithBiasSize: Int = weights.size / (numClasses - 1) + + private val weightsArray: Array[Double] = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.getClass}.") + } + /** * Constructs a [[LogisticRegressionModel]] with weights and intercept for binary classification. */ @@ -74,6 +83,7 @@ class LogisticRegressionModel ( * Sets the threshold that separates positive predictions from negative predictions * in Binary Logistic Regression. An example with prediction score greater than or equal to * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. + * It is only used for binary classification. */ @Experimental def setThreshold(threshold: Double): this.type = { @@ -84,6 +94,7 @@ class LogisticRegressionModel ( /** * :: Experimental :: * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. + * It is only used for binary classification. */ @Experimental def getThreshold: Option[Double] = threshold @@ -91,6 +102,7 @@ class LogisticRegressionModel ( /** * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. + * It is only used for binary classification. */ @Experimental def clearThreshold(): this.type = { @@ -106,7 +118,6 @@ class LogisticRegressionModel ( // If dataMatrix and weightMatrix have the same dimension, it's binary logistic regression. if (numClasses == 2) { - require(numFeatures == weightMatrix.size) val margin = dot(weightMatrix, dataMatrix) + intercept val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { @@ -114,30 +125,9 @@ class LogisticRegressionModel ( case None => score } } else { - val dataWithBiasSize = weightMatrix.size / (numClasses - 1) - - val weightsArray = weightMatrix match { - case dv: DenseVector => dv.values - case _ => - throw new IllegalArgumentException( - s"weights only supports dense vector but got type ${weightMatrix.getClass}.") - } - - val margins = (0 until numClasses - 1).map { i => - var margin = 0.0 - dataMatrix.foreachActive { (index, value) => - if (value != 0.0) margin += value * weightsArray((i * dataWithBiasSize) + index) - } - // Intercept is required to be added into margin. - if (dataMatrix.size + 1 == dataWithBiasSize) { - margin += weightsArray((i * dataWithBiasSize) + dataMatrix.size) - } - margin - } - /** - * Find the one with maximum margins. If the maxMargin is negative, then the prediction - * result will be the first class. + * Compute and find the one with maximum margins. If the maxMargin is negative, then the + * prediction result will be the first class. * * PS, if you want to compute the probabilities for each outcome instead of the outcome * with maximum probability, remember to subtract the maxMargin from margins if maxMargin @@ -145,13 +135,20 @@ class LogisticRegressionModel ( */ var bestClass = 0 var maxMargin = 0.0 - var i = 0 - while(i < margins.size) { - if (margins(i) > maxMargin) { - maxMargin = margins(i) + val withBias = dataMatrix.size + 1 == dataWithBiasSize + (0 until numClasses - 1).foreach { i => + var margin = 0.0 + dataMatrix.foreachActive { (index, value) => + if (value != 0.0) margin += value * weightsArray((i * dataWithBiasSize) + index) + } + // Intercept is required to be added into margin. + if (withBias) { + margin += weightsArray((i * dataWithBiasSize) + dataMatrix.size) + } + if (margin > maxMargin) { + maxMargin = margin bestClass = i + 1 } - i += 1 } bestClass.toDouble } From 757b2e91756ba49d7d1ab89abf19b00c7f5fd721 Mon Sep 17 00:00:00 2001 From: ksonj Date: Wed, 1 Apr 2015 17:23:57 -0700 Subject: [PATCH 611/817] [SPARK-6553] [pyspark] Support functools.partial as UDF Use `f.__repr__()` instead of `f.__name__` when instantiating `UserDefinedFunction`s, so `functools.partial`s may be used. Author: ksonj Closes #5206 from ksonj/partials and squashes the following commits: ea66f3d [ksonj] Inserted blank lines for PEP8 compliance d81b02b [ksonj] added tests for udf with partial function and callable object 2c76100 [ksonj] Makes UDFs work with all types of callables b814a12 [ksonj] support functools.partial as udf (cherry picked from commit 98f72dfc17853b570d05c20e97c78919682b6df6) Signed-off-by: Josh Rosen --- python/pyspark/sql/functions.py | 3 ++- python/pyspark/sql/tests.py | 31 +++++++++++++++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8a478fddf0e95..146ba6f3e0d98 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -123,7 +123,8 @@ def _create_judf(self): pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) jdt = ssql_ctx.parseDataType(self.returnType.json()) - judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, + fname = f.__name__ if hasattr(f, '__name__') else f.__class__.__name__ + judf = sc._jvm.UserDefinedPythonFunction(fname, bytearray(pickled_command), env, includes, sc.pythonExec, broadcast_vars, sc._javaAccumulator, jdt) return judf diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 258464b7f230d..b3a6a2c6a9229 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -25,6 +25,7 @@ import shutil import tempfile import pickle +import functools import py4j @@ -41,6 +42,7 @@ from pyspark.sql.types import * from pyspark.sql.types import UserDefinedType, _infer_type from pyspark.tests import ReusedPySparkTestCase +from pyspark.sql.functions import UserDefinedFunction class ExamplePointUDT(UserDefinedType): @@ -114,6 +116,35 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name, ignore_errors=True) + def test_udf_with_callable(self): + d = [Row(number=i, squared=i**2) for i in range(10)] + rdd = self.sc.parallelize(d) + data = self.sqlCtx.createDataFrame(rdd) + + class PlusFour: + def __call__(self, col): + if col is not None: + return col + 4 + + call = PlusFour() + pudf = UserDefinedFunction(call, LongType()) + res = data.select(pudf(data['number']).alias('plus_four')) + self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85) + + def test_udf_with_partial_function(self): + d = [Row(number=i, squared=i**2) for i in range(10)] + rdd = self.sc.parallelize(d) + data = self.sqlCtx.createDataFrame(rdd) + + def some_func(col, param): + if col is not None: + return col + param + + pfunc = functools.partial(some_func, param=4) + pudf = UserDefinedFunction(pfunc, LongType()) + res = data.select(pudf(data['number']).alias('plus_four')) + self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85) + def test_udf(self): self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() From 4815bc2128c7f6d4d21da730b8c72da087233b34 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 1 Apr 2015 18:17:07 -0700 Subject: [PATCH 612/817] [SPARK-6660][MLLIB] pythonToJava doesn't recognize object arrays davies Author: Xiangrui Meng Closes #5318 from mengxr/SPARK-6660 and squashes the following commits: 0f66ec2 [Xiangrui Meng] recognize object arrays ad8c42f [Xiangrui Meng] add a test for SPARK-6660 --- .../apache/spark/mllib/api/python/PythonMLLibAPI.scala | 5 ++++- python/pyspark/mllib/tests.py | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 662ec5fbed453..5995d6df97c15 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -1113,7 +1113,10 @@ private[spark] object SerDe extends Serializable { iter.flatMap { row => val obj = unpickle.loads(row) if (batched) { - obj.asInstanceOf[JArrayList[_]].asScala + obj match { + case list: JArrayList[_] => list.asScala + case arr: Array[_] => arr + } } else { Seq(obj) } diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 893fc6f491ab3..6e9c68ec8a5c1 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -36,6 +36,7 @@ else: import unittest +from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ DenseMatrix, Vectors, Matrices from pyspark.mllib.regression import LabeledPoint @@ -641,6 +642,13 @@ def test_idf_model(self): idf = model.idf() self.assertEqual(len(idf), 11) + +class SerDeTest(PySparkTestCase): + def test_to_java_object_rdd(self): # SPARK-6660 + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) + self.assertEqual(_to_java_object_rdd(data).count(), 10) + + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" From 899ebcb1448126f40be784ce42e69218e9a1ead7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 Apr 2015 18:36:06 -0700 Subject: [PATCH 613/817] [SPARK-6578] Small rewrite to make the logic more clear in MessageWithHeader.transferTo. Author: Reynold Xin Closes #5319 from rxin/SPARK-6578 and squashes the following commits: 7c62a64 [Reynold Xin] Small rewrite to make the logic more clear in transferTo. --- .../network/protocol/MessageWithHeader.java | 43 ++++++++++--------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java index 215a8517e8608..d686a951467cf 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -21,15 +21,15 @@ import java.nio.channels.WritableByteChannel; import com.google.common.base.Preconditions; -import com.google.common.primitives.Ints; import io.netty.buffer.ByteBuf; import io.netty.channel.FileRegion; import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCountUtil; /** - * A wrapper message that holds two separate pieces (a header and a body) to avoid - * copying the body's content. + * A wrapper message that holds two separate pieces (a header and a body). + * + * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. */ class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { @@ -63,32 +63,36 @@ public long transfered() { return totalBytesTransferred; } + /** + * This code is more complicated than you would think because we might require multiple + * transferTo invocations in order to transfer a single MessageWithHeader to avoid busy waiting. + * + * The contract is that the caller will ensure position is properly set to the total number + * of bytes transferred so far (i.e. value returned by transfered()). + */ @Override - public long transferTo(WritableByteChannel target, long position) throws IOException { + public long transferTo(final WritableByteChannel target, final long position) throws IOException { Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); - long written = 0; - - if (position < headerLength) { - written += copyByteBuf(header, target); + // Bytes written for header in this call. + long writtenHeader = 0; + if (header.readableBytes() > 0) { + writtenHeader = copyByteBuf(header, target); + totalBytesTransferred += writtenHeader; if (header.readableBytes() > 0) { - totalBytesTransferred += written; - return written; + return writtenHeader; } } + // Bytes written for body in this call. + long writtenBody = 0; if (body instanceof FileRegion) { - // Adjust the position. If the write is happening as part of the same call where the header - // (or some part of it) is written, `position` will be less than the header size, so we want - // to start from position 0 in the FileRegion object. Otherwise, we start from the position - // requested by the caller. - long bodyPos = position > headerLength ? position - headerLength : 0; - written += ((FileRegion)body).transferTo(target, bodyPos); + writtenBody = ((FileRegion) body).transferTo(target, totalBytesTransferred - headerLength); } else if (body instanceof ByteBuf) { - written += copyByteBuf((ByteBuf) body, target); + writtenBody = copyByteBuf((ByteBuf) body, target); } + totalBytesTransferred += writtenBody; - totalBytesTransferred += written; - return written; + return writtenHeader + writtenBody; } @Override @@ -102,5 +106,4 @@ private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOExcept buf.skipBytes(written); return written; } - } From 191524e7401fcdfae46dc7e6a64c28907b1b1c20 Mon Sep 17 00:00:00 2001 From: Chet Mancini Date: Wed, 1 Apr 2015 21:39:46 -0700 Subject: [PATCH 614/817] [SPARK-6658][SQL] Update DataFrame documentation to fix type references. First contribution here; would love to be getting some code contributions in soon. Let me know if there's anything about contribution process I should improve. Author: Chet Mancini Closes #5316 from chetmancini/SPARK_6658_dataframe_doc and squashes the following commits: 53b627a [Chet Mancini] [SQL] SPARK-6658: Update DataFrame documentation to refer to correct types --- .../main/scala/org/apache/spark/sql/DataFrame.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 19cfa15f27b09..ce0890906bf1b 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 @@ -273,7 +273,7 @@ class DataFrame private[sql]( def printSchema(): Unit = println(schema.treeString) /** - * Prints the plans (logical and physical) to the console for debugging purpose. + * Prints the plans (logical and physical) to the console for debugging purposes. * @group basic */ def explain(extended: Boolean): Unit = { @@ -285,7 +285,7 @@ class DataFrame private[sql]( } /** - * Only prints the physical plan to the console for debugging purpose. + * Only prints the physical plan to the console for debugging purposes. * @group basic */ def explain(): Unit = explain(extended = false) @@ -976,8 +976,8 @@ class DataFrame private[sql]( def javaRDD: JavaRDD[Row] = toJavaRDD /** - * Registers this RDD as a temporary table using the given name. The lifetime of this temporary - * table is tied to the [[SQLContext]] that was used to create this DataFrame. + * Registers this [[DataFrame]] as a temporary table using the given name. The lifetime of this + * temporary table is tied to the [[SQLContext]] that was used to create this DataFrame. * * @group basic */ @@ -1252,7 +1252,7 @@ class DataFrame private[sql]( //////////////////////////////////////////////////////////////////////////// /** - * Save this RDD to a JDBC database at `url` under the table name `table`. + * Save this [[DataFrame]] to a JDBC database at `url` under the table name `table`. * This will run a `CREATE TABLE` and a bunch of `INSERT INTO` statements. * If you pass `true` for `allowExisting`, it will drop any table with the * given name; if you pass `false`, it will throw if the table already @@ -1276,7 +1276,7 @@ class DataFrame private[sql]( } /** - * Save this RDD to a JDBC database at `url` under the table name `table`. + * Save this [[DataFrame]] to a JDBC database at `url` under the table name `table`. * Assumes the table already exists and has a compatible schema. If you * pass `true` for `overwrite`, it will `TRUNCATE` the table before * performing the `INSERT`s. From 2bc7fe7f7eb31b8f0591611b1e66b601bba8a4b7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 2 Apr 2015 12:56:34 +0800 Subject: [PATCH 615/817] Revert "[SPARK-6618][SQL] HiveMetastoreCatalog.lookupRelation should use fine-grained lock" This reverts commit 314afd0e2f08dd8d3333d3143712c2c79fa40d1e. --- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++--------- .../spark/sql/hive/execution/SQLQuerySuite.scala | 11 ----------- 2 files changed, 3 insertions(+), 20 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 2b5d031741a63..f0076cef13777 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -173,16 +173,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with def lookupRelation( tableIdentifier: Seq[String], - alias: Option[String]): LogicalPlan = { + alias: Option[String]): LogicalPlan = synchronized { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - val table = try { - synchronized { - client.getTable(databaseName, tblName) - } - } catch { + val table = try client.getTable(databaseName, tblName) catch { case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => throw new NoSuchTableException } @@ -204,9 +200,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } else { val partitions: Seq[Partition] = if (table.isPartitioned) { - synchronized { - HiveShim.getAllPartitionsOf(client, table).toSeq - } + HiveShim.getAllPartitionsOf(client, table).toSeq } else { Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 2065f0d60d92f..310c2bfdf1011 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -457,15 +457,4 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") setConf("spark.sql.hive.convertCTAS", originalConf) } - - test("sanity test for SPARK-6618") { - (1 to 100).par.map { i => - val tableName = s"SPARK_6618_table_$i" - sql(s"CREATE TABLE $tableName (col1 string)") - catalog.lookupRelation(Seq(tableName)) - table(tableName) - tables() - sql(s"DROP TABLE $tableName") - } - } } From 40df5d49bb5c80cd3a1e2d7c853c0b5ea901adf3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 1 Apr 2015 23:11:38 -0700 Subject: [PATCH 616/817] [SPARK-6663] [SQL] use Literal.create instread of constructor In order to do inbound checking and type conversion, we should use Literal.create() instead of constructor. Author: Davies Liu Closes #5320 from davies/literal and squashes the following commits: 1667604 [Davies Liu] fix style and add comment 5f8c0fd [Davies Liu] use Literal.create instread of constructor --- .../apache/spark/sql/catalyst/SqlParser.scala | 8 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 3 +- .../sql/catalyst/expressions/literals.scala | 7 +- .../sql/catalyst/optimizer/Optimizer.scala | 42 ++-- .../analysis/HiveTypeCoercionSuite.scala | 4 +- .../ExpressionEvaluationSuite.scala | 204 +++++++++--------- .../optimizer/ConstantFoldingSuite.scala | 70 +++--- .../sql/catalyst/trees/TreeNodeSuite.scala | 2 +- .../sql/execution/GeneratedAggregate.scala | 8 +- .../apache/spark/sql/parquet/newParquet.scala | 21 +- .../ParquetPartitionDiscoverySuite.scala | 20 +- .../spark/sql/hive/HiveInspectors.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 20 +- .../spark/sql/hive/HiveInspectorSuite.scala | 16 +- 16 files changed, 220 insertions(+), 213 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b176f7e729a42..89f4a19add1c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -316,13 +316,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val literal: Parser[Literal] = ( numericLiteral | booleanLiteral - | stringLit ^^ {case s => Literal(s, StringType) } - | NULL ^^^ Literal(null, NullType) + | stringLit ^^ {case s => Literal.create(s, StringType) } + | NULL ^^^ Literal.create(null, NullType) ) protected lazy val booleanLiteral: Parser[Literal] = - ( TRUE ^^^ Literal(true, BooleanType) - | FALSE ^^^ Literal(false, BooleanType) + ( TRUE ^^^ Literal.create(true, BooleanType) + | FALSE ^^^ Literal.create(false, BooleanType) ) protected lazy val numericLiteral: Parser[Literal] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c578d084a45b6..119cb9c3a4400 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -140,10 +140,10 @@ class Analyzer( case x: Expression if nonSelectedGroupExprSet.contains(x) => // if the input attribute in the Invalid Grouping Expression set of for this group // replace it with constant null - Literal(null, expr.dataType) + Literal.create(null, expr.dataType) case x if x == g.gid => // replace the groupingId with concrete value (the bit mask) - Literal(bitmask, IntegerType) + Literal.create(bitmask, IntegerType) }) result += GroupExpression(substitution) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 3c7b46e0702a2..9a33eb145273e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -115,7 +115,7 @@ trait HiveTypeCoercion { * the appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal("NaN", StringType) + val stringNaN = Literal.create("NaN", StringType) def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 30da4faa3f1c6..406de38d1c483 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -505,7 +505,8 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) private var count: Long = _ private val sum = MutableLiteral(zero.eval(null), calcType) - private def addFunction(value: Any) = Add(sum, Cast(Literal(value, expr.dataType), calcType)) + private def addFunction(value: Any) = Add(sum, + Cast(Literal.create(value, expr.dataType), calcType)) override def eval(input: Row): Any = { if (count == 0L) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 19f3fc9c2291a..0e2d593e94124 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -41,6 +41,8 @@ object Literal { case _ => throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } + + def create(v: Any, dataType: DataType): Literal = Literal(v, dataType) } /** @@ -62,7 +64,10 @@ object IntegerLiteral { } } -case class Literal(value: Any, dataType: DataType) extends LeafExpression { +/** + * In order to do type checking, use Literal.create() instead of constructor + */ +case class Literal protected (value: Any, dataType: DataType) extends LeafExpression { override def foldable: Boolean = true override def nullable: Boolean = value == null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c23d3b61887c6..93e69d409cb91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -218,12 +218,12 @@ object NullPropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case e @ Count(Literal(null, _)) => Cast(Literal(0L), e.dataType) - case e @ IsNull(c) if !c.nullable => Literal(false, BooleanType) - case e @ IsNotNull(c) if !c.nullable => Literal(true, BooleanType) - case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) - case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) - case e @ StructGetField(Literal(null, _), _, _) => Literal(null, e.dataType) - case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal(null, e.dataType) + case e @ IsNull(c) if !c.nullable => Literal.create(false, BooleanType) + case e @ IsNotNull(c) if !c.nullable => Literal.create(true, BooleanType) + case e @ GetItem(Literal(null, _), _) => Literal.create(null, e.dataType) + case e @ GetItem(_, Literal(null, _)) => Literal.create(null, e.dataType) + case e @ StructGetField(Literal(null, _), _, _) => Literal.create(null, e.dataType) + case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal.create(null, e.dataType) case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) case e @ Count(expr) if !expr.nullable => Count(Literal(1)) @@ -235,36 +235,36 @@ object NullPropagation extends Rule[LogicalPlan] { case _ => true } if (newChildren.length == 0) { - Literal(null, e.dataType) + Literal.create(null, e.dataType) } else if (newChildren.length == 1) { newChildren(0) } else { Coalesce(newChildren) } - case e @ Substring(Literal(null, _), _, _) => Literal(null, e.dataType) - case e @ Substring(_, Literal(null, _), _) => Literal(null, e.dataType) - case e @ Substring(_, _, Literal(null, _)) => Literal(null, e.dataType) + case e @ Substring(Literal(null, _), _, _) => Literal.create(null, e.dataType) + case e @ Substring(_, Literal(null, _), _) => Literal.create(null, e.dataType) + case e @ Substring(_, _, Literal(null, _)) => Literal.create(null, e.dataType) // Put exceptional cases above if any case e: BinaryArithmetic => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: BinaryComparison => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: StringRegexExpression => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: StringComparison => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } } @@ -284,13 +284,13 @@ object ConstantFolding extends Rule[LogicalPlan] { case l: Literal => l // Fold expressions that are foldable. - case e if e.foldable => Literal(e.eval(null), e.dataType) + case e if e.foldable => Literal.create(e.eval(null), e.dataType) // Fold "literal in (item1, item2, ..., literal, ...)" into true directly. case In(Literal(v, _), list) if list.exists { case Literal(candidate, _) if candidate == v => true case _ => false - } => Literal(true, BooleanType) + } => Literal.create(true, BooleanType) } } } @@ -647,7 +647,7 @@ object DecimalAggregates extends Rule[LogicalPlan] { case Average(e @ DecimalType.Expression(prec, scale)) if prec + 4 <= MAX_DOUBLE_DIGITS => Cast( - Divide(Average(UnscaledValue(e)), Literal(math.pow(10.0, scale), DoubleType)), + Divide(Average(UnscaledValue(e)), Literal.create(math.pow(10.0, scale), DoubleType)), DecimalType(prec + 4, scale + 4)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index ecbb54218d457..70aef1cac421a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -127,11 +127,11 @@ class HiveTypeCoercionSuite extends PlanTest { ruleTest( Coalesce(Literal(1.0) :: Literal(1) - :: Literal(1.0, FloatType) + :: Literal.create(1.0, FloatType) :: Nil), Coalesce(Cast(Literal(1.0), DoubleType) :: Cast(Literal(1), DoubleType) - :: Cast(Literal(1.0, FloatType), DoubleType) + :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) ruleTest( Coalesce(Literal(1L) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 1183a0d899dda..3dbefa40d2808 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -111,7 +111,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("3VL Not") { notTrueTable.foreach { case (v, answer) => - checkEvaluation(!Literal(v, BooleanType), answer) + checkEvaluation(!Literal.create(v, BooleanType), answer) } } @@ -155,7 +155,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test(s"3VL $name") { truthTable.foreach { case (l,r,answer) => - val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) + val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType)) checkEvaluation(expr, answer) } } @@ -175,12 +175,12 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Divide(Literal(1), Literal(0)), null) checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0), Literal(null, IntegerType)), null) - checkEvaluation(Divide(Literal(1), Literal(null, IntegerType)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(0)), null) - checkEvaluation(Divide(Literal(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(null, IntegerType)), null) + checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) } test("Remainder") { @@ -190,12 +190,12 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Remainder(Literal(1), Literal(0)), null) checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0), Literal(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(1), Literal(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(0)), null) - checkEvaluation(Remainder(Literal(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) } test("INSET") { @@ -222,14 +222,14 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(MaxOf(1L, 2L), 2L) checkEvaluation(MaxOf(2L, 1L), 2L) - checkEvaluation(MaxOf(Literal(null, IntegerType), 2), 2) - checkEvaluation(MaxOf(2, Literal(null, IntegerType)), 2) + checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) } test("LIKE literal Regular Expression") { - checkEvaluation(Literal(null, StringType).like("a"), null) - checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) - checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like("a"), null) + checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) checkEvaluation("abdef" like "abdef", true) checkEvaluation("a_%b" like "a\\__b", true) checkEvaluation("addb" like "a_%b", true) @@ -264,13 +264,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) + checkEvaluation(Literal.create(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } test("RLIKE literal Regular Expression") { - checkEvaluation(Literal(null, StringType) rlike "abdef", null) - checkEvaluation("abdef" rlike Literal(null, StringType), null) - checkEvaluation(Literal(null, StringType) rlike Literal(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal.create(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) @@ -381,7 +381,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(("abcdef" cast DoubleType).nullable === true) assert(("abcdef" cast FloatType).nullable === true) - checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) + checkEvaluation(Cast(Literal.create(null, IntegerType), ShortType), null) } test("date") { @@ -507,8 +507,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("array casting") { - val array = Literal(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) - val array_notNull = Literal(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) + val array = Literal.create(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) { val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) @@ -556,10 +556,10 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("map casting") { - val map = Literal( + val map = Literal.create( Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null), MapType(StringType, StringType, valueContainsNull = true)) - val map_notNull = Literal( + val map_notNull = Literal.create( Map("a" -> "123", "b" -> "abc", "c" -> ""), MapType(StringType, StringType, valueContainsNull = false)) @@ -617,14 +617,14 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("struct casting") { - val struct = Literal( + val struct = Literal.create( Row("123", "abc", "", null), StructType(Seq( StructField("a", StringType, nullable = true), StructField("b", StringType, nullable = true), StructField("c", StringType, nullable = true), StructField("d", StringType, nullable = true)))) - val struct_notNull = Literal( + val struct_notNull = Literal.create( Row("123", "abc", ""), StructType(Seq( StructField("a", StringType, nullable = false), @@ -712,7 +712,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("complex casting") { - val complex = Literal( + val complex = Literal.create( Row( Seq("123", "abc", ""), Map("a" -> "123", "b" -> "abc", "c" -> ""), @@ -755,30 +755,30 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(c2.isNull, true, row) checkEvaluation(c2.isNotNull, false, row) - checkEvaluation(Literal(1, ShortType).isNull, false) - checkEvaluation(Literal(1, ShortType).isNotNull, true) + checkEvaluation(Literal.create(1, ShortType).isNull, false) + checkEvaluation(Literal.create(1, ShortType).isNotNull, true) - checkEvaluation(Literal(null, ShortType).isNull, true) - checkEvaluation(Literal(null, ShortType).isNotNull, false) + checkEvaluation(Literal.create(null, ShortType).isNull, true) + checkEvaluation(Literal.create(null, ShortType).isNotNull, false) checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) - checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(If(c3, Literal("a", StringType), Literal("b", StringType)), "a", row) + checkEvaluation(If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) checkEvaluation(If(c3, c1, c2), "^Ba*n", row) checkEvaluation(If(c4, c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), - Literal("a", StringType), Literal("b", StringType)), "b", row) + checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), + Literal.create("a", StringType), Literal.create("b", StringType)), "b", row) checkEvaluation(c1 in (c1, c2), true, row) checkEvaluation( - Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType)), true, row) + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row) checkEvaluation( - Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType), c2), true, row) + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row) } test("case when") { @@ -793,9 +793,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(Literal(null, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal(false, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal(true, BooleanType), c4, c6)), "a", row) + checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) @@ -841,17 +841,17 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) - checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) - checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) + checkEvaluation(GetItem(Literal.create(null, typeMap), Literal("aa")), null, row) + checkEvaluation(GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(3, typeMap, true), - Literal(null, StringType)), null, row) + Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) - checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) - checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) + checkEvaluation(GetItem(Literal.create(null, typeArray), Literal(1)), null, row) + checkEvaluation(GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), - Literal(null, IntegerType)), null, row) + Literal.create(null, IntegerType)), null, row) def quickBuildGetField(expr: Expression, fieldName: String) = { expr.dataType match { @@ -864,7 +864,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { def quickResolve(u: UnresolvedGetField) = quickBuildGetField(u.child, u.fieldName) checkEvaluation(quickBuildGetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) - checkEvaluation(quickBuildGetField(Literal(null, typeS), "a"), null, row) + checkEvaluation(quickBuildGetField(Literal.create(null, typeS), "a"), null, row) val typeS_notNullable = StructType( StructField("a", StringType, nullable = false) @@ -874,8 +874,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) - assert(quickBuildGetField(Literal(null, typeS), "a").nullable === true) - assert(quickBuildGetField(Literal(null, typeS_notNullable), "a").nullable === true) + assert(quickBuildGetField(Literal.create(null, typeS), "a").nullable === true) + assert(quickBuildGetField(Literal.create(null, typeS_notNullable), "a").nullable === true) checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) @@ -890,13 +890,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val c4 = 'a.int.at(3) checkEvaluation(UnaryMinus(c1), -1, row) - checkEvaluation(UnaryMinus(Literal(100, IntegerType)), -100) + checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100) checkEvaluation(Add(c1, c4), null, row) checkEvaluation(Add(c1, c2), 3, row) - checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(Add(Literal(null, IntegerType), c2), null, row) - checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation(Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(-c1, -1, row) checkEvaluation(c1 + c2, 3, row) @@ -914,12 +914,12 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val c4 = 'a.double.at(3) checkEvaluation(UnaryMinus(c1), -1.1, row) - checkEvaluation(UnaryMinus(Literal(100.0, DoubleType)), -100.0) + checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0) checkEvaluation(Add(c1, c4), null, row) checkEvaluation(Add(c1, c2), 3.1, row) - checkEvaluation(Add(c1, Literal(null, DoubleType)), null, row) - checkEvaluation(Add(Literal(null, DoubleType), c2), null, row) - checkEvaluation(Add(Literal(null, DoubleType), Literal(null, DoubleType)), null, row) + checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) + checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) + checkEvaluation(Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) checkEvaluation(-c1, -1.1, row) checkEvaluation(c1 + c2, 3.1, row) @@ -940,9 +940,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(LessThan(c1, c4), null, row) checkEvaluation(LessThan(c1, c2), true, row) - checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row) - checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation(LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 < c2, true, row) checkEvaluation(c1 <= c2, true, row) @@ -954,8 +954,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(c1 <=> c4, false, row) checkEvaluation(c4 <=> c6, true, row) checkEvaluation(c3 <=> c5, true, row) - checkEvaluation(Literal(true) <=> Literal(null, BooleanType), false, row) - checkEvaluation(Literal(null, BooleanType) <=> Literal(true), false, row) + checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row) + checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row) } test("StringComparison") { @@ -966,17 +966,17 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(c1 contains "b", true, row) checkEvaluation(c1 contains "x", false, row) checkEvaluation(c2 contains "b", null, row) - checkEvaluation(c1 contains Literal(null, StringType), null, row) + checkEvaluation(c1 contains Literal.create(null, StringType), null, row) checkEvaluation(c1 startsWith "a", true, row) checkEvaluation(c1 startsWith "b", false, row) checkEvaluation(c2 startsWith "a", null, row) - checkEvaluation(c1 startsWith Literal(null, StringType), null, row) + checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row) checkEvaluation(c1 endsWith "c", true, row) checkEvaluation(c1 endsWith "b", false, row) checkEvaluation(c2 endsWith "b", null, row) - checkEvaluation(c1 endsWith Literal(null, StringType), null, row) + checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row) } test("Substring") { @@ -985,54 +985,54 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val s = 'a.string.at(0) // substring from zero position with less-than-full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) // substring from zero position with full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(7, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(7, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) // substring from zero position with greater-than-full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(100, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), "example", row) // substring from nonzero position with less-than-full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(2, IntegerType)), "xa", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), "xa", row) // substring from nonzero position with full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(6, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), "xample", row) // substring from nonzero position with greater-than-full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(100, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), "xample", row) // zero-length substring (within string bounds) - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(0, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), "", row) // zero-length substring (beyond string bounds) - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), "", row) // substring(null, _, _) -> null - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), null, new GenericRow(Array[Any](null))) // substring(_, null, _) -> null - checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row) + checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), null, row) // substring(_, _, null) -> null - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), null, row) // 2-arg substring from zero position - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) // 2-arg substring from nonzero position - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "xample", row) val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) - assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === false) + assert(Substring(s_notNull, Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -1050,7 +1050,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Sqrt(d), expected, row) } - checkEvaluation(Sqrt(Literal(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, new GenericRow(Array[Any](null))) checkEvaluation(Sqrt(-1), null, EmptyRow) checkEvaluation(Sqrt(-1.5), null, EmptyRow) } @@ -1064,22 +1064,22 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(BitwiseAnd(c1, c4), null, row) checkEvaluation(BitwiseAnd(c1, c2), 0, row) - checkEvaluation(BitwiseAnd(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseAnd(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseOr(c1, c4), null, row) checkEvaluation(BitwiseOr(c1, c2), 3, row) - checkEvaluation(BitwiseOr(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseOr(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseXor(c1, c4), null, row) checkEvaluation(BitwiseXor(c1, c2), 3, row) - checkEvaluation(BitwiseXor(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseXor(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseNot(c4), null, row) checkEvaluation(BitwiseNot(c1), -2, row) - checkEvaluation(BitwiseNot(Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 & c2, 0, row) checkEvaluation(c1 | c2, 3, row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index ef10c0aece716..a0efe9e2e7f6b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -182,33 +182,33 @@ class ConstantFoldingSuite extends PlanTest { IsNull(Literal(null)) as 'c1, IsNotNull(Literal(null)) as 'c2, - GetItem(Literal(null, ArrayType(IntegerType)), 1) as 'c3, - GetItem(Literal(Seq(1), ArrayType(IntegerType)), Literal(null, IntegerType)) as 'c4, + GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + GetItem(Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, UnresolvedGetField( - Literal(null, StructType(Seq(StructField("a", IntegerType, true)))), + Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), "a") as 'c5, - UnaryMinus(Literal(null, IntegerType)) as 'c6, + UnaryMinus(Literal.create(null, IntegerType)) as 'c6, Cast(Literal(null), IntegerType) as 'c7, - Not(Literal(null, BooleanType)) as 'c8, + Not(Literal.create(null, BooleanType)) as 'c8, - Add(Literal(null, IntegerType), 1) as 'c9, - Add(1, Literal(null, IntegerType)) as 'c10, + Add(Literal.create(null, IntegerType), 1) as 'c9, + Add(1, Literal.create(null, IntegerType)) as 'c10, - EqualTo(Literal(null, IntegerType), 1) as 'c11, - EqualTo(1, Literal(null, IntegerType)) as 'c12, + EqualTo(Literal.create(null, IntegerType), 1) as 'c11, + EqualTo(1, Literal.create(null, IntegerType)) as 'c12, - Like(Literal(null, StringType), "abc") as 'c13, - Like("abc", Literal(null, StringType)) as 'c14, + Like(Literal.create(null, StringType), "abc") as 'c13, + Like("abc", Literal.create(null, StringType)) as 'c14, - Upper(Literal(null, StringType)) as 'c15, + Upper(Literal.create(null, StringType)) as 'c15, - Substring(Literal(null, StringType), 0, 1) as 'c16, - Substring("abc", Literal(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal(null, IntegerType)) as 'c18, + Substring(Literal.create(null, StringType), 0, 1) as 'c16, + Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, + Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, - Contains(Literal(null, StringType), "abc") as 'c19, - Contains("abc", Literal(null, StringType)) as 'c20 + Contains(Literal.create(null, StringType), "abc") as 'c19, + Contains("abc", Literal.create(null, StringType)) as 'c20 ) val optimized = Optimize(originalQuery.analyze) @@ -219,31 +219,31 @@ class ConstantFoldingSuite extends PlanTest { Literal(true) as 'c1, Literal(false) as 'c2, - Literal(null, IntegerType) as 'c3, - Literal(null, IntegerType) as 'c4, - Literal(null, IntegerType) as 'c5, + Literal.create(null, IntegerType) as 'c3, + Literal.create(null, IntegerType) as 'c4, + Literal.create(null, IntegerType) as 'c5, - Literal(null, IntegerType) as 'c6, - Literal(null, IntegerType) as 'c7, - Literal(null, BooleanType) as 'c8, + Literal.create(null, IntegerType) as 'c6, + Literal.create(null, IntegerType) as 'c7, + Literal.create(null, BooleanType) as 'c8, - Literal(null, IntegerType) as 'c9, - Literal(null, IntegerType) as 'c10, + Literal.create(null, IntegerType) as 'c9, + Literal.create(null, IntegerType) as 'c10, - Literal(null, BooleanType) as 'c11, - Literal(null, BooleanType) as 'c12, + Literal.create(null, BooleanType) as 'c11, + Literal.create(null, BooleanType) as 'c12, - Literal(null, BooleanType) as 'c13, - Literal(null, BooleanType) as 'c14, + Literal.create(null, BooleanType) as 'c13, + Literal.create(null, BooleanType) as 'c14, - Literal(null, StringType) as 'c15, + Literal.create(null, StringType) as 'c15, - Literal(null, StringType) as 'c16, - Literal(null, StringType) as 'c17, - Literal(null, StringType) as 'c18, + Literal.create(null, StringType) as 'c16, + Literal.create(null, StringType) as 'c17, + Literal.create(null, StringType) as 'c18, - Literal(null, BooleanType) as 'c19, - Literal(null, BooleanType) as 'c20 + Literal.create(null, BooleanType) as 'c19, + Literal.create(null, BooleanType) as 'c20 ).analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index e7ce92a2160b6..274f3ede0045c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -90,7 +90,7 @@ class TreeNodeSuite extends FunSuite { } test("transform works on nodes with Option children") { - val dummy1 = Dummy(Some(Literal("1", StringType))) + val dummy1 = Dummy(Some(Literal.create("1", StringType))) val dummy2 = Dummy(None) val toZero: PartialFunction[Expression, Expression] = { case Literal(_, _) => Literal(0) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 89682d25ca7dc..a8018b9213f2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -93,7 +93,7 @@ case class GeneratedAggregate( } val currentSum = AttributeReference("currentSum", calcType, nullable = true)() - val initialValue = Literal(null, calcType) + val initialValue = Literal.create(null, calcType) // Coalasce avoids double calculation... // but really, common sub expression elimination would be better.... @@ -137,13 +137,13 @@ case class GeneratedAggregate( expr.dataType match { case DecimalType.Fixed(_, _) => If(EqualTo(currentCount, Literal(0L)), - Literal(null, a.dataType), + Literal.create(null, a.dataType), Cast(Divide( Cast(currentSum, DecimalType.Unlimited), Cast(currentCount, DecimalType.Unlimited)), a.dataType)) case _ => If(EqualTo(currentCount, Literal(0L)), - Literal(null, a.dataType), + Literal.create(null, a.dataType), Divide(Cast(currentSum, a.dataType), Cast(currentCount, a.dataType))) } @@ -156,7 +156,7 @@ case class GeneratedAggregate( case m @ Max(expr) => val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() - val initialValue = Literal(null, expr.dataType) + val initialValue = Literal.create(null, expr.dataType) val updateMax = MaxOf(currentMax, expr) AggregateEvaluation( 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 19800ad88c031..43f260d3ef8d3 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 @@ -872,9 +872,9 @@ private[sql] object ParquetRelation2 extends Logging { * PartitionValues( * Seq("a", "b", "c"), * Seq( - * Literal(42, IntegerType), - * Literal("hello", StringType), - * Literal(3.14, FloatType))) + * Literal.create(42, IntegerType), + * Literal.create("hello", StringType), + * Literal.create(3.14, FloatType))) * }}} */ private[parquet] def parsePartition( @@ -953,15 +953,16 @@ private[sql] object ParquetRelation2 extends Logging { raw: String, defaultPartitionName: String): Literal = { // First tries integral types - Try(Literal(Integer.parseInt(raw), IntegerType)) - .orElse(Try(Literal(JLong.parseLong(raw), LongType))) + Try(Literal.create(Integer.parseInt(raw), IntegerType)) + .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) // Then falls back to fractional types - .orElse(Try(Literal(JFloat.parseFloat(raw), FloatType))) - .orElse(Try(Literal(JDouble.parseDouble(raw), DoubleType))) - .orElse(Try(Literal(new JBigDecimal(raw), DecimalType.Unlimited))) + .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) + .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) + .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) // Then falls back to string .getOrElse { - if (raw == defaultPartitionName) Literal(null, NullType) else Literal(raw, StringType) + if (raw == defaultPartitionName) Literal.create(null, NullType) + else Literal.create(raw, StringType) } } @@ -980,7 +981,7 @@ private[sql] object ParquetRelation2 extends Logging { } literals.map { case l @ Literal(_, dataType) => - Literal(Cast(l, desiredType).eval(), desiredType) + Literal.create(Cast(l, desiredType).eval(), desiredType) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index adb3c9391f6c2..b7561ce7298cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -45,11 +45,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { assert(inferPartitionColumnValue(raw, defaultPartitionName) === literal) } - check("10", Literal(10, IntegerType)) - check("1000000000000000", Literal(1000000000000000L, LongType)) - check("1.5", Literal(1.5, FloatType)) - check("hello", Literal("hello", StringType)) - check(defaultPartitionName, Literal(null, NullType)) + check("10", Literal.create(10, IntegerType)) + check("1000000000000000", Literal.create(1000000000000000L, LongType)) + check("1.5", Literal.create(1.5, FloatType)) + check("hello", Literal.create("hello", StringType)) + check(defaultPartitionName, Literal.create(null, NullType)) } test("parse partition") { @@ -75,22 +75,22 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { "file://path/a=10", PartitionValues( ArrayBuffer("a"), - ArrayBuffer(Literal(10, IntegerType)))) + ArrayBuffer(Literal.create(10, IntegerType)))) check( "file://path/a=10/b=hello/c=1.5", PartitionValues( ArrayBuffer("a", "b", "c"), ArrayBuffer( - Literal(10, IntegerType), - Literal("hello", StringType), - Literal(1.5, FloatType)))) + Literal.create(10, IntegerType), + Literal.create("hello", StringType), + Literal.create(1.5, FloatType)))) check( "file://path/a=10/b_hello/c=1.5", PartitionValues( ArrayBuffer("c"), - ArrayBuffer(Literal(1.5, FloatType)))) + ArrayBuffer(Literal.create(1.5, FloatType)))) checkThrows[AssertionError]("file://path/=10", "Empty partition column name") checkThrows[AssertionError]("file://path/a=", "Empty partition column value") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 4afa2e71d77cc..921c6194c7b76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -593,7 +593,7 @@ private[hive] trait HiveInspectors { case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") // ideally, we don't test the foldable here(but in optimizer), however, some of the // Hive UDF / UDAF requires its argument to be constant objectinspector, we do it eagerly. - case _ if expr.foldable => toInspector(Literal(expr.eval(), expr.dataType)) + case _ if expr.foldable => toInspector(Literal.create(expr.eval(), expr.dataType)) // For those non constant expression, map to object inspector according to its data type case _ => toInspector(expr.dataType) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index cd8e7c09eea5b..5be09a11ad641 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1201,7 +1201,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C CreateArray(children.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => - Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) + Substring(nodeToExpr(string), nodeToExpr(pos), Literal.create(Integer.MAX_VALUE, IntegerType)) case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr)) @@ -1213,9 +1213,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C UnresolvedFunction(name, UnresolvedStar(None) :: Nil) /* Literals */ - case Token("TOK_NULL", Nil) => Literal(null, NullType) - case Token(TRUE(), Nil) => Literal(true, BooleanType) - case Token(FALSE(), Nil) => Literal(false, BooleanType) + case Token("TOK_NULL", Nil) => Literal.create(null, NullType) + case Token(TRUE(), Nil) => Literal.create(true, BooleanType) + case Token(FALSE(), Nil) => Literal.create(false, BooleanType) case Token("TOK_STRINGLITERALSEQUENCE", strings) => Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString) @@ -1226,21 +1226,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C try { if (ast.getText.endsWith("L")) { // Literal bigint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) } else if (ast.getText.endsWith("S")) { // Literal smallint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) } else if (ast.getText.endsWith("Y")) { // Literal tinyint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { // Literal decimal val strVal = ast.getText.stripSuffix("D").stripSuffix("B") v = Literal(Decimal(strVal)) } else { - v = Literal(ast.getText.toDouble, DoubleType) - v = Literal(ast.getText.toLong, LongType) - v = Literal(ast.getText.toInt, IntegerType) + v = Literal.create(ast.getText.toDouble, DoubleType) + v = Literal.create(ast.getText.toLong, LongType) + v = Literal.create(ast.getText.toInt, IntegerType) } } catch { case nfe: NumberFormatException => // Do nothing diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index 3181cfe40016c..c482c6de8a736 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -79,9 +79,9 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: - Literal(Seq[Int](1,2,3), ArrayType(IntegerType)) :: - Literal(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) :: - Literal(Row(1,2.0d,3.0f), + Literal.create(Seq[Int](1,2,3), ArrayType(IntegerType)) :: + Literal.create(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) :: + Literal.create(Row(1,2.0d,3.0f), StructType(StructField("c1", IntegerType) :: StructField("c2", DoubleType) :: StructField("c3", FloatType) :: Nil)) :: @@ -166,7 +166,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val constantData = constantExprs.map(_.eval()) val constantNullData = constantData.map(_ => null) val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType)) - val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal(null, e.dataType))) + val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) checkValues(constantData, constantData.zip(constantWritableOIs).map { case (d, oi) => unwrap(wrap(d, oi), oi) @@ -212,8 +212,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = row(0) :: row(0) :: Nil checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } test("wrap / unwrap Map Type") { @@ -222,7 +222,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = Map(row(0) -> row(1)) checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } } From 6562787b963204763a33e1c4e9d192db913af1fc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 1 Apr 2015 23:42:09 -0700 Subject: [PATCH 617/817] [SPARK-6627] Some clean-up in shuffle code. Before diving into review #4450 I did a look through the existing shuffle code to learn how it works. Unfortunately, there are some very confusing things in this code. This patch makes a few small changes to simplify things. It is not easily to concisely describe the changes because of how convoluted the issues were, but they are fairly small logically: 1. There is a trait named `ShuffleBlockManager` that only deals with one logical function which is retrieving shuffle block data given shuffle block coordinates. This trait has two implementors FileShuffleBlockManager and IndexShuffleBlockManager. Confusingly the vast majority of those implementations have nothing to do with this particular functionality. So I've renamed the trait to ShuffleBlockResolver and documented it. 2. The aforementioned trait had two almost identical methods, for no good reason. I removed one method (getBytes) and modified callers to use the other one. I think the behavior is preserved in all cases. 3. The sort shuffle code uses an identifier "0" in the reduce slot of a BlockID as a placeholder. I made it into a constant since it needs to be consistent across multiple places. I think for (3) there is actually a better solution that would avoid the need to do this type of workaround/hack in the first place, but it's more complex so I'm punting it for now. Author: Patrick Wendell Closes #5286 from pwendell/cleanup and squashes the following commits: c71fbc7 [Patrick Wendell] Open interface back up for testing f36edd5 [Patrick Wendell] Code review feedback d1c0494 [Patrick Wendell] Style fix a406079 [Patrick Wendell] [HOTFIX] Some clean-up in shuffle code. --- .../shuffle/FileShuffleBlockManager.scala | 7 +---- .../shuffle/IndexShuffleBlockManager.scala | 27 +++++++++---------- ...nager.scala => ShuffleBlockResolver.scala} | 14 ++++++---- .../apache/spark/shuffle/ShuffleManager.scala | 5 +++- .../apache/spark/shuffle/ShuffleWriter.scala | 2 +- .../shuffle/hash/HashShuffleManager.scala | 8 +++--- .../shuffle/sort/SortShuffleManager.scala | 9 ++++--- .../shuffle/sort/SortShuffleWriter.scala | 6 ++--- .../apache/spark/storage/BlockManager.scala | 14 ++++------ .../util/collection/ExternalSorter.scala | 6 +++-- .../hash/HashShuffleManagerSuite.scala | 2 +- .../spark/tools/StoragePerfTester.scala | 2 +- 12 files changed, 51 insertions(+), 51 deletions(-) rename core/src/main/scala/org/apache/spark/shuffle/{ShuffleBlockManager.scala => ShuffleBlockResolver.scala} (68%) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index d0178dfde6935..5be3ed771e534 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -67,7 +67,7 @@ private[spark] trait ShuffleWriterGroup { // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getHashBasedShuffleBlockData(). private[spark] class FileShuffleBlockManager(conf: SparkConf) - extends ShuffleBlockManager with Logging { + extends ShuffleBlockResolver with Logging { private val transportConf = SparkTransportConf.fromSparkConf(conf) @@ -175,11 +175,6 @@ class FileShuffleBlockManager(conf: SparkConf) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockData(blockId) - Some(segment.nioByteBuffer()) - } - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 87fd161e06c85..50edb5a34e333 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -27,6 +27,8 @@ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ +import IndexShuffleBlockManager.NOOP_REDUCE_ID + /** * Create and maintain the shuffle blocks' mapping between logic block and physical file location. * Data of shuffle blocks from the same map task are stored in a single consolidated data file. @@ -39,25 +41,18 @@ import org.apache.spark.storage._ // Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). private[spark] -class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { +class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { private lazy val blockManager = SparkEnv.get.blockManager private val transportConf = SparkTransportConf.fromSparkConf(conf) - /** - * Mapping to a single shuffleBlockId with reduce ID 0. - * */ - def consolidateId(shuffleId: Int, mapId: Int): ShuffleBlockId = { - ShuffleBlockId(shuffleId, mapId, 0) - } - def getDataFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, 0)) + blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } private def getIndexFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, 0)) + blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } /** @@ -97,10 +92,6 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - Some(getBlockData(blockId).nioByteBuffer()) - } - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index @@ -123,3 +114,11 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { override def stop(): Unit = {} } + +private[spark] object IndexShuffleBlockManager { + // No-op reduce ID used in interactions with disk store and BlockObjectWriter. + // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort + // shuffle outputs for several reduces are glommed into a single file. + // TODO: Avoid this entirely by having the DiskBlockObjectWriter not require a BlockId. + val NOOP_REDUCE_ID = 0 +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala similarity index 68% rename from core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala rename to core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index b521f0c7fc77e..4342b0d598b16 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -22,15 +22,19 @@ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId private[spark] -trait ShuffleBlockManager { +/** + * Implementers of this trait understand how to retrieve block data for a logical shuffle block + * identifier (i.e. map, reduce, and shuffle). Implementations may use files or file segments to + * encapsulate shuffle data. This is used by the BlockStore to abstract over different shuffle + * implementations when shuffle data is retrieved. + */ +trait ShuffleBlockResolver { type ShuffleId = Int /** - * Get shuffle block data managed by the local ShuffleBlockManager. - * @return Some(ByteBuffer) if block found, otherwise None. + * Retrieve the data for the specified block. If the data for that block is not available, + * throws an unspecified exception. */ - def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] - def getBlockData(blockId: ShuffleBlockId): ManagedBuffer def stop(): Unit diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index a44a8e1249256..978366d1a1d1b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -55,7 +55,10 @@ private[spark] trait ShuffleManager { */ def unregisterShuffle(shuffleId: Int): Boolean - def shuffleBlockManager: ShuffleBlockManager + /** + * Return a resolver capable of retrieving shuffle block data based on block coordinates. + */ + def shuffleBlockResolver: ShuffleBlockResolver /** Shut down this ShuffleManager. */ def stop(): Unit diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index b934480cfb9be..f6e6fe5defe09 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -23,7 +23,7 @@ import org.apache.spark.scheduler.MapStatus * Obtained inside a map task to write out records to the shuffle system. */ private[spark] trait ShuffleWriter[K, V] { - /** Write a bunch of records to this task's output */ + /** Write a sequence of records to this task's output */ def write(records: Iterator[_ <: Product2[K, V]]): Unit /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 62e0629b34400..2a7df8dd5bd83 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -53,20 +53,20 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) : ShuffleWriter[K, V] = { new HashShuffleWriter( - shuffleBlockManager, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + shuffleBlockResolver, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - shuffleBlockManager.removeShuffle(shuffleId) + shuffleBlockResolver.removeShuffle(shuffleId) } - override def shuffleBlockManager: FileShuffleBlockManager = { + override def shuffleBlockResolver: FileShuffleBlockManager = { fileShuffleBlockManager } /** Shut down this ShuffleManager. */ override def stop(): Unit = { - shuffleBlockManager.stop() + shuffleBlockResolver.stop() } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index bda30a56d808e..0497036192154 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -58,7 +58,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, V, _]] shuffleMapNumber.putIfAbsent(baseShuffleHandle.shuffleId, baseShuffleHandle.numMaps) new SortShuffleWriter( - shuffleBlockManager, baseShuffleHandle, mapId, context) + shuffleBlockResolver, baseShuffleHandle, mapId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ @@ -66,18 +66,19 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager if (shuffleMapNumber.containsKey(shuffleId)) { val numMaps = shuffleMapNumber.remove(shuffleId) (0 until numMaps).map{ mapId => - shuffleBlockManager.removeDataByMap(shuffleId, mapId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId) } } true } - override def shuffleBlockManager: IndexShuffleBlockManager = { + override def shuffleBlockResolver: IndexShuffleBlockManager = { indexShuffleBlockManager } /** Shut down this ShuffleManager. */ override def stop(): Unit = { - shuffleBlockManager.stop() + shuffleBlockResolver.stop() } } + diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 55ea0f17b156a..a066435df6fb0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -58,8 +58,7 @@ private[spark] class SortShuffleWriter[K, V, C]( // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't // care whether the keys get sorted in each partition; that will be done on the reduce side // if the operation being run is sortByKey. - sorter = new ExternalSorter[K, V, V]( - None, Some(dep.partitioner), None, dep.serializer) + sorter = new ExternalSorter[K, V, V](None, Some(dep.partitioner), None, dep.serializer) sorter.insertAll(records) } @@ -67,7 +66,7 @@ private[spark] class SortShuffleWriter[K, V, C]( // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) - val blockId = shuffleBlockManager.consolidateId(dep.shuffleId, mapId) + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) @@ -100,3 +99,4 @@ private[spark] class SortShuffleWriter[K, V, C]( } } } + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1dff09a75d038..fc31296f4deb3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -301,7 +301,7 @@ private[spark] class BlockManager( */ override def getBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { - shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) .asInstanceOf[Option[ByteBuffer]] @@ -439,14 +439,10 @@ private[spark] class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { - val shuffleBlockManager = shuffleManager.shuffleBlockManager - shuffleBlockManager.getBytes(blockId.asInstanceOf[ShuffleBlockId]) match { - case Some(bytes) => - Some(bytes) - case None => - throw new BlockException( - blockId, s"Block $blockId not found on disk, though it should be") - } + val shuffleBlockManager = shuffleManager.shuffleBlockResolver + // TODO: This should gracefully handle case where local block is not available. Currently + // downstream code will throw an exception. + Option(shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) } else { doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b962c101c91da..7bd3c7852a6b2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -664,6 +664,8 @@ private[spark] class ExternalSorter[K, V, C]( } /** + * Exposed for testing purposes. + * * Return an iterator over all the data written to this object, grouped by partition and * aggregated by the requested aggregator. For each partition we then have an iterator over its * contents, and these are expected to be accessed in order (you can't "skip ahead" to one @@ -673,7 +675,7 @@ private[spark] class ExternalSorter[K, V, C]( * For now, we just merge all the spilled files in once pass, but this can be modified to * support hierarchical merging. */ - def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { + def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { val usingMap = aggregator.isDefined val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer if (spills.isEmpty && partitionWriters == null) { @@ -781,7 +783,7 @@ private[spark] class ExternalSorter[K, V, C]( /** * Read a partition file back as an iterator (used in our iterator method) */ - def readPartitionFile(writer: BlockObjectWriter): Iterator[Product2[K, C]] = { + private def readPartitionFile(writer: BlockObjectWriter): Iterator[Product2[K, C]] = { if (writer.isOpen) { writer.commitAndClose() } diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 6790388f96603..b834dc0e735eb 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -54,7 +54,7 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test", conf) val shuffleBlockManager = - SparkEnv.get.shuffleManager.shuffleBlockManager.asInstanceOf[FileShuffleBlockManager] + SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[FileShuffleBlockManager] val shuffle1 = shuffleBlockManager.forMapTask(1, 1, 1, new JavaSerializer(conf), new ShuffleWriteMetrics) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 15ee95070a3d3..6b666a0384879 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -59,7 +59,7 @@ object StoragePerfTester { val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong) = { - val shuffle = hashShuffleManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, + val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { From 424e987dfebbbaa37f4496d44090d469a931ce76 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 2 Apr 2015 17:57:01 +0800 Subject: [PATCH 618/817] [SPARK-6672][SQL] convert row to catalyst in createDataFrame(RDD[Row], ...) We assume that `RDD[Row]` contains Scala types. So we need to convert them into catalyst types in createDataFrame. liancheng Author: Xiangrui Meng Closes #5329 from mengxr/SPARK-6672 and squashes the following commits: 2d52644 [Xiangrui Meng] set needsConversion = false in jsonRDD 06896e4 [Xiangrui Meng] add createDataFrame without conversion 4a3767b [Xiangrui Meng] convert Row to catalyst --- .../spark/sql/catalyst/ScalaReflection.scala | 5 +++++ .../org/apache/spark/sql/DataFrame.scala | 3 ++- .../org/apache/spark/sql/SQLContext.scala | 20 ++++++++++++++++--- .../apache/spark/sql/parquet/newParquet.scala | 3 ++- .../apache/spark/sql/sources/commands.scala | 3 ++- .../spark/sql/test/ExamplePointUDT.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 9 ++++++++- 7 files changed, 37 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 2220970085462..8bfd0471d9c7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -72,6 +72,11 @@ trait ScalaReflection { case (d: BigDecimal, _) => Decimal(d) case (d: java.math.BigDecimal, _) => Decimal(d) case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) + case (r: Row, structType: StructType) => + new GenericRow( + r.toSeq.zip(structType.fields).map { case (elem, field) => + convertToCatalyst(elem, field.dataType) + }.toArray) case (other, _) => other } 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 ce0890906bf1b..34be17325b2b0 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 @@ -904,7 +904,8 @@ class DataFrame private[sql]( */ override def repartition(numPartitions: Int): DataFrame = { sqlContext.createDataFrame( - queryExecution.toRdd.map(_.copy()).repartition(numPartitions), schema) + queryExecution.toRdd.map(_.copy()).repartition(numPartitions), + schema, needsConversion = false) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 1794936a52c6d..39dd14e796f06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -392,9 +392,23 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @DeveloperApi def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema, needsConversion = true) + } + + /** + * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be + * converted to Catalyst rows. + */ + private[sql] + def createDataFrame(rowRDD: RDD[Row], schema: StructType, needsConversion: Boolean) = { // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. - val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) + val catalystRows = if (needsConversion) { + rowRDD.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row]) + } else { + rowRDD + } + val logicalPlan = LogicalRDD(schema.toAttributes, catalystRows)(self) DataFrame(this, logicalPlan) } @@ -604,7 +618,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) } /** @@ -633,7 +647,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) } /** 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 43f260d3ef8d3..e12531480ce92 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 @@ -122,7 +122,8 @@ private[sql] class DefaultSource val df = sqlContext.createDataFrame( data.queryExecution.toRdd, - data.schema.asNullable) + data.schema.asNullable, + needsConversion = false) val createdRelation = createRelation(sqlContext, parameters, df.schema).asInstanceOf[ParquetRelation2] createdRelation.insert(df, overwrite = mode == SaveMode.Overwrite) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 9bbe06e59ba30..dbdb0d39c26a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -31,7 +31,8 @@ private[sql] case class InsertIntoDataSource( val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] val data = DataFrame(sqlContext, query) // Apply the schema of the existing table to the new data. - val df = sqlContext.createDataFrame(data.queryExecution.toRdd, logicalRelation.schema) + val df = sqlContext.createDataFrame( + data.queryExecution.toRdd, logicalRelation.schema, needsConversion = false) relation.insert(df, overwrite) // Invalidate the cache. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index c11d0ae5bf1cc..2fdd798b44bb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ * @param y y coordinate */ @SQLUserDefinedType(udt = classOf[ExamplePointUDT]) -private[sql] class ExamplePoint(val x: Double, val y: Double) +private[sql] class ExamplePoint(val x: Double, val y: Double) extends Serializable /** * User-defined type for [[ExamplePoint]]. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 6761d996fd975..5297cc01eddfc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -21,7 +21,7 @@ import scala.language.postfixOps import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, TestSQLContext} import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.test.TestSQLContext.sql @@ -506,4 +506,11 @@ class DataFrameSuite extends QueryTest { testData.select($"*").show() testData.select($"*").show(1000) } + + test("createDataFrame(RDD[Row], StructType) should convert UDTs (SPARK-6672)") { + val rowRDD = TestSQLContext.sparkContext.parallelize(Seq(Row(new ExamplePoint(1.0, 2.0)))) + val schema = StructType(Array(StructField("point", new ExamplePointUDT(), false))) + val df = TestSQLContext.createDataFrame(rowRDD, schema) + df.rdd.collect() + } } From 0cce5451adfc6bf4661bcf67aca3db26376455fe Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 2 Apr 2015 12:18:33 -0700 Subject: [PATCH 619/817] [SPARK-6667] [PySpark] remove setReuseAddress The reused address on server side had caused the server can not acknowledge the connected connections, remove it. This PR will retry once after timeout, it also add a timeout at client side. Author: Davies Liu Closes #5324 from davies/collect_hang and squashes the following commits: e5a51a2 [Davies Liu] remove setReuseAddress 7977c2f [Davies Liu] do retry on client side b838f35 [Davies Liu] retry after timeout --- core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala | 1 - python/pyspark/rdd.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 19f4c95fcad74..36cf2af0857dd 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -605,7 +605,6 @@ private[spark] object PythonRDD extends Logging { */ private def serveIterator[T](items: Iterator[T], threadName: String): Int = { val serverSocket = new ServerSocket(0, 1) - serverSocket.setReuseAddress(true) // Close the socket if no connection in 3 seconds serverSocket.setSoTimeout(3000) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c337a43c8a7fc..2d05611321ed6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -113,6 +113,7 @@ def _parse_memory(s): def _load_from_socket(port, serializer): sock = socket.socket() + sock.settimeout(3) try: sock.connect(("localhost", port)) rf = sock.makefile("rb", 65536) From e3202aa2e9bd140effbcf2a7a02b90cb077e760b Mon Sep 17 00:00:00 2001 From: Hung Lin Date: Thu, 2 Apr 2015 14:01:43 -0700 Subject: [PATCH 620/817] SPARK-6414: Spark driver failed with NPE on job cancelation Use Option for ActiveJob.properties to avoid NPE bug Author: Hung Lin Closes #5124 from hunglin/SPARK-6414 and squashes the following commits: 2290b6b [Hung Lin] [SPARK-6414][core] Fix NPE in SparkContext.cancelJobGroup() --- .../scala/org/apache/spark/SparkContext.scala | 4 +--- .../apache/spark/scheduler/DAGScheduler.scala | 10 +++++----- .../org/apache/spark/SparkContextSuite.scala | 20 ++++++++++++++++++- 3 files changed, 25 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a70be16f77eeb..3904f7d1060c5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -433,6 +433,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Thread Local variable that can be used by users to pass information down the stack private val localProperties = new InheritableThreadLocal[Properties] { override protected def childValue(parent: Properties): Properties = new Properties(parent) + override protected def initialValue(): Properties = new Properties() } /** @@ -474,9 +475,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Spark fair scheduler pool. */ def setLocalProperty(key: String, value: String) { - if (localProperties.get() == null) { - localProperties.set(new Properties()) - } if (value == null) { localProperties.get.remove(key) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d35b4f9dbaf88..7227fa9da4317 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -493,7 +493,7 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null): JobWaiter[U] = { + properties: Properties): JobWaiter[U] = { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length partitions.find(p => p >= maxPartitions || p < 0).foreach { p => @@ -522,7 +522,7 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null): Unit = { + properties: Properties): Unit = { val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { @@ -542,7 +542,7 @@ class DAGScheduler( evaluator: ApproximateEvaluator[U, R], callSite: CallSite, timeout: Long, - properties: Properties = null): PartialResult[R] = { + properties: Properties): PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray @@ -689,7 +689,7 @@ class DAGScheduler( // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. val activeInGroup = activeJobs.filter(activeJob => - groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + Option(activeJob.properties).exists(_.get(SparkContext.SPARK_JOB_GROUP_ID) == groupId)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) submitWaitingStages() @@ -736,7 +736,7 @@ class DAGScheduler( allowLocal: Boolean, callSite: CallSite, listener: JobListener, - properties: Properties = null) { + properties: Properties) { var finalStage: ResultStage = null try { // New stage creation may throw an exception if, for example, jobs are run on a diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index b07c4d93db4e6..c7301a30d8b11 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.io.File +import java.util.concurrent.TimeUnit import com.google.common.base.Charsets._ import com.google.common.io.Files @@ -25,9 +26,11 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.hadoop.io.BytesWritable - import org.apache.spark.util.Utils +import scala.concurrent.Await +import scala.concurrent.duration.Duration + class SparkContextSuite extends FunSuite with LocalSparkContext { test("Only one SparkContext may be active at a time") { @@ -173,4 +176,19 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } + + test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val future = sc.parallelize(Seq(0)).foreachAsync(_ => {Thread.sleep(1000L)}) + sc.cancelJobGroup("nonExistGroupId") + Await.ready(future, Duration(2, TimeUnit.SECONDS)) + + // In SPARK-6414, sc.cancelJobGroup will cause NullPointerException and cause + // SparkContext to shutdown, so the following assertion will fail. + assert(sc.parallelize(1 to 10).count() == 10L) + } finally { + sc.stop() + } + } } From 4214e50fc32de1478584d8edfa3a35576c12c025 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Apr 2015 16:01:03 -0700 Subject: [PATCH 621/817] [SQL] Throw UnsupportedOperationException instead of NotImplementedError NotImplementedError in scala 2.10 is a fatal exception, which is not very nice to throw when not actually fatal. Author: Michael Armbrust Closes #5315 from marmbrus/throwUnsupported and squashes the following commits: c29e03b [Michael Armbrust] [SQL] Throw UnsupportedOperationException instead of NotImplementedError 052e05b [Michael Armbrust] [SQL] Throw UnsupportedOperationException instead of NotImplementedError --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 5 ++--- .../scala/org/apache/spark/sql/hive/StatisticsSuite.scala | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 6bb1c47dba920..46991fbd68cde 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -184,9 +184,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.client.alterTable(tableFullName, new Table(hiveTTable)) } case otherRelation => - throw new NotImplementedError( - s"Analyze has only implemented for Hive tables, " + - s"but $tableName is a ${otherRelation.nodeName}") + throw new UnsupportedOperationException( + s"Analyze only works for Hive tables, but $tableName is a ${otherRelation.nodeName}") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 1e05a024b8807..ccd0e5aa51f95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -120,7 +120,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // Try to analyze a temp table sql("""SELECT * FROM src""").registerTempTable("tempTable") - intercept[NotImplementedError] { + intercept[UnsupportedOperationException] { analyze("tempTable") } catalog.unregisterTable(Seq("tempTable")) From 251698fb7335a3bb465f1cd0c29e7e74e0361f4a Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 2 Apr 2015 16:02:31 -0700 Subject: [PATCH 622/817] [SPARK-6655][SQL] We need to read the schema of a data source table stored in spark.sql.sources.schema property https://issues.apache.org/jira/browse/SPARK-6655 Author: Yin Huai Closes #5313 from yhuai/SPARK-6655 and squashes the following commits: 1e00c03 [Yin Huai] Unnecessary change. f131bd9 [Yin Huai] Fix. f1218c1 [Yin Huai] Failed test. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 +++++++++++---- .../sql/hive/MetastoreDataSourcesSuite.scala | 23 +++++++++++++++++++ 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f0076cef13777..14cdb420731cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -70,7 +70,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val table = synchronized { client.getTable(in.database, in.name) } - val userSpecifiedSchema = + + def schemaStringFromParts: Option[String] = { Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => val parts = (0 until numParts.toInt).map { index => val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") @@ -82,10 +83,19 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with part } - // Stick all parts back to a single schema string in the JSON representation - // and convert it back to a StructType. - DataType.fromJson(parts.mkString).asInstanceOf[StructType] + // Stick all parts back to a single schema string. + parts.mkString } + } + + // Originally, we used spark.sql.sources.schema to store the schema of a data source table. + // After SPARK-6024, we removed this flag. + // Although we are not using spark.sql.sources.schema any more, we need to still support. + val schemaString = + Option(table.getProperty("spark.sql.sources.schema")).orElse(schemaStringFromParts) + + val userSpecifiedSchema = + schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index e5ad0bf552073..e09c702c8969e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -25,6 +25,8 @@ import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.TableType +import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql._ @@ -682,6 +684,27 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { assert(schema === actualSchema) } + test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") { + val tableName = "spark6655" + val schema = StructType(StructField("int", IntegerType, true) :: Nil) + // Manually create the metadata in metastore. + val tbl = new Table("default", tableName) + tbl.setProperty("spark.sql.sources.provider", "json") + tbl.setProperty("spark.sql.sources.schema", schema.json) + tbl.setProperty("EXTERNAL", "FALSE") + tbl.setTableType(TableType.MANAGED_TABLE) + tbl.setSerdeParam("path", catalog.hiveDefaultTableFilePath(tableName)) + catalog.synchronized { + catalog.client.createTable(tbl) + } + + invalidateTable(tableName) + val actualSchema = table(tableName).schema + assert(schema === actualSchema) + sql(s"drop table $tableName") + } + + test("insert into a table") { def createDF(from: Int, to: Int): DataFrame = createDataFrame((from to to).map(i => Tuple2(i, s"str$i"))).toDF("c1", "c2") From d3944b6f2aeb36629bf89207629cc5e55d327241 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 2 Apr 2015 16:15:34 -0700 Subject: [PATCH 623/817] [Minor] [SQL] Follow-up of PR #5210 This PR addresses rxin's comments in PR #5210. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5219) Author: Cheng Lian Closes #5219 from liancheng/spark-6554-followup and squashes the following commits: 41f3a09 [Cheng Lian] Addresses comments in #5210 --- .../scala/org/apache/spark/sql/parquet/newParquet.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) 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 e12531480ce92..583bac42fdcce 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 @@ -434,17 +434,18 @@ private[sql] case class ParquetRelation2( FileInputFormat.setInputPaths(job, selectedFiles.map(_.getPath): _*) } - // Push down filters when possible. Notice that not all filters can be converted to Parquet - // filter predicate. Here we try to convert each individual predicate and only collect those - // convertible ones. + // Try to push down filters when filter push-down is enabled. if (sqlContext.conf.parquetFilterPushDown) { + val partitionColNames = partitionColumns.map(_.name).toSet predicates // Don't push down predicates which reference partition columns .filter { pred => - val partitionColNames = partitionColumns.map(_.name).toSet val referencedColNames = pred.references.map(_.name).toSet referencedColNames.intersect(partitionColNames).isEmpty } + // 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) .reduceOption(FilterApi.and) .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) From 5db89127e72630aec7c5552f2c84018ae18d03fe Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 2 Apr 2015 16:46:50 -0700 Subject: [PATCH 624/817] [SPARK-6618][SPARK-6669][SQL] Lock Hive metastore client correctly. Author: Yin Huai Author: Michael Armbrust Closes #5333 from yhuai/lookupRelationLock and squashes the following commits: 59c884f [Michael Armbrust] [SQL] Lock metastore client in analyzeTable 7667030 [Yin Huai] Merge pull request #2 from marmbrus/pr/5333 e4a9b0b [Michael Armbrust] Correctly lock on MetastoreCatalog d6fc32f [Yin Huai] Missing `)`. 1e241af [Yin Huai] Protect InsertIntoHive. fee7e9c [Yin Huai] A test? 5416b0f [Yin Huai] Just protect client. --- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 +++-- .../hive/execution/InsertIntoHiveTable.scala | 51 +++++++++++-------- .../sql/hive/execution/SQLQuerySuite.scala | 11 ++++ 4 files changed, 53 insertions(+), 27 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 46991fbd68cde..7c6a7df2bd01e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -181,7 +181,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val tableFullName = relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName - catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + catalog.synchronized { + catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + } } case otherRelation => throw new UnsupportedOperationException( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 14cdb420731cd..bbd920a4051de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -67,7 +67,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = synchronized { + val table = HiveMetastoreCatalog.this.synchronized { client.getTable(in.database, in.name) } @@ -183,12 +183,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with def lookupRelation( tableIdentifier: Seq[String], - alias: Option[String]): LogicalPlan = synchronized { + alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - val table = try client.getTable(databaseName, tblName) catch { + val table = try { + synchronized { + client.getTable(databaseName, tblName) + } + } catch { case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => throw new NoSuchTableException } @@ -210,7 +214,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } else { val partitions: Seq[Partition] = if (table.isPartitioned) { - HiveShim.getAllPartitionsOf(client, table).toSeq + synchronized { + HiveShim.getAllPartitionsOf(client, table).toSeq + } } else { Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index cdf012b5117be..6c96747439683 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -50,7 +50,7 @@ case class InsertIntoHiveTable( @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) - @transient private lazy val db = Hive.get(sc.hiveconf) + @transient private lazy val catalog = sc.catalog private def newSerializer(tableDesc: TableDesc): Serializer = { val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] @@ -199,38 +199,45 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - db.validatePartitionNameCharacters(partVals) + catalog.synchronized { + catalog.client.validatePartitionNameCharacters(partVals) + } // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false if (numDynamicPartitions > 0) { - db.loadDynamicPartitions( - outputPath, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - numDynamicPartitions, - holdDDLTime, - isSkewedStoreAsSubdir - ) + catalog.synchronized { + catalog.client.loadDynamicPartitions( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + numDynamicPartitions, + holdDDLTime, + isSkewedStoreAsSubdir) + } } else { - db.loadPartition( + catalog.synchronized { + catalog.client.loadPartition( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } + } + } else { + catalog.synchronized { + catalog.client.loadTable( outputPath, qualifiedTableName, - orderedPartitionSpec, overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + holdDDLTime) } - } else { - db.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) } // Invalidate the cache. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 310c2bfdf1011..2065f0d60d92f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -457,4 +457,15 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") setConf("spark.sql.hive.convertCTAS", originalConf) } + + test("sanity test for SPARK-6618") { + (1 to 100).par.map { i => + val tableName = s"SPARK_6618_table_$i" + sql(s"CREATE TABLE $tableName (col1 string)") + catalog.lookupRelation(Seq(tableName)) + table(tableName) + tables() + sql(s"DROP TABLE $tableName") + } + } } From dfd2982bc7047732197f1d9ad77221e9c6076fc2 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 2 Apr 2015 17:20:31 -0700 Subject: [PATCH 625/817] [SQL][Minor] Use analyzed logical instead of unresolved in HiveComparisonTest Some internal unit test failed due to the logical plan node in pattern matching in `HiveComparisonTest`, e.g. https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala#L137 Which will may call the `output` function on an unresolved logical plan. Author: Cheng Hao Closes #4946 from chenghao-intel/logical and squashes the following commits: 432ecb3 [Cheng Hao] Use analyzed instead of logical in HiveComparisonTest --- .../apache/spark/sql/hive/execution/HiveComparisonTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 8f3285242091c..a5ec312ee430c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -138,7 +138,7 @@ abstract class HiveComparisonTest case _ => plan.children.iterator.exists(isSorted) } - val orderedAnswer = hiveQuery.logical match { + val orderedAnswer = hiveQuery.analyzed match { // Clean out non-deterministic time schema info. // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. @@ -299,7 +299,7 @@ abstract class HiveComparisonTest val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. - hiveQueries.foreach(_.logical) + hiveQueries.foreach(_.analyzed) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { case ((queryString, i), hiveQuery, cachedAnswerFile)=> try { From 947802cb0de581e51f8141f6663e896de3d753ce Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Thu, 2 Apr 2015 17:23:51 -0700 Subject: [PATCH 626/817] [SPARK-6243][SQL] The Operation of match did not conside the scenarios that order.dataType does not match NativeType It did not conside that order.dataType does not match NativeType. So i add "case other => ..." for other cenarios. Author: DoingDone9 <799203320@qq.com> Closes #4959 from DoingDone9/case_ and squashes the following commits: 6278846 [DoingDone9] Update rows.scala cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/sql/catalyst/expressions/rows.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index a8983df208318..0a275b84086cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -224,6 +224,7 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { n.ordering.asInstanceOf[Ordering[Any]].compare(left, right) case n: NativeType if order.direction == Descending => n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) + case other => sys.error(s"Type $other does not support ordered operations") } if (comparison != 0) return comparison } From 052dee0707830cfd3cd8821ecc3471a37ede294a Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Apr 2015 18:30:55 -0700 Subject: [PATCH 627/817] [SPARK-6686][SQL] Use resolved output instead of names for toDF rename This is a workaround for a problem reported on the user list. This doesn't fix the core problem, but in general is a more robust way to do renames. Author: Michael Armbrust Closes #5337 from marmbrus/toDFrename and squashes the following commits: 6a3159d [Michael Armbrust] [SPARK-6686][SQL] Use resolved output instead of names for toDF rename --- .../src/main/scala/org/apache/spark/sql/DataFrame.scala | 4 ++-- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) 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 34be17325b2b0..5c6016a4a2ce2 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 @@ -240,8 +240,8 @@ class DataFrame private[sql]( s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" + s"New column names (${colNames.size}): " + colNames.mkString(", ")) - val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => - apply(oldName).as(newName) + val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => + Column(oldAttribute).as(newName) } select(newCols :_*) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 5297cc01eddfc..1db0cf7daac03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -60,6 +60,14 @@ class DataFrameSuite extends QueryTest { assert($"test".toString === "test") } + test("rename nested groupby") { + val df = Seq((1,(1,1))).toDF() + + checkAnswer( + df.groupBy("_1").agg(col("_1"), sum("_2._1")).toDF("key", "total"), + Row(1, 1) :: Nil) + } + test("invalid plan toString, debug mode") { val oldSetting = TestSQLContext.conf.dataFrameEagerAnalysis TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "true") From 45134ec920c3766c22aefd4366b4b60ec99bd810 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 2 Apr 2015 19:48:55 -0700 Subject: [PATCH 628/817] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops. This fixes the thread leak. I also changed the unit test to keep track of allocated contexts and make sure they're closed after tests are run; this is needed since some tests use this pattern: val sc = createContext() doSomethingThatMayThrow() sc.stop() Author: Marcelo Vanzin Closes #5311 from vanzin/SPARK-6650 and squashes the following commits: 652c73b [Marcelo Vanzin] Nits. 5711512 [Marcelo Vanzin] More exception safety. cc5a744 [Marcelo Vanzin] Stop alloc manager before scheduler. 9886f69 [Marcelo Vanzin] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops. --- .../spark/ExecutorAllocationManager.scala | 38 ++++++++-------- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../ExecutorAllocationManagerSuite.scala | 44 ++++++++++++------- 3 files changed, 49 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 21c6e6ffa6666..9385f557c4614 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -17,10 +17,12 @@ package org.apache.spark +import java.util.concurrent.{Executors, TimeUnit} + import scala.collection.mutable import org.apache.spark.scheduler._ -import org.apache.spark.util.{SystemClock, Clock} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. @@ -129,6 +131,10 @@ private[spark] class ExecutorAllocationManager( // Listener for Spark events that impact the allocation policy private val listener = new ExecutorAllocationListener + // Executor that handles the scheduling task. + private val executor = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("spark-dynamic-executor-allocation")) + /** * Verify that the settings specified through the config are valid. * If not, throw an appropriate exception. @@ -173,32 +179,24 @@ private[spark] class ExecutorAllocationManager( } /** - * Register for scheduler callbacks to decide when to add and remove executors. + * Register for scheduler callbacks to decide when to add and remove executors, and start + * the scheduling task. */ def start(): Unit = { listenerBus.addListener(listener) - startPolling() + + val scheduleTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(schedule()) + } + executor.scheduleAtFixedRate(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) } /** - * Start the main polling thread that keeps track of when to add and remove executors. + * Stop the allocation manager. */ - private def startPolling(): Unit = { - val t = new Thread { - override def run(): Unit = { - while (true) { - try { - schedule() - } catch { - case e: Exception => logError("Exception in dynamic executor allocation thread!", e) - } - Thread.sleep(intervalMillis) - } - } - } - t.setName("spark-dynamic-executor-allocation") - t.setDaemon(true) - t.start() + def stop(): Unit = { + executor.shutdown() + executor.awaitTermination(10, TimeUnit.SECONDS) } /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3904f7d1060c5..5b3778ead6994 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1136,7 +1136,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Return whether dynamically adjusting the amount of resources allocated to * this application is supported. This is currently only available for YARN. */ - private[spark] def supportDynamicAllocation = + private[spark] def supportDynamicAllocation = master.contains("yarn") || dynamicAllocationTesting /** @@ -1400,6 +1400,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli env.metricsSystem.report() metadataCleaner.cancel() cleaner.foreach(_.stop()) + executorAllocationManager.foreach(_.stop()) dagScheduler.stop() dagScheduler = null listenerBus.stop() diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index abfcee75728dc..3ded1e4af8742 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import scala.collection.mutable -import org.scalatest.{FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -28,10 +28,20 @@ import org.apache.spark.util.ManualClock /** * Test add and remove behavior of ExecutorAllocationManager. */ -class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { +class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter { import ExecutorAllocationManager._ import ExecutorAllocationManagerSuite._ + private val contexts = new mutable.ListBuffer[SparkContext]() + + before { + contexts.clear() + } + + after { + contexts.foreach(_.stop()) + } + test("verify min/max executors") { val conf = new SparkConf() .setMaster("local") @@ -39,18 +49,19 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") val sc0 = new SparkContext(conf) + contexts += sc0 assert(sc0.executorAllocationManager.isDefined) sc0.stop() // Min < 0 val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") - intercept[SparkException] { new SparkContext(conf1) } + intercept[SparkException] { contexts += new SparkContext(conf1) } SparkEnv.get.stop() SparkContext.clearActiveContext() // Max < 0 val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") - intercept[SparkException] { new SparkContext(conf2) } + intercept[SparkException] { contexts += new SparkContext(conf2) } SparkEnv.get.stop() SparkContext.clearActiveContext() @@ -665,16 +676,6 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("executor-2")) assert(!removeTimes(manager).contains("executor-1")) } -} - -/** - * Helper methods for testing ExecutorAllocationManager. - * This includes methods to access private methods and fields in ExecutorAllocationManager. - */ -private object ExecutorAllocationManagerSuite extends PrivateMethodTester { - private val schedulerBacklogTimeout = 1L - private val sustainedSchedulerBacklogTimeout = 2L - private val executorIdleTimeout = 3L private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = { val conf = new SparkConf() @@ -688,9 +689,22 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { sustainedSchedulerBacklogTimeout.toString) .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) .set("spark.dynamicAllocation.testing", "true") - new SparkContext(conf) + val sc = new SparkContext(conf) + contexts += sc + sc } +} + +/** + * Helper methods for testing ExecutorAllocationManager. + * This includes methods to access private methods and fields in ExecutorAllocationManager. + */ +private object ExecutorAllocationManagerSuite extends PrivateMethodTester { + private val schedulerBacklogTimeout = 1L + private val sustainedSchedulerBacklogTimeout = 2L + private val executorIdleTimeout = 3L + private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = { new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details") } From 4b82bd730a24f96d94dfea87420cfaa4253a5ccb Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 2 Apr 2015 20:23:08 -0700 Subject: [PATCH 629/817] [SPARK-6575][SQL] Converted Parquet Metastore tables no longer cache metadata https://issues.apache.org/jira/browse/SPARK-6575 Author: Yin Huai Closes #5339 from yhuai/parquetRelationCache and squashes the following commits: 83d9846 [Yin Huai] Remove unnecessary change. c0dc7a4 [Yin Huai] Cache converted parquet relations. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 61 +++++++++- .../apache/spark/sql/hive/parquetSuites.scala | 112 ++++++++++++++++++ 2 files changed, 167 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bbd920a4051de..76d329a3ddcdf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -116,7 +116,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } override def refreshTable(databaseName: String, tableName: String): Unit = { - cachedDataSourceTables.refresh(QualifiedTableName(databaseName, tableName).toLowerCase) + // refresh table does not eagerly reload the cache. It just invalidate the cache. + // Next time when we use the table, it will be populated in the cache. + invalidateTable(databaseName, tableName) } def invalidateTable(databaseName: String, tableName: String): Unit = { @@ -229,13 +231,42 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with private def convertToParquetRelation(metastoreRelation: MetastoreRelation): LogicalRelation = { val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging - val parquetOptions = Map( - ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json, - ParquetRelation2.MERGE_SCHEMA -> mergeSchema.toString) // NOTE: Instead of passing Metastore schema directly to `ParquetRelation2`, we have to // serialize the Metastore schema to JSON and pass it as a data source option because of the // evil case insensitivity issue, which is reconciled within `ParquetRelation2`. + val parquetOptions = Map( + ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json, + ParquetRelation2.MERGE_SCHEMA -> mergeSchema.toString) + val tableIdentifier = + QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) + + def getCached( + tableIdentifier: QualifiedTableName, + pathsInMetastore: Seq[String], + schemaInMetastore: StructType, + partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { + cachedDataSourceTables.getIfPresent(tableIdentifier) match { + case null => None // Cache miss + case logical @ LogicalRelation(parquetRelation: ParquetRelation2) => + // If we have the same paths, same schema, and same partition spec, + // we will use the cached Parquet Relation. + val useCached = + parquetRelation.paths == pathsInMetastore && + logical.schema.sameType(metastoreSchema) && + parquetRelation.maybePartitionSpec == partitionSpecInMetastore + + if (useCached) Some(logical) else None + case other => + logWarning( + s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} shold be stored " + + s"as Parquet. However, we are getting a ${other} from the metastore cache. " + + s"This cached entry will be invalidated.") + cachedDataSourceTables.invalidate(tableIdentifier) + None + } + } + if (metastoreRelation.hiveQlTable.isPartitioned) { val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) val partitionColumnDataTypes = partitionSchema.map(_.dataType) @@ -248,10 +279,28 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } val partitionSpec = PartitionSpec(partitionSchema, partitions) val paths = partitions.map(_.path) - LogicalRelation(ParquetRelation2(paths, parquetOptions, None, Some(partitionSpec))(hive)) + + val cached = getCached(tableIdentifier, paths, metastoreSchema, Some(partitionSpec)) + val parquetRelation = cached.getOrElse { + val created = + LogicalRelation(ParquetRelation2(paths, parquetOptions, None, Some(partitionSpec))(hive)) + cachedDataSourceTables.put(tableIdentifier, created) + created + } + + parquetRelation } else { val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) - LogicalRelation(ParquetRelation2(paths, parquetOptions)(hive)) + + val cached = getCached(tableIdentifier, paths, metastoreSchema, None) + val parquetRelation = cached.getOrElse { + val created = + LogicalRelation(ParquetRelation2(paths, parquetOptions)(hive)) + cachedDataSourceTables.put(tableIdentifier, created) + created + } + + parquetRelation } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 432d65a874518..2ad6e867262b1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -26,8 +26,10 @@ import org.apache.spark.sql.{QueryTest, SQLConf, SaveMode} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan +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._ +import org.apache.spark.sql.json.JSONRelation import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.SaveMode @@ -390,6 +392,116 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { sql("DROP TABLE ms_convert") } + + test("Caching converted data source Parquet Relations") { + def checkCached(tableIdentifer: catalog.QualifiedTableName): Unit = { + // Converted test_parquet should be cached. + catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) match { + case null => fail("Converted test_parquet should be cached in the cache.") + case logical @ LogicalRelation(parquetRelation: ParquetRelation2) => // OK + case other => + fail( + "The cached test_parquet should be a Parquet Relation. " + + s"However, $other is returned form the cache.") + } + } + + sql("DROP TABLE IF EXISTS test_insert_parquet") + sql("DROP TABLE IF EXISTS test_parquet_partitioned_cache_test") + + sql( + """ + |create table test_insert_parquet + |( + | intField INT, + | stringField STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + var tableIdentifer = catalog.QualifiedTableName("default", "test_insert_parquet") + + // First, make sure the converted test_parquet is not cached. + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + // Table lookup will make the table cached. + table("test_insert_parquet") + checkCached(tableIdentifer) + // For insert into non-partitioned table, we will do the conversion, + // so the converted test_insert_parquet should be cached. + invalidateTable("test_insert_parquet") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + sql( + """ + |INSERT INTO TABLE test_insert_parquet + |select a, b from jt + """.stripMargin) + checkCached(tableIdentifer) + // Make sure we can read the data. + checkAnswer( + sql("select * from test_insert_parquet"), + sql("select a, b from jt").collect()) + // Invalidate the cache. + invalidateTable("test_insert_parquet") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + + // Create a partitioned table. + sql( + """ + |create table test_parquet_partitioned_cache_test + |( + | intField INT, + | stringField STRING + |) + |PARTITIONED BY (date string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + tableIdentifer = catalog.QualifiedTableName("default", "test_parquet_partitioned_cache_test") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + sql( + """ + |INSERT INTO TABLE test_parquet_partitioned_cache_test + |PARTITION (date='2015-04-01') + |select a, b from jt + """.stripMargin) + // Right now, insert into a partitioned Parquet is not supported in data source Parquet. + // So, we expect it is not cached. + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") + sql( + """ + |INSERT INTO TABLE test_parquet_partitioned_cache_test + |PARTITION (date='2015-04-02') + |select a, b from jt + """.stripMargin) + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + + // Make sure we can cache the partitioned table. + table("test_parquet_partitioned_cache_test") + checkCached(tableIdentifer) + // Make sure we can read the data. + checkAnswer( + sql("select STRINGField, date, intField from test_parquet_partitioned_cache_test"), + sql( + """ + |select b, '2015-04-01', a FROM jt + |UNION ALL + |select b, '2015-04-02', a FROM jt + """.stripMargin).collect()) + + invalidateTable("test_parquet_partitioned_cache_test") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + + sql("DROP TABLE test_insert_parquet") + sql("DROP TABLE test_parquet_partitioned_cache_test") + } } class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { From 8a0aa81ca37d337423db60edb09cf264cc2c6498 Mon Sep 17 00:00:00 2001 From: KaiXinXiaoLei Date: Thu, 2 Apr 2015 20:24:31 -0700 Subject: [PATCH 630/817] [CORE] The descriptionof jobHistory config should be spark.history.fs.logDirectory The config option is spark.history.fs.logDirectory, not spark.fs.history.logDirectory. So the descriptionof should be changed. Thanks. Author: KaiXinXiaoLei Closes #5332 from KaiXinXiaoLei/historyConfig and squashes the following commits: 5ffbfb5 [KaiXinXiaoLei] the describe of jobHistory config is error --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 80c9c13ddec1e..9d40d8c8fd7a8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -118,7 +118,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (!fs.exists(path)) { var msg = s"Log directory specified does not exist: $logDir." if (logDir == DEFAULT_LOG_DIR) { - msg += " Did you configure the correct one through spark.fs.history.logDirectory?" + msg += " Did you configure the correct one through spark.history.fs.logDirectory?" } throw new IllegalArgumentException(msg) } From 6e1c1ec67bc4d7e5700f523ec08db6bb25bd2302 Mon Sep 17 00:00:00 2001 From: freeman Date: Thu, 2 Apr 2015 21:37:44 -0700 Subject: [PATCH 631/817] [SPARK-6345][STREAMING][MLLIB] Fix for training with prediction This patch fixes a reported bug causing model updates to not properly propagate to model predictions during streaming regression. These minor changes in model declaration fix the problem, and I expanded the tests to include the scenario in which the bug was arising. The two new tests failed prior to the patch and now pass. cc mengxr Author: freeman Closes #5037 from freeman-lab/train-predict-fix and squashes the following commits: 3af953e [freeman] Expand test coverage to include combined training and prediction 8f84fc8 [freeman] Move model declaration --- .../StreamingLogisticRegressionWithSGD.scala | 2 ++ .../regression/StreamingLinearAlgorithm.scala | 6 ++-- .../StreamingLinearRegressionWithSGD.scala | 2 ++ .../StreamingLogisticRegressionSuite.scala | 27 ++++++++++++++++++ .../StreamingLinearRegressionSuite.scala | 28 +++++++++++++++++++ 5 files changed, 62 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala index b89f38cf5aba4..7d33df3221fbf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala @@ -63,6 +63,8 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( protected val algorithm = new LogisticRegressionWithSGD( stepSize, numIterations, regParam, miniBatchFraction) + protected var model: Option[LogisticRegressionModel] = None + /** Set the step size for gradient descent. Default: 0.1. */ def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index ce95c063db970..cea8f3f47307b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -60,7 +60,7 @@ abstract class StreamingLinearAlgorithm[ A <: GeneralizedLinearAlgorithm[M]] extends Logging { /** The model to be updated and used for prediction. */ - protected var model: Option[M] = None + protected var model: Option[M] /** The algorithm to use for updating. */ protected val algorithm: A @@ -114,7 +114,7 @@ abstract class StreamingLinearAlgorithm[ if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction.") } - data.map(model.get.predict) + data.map{x => model.get.predict(x)} } /** Java-friendly version of `predictOn`. */ @@ -132,7 +132,7 @@ abstract class StreamingLinearAlgorithm[ if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction") } - data.mapValues(model.get.predict) + data.mapValues{x => model.get.predict(x)} } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index e5e6301127a28..a49153bf73c0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -59,6 +59,8 @@ class StreamingLinearRegressionWithSGD private[mllib] ( val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + protected var model: Option[LinearRegressionModel] = None + /** Set the step size for gradient descent. Default: 0.1. */ def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index 8b3e6e5ce9249..d50c43d439187 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -132,4 +132,31 @@ class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { assert(errors.forall(x => x <= 0.4)) } + // Test training combined with prediction + test("training and prediction") { + // create model initialized with zero weights + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.dense(-0.1)) + .setStepSize(0.01) + .setNumIterations(10) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LogisticRegressionSuite.generateLogisticInput(0.0, 5.0, nPoints, 42 * (i + 1)) + } + + // train and predict + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // assert that prediction error improves, ensuring that the updated model is being used + val error = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints).toList + assert(error.head > 0.8 & error.last < 0.2) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 70b43ddb7daf5..24fd8df691817 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -139,4 +139,32 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) assert(errors.forall(x => x <= 0.1)) } + + // Test training combined with prediction + test("training and prediction") { + // create model initialized with zero weights + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0, 0.0)) + .setStepSize(0.2) + .setNumIterations(25) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), nPoints, 42 * (i + 1)) + } + + // train and predict + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // assert that prediction error improves, ensuring that the updated model is being used + val error = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints).toList + assert((error.head - error.last) > 2) + } } From 440ea31b76aa7e813436271fd63880c7bcd69157 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 2 Apr 2015 22:54:30 -0700 Subject: [PATCH 632/817] [SPARK-6621][Core] Fix the bug that calling EventLoop.stop in EventLoop.onReceive/onError/onStart doesn't call onStop Author: zsxwing Closes #5280 from zsxwing/SPARK-6621 and squashes the following commits: 521125e [zsxwing] Fix the bug that calling EventLoop.stop in EventLoop.onReceive and EventLoop.onError doesn't call onStop --- .../org/apache/spark/util/EventLoop.scala | 18 ++++- .../apache/spark/util/EventLoopSuite.scala | 72 +++++++++++++++++++ 2 files changed, 87 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index b0ed908b84424..e9b2b8d24b476 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -76,9 +76,21 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging { def stop(): Unit = { if (stopped.compareAndSet(false, true)) { eventThread.interrupt() - eventThread.join() - // Call onStop after the event thread exits to make sure onReceive happens before onStop - onStop() + var onStopCalled = false + try { + eventThread.join() + // Call onStop after the event thread exits to make sure onReceive happens before onStop + onStopCalled = true + onStop() + } catch { + case ie: InterruptedException => + Thread.currentThread().interrupt() + if (!onStopCalled) { + // ie is thrown from `eventThread.join()`. Otherwise, we should not call `onStop` since + // it's already called. + onStop() + } + } } else { // Keep quiet to allow calling `stop` multiple times. } diff --git a/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala index 1026cb2aa7cae..47b535206c949 100644 --- a/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala @@ -203,4 +203,76 @@ class EventLoopSuite extends FunSuite with Timeouts { assert(!eventLoop.isActive) } } + + test("EventLoop: stop() in onStart should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onStart(): Unit = { + stop() + } + + override def onReceive(event: Int): Unit = { + } + + override def onError(e: Throwable): Unit = { + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } + + test("EventLoop: stop() in onReceive should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + stop() + } + + override def onError(e: Throwable): Unit = { + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } + + test("EventLoop: stop() in onError should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + throw new RuntimeException("Oops") + } + + override def onError(e: Throwable): Unit = { + stop() + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } } From c42c3fc7f7b79a1f6ce990d39b5d9d14ab19fcf0 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 3 Apr 2015 14:40:36 +0800 Subject: [PATCH 633/817] [SPARK-6575][SQL] Converted Parquet Metastore tables no longer cache metadata https://issues.apache.org/jira/browse/SPARK-6575 Author: Yin Huai This patch had conflicts when merged, resolved by Committer: Cheng Lian Closes #5339 from yhuai/parquetRelationCache and squashes the following commits: b0e1a42 [Yin Huai] Address comments. 83d9846 [Yin Huai] Remove unnecessary change. c0dc7a4 [Yin Huai] Cache converted parquet relations. --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 28 +++++++++++++------ .../spark/sql/hive/execution/commands.scala | 5 ++-- .../apache/spark/sql/hive/parquetSuites.scala | 2 -- 3 files changed, 23 insertions(+), 12 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 76d329a3ddcdf..c4da34ae645b8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -116,8 +116,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } override def refreshTable(databaseName: String, tableName: String): Unit = { - // refresh table does not eagerly reload the cache. It just invalidate the cache. + // refreshTable does not eagerly reload the cache. It just invalidate the cache. // Next time when we use the table, it will be populated in the cache. + // Since we also cache ParquetRealtions converted from Hive Parquet tables and + // adding converted ParquetRealtions into the cache is not defined in the load function + // of the cache (instead, we add the cache entry in convertToParquetRelation), + // it is better at here to invalidate the cache to avoid confusing waring logs from the + // cache loader (e.g. cannot find data source provider, which is only defined for + // data source table.). invalidateTable(databaseName, tableName) } @@ -242,21 +248,27 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) def getCached( - tableIdentifier: QualifiedTableName, - pathsInMetastore: Seq[String], - schemaInMetastore: StructType, - partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { + tableIdentifier: QualifiedTableName, + pathsInMetastore: Seq[String], + schemaInMetastore: StructType, + partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(parquetRelation: ParquetRelation2) => + case logical@LogicalRelation(parquetRelation: ParquetRelation2) => // If we have the same paths, same schema, and same partition spec, // we will use the cached Parquet Relation. val useCached = - parquetRelation.paths == pathsInMetastore && + parquetRelation.paths.toSet == pathsInMetastore.toSet && logical.schema.sameType(metastoreSchema) && parquetRelation.maybePartitionSpec == partitionSpecInMetastore - if (useCached) Some(logical) else None + if (useCached) { + Some(logical) + } else { + // If the cached relation is not updated, we invalidate it right away. + cachedDataSourceTables.invalidate(tableIdentifier) + None + } case other => logWarning( s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} shold be stored " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 4345ffbf30f77..99dc58646ddd6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -58,12 +58,13 @@ case class DropTable( try { hiveContext.cacheManager.tryUncacheQuery(hiveContext.table(tableName)) } catch { - // This table's metadata is not in + // This table's metadata is not in Hive metastore (e.g. the table does not exist). case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException => + case _: org.apache.spark.sql.catalyst.analysis.NoSuchTableException => // Other Throwables can be caused by users providing wrong parameters in OPTIONS // (e.g. invalid paths). We catch it and log a warning message. // Users should be able to drop such kinds of tables regardless if there is an error. - case e: Throwable => log.warn(s"${e.getMessage}") + case e: Throwable => log.warn(s"${e.getMessage}", e) } hiveContext.invalidateTable(tableName) hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 2ad6e867262b1..1319c81dfc131 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -473,7 +473,6 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { // Right now, insert into a partitioned Parquet is not supported in data source Parquet. // So, we expect it is not cached. assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) - conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") sql( """ |INSERT INTO TABLE test_parquet_partitioned_cache_test @@ -481,7 +480,6 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { |select a, b from jt """.stripMargin) assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) - conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") // Make sure we can cache the partitioned table. table("test_parquet_partitioned_cache_test") From 82701ee25fda64f03899713bc56f82ca6f278151 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Apr 2015 01:25:02 -0700 Subject: [PATCH 634/817] [SPARK-6428] Turn on explicit type checking for public methods. This builds on my earlier pull requests and turns on the explicit type checking in scalastyle. Author: Reynold Xin Closes #5342 from rxin/SPARK-6428 and squashes the following commits: 7b531ab [Reynold Xin] import ordering 2d9a8a5 [Reynold Xin] jl e668b1c [Reynold Xin] override 9b9e119 [Reynold Xin] Parenthesis. 82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods. --- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../apache/spark/api/java/JavaRDDLike.scala | 53 ++++++++++------ .../apache/spark/examples/LocalKMeans.scala | 4 +- .../org/apache/spark/examples/LocalLR.scala | 4 +- .../org/apache/spark/examples/LogQuery.scala | 4 +- .../org/apache/spark/examples/SparkLR.scala | 4 +- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/examples/bagel/PageRankUtils.scala | 2 +- .../spark/examples/mllib/MovieLensALS.scala | 4 +- .../examples/streaming/ActorWordCount.scala | 6 +- .../RecoverableNetworkWordCount.scala | 3 +- .../examples/streaming/ZeroMQWordCount.scala | 6 +- .../clickstream/PageViewGenerator.scala | 2 +- .../streaming/flume/FlumeInputDStream.scala | 12 ++-- .../kafka/DirectKafkaInputDStream.scala | 5 +- .../spark/streaming/kafka/KafkaRDD.scala | 4 +- .../twitter/TwitterInputDStream.scala | 2 +- .../streaming/zeromq/ZeroMQReceiver.scala | 13 ++-- .../org/apache/spark/graphx/EdgeContext.scala | 3 +- .../apache/spark/graphx/EdgeDirection.scala | 12 ++-- .../org/apache/spark/graphx/EdgeTriplet.scala | 2 +- .../spark/graphx/impl/EdgePartition.scala | 14 ++--- .../spark/graphx/impl/EdgeRDDImpl.scala | 4 +- .../graphx/impl/ReplicatedVertexView.scala | 2 +- .../spark/graphx/impl/VertexRDDImpl.scala | 4 +- .../graphx/lib/ConnectedComponents.scala | 2 +- .../spark/graphx/lib/LabelPropagation.scala | 4 +- .../apache/spark/graphx/lib/PageRank.scala | 2 +- .../GraphXPrimitiveKeyOpenHashMap.scala | 8 +-- .../apache/spark/mllib/feature/Word2Vec.scala | 2 +- scalastyle-config.xml | 2 +- .../apache/spark/sql/AnalysisException.scala | 2 +- .../spark/sql/catalyst/analysis/package.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 3 +- .../apache/spark/streaming/Checkpoint.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 12 ++-- .../streaming/api/java/JavaPairDStream.scala | 2 +- .../api/java/JavaStreamingContext.scala | 10 +-- .../spark/streaming/dstream/DStream.scala | 2 +- .../tools/JavaAPICompletenessChecker.scala | 4 +- .../spark/tools/StoragePerfTester.scala | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 62 ++++++++++--------- .../spark/deploy/yarn/ExecutorRunnable.scala | 6 +- 46 files changed, 170 insertions(+), 142 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a023712be1166..8441bb3a3047e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -661,7 +661,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ - def fn = (x: V) => f.call(x).asScala + def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 18ccd625fc8d1..db4e996feb31c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -192,7 +192,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) */ def sortBy[S](f: JFunction[T, S], ascending: Boolean, numPartitions: Int): JavaRDD[T] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x) + def fn: (T) => S = (x: T) => f.call(x) import com.google.common.collect.Ordering // shadows scala.math.Ordering implicit val ordering = Ordering.natural().asInstanceOf[Ordering[S]] implicit val ctag: ClassTag[S] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 8da42934a7d96..8bf0627fc420d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -17,8 +17,9 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList, Iterator => JIterator} +import java.{lang => jl} import java.lang.{Iterable => JIterable, Long => JLong} +import java.util.{Comparator, List => JList, Iterator => JIterator} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -93,7 +94,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * of the original partition. */ def mapPartitionsWithIndex[R]( - f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]], + f: JFunction2[jl.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag) @@ -109,7 +110,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to all elements of this RDD. */ def mapToPair[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - def cm = implicitly[ClassTag[(K2, V2)]] + def cm: ClassTag[(K2, V2)] = implicitly[ClassTag[(K2, V2)]] new JavaPairRDD(rdd.map[(K2, V2)](f)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -119,7 +120,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala JavaRDD.fromRDD(rdd.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -129,8 +130,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMapToDouble(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala - new JavaDoubleRDD(rdd.flatMap(fn).map((x: java.lang.Double) => x.doubleValue())) + def fn: (T) => Iterable[jl.Double] = (x: T) => f.call(x).asScala + new JavaDoubleRDD(rdd.flatMap(fn).map((x: jl.Double) => x.doubleValue())) } /** @@ -139,8 +140,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala - def cm = implicitly[ClassTag[(K2, V2)]] + def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala + def cm: ClassTag[(K2, V2)] = implicitly[ClassTag[(K2, V2)]] JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -148,7 +149,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD. */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaRDD[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaRDD.fromRDD(rdd.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -157,7 +160,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -166,8 +171,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD. */ def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) - new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: java.lang.Double) => x.doubleValue())) + def fn: (Iterator[T]) => Iterator[jl.Double] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } + new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: jl.Double) => x.doubleValue())) } /** @@ -175,7 +182,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -184,7 +193,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[jl.Double] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) .map(x => x.doubleValue())) } @@ -194,7 +205,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaPairRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -277,8 +290,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def zipPartitions[U, V]( other: JavaRDDLike[U, _], f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V]): JavaRDD[V] = { - def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator( - f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) + def fn: (Iterator[T], Iterator[U]) => Iterator[V] = { + (x: Iterator[T], y: Iterator[U]) => asScalaIterator( + f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) + } JavaRDD.fromRDD( rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } @@ -441,8 +456,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final * combine step happens locally on the master, equivalent to running a single reduce task. */ - def countByValue(): java.util.Map[T, java.lang.Long] = - mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) + def countByValue(): java.util.Map[T, jl.Long] = + mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new jl.Long(x._2))))) /** * (Experimental) Approximate version of countByValue(). diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index 17624c20cff3d..f73eac1e2b906 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -40,8 +40,8 @@ object LocalKMeans { val convergeDist = 0.001 val rand = new Random(42) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DenseVector[Double]] = { + def generatePoint(i: Int): DenseVector[Double] = { DenseVector.fill(D){rand.nextDouble * R} } Array.tabulate(N)(generatePoint) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 92a683ad57ea1..a55e0dc8d36c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -37,8 +37,8 @@ object LocalLR { case class DataPoint(x: Vector[Double], y: Double) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DataPoint] = { + def generatePoint(i: Int): DataPoint = { val y = if(i % 2 == 0) -1 else 1 val x = DenseVector.fill(D){rand.nextGaussian + y * R} DataPoint(x, y) diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 74620ad007d83..32e02eab8b031 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -54,8 +54,8 @@ object LogQuery { // scalastyle:on /** Tracks the total query count and number of aggregate bytes for a particular group. */ class Stats(val count: Int, val numBytes: Int) extends Serializable { - def merge(other: Stats) = new Stats(count + other.count, numBytes + other.numBytes) - override def toString = "bytes=%s\tn=%s".format(numBytes, count) + def merge(other: Stats): Stats = new Stats(count + other.count, numBytes + other.numBytes) + override def toString: String = "bytes=%s\tn=%s".format(numBytes, count) } def extractKey(line: String): (String, String, String) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 257a7d29f922a..8c01a60844620 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -42,8 +42,8 @@ object SparkLR { case class DataPoint(x: Vector[Double], y: Double) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DataPoint] = { + def generatePoint(i: Int): DataPoint = { val y = if(i % 2 == 0) -1 else 1 val x = DenseVector.fill(D){rand.nextGaussian + y * R} DataPoint(x, y) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index f7f83086df3db..772cd897f5140 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -31,7 +31,7 @@ object SparkTC { val numVertices = 100 val rand = new Random(42) - def generateGraph = { + def generateGraph: Seq[(Int, Int)] = { val edges: mutable.Set[(Int, Int)] = mutable.Set.empty while (edges.size < numEdges) { val from = rand.nextInt(numVertices) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index e322d4ce5a745..ab6e63deb3c95 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -90,7 +90,7 @@ class PRMessage() extends Message[String] with Serializable { } class CustomPartitioner(partitions: Int) extends Partitioner { - def numPartitions = partitions + def numPartitions: Int = partitions def getPartition(key: Any): Int = { val hash = key match { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 1f4ca4fbe7778..0bc36ea65e1ab 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -178,7 +178,9 @@ object MovieLensALS { def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], implicitPrefs: Boolean) : Double = { - def mapPredictedRating(r: Double) = if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r + def mapPredictedRating(r: Double): Double = { + if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r + } val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) val predictionsAndRatings = predictions.map{ x => diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index b433082dce1a2..92867b44be138 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -85,13 +85,13 @@ extends Actor with ActorHelper { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) - override def preStart = remotePublisher ! SubscribeReceiver(context.self) + override def preStart(): Unit = remotePublisher ! SubscribeReceiver(context.self) - def receive = { + def receive: PartialFunction[Any, Unit] = { case msg => store(msg.asInstanceOf[T]) } - override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) + override def postStop(): Unit = remotePublisher ! UnsubscribeReceiver(context.self) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index c3a05c89d817e..751b30ea15782 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -55,7 +55,8 @@ import org.apache.spark.util.IntParam */ object RecoverableNetworkWordCount { - def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) = { + def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) + : StreamingContext = { // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 6510c70bd1866..e99d1baa72b9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.SparkConf */ object SimpleZeroMQPublisher { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: SimpleZeroMQPublisher ") System.exit(1) @@ -45,7 +45,7 @@ object SimpleZeroMQPublisher { val acs: ActorSystem = ActorSystem() val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - implicit def stringToByteString(x: String) = ByteString(x) + implicit def stringToByteString(x: String): ByteString = ByteString(x) val messages: List[ByteString] = List("words ", "may ", "count ") while (true) { Thread.sleep(1000) @@ -86,7 +86,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(sparkConf, Seconds(2)) - def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator + def bytesToStringIterator(x: Seq[ByteString]): Iterator[String] = x.map(_.utf8String).iterator // For this stream, a zeroMQ publisher should be running. val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 8402491b62671..54d996b8ac990 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -94,7 +94,7 @@ object PageViewGenerator { while (true) { val socket = listener.accept() new Thread() { - override def run = { + override def run(): Unit = { println("Got client connected from: " + socket.getInetAddress) val out = new PrintWriter(socket.getOutputStream(), true) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 2de2a7926bfd1..60e2994431b38 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -37,8 +37,7 @@ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver -import org.jboss.netty.channel.ChannelPipelineFactory -import org.jboss.netty.channel.Channels +import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.compression._ @@ -187,8 +186,8 @@ class FlumeReceiver( logInfo("Flume receiver stopped") } - override def preferredLocation = Some(host) - + override def preferredLocation: Option[String] = Option(host) + /** A Netty Pipeline factory that will decompress incoming data from * and the Netty client and compress data going back to the client. * @@ -198,13 +197,12 @@ class FlumeReceiver( */ private[streaming] class CompressionChannelPipelineFactory extends ChannelPipelineFactory { - - def getPipeline() = { + def getPipeline(): ChannelPipeline = { val pipeline = Channels.pipeline() val encoder = new ZlibEncoder(6) pipeline.addFirst("deflater", encoder) pipeline.addFirst("inflater", new ZlibDecoder()) pipeline + } } } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 04e65cb3d708c..1b1fc8051d052 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -129,8 +129,9 @@ class DirectKafkaInputDStream[ private[streaming] class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def batchForTime = data.asInstanceOf[mutable.HashMap[ - Time, Array[OffsetRange.OffsetRangeTuple]]] + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } override def update(time: Time) { batchForTime.clear() diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 6d465bcb6bfc0..4a83b715fa89d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -155,7 +155,7 @@ class KafkaRDD[ .dropWhile(_.offset < requestOffset) } - override def close() = consumer.close() + override def close(): Unit = consumer.close() override def getNext(): R = { if (iter == null || !iter.hasNext) { @@ -207,7 +207,7 @@ object KafkaRDD { fromOffsets: Map[TopicAndPartition, Long], untilOffsets: Map[TopicAndPartition, LeaderOffset], messageHandler: MessageAndMetadata[K, V] => R - ): KafkaRDD[K, V, U, T, R] = { + ): KafkaRDD[K, V, U, T, R] = { val leaders = untilOffsets.map { case (tp, lo) => tp -> (lo.host, lo.port) }.toMap diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 4eacc47da5699..7cf02d85d73d3 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -70,7 +70,7 @@ class TwitterReceiver( try { val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) newTwitterStream.addListener(new StatusListener { - def onStatus(status: Status) = { + def onStatus(status: Status): Unit = { store(status) } // Unimplemented diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 554705878ee78..588e6bac7b14a 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -29,13 +29,16 @@ import org.apache.spark.streaming.receiver.ActorHelper /** * A receiver to subscribe to ZeroMQ stream. */ -private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T]) +private[streaming] class ZeroMQReceiver[T: ClassTag]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] => Iterator[T]) extends Actor with ActorHelper with Logging { - override def preStart() = ZeroMQExtension(context.system) - .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) + override def preStart(): Unit = { + ZeroMQExtension(context.system) + .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) + } def receive: Receive = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index d8be02e2023d5..23430179f12ec 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -62,7 +62,6 @@ object EdgeContext { * , _ + _) * }}} */ - def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]) = + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] = Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) } - diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 6f03eb1439773..058c8c8aa1b24 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -34,12 +34,12 @@ class EdgeDirection private (private val name: String) extends Serializable { override def toString: String = "EdgeDirection." + name - override def equals(o: Any) = o match { + override def equals(o: Any): Boolean = o match { case other: EdgeDirection => other.name == name case _ => false } - override def hashCode = name.hashCode + override def hashCode: Int = name.hashCode } @@ -48,14 +48,14 @@ class EdgeDirection private (private val name: String) extends Serializable { */ object EdgeDirection { /** Edges arriving at a vertex. */ - final val In = new EdgeDirection("In") + final val In: EdgeDirection = new EdgeDirection("In") /** Edges originating from a vertex. */ - final val Out = new EdgeDirection("Out") + final val Out: EdgeDirection = new EdgeDirection("Out") /** Edges originating from *or* arriving at a vertex of interest. */ - final val Either = new EdgeDirection("Either") + final val Either: EdgeDirection = new EdgeDirection("Either") /** Edges originating from *and* arriving at a vertex of interest. */ - final val Both = new EdgeDirection("Both") + final val Both: EdgeDirection = new EdgeDirection("Both") } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index 9d473d5ebda44..c8790cac3d8a0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -62,7 +62,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { def vertexAttr(vid: VertexId): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } - override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + override def toString: String = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 373af75448374..c561570809253 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -324,7 +324,7 @@ class EdgePartition[ * * @return an iterator over edges in the partition */ - def iterator = new Iterator[Edge[ED]] { + def iterator: Iterator[Edge[ED]] = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] private[this] var pos = 0 @@ -351,7 +351,7 @@ class EdgePartition[ override def hasNext: Boolean = pos < EdgePartition.this.size - override def next() = { + override def next(): EdgeTriplet[VD, ED] = { val triplet = new EdgeTriplet[VD, ED] val localSrcId = localSrcIds(pos) val localDstId = localDstIds(pos) @@ -518,11 +518,11 @@ private class AggregatingEdgeContext[VD, ED, A]( _attr = attr } - override def srcId = _srcId - override def dstId = _dstId - override def srcAttr = _srcAttr - override def dstAttr = _dstAttr - override def attr = _attr + override def srcId: VertexId = _srcId + override def dstId: VertexId = _dstId + override def srcAttr: VD = _srcAttr + override def dstAttr: VD = _dstAttr + override def attr: ED = _attr override def sendToSrc(msg: A) { send(_localSrcId, msg) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 43a3aea0f6196..c88b2f65a86cd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -70,9 +70,9 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } - override def getStorageLevel = partitionsRDD.getStorageLevel + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel - override def checkpoint() = { + override def checkpoint(): Unit = { partitionsRDD.checkpoint() } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 8ab255bd4038c..1df86449fa0c2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -50,7 +50,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped to * match. */ - def reverse() = { + def reverse(): ReplicatedVertexView[VD, ED] = { val newEdges = edges.mapEdgePartitions((pid, part) => part.reverse) new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 349c8545bf201..33ac7b0ed6095 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -71,9 +71,9 @@ class VertexRDDImpl[VD] private[graphx] ( this } - override def getStorageLevel = partitionsRDD.getStorageLevel + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel - override def checkpoint() = { + override def checkpoint(): Unit = { partitionsRDD.checkpoint() } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index e2f6cc138958e..859f896039047 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -37,7 +37,7 @@ object ConnectedComponents { */ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(edge: EdgeTriplet[VertexId, ED]) = { + def sendMessage(edge: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, VertexId)] = { if (edge.srcAttr < edge.dstAttr) { Iterator((edge.dstId, edge.srcAttr)) } else if (edge.srcAttr > edge.dstAttr) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala index 82e9e06515179..2bcf8684b8b8e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -43,7 +43,7 @@ object LabelPropagation { */ def run[VD, ED: ClassTag](graph: Graph[VD, ED], maxSteps: Int): Graph[VertexId, ED] = { val lpaGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(e: EdgeTriplet[VertexId, ED]) = { + def sendMessage(e: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, Map[VertexId, VertexId])] = { Iterator((e.srcId, Map(e.dstAttr -> 1L)), (e.dstId, Map(e.srcAttr -> 1L))) } def mergeMessage(count1: Map[VertexId, Long], count2: Map[VertexId, Long]) @@ -54,7 +54,7 @@ object LabelPropagation { i -> (count1Val + count2Val) }.toMap } - def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]) = { + def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]): VertexId = { if (message.isEmpty) attr else message.maxBy(_._2)._1 } val initialMessage = Map[VertexId, Long]() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 570440ba4441f..042e366a29f58 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -156,7 +156,7 @@ object PageRank extends Logging { (newPR, newPR - oldPR) } - def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]): Iterator[(VertexId, Double)] = { if (edge.srcAttr._2 > tol) { Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) } else { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 57b01b6f2e1fb..e2754ea699da9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -56,7 +56,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, private var _oldValues: Array[V] = null - override def size = keySet.size + override def size: Int = keySet.size /** Get the value for a given key */ def apply(k: K): V = { @@ -112,7 +112,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = 0 var nextPair: (K, V) = computeNextPair() @@ -128,9 +128,9 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair 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 9ee7e4a66b535..b2d9053f70145 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 @@ -522,7 +522,7 @@ object Word2VecModel extends Loader[Word2VecModel] { new Word2VecModel(word2VecMap) } - def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]) = { + def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = { val sqlContext = new SQLContext(sc) import sqlContext.implicits._ diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 459a5035d4984..7168d5b2a8e26 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -137,7 +137,7 @@ - + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 34fedead44db3..f9992185a4563 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -30,7 +30,7 @@ class AnalysisException protected[sql] ( val startPosition: Option[Int] = None) extends Exception with Serializable { - def withPosition(line: Option[Int], startPosition: Option[Int]) = { + def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { val newException = new AnalysisException(message, line, startPosition) newException.setStackTrace(getStackTrace) newException diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index c61c395cb4bb1..7731336d247db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -44,7 +44,7 @@ package object analysis { } /** Catches any AnalysisExceptions thrown by `f` and attaches `t`'s position if any. */ - def withPosition[A](t: TreeNode[_])(f: => A) = { + def withPosition[A](t: TreeNode[_])(f: => A): A = { try f catch { case a: AnalysisException => throw a.withPosition(t.origin.line, t.origin.startPosition) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index eb46b46ca5bf4..319de710fbc3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -204,7 +204,7 @@ private[sql] object ResolvedDataSource { provider: String, options: Map[String, String]): ResolvedDataSource = { val clazz: Class[_] = lookupDataSource(provider) - def className = clazz.getCanonicalName + def className: String = clazz.getCanonicalName val relation = userSpecifiedSchema match { case Some(schema: StructType) => clazz.newInstance() match { case dataSource: SchemaRelationProvider => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c4da34ae645b8..ae5ce4cf4c7e7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -861,7 +861,7 @@ private[hive] case class MetastoreRelation /** An attribute map for determining the ordinal for non-partition columns. */ val columnOrdinals = AttributeMap(attributes.zipWithIndex) - override def newInstance() = { + override def newInstance(): MetastoreRelation = { MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5be09a11ad641..077e64133faad 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -659,7 +659,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C AttributeReference("value", StringType)()), true) } - def matchSerDe(clause: Seq[ASTNode]) = clause match { + def matchSerDe(clause: Seq[ASTNode]) + : (Seq[(String, String)], String, Seq[(String, String)]) = clause match { case Token("TOK_SERDEPROPS", propsClause) :: Nil => val rowFormat = propsClause.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) 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 f73b463d07779..28703ef8129b3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -234,7 +234,7 @@ object CheckpointReader extends Logging { val checkpointPath = new Path(checkpointDir) // TODO(rxin): Why is this a def?! - def fs = checkpointPath.getFileSystem(hadoopConf) + def fs: FileSystem = checkpointPath.getFileSystem(hadoopConf) // Try to find the checkpoint files val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 73030e15c5661..808dcc174cf9a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -169,7 +169,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala new JavaDStream(dstream.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -179,7 +179,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala def cm: ClassTag[(K2, V2)] = fakeClassTag new JavaPairDStream(dstream.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -190,7 +190,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of the RDD. */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaDStream(dstream.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -201,7 +203,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) : JavaPairDStream[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaPairDStream(dstream.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index f94f2d0e8bd31..93baad19e3ee1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -526,7 +526,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = { import scala.collection.JavaConverters._ - def fn = (x: V) => f.apply(x).asScala + def fn: (V) => Iterable[U] = (x: V) => f.apply(x).asScala implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.flatMapValues(fn) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index e3db01c1e12c6..4095a7cc84946 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -192,7 +192,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) : JavaReceiverInputDStream[T] = { - def fn = (x: InputStream) => converter.call(x).toIterator + def fn: (InputStream) => Iterator[T] = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.socketStream(hostname, port, fn, storageLevel) @@ -313,7 +313,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cmk: ClassTag[K] = ClassTag(kClass) implicit val cmv: ClassTag[V] = ClassTag(vClass) implicit val cmf: ClassTag[F] = ClassTag(fClass) - def fn = (x: Path) => filter.call(x).booleanValue() + def fn: (Path) => Boolean = (x: Path) => filter.call(x).booleanValue() ssc.fileStream[K, V, F](directory, fn, newFilesOnly) } @@ -344,7 +344,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cmk: ClassTag[K] = ClassTag(kClass) implicit val cmv: ClassTag[V] = ClassTag(vClass) implicit val cmf: ClassTag[F] = ClassTag(fClass) - def fn = (x: Path) => filter.call(x).booleanValue() + def fn: (Path) => Boolean = (x: Path) => filter.call(x).booleanValue() ssc.fileStream[K, V, F](directory, fn, newFilesOnly, conf) } @@ -625,7 +625,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean): Unit = ssc.stop(stopSparkContext) /** * Stop the execution of the streams. @@ -633,7 +633,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @param stopGracefully Stop gracefully by waiting for the processing of all * received data to be completed */ - def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { + def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = { ssc.stop(stopSparkContext, stopGracefully) } 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 795c5aa6d585b..24f99a2b929f5 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 @@ -839,7 +839,7 @@ object DStream { /** Filtering function that excludes non-user classes for a streaming application */ def streamingExclustionFunction(className: String): Boolean = { - def doesMatch(r: Regex) = r.findFirstIn(className).isDefined + def doesMatch(r: Regex): Boolean = r.findFirstIn(className).isDefined val isSparkClass = doesMatch(SPARK_CLASS_REGEX) val isSparkExampleClass = doesMatch(SPARK_EXAMPLES_CLASS_REGEX) val isSparkStreamingTestClass = doesMatch(SPARK_STREAMING_TESTCLASS_REGEX) diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 8d0f09933c8d3..583823c90c5c6 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -17,7 +17,7 @@ package org.apache.spark.tools -import java.lang.reflect.Method +import java.lang.reflect.{Type, Method} import scala.collection.mutable.ArrayBuffer import scala.language.existentials @@ -302,7 +302,7 @@ object JavaAPICompletenessChecker { private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") - def toComparisionKey(method: Method) = + def toComparisionKey(method: Method): (Class[_], String, Type) = (method.getReturnType, method.getName, method.getGenericReturnType) val interfaces = method.getDeclaringClass.getInterfaces.filter { i => excludedInterfaces.contains(i.getName) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 6b666a0384879..f2d135397ce2f 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Writes simulated shuffle output from several threads and records the observed throughput. */ object StoragePerfTester { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { /** Total amount of data to generate. Distributed evenly amongst maps and reduce splits. */ val dataSizeMb = Utils.memoryStringToMb(sys.env.getOrElse("OUTPUT_DATA", "1g")) @@ -58,7 +58,7 @@ object StoragePerfTester { val sc = new SparkContext("local[4]", "Write Tester", conf) val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] - def writeOutputBytes(mapId: Int, total: AtomicLong) = { + def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers @@ -78,7 +78,7 @@ object StoragePerfTester { val totalBytes = new AtomicLong() for (task <- 1 to numMaps) { executor.submit(new Runnable() { - override def run() = { + override def run(): Unit = { try { writeOutputBytes(task, totalBytes) latch.countDown() 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 3d18690cd9cbf..455554eea0597 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 @@ -162,7 +162,7 @@ private[spark] class ApplicationMaster( * status to SUCCEEDED in cluster mode to handle if the user calls System.exit * from the application code. */ - final def getDefaultFinalStatus() = { + final def getDefaultFinalStatus(): FinalApplicationStatus = { if (isClusterMode) { FinalApplicationStatus.SUCCEEDED } else { @@ -175,31 +175,35 @@ private[spark] class ApplicationMaster( * This means the ResourceManager will not retry the application attempt on your behalf if * a failure occurred. */ - final def unregister(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { - if (!unregistered) { - logInfo(s"Unregistering ApplicationMaster with $status" + - Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) - unregistered = true - client.unregister(status, Option(diagnostics).getOrElse("")) + final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = { + synchronized { + if (!unregistered) { + logInfo(s"Unregistering ApplicationMaster with $status" + + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) + unregistered = true + client.unregister(status, Option(diagnostics).getOrElse("")) + } } } - final def finish(status: FinalApplicationStatus, code: Int, msg: String = null) = synchronized { - if (!finished) { - val inShutdown = Utils.inShutdown() - logInfo(s"Final app status: ${status}, exitCode: ${code}" + - Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) - exitCode = code - finalStatus = status - finalMsg = msg - finished = true - if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) { - logDebug("shutting down reporter thread") - reporterThread.interrupt() - } - if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) { - logDebug("shutting down user thread") - userClassThread.interrupt() + final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = { + synchronized { + if (!finished) { + val inShutdown = Utils.inShutdown() + logInfo(s"Final app status: $status, exitCode: $code" + + Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) + exitCode = code + finalStatus = status + finalMsg = msg + finished = true + if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) { + logDebug("shutting down reporter thread") + reporterThread.interrupt() + } + if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) { + logDebug("shutting down user thread") + userClassThread.interrupt() + } } } } @@ -506,7 +510,7 @@ private[spark] class ApplicationMaster( private class AMActor(driverUrl: String, isClusterMode: Boolean) extends Actor { var driver: ActorSelection = _ - override def preStart() = { + override def preStart(): Unit = { logInfo("Listen to driver: " + driverUrl) driver = context.actorSelection(driverUrl) // Send a hello message to establish the connection, after which @@ -520,7 +524,7 @@ private[spark] class ApplicationMaster( } } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") // In cluster mode, do not rely on the disassociated event to exit @@ -567,7 +571,7 @@ object ApplicationMaster extends Logging { private var master: ApplicationMaster = _ - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { SignalLogger.register(log) val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => @@ -576,11 +580,11 @@ object ApplicationMaster extends Logging { } } - private[spark] def sparkContextInitialized(sc: SparkContext) = { + private[spark] def sparkContextInitialized(sc: SparkContext): Unit = { master.sparkContextInitialized(sc) } - private[spark] def sparkContextStopped(sc: SparkContext) = { + private[spark] def sparkContextStopped(sc: SparkContext): Boolean = { master.sparkContextStopped(sc) } @@ -592,7 +596,7 @@ object ApplicationMaster extends Logging { */ object ExecutorLauncher { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { ApplicationMaster.main(args) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index c1d3f7320f53c..1ce10d906ab23 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -59,15 +59,15 @@ class ExecutorRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) lazy val env = prepareEnvironment(container) - def run = { + override def run(): Unit = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() nmClient.init(yarnConf) nmClient.start() - startContainer + startContainer() } - def startContainer = { + def startContainer(): java.util.Map[String, ByteBuffer] = { logInfo("Setting up ContainerLaunchContext") val ctx = Records.newRecord(classOf[ContainerLaunchContext]) From b0d884f044fea1c954da77073f3556cd9ab1e922 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 3 Apr 2015 09:48:37 +0100 Subject: [PATCH 635/817] [SPARK-6560][CORE] Do not suppress exceptions from writer.write. If there is a failure in the Hadoop backend while calling writer.write, we should remember this original exception, and try to call writer.close(), but if that fails as well, still report the original exception. Note that, if writer.write fails, it is likely that writer was left in an invalid state, and so actually makes it more likely that writer.close will also fail. Which just increases the chances for writer.write's exception to be suppressed. This patch introduces an admittedly potentially too cute Utils.tryWithSafeFinally method to handle the try/finally gyrations. Author: Stephen Haberman Closes #5223 from stephenh/do_not_suppress_writer_exception and squashes the following commits: c7ad53f [Stephen Haberman] [SPARK-6560][CORE] Do not suppress exceptions from writer.write. --- .../org/apache/spark/MapOutputTracker.scala | 11 +++-- .../apache/spark/api/python/PythonRDD.scala | 8 ++-- .../spark/broadcast/HttpBroadcast.scala | 19 +++++--- .../master/FileSystemPersistenceEngine.scala | 5 +- .../deploy/rest/StandaloneRestClient.scala | 8 +++- .../org/apache/spark/rdd/CheckpointRDD.scala | 8 +++- .../apache/spark/rdd/PairRDDFunctions.scala | 9 ++-- .../shuffle/IndexShuffleBlockManager.scala | 6 +-- .../spark/storage/BlockObjectWriter.scala | 16 ++++--- .../org/apache/spark/storage/DiskStore.scala | 18 ++++---- .../scala/org/apache/spark/util/Utils.scala | 46 +++++++++++++++++-- .../util/collection/ExternalSorter.scala | 26 ++++------- 12 files changed, 118 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index c9426c5de23a2..5718951451afc 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -356,11 +356,14 @@ private[spark] object MapOutputTracker extends Logging { def serializeMapStatuses(statuses: Array[MapStatus]): Array[Byte] = { val out = new ByteArrayOutputStream val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) - // Since statuses can be modified in parallel, sync on it - statuses.synchronized { - objOut.writeObject(statuses) + Utils.tryWithSafeFinally { + // Since statuses can be modified in parallel, sync on it + statuses.synchronized { + objOut.writeObject(statuses) + } + } { + objOut.close() } - objOut.close() out.toByteArray } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 36cf2af0857dd..b1ffba4c546bf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -614,9 +614,9 @@ private[spark] object PythonRDD extends Logging { try { val sock = serverSocket.accept() val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) - try { + Utils.tryWithSafeFinally { writeIteratorToStream(items, out) - } finally { + } { out.close() } } catch { @@ -862,9 +862,9 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial val file = File.createTempFile("broadcast", "", dir) path = file.getAbsolutePath val out = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { Utils.copyStream(in, out) - } finally { + } { out.close() } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 74ccfa6d3c9a3..4457c75e8b0fc 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -165,7 +165,7 @@ private[broadcast] object HttpBroadcast extends Logging { private def write(id: Long, value: Any) { val file = getFile(id) val fileOutputStream = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { val out: OutputStream = { if (compress) { compressionCodec.compressedOutputStream(fileOutputStream) @@ -175,10 +175,13 @@ private[broadcast] object HttpBroadcast extends Logging { } val ser = SparkEnv.get.serializer.newInstance() val serOut = ser.serializeStream(out) - serOut.writeObject(value) - serOut.close() + Utils.tryWithSafeFinally { + serOut.writeObject(value) + } { + serOut.close() + } files += file - } finally { + } { fileOutputStream.close() } } @@ -212,9 +215,11 @@ private[broadcast] object HttpBroadcast extends Logging { } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) - val obj = serIn.readObject[T]() - serIn.close() - obj + Utils.tryWithSafeFinally { + serIn.readObject[T]() + } { + serIn.close() + } } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 32499b3a784a1..f459ed5b3a1a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import akka.serialization.Serialization import org.apache.spark.Logging +import org.apache.spark.util.Utils /** @@ -59,9 +60,9 @@ private[master] class FileSystemPersistenceEngine( val serializer = serialization.findSerializerFor(value) val serialized = serializer.toBinary(value) val out = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { out.write(serialized) - } finally { + } { out.close() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index 420442f7564cc..a3539e44bd2f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -27,6 +27,7 @@ import com.fasterxml.jackson.core.JsonProcessingException import com.google.common.base.Charsets import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.Utils /** * A client that submits applications to the standalone Master using a REST protocol. @@ -148,8 +149,11 @@ private[deploy] class StandaloneRestClient extends Logging { conn.setRequestProperty("charset", "utf-8") conn.setDoOutput(true) val out = new DataOutputStream(conn.getOutputStream) - out.write(json.getBytes(Charsets.UTF_8)) - out.close() + Utils.tryWithSafeFinally { + out.write(json.getBytes(Charsets.UTF_8)) + } { + out.close() + } readResponse(conn) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 1c13e2c372845..760c0fa3ac96a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -112,8 +113,11 @@ private[spark] object CheckpointRDD extends Logging { } val serializer = env.serializer.newInstance() val serializeStream = serializer.serializeStream(fileOutputStream) - serializeStream.writeAll(iterator) - serializeStream.close() + Utils.tryWithSafeFinally { + serializeStream.writeAll(iterator) + } { + serializeStream.close() + } if (!fs.rename(tempOutputPath, finalOutputPath)) { if (!fs.exists(finalOutputPath)) { 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 6b4f097ea9ae5..bf1303d39592d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -995,7 +995,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] require(writer != null, "Unable to obtain RecordWriter") var recordsWritten = 0L - try { + Utils.tryWithSafeFinally { while (iter.hasNext) { val pair = iter.next() writer.write(pair._1, pair._2) @@ -1004,7 +1004,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) recordsWritten += 1 } - } finally { + } { writer.close(hadoopContext) } committer.commitTask(hadoopContext) @@ -1068,7 +1068,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() var recordsWritten = 0L - try { + + Utils.tryWithSafeFinally { while (iter.hasNext) { val record = iter.next() writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) @@ -1077,7 +1078,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) recordsWritten += 1 } - } finally { + } { writer.close() } writer.commit() diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 50edb5a34e333..a1741e2875c16 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -26,6 +26,7 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ +import org.apache.spark.util.Utils import IndexShuffleBlockManager.NOOP_REDUCE_ID @@ -78,16 +79,15 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) - try { + Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. var offset = 0L out.writeLong(offset) - for (length <- lengths) { offset += length out.writeLong(offset) } - } finally { + } { out.close() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index f703e50b6b0ac..0dfc91dfaff85 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -23,6 +23,7 @@ import java.nio.channels.FileChannel import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.util.Utils /** * An interface for writing JVM objects to some underlying storage. This interface allows @@ -140,14 +141,17 @@ private[spark] class DiskBlockObjectWriter( override def close() { if (initialized) { - if (syncWrites) { - // Force outstanding writes to disk and track how long it takes - objOut.flush() - callWithTiming { - fos.getFD.sync() + Utils.tryWithSafeFinally { + if (syncWrites) { + // Force outstanding writes to disk and track how long it takes + objOut.flush() + callWithTiming { + fos.getFD.sync() + } } + } { + objOut.close() } - objOut.close() channel = null bs = null diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 61ef5ff168791..4b232ae7d3180 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -46,10 +46,13 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val channel = new FileOutputStream(file).getChannel - while (bytes.remaining > 0) { - channel.write(bytes) + Utils.tryWithSafeFinally { + while (bytes.remaining > 0) { + channel.write(bytes) + } + } { + channel.close() } - channel.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) @@ -75,9 +78,9 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) try { - try { + Utils.tryWithSafeFinally { blockManager.dataSerializeStream(blockId, outputStream, values) - } finally { + } { // Close outputStream here because it should be closed before file is deleted. outputStream.close() } @@ -106,8 +109,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc private def getBytes(file: File, offset: Long, length: Long): Option[ByteBuffer] = { val channel = new RandomAccessFile(file, "r").getChannel - - try { + Utils.tryWithSafeFinally { // For small files, directly read rather than memory map if (length < minMemoryMapBytes) { val buf = ByteBuffer.allocate(length.toInt) @@ -123,7 +125,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } else { Some(channel.map(MapMode.READ_ONLY, offset, length)) } - } finally { + } { channel.close() } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bb8bd1015668a..7c85e28679f1d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -313,7 +313,7 @@ private[spark] object Utils extends Logging { transferToEnabled: Boolean = false): Long = { var count = 0L - try { + tryWithSafeFinally { if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. @@ -353,7 +353,7 @@ private[spark] object Utils extends Logging { } } count - } finally { + } { if (closeStreams) { try { in.close() @@ -1214,6 +1214,44 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code, then a finally block, but if exceptions happen in + * the finally block, do not suppress the original exception. + * + * This is primarily an issue with `finally { out.close() }` blocks, where + * close needs to be called to clean up `out`, but if an exception happened + * in `out.write`, it's likely `out` may be corrupted and `out.close` will + * fail as well. This would then suppress the original/likely more meaningful + * exception from the original `out.write` call. + */ + def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = { + // It would be nice to find a method on Try that did this + var originalThrowable: Throwable = null + try { + block + } catch { + case t: Throwable => + // Purposefully not using NonFatal, because even fatal exceptions + // we don't want to have our finallyBlock suppress + originalThrowable = t + throw originalThrowable + } finally { + try { + finallyBlock + } catch { + case t: Throwable => + if (originalThrowable != null) { + // We could do originalThrowable.addSuppressed(t), but it's + // not available in JDK 1.6. + logWarning(s"Suppressing exception in finally: " + t.getMessage, t) + throw originalThrowable + } else { + throw t + } + } + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when @@ -2074,7 +2112,7 @@ private[spark] class RedirectThread( override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - try { + Utils.tryWithSafeFinally { val buf = new Array[Byte](1024) var len = in.read(buf) while (len != -1) { @@ -2082,7 +2120,7 @@ private[spark] class RedirectThread( out.flush() len = in.read(buf) } - } finally { + } { if (propagateEof) { out.close() } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 7bd3c7852a6b2..035f3767ff554 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -728,25 +728,19 @@ private[spark] class ExternalSorter[K, V, C]( // this simple we spill out the current in-memory collection so that everything is in files. spillToPartitionFiles(if (aggregator.isDefined) map else buffer) partitionWriters.foreach(_.commitAndClose()) - var out: FileOutputStream = null - var in: FileInputStream = null + val out = new FileOutputStream(outputFile, true) val writeStartTime = System.nanoTime - try { - out = new FileOutputStream(outputFile, true) + util.Utils.tryWithSafeFinally { for (i <- 0 until numPartitions) { - in = new FileInputStream(partitionWriters(i).fileSegment().file) - val size = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) - in.close() - in = null - lengths(i) = size - } - } finally { - if (out != null) { - out.close() - } - if (in != null) { - in.close() + val in = new FileInputStream(partitionWriters(i).fileSegment().file) + util.Utils.tryWithSafeFinally { + lengths(i) = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) + } { + in.close() + } } + } { + out.close() context.taskMetrics.shuffleWriteMetrics.foreach( _.incShuffleWriteTime(System.nanoTime - writeStartTime)) } From b52c7f9fc87a1b9a039724e1dac8b30554f75196 Mon Sep 17 00:00:00 2001 From: Omede Firouz Date: Fri, 3 Apr 2015 10:26:43 +0100 Subject: [PATCH 636/817] [MLLIB] Remove println in LogisticRegression.scala There's no corresponding printing in linear regression. Here was my previous PR (something weird happened and I can't reopen it) https://github.com/apache/spark/pull/5272 Author: Omede Firouz Closes #5338 from oefirouz/println and squashes the following commits: 3f3dbf4 [Omede Firouz] [MLLIB] Remove println --- .../org/apache/spark/ml/classification/LogisticRegression.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 21f61d80dd95a..49c00f77480e8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -180,7 +180,6 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { - println(s"LR.predict with threshold: ${paramMap(threshold)}") if (score(features) > paramMap(threshold)) 1 else 0 } From 512a2f191a6b53699373b6588f316b4437050425 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Fri, 3 Apr 2015 09:49:50 -0700 Subject: [PATCH 637/817] [SPARK-6615][MLLIB] Python API for Word2Vec This is the sub-task of SPARK-6254. Wrap missing method for `Word2Vec` and `Word2VecModel`. Author: lewuathe Closes #5296 from Lewuathe/SPARK-6615 and squashes the following commits: f14c304 [lewuathe] Reorder tests 1d326b9 [lewuathe] Merge master e2bedfb [lewuathe] Modify test cases afb866d [lewuathe] [SPARK-6615] Python API for Word2Vec --- .../mllib/api/python/PythonMLLibAPI.scala | 8 +++- python/pyspark/mllib/feature.py | 18 +++++++- python/pyspark/mllib/tests.py | 45 ++++++++++++++++--- 3 files changed, 64 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 5995d6df97c15..6c386cacfb7ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -476,13 +476,15 @@ private[python] class PythonMLLibAPI extends Serializable { learningRate: Double, numPartitions: Int, numIterations: Int, - seed: Long): Word2VecModelWrapper = { + seed: Long, + minCount: Int): Word2VecModelWrapper = { val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) .setNumPartitions(numPartitions) .setNumIterations(numIterations) .setSeed(seed) + .setMinCount(minCount) try { val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) new Word2VecModelWrapper(model) @@ -516,6 +518,10 @@ private[python] class PythonMLLibAPI extends Serializable { val words = result.map(_._1) List(words, similarity).map(_.asInstanceOf[Object]).asJava } + + def getVectors: JMap[String, JList[Float]] = { + model.getVectors.map({case (k, v) => (k, v.toList.asJava)}).asJava + } } /** diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 4bfe3014ef748..3cda1205e1391 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -337,6 +337,12 @@ def findSynonyms(self, word, num): words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) + def getVectors(self): + """ + Returns a map of words to their vector representations. + """ + return self.call("getVectors") + class Word2Vec(object): """ @@ -379,6 +385,7 @@ def __init__(self): self.numPartitions = 1 self.numIterations = 1 self.seed = random.randint(0, sys.maxint) + self.minCount = 5 def setVectorSize(self, vectorSize): """ @@ -417,6 +424,14 @@ def setSeed(self, seed): self.seed = seed return self + def setMinCount(self, minCount): + """ + Sets minCount, the minimum number of times a token must appear + to be included in the word2vec model's vocabulary (default: 5). + """ + self.minCount = minCount + return self + def fit(self, data): """ Computes the vector representation of each word in vocabulary. @@ -428,7 +443,8 @@ def fit(self, data): raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed)) + int(self.numIterations), long(self.seed), + int(self.minCount)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 6e9c68ec8a5c1..dd3b66ce67457 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -42,6 +42,7 @@ from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics +from pyspark.mllib.feature import Word2Vec from pyspark.mllib.feature import IDF from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext @@ -630,6 +631,12 @@ def test_right_number_of_results(self): self.assertIsNotNone(chi[1000]) +class SerDeTest(PySparkTestCase): + def test_to_java_object_rdd(self): # SPARK-6660 + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) + self.assertEqual(_to_java_object_rdd(data).count(), 10) + + class FeatureTest(PySparkTestCase): def test_idf_model(self): data = [ @@ -643,11 +650,39 @@ def test_idf_model(self): self.assertEqual(len(idf), 11) -class SerDeTest(PySparkTestCase): - def test_to_java_object_rdd(self): # SPARK-6660 - data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) - self.assertEqual(_to_java_object_rdd(data).count(), 10) - +class Word2VecTests(PySparkTestCase): + def test_word2vec_setters(self): + data = [ + ["I", "have", "a", "pen"], + ["I", "like", "soccer", "very", "much"], + ["I", "live", "in", "Tokyo"] + ] + model = Word2Vec() \ + .setVectorSize(2) \ + .setLearningRate(0.01) \ + .setNumPartitions(2) \ + .setNumIterations(10) \ + .setSeed(1024) \ + .setMinCount(3) + self.assertEquals(model.vectorSize, 2) + self.assertTrue(model.learningRate < 0.02) + self.assertEquals(model.numPartitions, 2) + self.assertEquals(model.numIterations, 10) + self.assertEquals(model.seed, 1024) + self.assertEquals(model.minCount, 3) + + def test_word2vec_get_vectors(self): + data = [ + ["a", "b", "c", "d", "e", "f", "g"], + ["a", "b", "c", "d", "e", "f"], + ["a", "b", "c", "d", "e"], + ["a", "b", "c", "d"], + ["a", "b", "c"], + ["a", "b"], + ["a"] + ] + model = Word2Vec().fit(self.sc.parallelize(data)) + self.assertEquals(len(model.getVectors()), 3) if __name__ == "__main__": if not _have_scipy: From dc6dff248d8f5d7de22af64b0586dfe3885731df Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 3 Apr 2015 18:31:48 +0100 Subject: [PATCH 638/817] [Minor][SQL] Fix typo Just fix a typo. Author: Liang-Chi Hsieh Closes #5352 from viirya/fix_a_typo and squashes the following commits: 303b2d2 [Liang-Chi Hsieh] Fix typo. --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ae5ce4cf4c7e7..315fab673da5c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -271,7 +271,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } case other => logWarning( - s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} shold be stored " + + s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be stored " + s"as Parquet. However, we are getting a ${other} from the metastore cache. " + s"This cached entry will be invalidated.") cachedDataSourceTables.invalidate(tableIdentifier) From c23ba81b8cf86c3a085de8ddfef9403ff6fcd87f Mon Sep 17 00:00:00 2001 From: guowei2 Date: Sat, 4 Apr 2015 02:02:30 +0800 Subject: [PATCH 639/817] [SPARK-5203][SQL] fix union with different decimal type When union non-decimal types with decimals, we use the following rules: - FIRST `intTypeToFixed`, then fixed union decimals with precision/scale p1/s2 and p2/s2 will be promoted to DecimalType(max(p1, p2), max(s1, s2)) - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, but note that unlimited decimals are considered bigger than doubles in WidenTypes) Author: guowei2 Closes #4004 from guowei2/SPARK-5203 and squashes the following commits: ff50f5f [guowei2] fix code style 11df1bf [guowei2] fix decimal union with double, double->Decimal(15,15) 0f345f9 [guowei2] fix structType merge with decimal 101ed4d [guowei2] fix build error after rebase 0b196e4 [guowei2] code style fe2c2ca [guowei2] handle union decimal precision in 'DecimalPrecision' 421d840 [guowei2] fix union types for decimal precision ef2c661 [guowei2] fix union with different decimal type --- .../catalyst/analysis/HiveTypeCoercion.scala | 190 ++++++++++++------ .../apache/spark/sql/types/dataTypes.scala | 5 +- .../analysis/DecimalPrecisionSuite.scala | 30 ++- .../sql/hive/execution/SQLQuerySuite.scala | 11 + 4 files changed, 167 insertions(+), 69 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 9a33eb145273e..3aeb964994d37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -285,6 +285,7 @@ trait HiveTypeCoercion { * Calculates and propagates precision for fixed-precision decimals. Hive has a number of * rules for this based on the SQL standard and MS SQL: * https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf + * https://msdn.microsoft.com/en-us/library/ms190476.aspx * * In particular, if we have expressions e1 and e2 with precision/scale p1/s2 and p2/s2 * respectively, then the following operations have the following precision / scale: @@ -296,6 +297,7 @@ trait HiveTypeCoercion { * e1 * e2 p1 + p2 + 1 s1 + s2 * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) + * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) * sum(e1) p1 + 10 s1 * avg(e1) p1 + 4 s1 + 4 * @@ -311,7 +313,12 @@ trait HiveTypeCoercion { * - SHORT gets turned into DECIMAL(5, 0) * - INT gets turned into DECIMAL(10, 0) * - LONG gets turned into DECIMAL(20, 0) - * - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, + * - FLOAT and DOUBLE + * 1. Union operation: + * FLOAT gets turned into DECIMAL(7, 7), DOUBLE gets turned into DECIMAL(15, 15) (this is the + * same as Hive) + * 2. Other operation: + * FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, * but note that unlimited decimals are considered bigger than doubles in WidenTypes) */ // scalastyle:on @@ -328,76 +335,127 @@ trait HiveTypeCoercion { def isFloat(t: DataType): Boolean = t == FloatType || t == DoubleType - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes whose children have not been resolved yet - case e if !e.childrenResolved => e + // Conversion rules for float and double into fixed-precision decimals + val floatTypeToFixed: Map[DataType, DecimalType] = Map( + FloatType -> DecimalType(7, 7), + DoubleType -> DecimalType(15, 15) + ) - case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Add(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - ) - - case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Subtract(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - ) - - case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Multiply(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(p1 + p2 + 1, s1 + s2) - ) - - case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Divide(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) - ) - - case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Remainder(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) - ) - - case LessThan(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case GreaterThan(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - // Promote integers inside a binary expression with fixed-precision decimals to decimals, - // and fixed-precision decimals in an expression with floats / doubles to doubles - case b: BinaryExpression if b.left.dataType != b.right.dataType => - (b.left.dataType, b.right.dataType) match { - case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => - b.makeCopy(Array(Cast(b.left, intTypeToFixed(t)), b.right)) - case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => - b.makeCopy(Array(b.left, Cast(b.right, intTypeToFixed(t)))) - case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(b.left, Cast(b.right, DoubleType))) - case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(b.left, DoubleType), b.right)) - case _ => - b + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // fix decimal precision for union + case u @ Union(left, right) if u.childrenResolved && !u.resolved => + val castedInput = left.output.zip(right.output).map { + case (l, r) if l.dataType != r.dataType => + (l.dataType, r.dataType) match { + case (DecimalType.Fixed(p1, s1), DecimalType.Fixed(p2, s2)) => + // Union decimals with precision/scale p1/s2 and p2/s2 will be promoted to + // DecimalType(max(s1, s2) + max(p1-s1, p2-s2), max(s1, s2)) + val fixedType = DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2), max(s1, s2)) + (Alias(Cast(l, fixedType), l.name)(), Alias(Cast(r, fixedType), r.name)()) + case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => + (Alias(Cast(l, intTypeToFixed(t)), l.name)(), r) + case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => + (l, Alias(Cast(r, intTypeToFixed(t)), r.name)()) + case (t, DecimalType.Fixed(p, s)) if floatTypeToFixed.contains(t) => + (Alias(Cast(l, floatTypeToFixed(t)), l.name)(), r) + case (DecimalType.Fixed(p, s), t) if floatTypeToFixed.contains(t) => + (l, Alias(Cast(r, floatTypeToFixed(t)), r.name)()) + case _ => (l, r) + } + case other => other } - // TODO: MaxOf, MinOf, etc might want other rules + val (castedLeft, castedRight) = castedInput.unzip - // SUM and AVERAGE are handled by the implementations of those expressions + val newLeft = + if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { + Project(castedLeft, left) + } else { + left + } + + val newRight = + if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { + Project(castedRight, right) + } else { + right + } + + Union(newLeft, newRight) + + // fix decimal precision for expressions + case q => q.transformExpressions { + // Skip nodes whose children have not been resolved yet + case e if !e.childrenResolved => e + + case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Add(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Subtract(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Multiply(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 + p2 + 1, s1 + s2) + ) + + case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Divide(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) + ) + + case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Remainder(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) + ) + + case LessThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + // Promote integers inside a binary expression with fixed-precision decimals to decimals, + // and fixed-precision decimals in an expression with floats / doubles to doubles + case b: BinaryExpression if b.left.dataType != b.right.dataType => + (b.left.dataType, b.right.dataType) match { + case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => + b.makeCopy(Array(Cast(b.left, intTypeToFixed(t)), b.right)) + case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => + b.makeCopy(Array(b.left, Cast(b.right, intTypeToFixed(t)))) + case (t, DecimalType.Fixed(p, s)) if isFloat(t) => + b.makeCopy(Array(b.left, Cast(b.right, DoubleType))) + case (DecimalType.Fixed(p, s), t) if isFloat(t) => + b.makeCopy(Array(Cast(b.left, DoubleType), b.right)) + case _ => + b + } + + // TODO: MaxOf, MinOf, etc might want other rules + + // SUM and AVERAGE are handled by the implementations of those expressions + } } + } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 952cf5c75688d..cdf2bc68d9c5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.types import java.sql.Timestamp import scala.collection.mutable.ArrayBuffer +import scala.math._ import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} @@ -934,7 +935,9 @@ object StructType { case (DecimalType.Fixed(leftPrecision, leftScale), DecimalType.Fixed(rightPrecision, rightScale)) => - DecimalType(leftPrecision.max(rightPrecision), leftScale.max(rightScale)) + DecimalType( + max(leftScale, rightScale) + max(leftPrecision - leftScale, rightPrecision - rightScale), + max(leftScale, rightScale)) case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) if leftUdt.userClass == rightUdt.userClass => leftUdt diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index bc2ec754d5865..67bec999dfbd1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation} +import org.apache.spark.sql.catalyst.plans.logical.{Union, Project, LocalRelation} import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -31,7 +31,8 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { AttributeReference("d1", DecimalType(2, 1))(), AttributeReference("d2", DecimalType(5, 2))(), AttributeReference("u", DecimalType.Unlimited)(), - AttributeReference("f", FloatType)() + AttributeReference("f", FloatType)(), + AttributeReference("b", DoubleType)() ) val i: Expression = UnresolvedAttribute("i") @@ -39,6 +40,7 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { val d2: Expression = UnresolvedAttribute("d2") val u: Expression = UnresolvedAttribute("u") val f: Expression = UnresolvedAttribute("f") + val b: Expression = UnresolvedAttribute("b") before { catalog.registerTable(Seq("table"), relation) @@ -58,6 +60,17 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { assert(comparison.right.dataType === expectedType) } + private def checkUnion(left: Expression, right: Expression, expectedType: DataType): Unit = { + val plan = + Union(Project(Seq(Alias(left, "l")()), relation), + Project(Seq(Alias(right, "r")()), relation)) + val (l, r) = analyzer(plan).collect { + case Union(left, right) => (left.output.head, right.output.head) + }.head + assert(l.dataType === expectedType) + assert(r.dataType === expectedType) + } + test("basic operations") { checkType(Add(d1, d2), DecimalType(6, 2)) checkType(Subtract(d1, d2), DecimalType(6, 2)) @@ -82,6 +95,19 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) } + test("decimal precision for union") { + checkUnion(d1, i, DecimalType(11, 1)) + checkUnion(i, d2, DecimalType(12, 2)) + checkUnion(d1, d2, DecimalType(5, 2)) + checkUnion(d2, d1, DecimalType(5, 2)) + checkUnion(d1, f, DecimalType(8, 7)) + checkUnion(f, d2, DecimalType(10, 7)) + checkUnion(d1, b, DecimalType(16, 15)) + checkUnion(b, d2, DecimalType(18, 15)) + checkUnion(d1, u, DecimalType.Unlimited) + checkUnion(u, d2, DecimalType.Unlimited) + } + test("bringing in primitive types") { checkType(Add(d1, i), DecimalType(12, 1)) checkType(Add(d1, f), DoubleType) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 2065f0d60d92f..817b9dcb8f505 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -468,4 +468,15 @@ class SQLQuerySuite extends QueryTest { sql(s"DROP TABLE $tableName") } } + + test("SPARK-5203 union with different decimal precision") { + Seq.empty[(Decimal, Decimal)] + .toDF("d1", "d2") + .select($"d1".cast(DecimalType(10, 15)).as("d")) + .registerTempTable("dn") + + sql("select d from dn union all select d * 2 from dn") + .queryExecution.analyzed + } + } From 2c43ea38ee0db6b292c14baf6bc6f8d16f509c9d Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 3 Apr 2015 19:23:11 +0100 Subject: [PATCH 640/817] [SPARK-6492][CORE] SparkContext.stop() can deadlock when DAGSchedulerEventProcessLoop dies I've added a timeout and retry loop around the SparkContext shutdown code that should fix this deadlock. If a SparkContext shutdown is in progress when another thread comes knocking, it will wait for 10 seconds for the lock, then fall through where the outer loop will re-submit the request. Author: Ilya Ganelin Closes #5277 from ilganeli/SPARK-6492 and squashes the following commits: 8617a7e [Ilya Ganelin] Resolved merge conflict 2fbab66 [Ilya Ganelin] Added MIMA Exclude a0e2c70 [Ilya Ganelin] Deleted stale imports fa28ce7 [Ilya Ganelin] reverted to just having a single stopped 76fc825 [Ilya Ganelin] Updated to use atomic booleans instead of the synchronized vars 6e8a7f7 [Ilya Ganelin] Removing unecessary null check for now since i'm not fixing stop ordering yet cdf7073 [Ilya Ganelin] [SPARK-6492] Moved stopped=true back to the start of the shutdown sequence so this can be addressed in a seperate PR 7fb795b [Ilya Ganelin] Spacing b7a0c5c [Ilya Ganelin] Import ordering df8224f [Ilya Ganelin] Added comment for added lock 343cb94 [Ilya Ganelin] [SPARK-6492] Added timeout/retry logic to fix a deadlock in SparkContext shutdown --- .../scala/org/apache/spark/SparkContext.scala | 59 ++++++++++--------- project/MimaExcludes.scala | 4 ++ 2 files changed, 34 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5b3778ead6994..abf81e312d8e6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -23,7 +23,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID import scala.collection.{Map, Set} @@ -95,10 +95,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val startTime = System.currentTimeMillis() - @volatile private var stopped: Boolean = false + private val stopped: AtomicBoolean = new AtomicBoolean(false) private def assertNotStopped(): Unit = { - if (stopped) { + if (stopped.get()) { throw new IllegalStateException("Cannot call methods on a stopped SparkContext") } } @@ -1390,33 +1390,34 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli addedJars.clear() } - /** Shut down the SparkContext. */ + // Shut down the SparkContext. def stop() { - SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - if (!stopped) { - stopped = true - postApplicationEnd() - ui.foreach(_.stop()) - env.metricsSystem.report() - metadataCleaner.cancel() - cleaner.foreach(_.stop()) - executorAllocationManager.foreach(_.stop()) - dagScheduler.stop() - dagScheduler = null - listenerBus.stop() - eventLogger.foreach(_.stop()) - env.actorSystem.stop(heartbeatReceiver) - progressBar.foreach(_.stop()) - taskScheduler = null - // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) - logInfo("Successfully stopped SparkContext") - SparkContext.clearActiveContext() - } else { - logInfo("SparkContext already stopped") - } + // Use the stopping variable to ensure no contention for the stop scenario. + // Still track the stopped variable for use elsewhere in the code. + + if (!stopped.compareAndSet(false, true)) { + logInfo("SparkContext already stopped.") + return } + + postApplicationEnd() + ui.foreach(_.stop()) + env.metricsSystem.report() + metadataCleaner.cancel() + cleaner.foreach(_.stop()) + executorAllocationManager.foreach(_.stop()) + dagScheduler.stop() + dagScheduler = null + listenerBus.stop() + eventLogger.foreach(_.stop()) + env.actorSystem.stop(heartbeatReceiver) + progressBar.foreach(_.stop()) + taskScheduler = null + // TODO: Cache.stop()? + env.stop() + SparkEnv.set(null) + SparkContext.clearActiveContext() + logInfo("Successfully stopped SparkContext") } @@ -1478,7 +1479,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - if (stopped) { + if (stopped.get()) { throw new IllegalStateException("SparkContext has been shutdown") } val callSite = getCallSite diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 54500f7c2701f..c2d828f982fe0 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,6 +60,10 @@ object MimaExcludes { ) ++ Seq( // SPARK-6510 Add a Graph#minus method acting as Set#difference ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") + ) ++ Seq( + // SPARK-6492 Fix deadlock in SparkContext.stop() + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.org$" + + "apache$spark$SparkContext$$SPARK_CONTEXT_CONSTRUCTOR_LOCK") ) case v if v.startsWith("1.3") => From 88504b75ee610e14d7ceed8b038fa698a3d14f81 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 3 Apr 2015 11:44:27 -0700 Subject: [PATCH 641/817] [SPARK-6640][Core] Fix the race condition of creating HeartbeatReceiver and retrieving HeartbeatReceiver This PR moved the code of creating `HeartbeatReceiver` above the code of creating `schedulerBackend` to resolve the race condition. Author: zsxwing Closes #5306 from zsxwing/SPARK-6640 and squashes the following commits: 840399d [zsxwing] Don't send TaskScheduler through Akka a90616a [zsxwing] Fix docs dd202c7 [zsxwing] Fix typo d7c250d [zsxwing] Fix the race condition of creating HeartbeatReceiver and retrieving HeartbeatReceiver --- .../org/apache/spark/HeartbeatReceiver.scala | 32 +++++++++++++++---- .../scala/org/apache/spark/SparkContext.scala | 10 ++++-- 2 files changed, 33 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 8435e1ea2611c..9f8ad03b91e85 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -37,6 +37,12 @@ private[spark] case class Heartbeat( taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) +/** + * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is + * created. + */ +private[spark] case object TaskSchedulerIsSet + private[spark] case object ExpireDeadHosts private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) @@ -44,9 +50,11 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** * Lives in the driver to receive heartbeats from executors.. */ -private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskScheduler) +private[spark] class HeartbeatReceiver(sc: SparkContext) extends Actor with ActorLogReceive with Logging { + private var scheduler: TaskScheduler = null + // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] @@ -71,12 +79,22 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule } override def receiveWithLogging: PartialFunction[Any, Unit] = { - case Heartbeat(executorId, taskMetrics, blockManagerId) => - val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, taskMetrics, blockManagerId) - val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) - executorLastSeen(executorId) = System.currentTimeMillis() - sender ! response + case TaskSchedulerIsSet => + scheduler = sc.taskScheduler + case heartbeat @ Heartbeat(executorId, taskMetrics, blockManagerId) => + if (scheduler != null) { + val unknownExecutor = !scheduler.executorHeartbeatReceived( + executorId, taskMetrics, blockManagerId) + val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) + executorLastSeen(executorId) = System.currentTimeMillis() + sender ! response + } else { + // Because Executor will sleep several seconds before sending the first "Heartbeat", this + // case rarely happens. However, if it really happens, log it and ask the executor to + // register itself again. + logWarning(s"Dropping $heartbeat because TaskScheduler is not ready yet") + sender ! HeartbeatResponse(reregisterBlockManager = true) + } case ExpireDeadHosts => expireDeadHosts() } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index abf81e312d8e6..fd1838976ee22 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -356,11 +356,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val sparkUser = Utils.getCurrentUserName() executorEnvs("SPARK_USER") = sparkUser + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will + // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) + private val heartbeatReceiver = env.actorSystem.actorOf( + Props(new HeartbeatReceiver(this)), "HeartbeatReceiver") + // Create and start the scheduler private[spark] var (schedulerBackend, taskScheduler) = SparkContext.createTaskScheduler(this, master) - private val heartbeatReceiver = env.actorSystem.actorOf( - Props(new HeartbeatReceiver(this, taskScheduler)), "HeartbeatReceiver") + + heartbeatReceiver ! TaskSchedulerIsSet + @volatile private[spark] var dagScheduler: DAGScheduler = _ try { dagScheduler = new DAGScheduler(this) From ffe8cc9a25454ee4f451f6ee3ec6d1e934b47ca2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 Apr 2015 11:53:07 -0700 Subject: [PATCH 642/817] Closes #3158 From 14632b7942c02a332c4d3814fb6b2611e3f76fc7 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 3 Apr 2015 11:54:31 -0700 Subject: [PATCH 643/817] [SPARK-6688] [core] Always use resolved URIs in EventLoggingListener. Author: Marcelo Vanzin Closes #5340 from vanzin/SPARK-6688 and squashes the following commits: ccfddd9 [Marcelo Vanzin] Resolve at the source. 20d2a34 [Marcelo Vanzin] [SPARK-6688] [core] Always use resolved URIs in EventLoggingListener. --- .../scala/org/apache/spark/SparkContext.scala | 6 +++-- .../spark/deploy/ApplicationDescription.scala | 6 +++-- .../scheduler/EventLoggingListener.scala | 10 ++++----- .../history/FsHistoryProviderSuite.scala | 2 +- .../scheduler/EventLoggingListenerSuite.scala | 22 ++++++++++++------- .../spark/scheduler/ReplayListenerSuite.scala | 3 ++- 6 files changed, 30 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fd1838976ee22..3b73a8a8fd850 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -227,9 +227,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val appName = conf.get("spark.app.name") private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - private[spark] val eventLogDir: Option[String] = { + private[spark] val eventLogDir: Option[URI] = { if (isEventLogEnabled) { - Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) + val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) + .stripSuffix("/") + Some(Utils.resolveURI(unresolvedDir)) } else { None } diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 3d0d68de8f495..b7ae9c1fc0a23 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,13 +17,15 @@ package org.apache.spark.deploy +import java.net.URI + private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, var appUiUrl: String, - val eventLogDir: Option[String] = None, + val eventLogDir: Option[URI] = None, // short name of compression codec used when writing event logs, if any (e.g. lzf) val eventLogCodec: Option[String] = None) extends Serializable { @@ -36,7 +38,7 @@ private[spark] class ApplicationDescription( memoryPerSlave: Int = memoryPerSlave, command: Command = command, appUiUrl: String = appUiUrl, - eventLogDir: Option[String] = eventLogDir, + eventLogDir: Option[URI] = eventLogDir, eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = new ApplicationDescription( name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index c0d889360ae99..08e7727db2fde 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -47,21 +47,21 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ private[spark] class EventLoggingListener( appId: String, - logBaseDir: String, + logBaseDir: URI, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appId: String, logBaseDir: String, sparkConf: SparkConf) = + def this(appId: String, logBaseDir: URI, sparkConf: SparkConf) = this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf) + private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) private val compressionCodec = if (shouldCompress) { Some(CompressionCodec.createCodec(sparkConf)) @@ -259,13 +259,13 @@ private[spark] object EventLoggingListener extends Logging { * @return A path which consists of file-system-safe characters. */ def getLogPath( - logBaseDir: String, + logBaseDir: URI, appId: String, compressionCodecName: Option[String] = None): String = { val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase // e.g. app_123, app_123.lzf val logName = sanitizedAppId + compressionCodecName.map { "." + _ }.getOrElse("") - Utils.resolveURI(logBaseDir).toString.stripSuffix("/") + "/" + logName + logBaseDir.toString.stripSuffix("/") + "/" + logName } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index e908ba604ebed..fcae603c7d18e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -50,7 +50,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers inProgress: Boolean, codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId) + val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId) val logPath = new URI(logUri).getPath + ip new File(logPath) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 448258a754153..30ee63e78d9d8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -61,7 +61,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Verify log file exist") { // Verify logging directory exists val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", testDirPath.toUri().toString(), conf) + val eventLogger = new EventLoggingListener("test", testDirPath.toUri(), conf) eventLogger.start() val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) @@ -95,7 +95,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef } test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, "test") + val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test") val logPath = new URI(logUri).getPath // Create file before writing the event log new FileOutputStream(new File(logPath)).close() @@ -107,16 +107,19 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Event log name") { // without compression - assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath("/base-dir", "app1")) + assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath( + Utils.resolveURI("/base-dir"), "app1")) // with compression assert(s"file:/base-dir/app1.lzf" === - EventLoggingListener.getLogPath("/base-dir", "app1", Some("lzf"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", Some("lzf"))) // illegal characters in app ID assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === - EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1")) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + "a fine:mind$dollar{bills}.1")) // illegal characters in app ID with compression assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === - EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1", Some("lz4"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + "a fine:mind$dollar{bills}.1", Some("lz4"))) } /* ----------------- * @@ -137,7 +140,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val conf = getLoggingConf(testDirPath, compressionCodec) extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") - val eventLogger = new EventLoggingListener(logName, testDirPath.toUri().toString(), conf) + val eventLogger = new EventLoggingListener(logName, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -173,12 +176,15 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef * This runs a simple Spark job and asserts that the expected events are logged when expected. */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { + // Set defaultFS to something that would cause an exception, to make sure we don't run + // into SPARK-6688. val conf = getLoggingConf(testDirPath, compressionCodec) + .set("spark.hadoop.fs.defaultFS", "unsupported://example.com") val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val eventLogPath = eventLogger.logPath - val expectedLogDir = testDir.toURI().toString() + val expectedLogDir = testDir.toURI() assert(eventLogPath === EventLoggingListener.getLogPath( expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 601694f57aad0..6de6d2fec622a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, PrintWriter} +import java.net.URI import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -145,7 +146,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * log the events. */ private class EventMonster(conf: SparkConf) - extends EventLoggingListener("test", "testdir", conf) { + extends EventLoggingListener("test", new URI("testdir"), conf) { override def start() { } From 26b415e15970d02523f0df459557b09ffda0c8c1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 3 Apr 2015 12:35:00 -0700 Subject: [PATCH 644/817] [SPARK-6647][SQL] Make trait StringComparison as BinaryPredicate and fix unit tests of string data source Filter Now trait `StringComparison` is a `BinaryExpression`. In fact, it should be a `BinaryPredicate`. By making `StringComparison` as `BinaryPredicate`, we can throw error when a `expressions.Predicate` can't translate to a data source `Filter` in function `selectFilters`. Without this modification, because we will wrap a `Filter` outside the scanned results in `pruneFilterProjectRaw`, we can't detect about something is wrong in translating predicates to filters in `selectFilters`. The unit test of #5285 demonstrates such problem. In that pr, even `expressions.Contains` is not properly translated to `sources.StringContains`, the filtering is still performed by the `Filter` and so the test passes. Of course, by doing this modification, all `expressions.Predicate` classes need to have its data source `Filter` correspondingly. There is a small bug in `FilteredScanSuite` for doing `StringEndsWith` filter. This pr also fixes it. Author: Liang-Chi Hsieh Closes #5309 from viirya/translate_predicate and squashes the following commits: b176385 [Liang-Chi Hsieh] Address comment. 275a493 [Liang-Chi Hsieh] More properly test for StringStartsWith, StringEndsWith and StringContains. caf2347 [Liang-Chi Hsieh] Make trait StringComparison as BinaryPredicate and throw error when Predicate can't translate to data source Filter. --- .../expressions/stringOperations.scala | 11 ++++---- .../spark/sql/sources/FilteredScanSuite.scala | 28 +++++++++++++------ 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 3cdca4e9dd2d1..acfbbace608ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -156,12 +156,11 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE /** A base trait for functions that compare two strings, returning a boolean. */ trait StringComparison { - self: BinaryExpression => + self: BinaryPredicate => - type EvaluatedType = Any + override type EvaluatedType = Any override def nullable: Boolean = left.nullable || right.nullable - override def dataType: DataType = BooleanType def compare(l: String, r: String): Boolean @@ -184,7 +183,7 @@ trait StringComparison { * A function that returns true if the string `left` contains the string `right`. */ case class Contains(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { + extends BinaryPredicate with StringComparison { override def compare(l: String, r: String): Boolean = l.contains(r) } @@ -192,7 +191,7 @@ case class Contains(left: Expression, right: Expression) * A function that returns true if the string `left` starts with the string `right`. */ case class StartsWith(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { + extends BinaryPredicate with StringComparison { override def compare(l: String, r: String): Boolean = l.startsWith(r) } @@ -200,7 +199,7 @@ case class StartsWith(left: Expression, right: Expression) * A function that returns true if the string `left` ends with the string `right`. */ case class EndsWith(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { + extends BinaryPredicate with StringComparison { override def compare(l: String, r: String): Boolean = l.endsWith(r) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 72ddc0ea2c8cb..773bd1602d5e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -45,7 +45,9 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) - case "c" => (i: Int) => Seq((i - 1 + 'a').toChar.toString * 10) + case "c" => (i: Int) => + val c = (i - 1 + 'a').toChar.toString + Seq(c * 5 + c.toUpperCase() * 5) } FiltersPushed.list = filters @@ -77,7 +79,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL } def eval(a: Int) = { - val c = (a - 1 + 'a').toChar.toString * 10 + val c = (a - 1 + 'a').toChar.toString * 5 + (a - 1 + 'a').toChar.toString.toUpperCase() * 5 !filters.map(translateFilterOnA(_)(a)).contains(false) && !filters.map(translateFilterOnC(_)(c)).contains(false) } @@ -110,7 +112,8 @@ class FilteredScanSuite extends DataSourceTest { sqlTest( "SELECT * FROM oneToTenFiltered", - (1 to 10).map(i => Row(i, i * 2, (i - 1 + 'a').toChar.toString * 10)).toSeq) + (1 to 10).map(i => Row(i, i * 2, (i - 1 + 'a').toChar.toString * 5 + + (i - 1 + 'a').toChar.toString.toUpperCase() * 5)).toSeq) sqlTest( "SELECT a, b FROM oneToTenFiltered", @@ -182,15 +185,15 @@ class FilteredScanSuite extends DataSourceTest { sqlTest( "SELECT a, b, c FROM oneToTenFiltered WHERE c like 'c%'", - Seq(Row(3, 3 * 2, "c" * 10))) + Seq(Row(3, 3 * 2, "c" * 5 + "C" * 5))) sqlTest( - "SELECT a, b, c FROM oneToTenFiltered WHERE c like 'd%'", - Seq(Row(4, 4 * 2, "d" * 10))) + "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%D'", + Seq(Row(4, 4 * 2, "d" * 5 + "D" * 5))) sqlTest( - "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%e%'", - Seq(Row(5, 5 * 2, "e" * 10))) + "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%eE%'", + Seq(Row(5, 5 * 2, "e" * 5 + "E" * 5))) testPushDown("SELECT * FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT a FROM oneToTenFiltered WHERE A = 1", 1) @@ -222,6 +225,15 @@ class FilteredScanSuite extends DataSourceTest { testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", 4) testPushDown("SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", 5) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like 'c%'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like 'C%'", 0) + + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%D'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%d'", 0) + + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%eE%'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%Ee%'", 0) + def testPushDown(sqlString: String, expectedCount: Int): Unit = { test(s"PushDown Returns $expectedCount: $sqlString") { val queryExecution = sql(sqlString).queryExecution From 9b40c17ab161b64933539abeefde443cb4f98673 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 3 Apr 2015 15:22:21 -0700 Subject: [PATCH 645/817] [SPARK-6700] disable flaky test Author: Davies Liu Closes #5356 from davies/flaky and squashes the following commits: 08955f4 [Davies Liu] disable flaky test --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 0e37276ba724b..c06c0105670c0 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -143,7 +143,8 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } } - test("run Python application in yarn-cluster mode") { + // Enable this once fix SPARK-6700 + ignore("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) val pyFile = new File(tempDir, "test2.py") From da25c86d64ff9ce80f88186ba083f6c21dd9a568 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 4 Apr 2015 23:26:10 +0800 Subject: [PATCH 646/817] [SQL] Use path.makeQualified in newParquet. Author: Yin Huai Closes #5353 from yhuai/wrongFS and squashes the following commits: 849603b [Yin Huai] Not use deprecated method. 6d6ae34 [Yin Huai] Use path.makeQualified. --- .../main/scala/org/apache/spark/sql/parquet/newParquet.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 583bac42fdcce..0dce3623a66df 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 @@ -268,7 +268,8 @@ private[sql] case class ParquetRelation2( // containing Parquet files (e.g. partitioned Parquet table). val baseStatuses = paths.distinct.map { p => val fs = FileSystem.get(URI.create(p), sparkContext.hadoopConfiguration) - val qualified = fs.makeQualified(new Path(p)) + val path = new Path(p) + val qualified = path.makeQualified(fs.getUri, fs.getWorkingDirectory) if (!fs.exists(qualified) && maybeSchema.isDefined) { fs.mkdirs(qualified) From 7bca62f79056e592cf07b49d8b8d04c59dea25fc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 5 Apr 2015 00:20:43 +0800 Subject: [PATCH 647/817] [SPARK-6607][SQL] Check invalid characters for Parquet schema and show error messages '(' and ')' are special characters used in Parquet schema for type annotation. When we run an aggregation query, we will obtain attribute name such as "MAX(a)". If we directly store the generated DataFrame as Parquet file, it causes failure when reading and parsing the stored schema string. Several methods can be adopted to solve this. This pr uses a simplest one to just replace attribute names before generating Parquet schema based on these attributes. Another possible method might be modifying all aggregation expression names from "func(column)" to "func[column]". Author: Liang-Chi Hsieh Closes #5263 from viirya/parquet_aggregation_name and squashes the following commits: 2d70542 [Liang-Chi Hsieh] Address comment. 463dff4 [Liang-Chi Hsieh] Instead of replacing special chars, showing error message to user to suggest using Alias. 1de001d [Liang-Chi Hsieh] Replace special characters '(' and ')' of Parquet schema. --- .../apache/spark/sql/parquet/ParquetTypes.scala | 14 ++++++++++++++ .../apache/spark/sql/hive/parquetSuites.scala | 16 ++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index da668f068613b..60e1bec4db8e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -390,6 +390,7 @@ private[parquet] object ParquetTypesConverter extends Logging { def convertFromAttributes(attributes: Seq[Attribute], toThriftSchemaNames: Boolean = false): MessageType = { + checkSpecialCharacters(attributes) val fields = attributes.map( attribute => fromDataType(attribute.dataType, attribute.name, attribute.nullable, @@ -404,7 +405,20 @@ private[parquet] object ParquetTypesConverter extends Logging { } } + private def checkSpecialCharacters(schema: Seq[Attribute]) = { + // ,;{}()\n\t= and space character are special characters in Parquet schema + schema.map(_.name).foreach { name => + if (name.matches(".*[ ,;{}()\n\t=].*")) { + sys.error( + s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\n\t=". + |Please use alias to rename it. + """.stripMargin.split("\n").mkString(" ")) + } + } + } + def convertToString(schema: Seq[Attribute]): String = { + checkSpecialCharacters(schema) StructType.fromAttributes(schema).json } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 1319c81dfc131..5f71e1bbc2d2e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -688,6 +688,22 @@ class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { sql("DROP TABLE alwaysNullable") } + + test("Aggregation attribute names can't contain special chars \" ,;{}()\\n\\t=\"") { + val tempDir = Utils.createTempDir() + val filePath = new File(tempDir, "testParquet").getCanonicalPath + val filePath2 = new File(tempDir, "testParquet2").getCanonicalPath + + val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").max("y.int") + intercept[RuntimeException](df2.saveAsParquetFile(filePath)) + + val df3 = df2.toDF("str", "max_int") + df3.saveAsParquetFile(filePath2) + val df4 = parquetFile(filePath2) + checkAnswer(df4, Row("1", 1) :: Row("2", 2) :: Row("3", 3) :: Nil) + assert(df4.columns === Array("str", "max_int")) + } } class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { From f15806a8f8ca34288ddb2d74b9ff1972c8374b59 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sat, 4 Apr 2015 11:52:05 -0700 Subject: [PATCH 648/817] [SPARK-6602][Core] Replace direct use of Akka with Spark RPC interface - part 1 This PR replaced the following `Actor`s to `RpcEndpoint`: 1. HeartbeatReceiver 1. ExecutorActor 1. BlockManagerMasterActor 1. BlockManagerSlaveActor 1. CoarseGrainedExecutorBackend and subclasses 1. CoarseGrainedSchedulerBackend.DriverActor This is the first PR. I will split the work of SPARK-6602 to several PRs for code review. Author: zsxwing Closes #5268 from zsxwing/rpc-rewrite and squashes the following commits: 287e9f8 [zsxwing] Fix the code style 26c56b7 [zsxwing] Merge branch 'master' into rpc-rewrite 9cc825a [zsxwing] Rmove setupThreadSafeEndpoint and add ThreadSafeRpcEndpoint 30a9036 [zsxwing] Make self return null after stopping RpcEndpointRef; fix docs and error messages 705245d [zsxwing] Fix some bugs after rebasing the changes on the master 003cf80 [zsxwing] Update CoarseGrainedExecutorBackend and CoarseGrainedSchedulerBackend to use RpcEndpoint 7d0e6dc [zsxwing] Update BlockManagerSlaveActor to use RpcEndpoint f5d6543 [zsxwing] Update BlockManagerMaster to use RpcEndpoint 30e3f9f [zsxwing] Update ExecutorActor to use RpcEndpoint 478b443 [zsxwing] Update HeartbeatReceiver to use RpcEndpoint --- .../org/apache/spark/HeartbeatReceiver.scala | 66 +++++--- .../scala/org/apache/spark/SparkContext.scala | 23 +-- .../scala/org/apache/spark/SparkEnv.scala | 13 +- .../CoarseGrainedExecutorBackend.scala | 79 +++++----- .../org/apache/spark/executor/Executor.scala | 18 +-- ...utorActor.scala => ExecutorEndpoint.scala} | 18 ++- .../scala/org/apache/spark/rpc/RpcEnv.scala | 39 +++-- .../apache/spark/rpc/akka/AkkaRpcEnv.scala | 10 +- .../apache/spark/scheduler/DAGScheduler.scala | 11 +- .../cluster/CoarseGrainedClusterMessage.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 148 +++++++++--------- .../scheduler/cluster/ExecutorData.scala | 8 +- .../cluster/SimrSchedulerBackend.scala | 13 +- .../cluster/SparkDeploySchedulerBackend.scala | 14 +- .../cluster/YarnSchedulerBackend.scala | 93 +++++------ .../mesos/CoarseMesosSchedulerBackend.scala | 4 +- .../spark/scheduler/local/LocalBackend.scala | 48 +++--- .../apache/spark/storage/BlockManager.scala | 22 +-- .../spark/storage/BlockManagerMaster.scala | 72 ++++----- ...scala => BlockManagerMasterEndpoint.scala} | 119 +++++++------- .../spark/storage/BlockManagerMessages.scala | 7 +- ....scala => BlockManagerSlaveEndpoint.scala} | 44 +++--- .../scala/org/apache/spark/util/Utils.scala | 10 ++ .../apache/spark/HeartbeatReceiverSuite.scala | 81 ++++++++++ .../org/apache/spark/rpc/RpcEnvSuite.scala | 14 +- .../BlockManagerReplicationSuite.scala | 28 ++-- .../spark/storage/BlockManagerSuite.scala | 37 ++--- .../streaming/ReceivedBlockHandlerSuite.scala | 25 ++- .../spark/deploy/yarn/ApplicationMaster.scala | 86 +++++----- .../spark/deploy/yarn/YarnAllocator.scala | 2 +- 30 files changed, 616 insertions(+), 542 deletions(-) rename core/src/main/scala/org/apache/spark/executor/{ExecutorActor.scala => ExecutorEndpoint.scala} (67%) rename core/src/main/scala/org/apache/spark/storage/{BlockManagerMasterActor.scala => BlockManagerMasterEndpoint.scala} (83%) rename core/src/main/scala/org/apache/spark/storage/{BlockManagerSlaveActor.scala => BlockManagerSlaveEndpoint.scala} (61%) create mode 100644 core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 9f8ad03b91e85..5871b8c869f03 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -17,15 +17,15 @@ package org.apache.spark -import scala.concurrent.duration._ -import scala.collection.mutable +import java.util.concurrent.{ScheduledFuture, TimeUnit, Executors} -import akka.actor.{Actor, Cancellable} +import scala.collection.mutable import org.apache.spark.executor.TaskMetrics +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext} import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.util.Utils /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -51,9 +51,11 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) * Lives in the driver to receive heartbeats from executors.. */ private[spark] class HeartbeatReceiver(sc: SparkContext) - extends Actor with ActorLogReceive with Logging { + extends ThreadSafeRpcEndpoint with Logging { + + override val rpcEnv: RpcEnv = sc.env.rpcEnv - private var scheduler: TaskScheduler = null + private[spark] var scheduler: TaskScheduler = null // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] @@ -69,34 +71,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) - private var timeoutCheckingTask: Cancellable = null - - override def preStart(): Unit = { - import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, - checkTimeoutIntervalMs.milliseconds, self, ExpireDeadHosts) - super.preStart() + private var timeoutCheckingTask: ScheduledFuture[_] = null + + private val timeoutCheckingThread = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("heartbeat-timeout-checking-thread")) + + private val killExecutorThread = Executors.newSingleThreadExecutor( + Utils.namedThreadFactory("kill-executor-thread")) + + override def onStart(): Unit = { + timeoutCheckingTask = timeoutCheckingThread.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ExpireDeadHosts)) + } + }, 0, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS) } - - override def receiveWithLogging: PartialFunction[Any, Unit] = { + + override def receive: PartialFunction[Any, Unit] = { + case ExpireDeadHosts => + expireDeadHosts() case TaskSchedulerIsSet => scheduler = sc.taskScheduler + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case heartbeat @ Heartbeat(executorId, taskMetrics, blockManagerId) => if (scheduler != null) { val unknownExecutor = !scheduler.executorHeartbeatReceived( executorId, taskMetrics, blockManagerId) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) executorLastSeen(executorId) = System.currentTimeMillis() - sender ! response + context.reply(response) } else { // Because Executor will sleep several seconds before sending the first "Heartbeat", this // case rarely happens. However, if it really happens, log it and ask the executor to // register itself again. logWarning(s"Dropping $heartbeat because TaskScheduler is not ready yet") - sender ! HeartbeatResponse(reregisterBlockManager = true) + context.reply(HeartbeatResponse(reregisterBlockManager = true)) } - case ExpireDeadHosts => - expireDeadHosts() } private def expireDeadHosts(): Unit = { @@ -109,17 +121,25 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) scheduler.executorLost(executorId, SlaveLost("Executor heartbeat " + s"timed out after ${now - lastSeenMs} ms")) if (sc.supportDynamicAllocation) { - sc.killExecutor(executorId) + // Asynchronously kill the executor to avoid blocking the current thread + killExecutorThread.submit(new Runnable { + override def run(): Unit = sc.killExecutor(executorId) + }) } executorLastSeen.remove(executorId) } } } - override def postStop(): Unit = { + override def onStop(): Unit = { if (timeoutCheckingTask != null) { - timeoutCheckingTask.cancel() + timeoutCheckingTask.cancel(true) } - super.postStop() + timeoutCheckingThread.shutdownNow() + killExecutorThread.shutdownNow() } } + +object HeartbeatReceiver { + val ENDPOINT_NAME = "HeartbeatReceiver" +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3b73a8a8fd850..942c5975ece6d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -32,8 +32,6 @@ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} -import akka.actor.Props - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, @@ -48,12 +46,13 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} -import org.apache.spark.executor.TriggerThreadDump +import org.apache.spark.executor.{ExecutorEndpoint, TriggerThreadDump} import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ +import org.apache.spark.rpc.RpcAddress import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} @@ -360,14 +359,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) - private val heartbeatReceiver = env.actorSystem.actorOf( - Props(new HeartbeatReceiver(this)), "HeartbeatReceiver") + private val heartbeatReceiver = env.rpcEnv.setupEndpoint( + HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) // Create and start the scheduler private[spark] var (schedulerBackend, taskScheduler) = SparkContext.createTaskScheduler(this, master) - heartbeatReceiver ! TaskSchedulerIsSet + heartbeatReceiver.send(TaskSchedulerIsSet) @volatile private[spark] var dagScheduler: DAGScheduler = _ try { @@ -455,10 +454,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (executorId == SparkContext.DRIVER_IDENTIFIER) { Some(Utils.getThreadDump()) } else { - val (host, port) = env.blockManager.master.getActorSystemHostPortForExecutor(executorId).get - val actorRef = AkkaUtils.makeExecutorRef("ExecutorActor", conf, host, port, env.actorSystem) - Some(AkkaUtils.askWithReply[Array[ThreadStackTrace]](TriggerThreadDump, actorRef, - AkkaUtils.numRetries(conf), AkkaUtils.retryWaitMs(conf), AkkaUtils.askTimeout(conf))) + val (host, port) = env.blockManager.master.getRpcHostPortForExecutor(executorId).get + val endpointRef = env.rpcEnv.setupEndpointRef( + SparkEnv.executorActorSystemName, + RpcAddress(host, port), + ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME) + Some(endpointRef.askWithReply[Array[ThreadStackTrace]](TriggerThreadDump)) } } catch { case e: Exception => @@ -1418,7 +1419,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli dagScheduler = null listenerBus.stop() eventLogger.foreach(_.stop()) - env.actorSystem.stop(heartbeatReceiver) + env.rpcEnv.stop(heartbeatReceiver) progressBar.foreach(_.stop()) taskScheduler = null // TODO: Cache.stop()? diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 4a2ed82a40dec..55be0a59fedd9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -295,7 +295,9 @@ object SparkEnv extends Logging { } } - def registerOrLookupEndpoint(name: String, endpointCreator: => RpcEndpoint): RpcEndpointRef = { + def registerOrLookupEndpoint( + name: String, endpointCreator: => RpcEndpoint): + RpcEndpointRef = { if (isDriver) { logInfo("Registering " + name) rpcEnv.setupEndpoint(name, endpointCreator) @@ -334,12 +336,13 @@ object SparkEnv extends Logging { new NioBlockTransferService(conf, securityManager) } - val blockManagerMaster = new BlockManagerMaster(registerOrLookup( - "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) + val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( + BlockManagerMaster.DRIVER_ENDPOINT_NAME, + new BlockManagerMasterEndpoint(rpcEnv, isLocal, conf, listenerBus)), + conf, isDriver) // NB: blockManager is not valid until initialize() is called later. - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + val blockManager = new BlockManager(executorId, rpcEnv, blockManagerMaster, serializer, conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager, numUsableCores) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 900e678ee02ef..8300f9f2190b9 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,39 +21,45 @@ import java.net.URL import java.nio.ByteBuffer import scala.collection.mutable -import scala.concurrent.Await +import scala.util.{Failure, Success} -import akka.actor.{Actor, ActorSelection, Props} -import akka.pattern.Patterns -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.rpc._ +import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( + override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, hostPort: String, cores: Int, userClassPath: Seq[URL], env: SparkEnv) - extends Actor with ActorLogReceive with ExecutorBackend with Logging { + extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") var executor: Executor = null - var driver: ActorSelection = null + @volatile var driver: Option[RpcEndpointRef] = None - override def preStart() { + override def onStart() { + import scala.concurrent.ExecutionContext.Implicits.global logInfo("Connecting to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - driver ! RegisterExecutor(executorId, hostPort, cores, extractLogUrls) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => + driver = Some(ref) + ref.sendWithReply[RegisteredExecutor.type]( + RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls)) + } onComplete { + case Success(msg) => Utils.tryLogNonFatalError { + Option(self).foreach(_.send(msg)) // msg must be RegisteredExecutor + } + case Failure(e) => logError(s"Cannot register with driver: $driverUrl", e) + } } def extractLogUrls: Map[String, String] = { @@ -62,7 +68,7 @@ private[spark] class CoarseGrainedExecutorBackend( .map(e => (e._1.substring(prefix.length).toLowerCase, e._2)) } - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receive: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) @@ -92,23 +98,28 @@ private[spark] class CoarseGrainedExecutorBackend( executor.killTask(taskId, interruptThread) } - case x: DisassociatedEvent => - if (x.remoteAddress == driver.anchorPath.address) { - logError(s"Driver $x disassociated! Shutting down.") - System.exit(1) - } else { - logWarning(s"Received irrelevant DisassociatedEvent $x") - } - case StopExecutor => logInfo("Driver commanded a shutdown") executor.stop() - context.stop(self) - context.system.shutdown() + stop() + rpcEnv.shutdown() + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (driver.exists(_.address == remoteAddress)) { + logError(s"Driver $remoteAddress disassociated! Shutting down.") + System.exit(1) + } else { + logWarning(s"An unknown ($remoteAddress) driver disconnected.") + } } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - driver ! StatusUpdate(executorId, taskId, state, data) + val msg = StatusUpdate(executorId, taskId, state, data) + driver match { + case Some(driverRef) => driverRef.send(msg) + case None => logWarning(s"Drop $msg because has not yet connected to driver") + } } } @@ -132,16 +143,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Bootstrap to fetch the driver's Spark properties. val executorConf = new SparkConf val port = executorConf.getInt("spark.executor.port", 0) - val (fetcher, _) = AkkaUtils.createActorSystem( + val fetcher = RpcEnv.create( "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf)) - val driver = fetcher.actorSelection(driverUrl) - val timeout = AkkaUtils.askTimeout(executorConf) - val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) - val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] ++ + val driver = fetcher.setupEndpointRefByURI(driverUrl) + val props = driver.askWithReply[Seq[(String, String)]](RetrieveSparkProps) ++ Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() @@ -162,16 +171,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val boundPort = env.conf.getInt("spark.executor.port", 0) assert(boundPort != 0) - // Start the CoarseGrainedExecutorBackend actor. + // Start the CoarseGrainedExecutorBackend endpoint. val sparkHostPort = hostname + ":" + boundPort - env.actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, userClassPath, env), - name = "Executor") + env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend( + env.rpcEnv, driverUrl, executorId, sparkHostPort, cores, userClassPath, env)) workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } - env.actorSystem.awaitTermination() + env.rpcEnv.awaitTermination() } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bf3135ef081c1..14f99a464b6e9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -27,8 +27,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import akka.actor.Props - import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} @@ -88,9 +86,9 @@ private[spark] class Executor( env.blockManager.initialize(conf.getAppId) } - // Create an actor for receiving RPCs from the driver - private val executorActor = env.actorSystem.actorOf( - Props(new ExecutorActor(executorId)), "ExecutorActor") + // Create an RpcEndpoint for receiving RPCs from the driver + private val executorEndpoint = env.rpcEnv.setupEndpoint( + ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) // Whether to load classes in user jars before those in Spark jars private val userClassPathFirst: Boolean = { @@ -139,7 +137,7 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - env.actorSystem.stop(executorActor) + env.rpcEnv.stop(executorEndpoint) isStopped = true threadPool.shutdown() if (!isLocal) { @@ -391,11 +389,8 @@ private[spark] class Executor( } } - private val timeout = AkkaUtils.lookupTimeout(conf) - private val retryAttempts = AkkaUtils.numRetries(conf) - private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) private val heartbeatReceiverRef = - AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + RpcUtils.makeDriverRef(HeartbeatReceiver.ENDPOINT_NAME, conf, env.rpcEnv) /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { @@ -426,8 +421,7 @@ private[spark] class Executor( val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) try { - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) + val response = heartbeatReceiverRef.askWithReply[HeartbeatResponse](message) if (response.reregisterBlockManager) { logWarning("Told to re-register on heartbeat") env.blockManager.reregister() diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala index 3e47d13f7545d..cf362f8464735 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala @@ -17,10 +17,8 @@ package org.apache.spark.executor -import akka.actor.Actor -import org.apache.spark.Logging - -import org.apache.spark.util.{Utils, ActorLogReceive} +import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} +import org.apache.spark.util.Utils /** * Driver -> Executor message to trigger a thread dump. @@ -28,14 +26,18 @@ import org.apache.spark.util.{Utils, ActorLogReceive} private[spark] case object TriggerThreadDump /** - * Actor that runs inside of executors to enable driver -> executor RPC. + * [[RpcEndpoint]] that runs inside of executors to enable driver -> executor RPC. */ private[spark] -class ExecutorActor(executorId: String) extends Actor with ActorLogReceive with Logging { +class ExecutorEndpoint(override val rpcEnv: RpcEnv, executorId: String) extends RpcEndpoint { - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case TriggerThreadDump => - sender ! Utils.getThreadDump() + context.reply(Utils.getThreadDump()) } } + +object ExecutorEndpoint { + val EXECUTOR_ENDPOINT_NAME = "ExecutorEndpoint" +} diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index 7985941d949c0..d47e41abcfa50 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -40,10 +40,7 @@ private[spark] abstract class RpcEnv(conf: SparkConf) { /** * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement - * [[RpcEndpoint.self]]. - * - * Note: This method won't return null. `IllegalArgumentException` will be thrown if calling this - * on a non-existent endpoint. + * [[RpcEndpoint.self]]. Return `null` if the corresponding [[RpcEndpointRef]] does not exist. */ private[rpc] def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef @@ -58,20 +55,6 @@ private[spark] abstract class RpcEnv(conf: SparkConf) { */ def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef - /** - * Register a [[RpcEndpoint]] with a name and return its [[RpcEndpointRef]]. [[RpcEnv]] should - * make sure thread-safely sending messages to [[RpcEndpoint]]. - * - * Thread-safety means processing of one message happens before processing of the next message by - * the same [[RpcEndpoint]]. In the other words, changes to internal fields of a [[RpcEndpoint]] - * are visible when processing the next message, and fields in the [[RpcEndpoint]] need not be - * volatile or equivalent. - * - * However, there is no guarantee that the same thread will be executing the same [[RpcEndpoint]] - * for different messages. - */ - def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef - /** * Retrieve the [[RpcEndpointRef]] represented by `uri` asynchronously. */ @@ -181,7 +164,7 @@ private[spark] trait RpcEnvFactory { * constructor onStart receive* onStop * * Note: `receive` can be called concurrently. If you want `receive` is thread-safe, please use - * [[RpcEnv.setupThreadSafeEndpoint]] + * [[ThreadSafeRpcEndpoint]] * * If any error is thrown from one of [[RpcEndpoint]] methods except `onError`, `onError` will be * invoked with the cause. If `onError` throws an error, [[RpcEnv]] will ignore it. @@ -195,7 +178,7 @@ private[spark] trait RpcEndpoint { /** * The [[RpcEndpointRef]] of this [[RpcEndpoint]]. `self` will become valid when `onStart` is - * called. + * called. And `self` will become `null` when `onStop` is called. * * Note: Because before `onStart`, [[RpcEndpoint]] has not yet been registered and there is not * valid [[RpcEndpointRef]] for it. So don't call `self` before `onStart` is called. @@ -278,6 +261,19 @@ private[spark] trait RpcEndpoint { } } +/** + * A trait that requires RpcEnv thread-safely sending messages to it. + * + * Thread-safety means processing of one message happens before processing of the next message by + * the same [[ThreadSafeRpcEndpoint]]. In the other words, changes to internal fields of a + * [[ThreadSafeRpcEndpoint]] are visible when processing the next message, and fields in the + * [[ThreadSafeRpcEndpoint]] need not be volatile or equivalent. + * + * However, there is no guarantee that the same thread will be executing the same + * [[ThreadSafeRpcEndpoint]] for different messages. + */ +trait ThreadSafeRpcEndpoint extends RpcEndpoint + /** * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe. */ @@ -407,7 +403,8 @@ private[spark] object RpcAddress { } /** - * A callback that [[RpcEndpoint]] can use it to send back a message or failure. + * A callback that [[RpcEndpoint]] can use it to send back a message or failure. It's thread-safe + * and can be called in any thread. */ private[spark] trait RpcCallContext { diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala index 769d59b7b3343..9e06147dff1ed 100644 --- a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala @@ -82,17 +82,9 @@ private[spark] class AkkaRpcEnv private[akka] ( /** * Retrieve the [[RpcEndpointRef]] of `endpoint`. */ - override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { - val endpointRef = endpointToRef.get(endpoint) - require(endpointRef != null, s"Cannot find RpcEndpointRef of ${endpoint} in ${this}") - endpointRef - } + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = endpointToRef.get(endpoint) override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { - setupThreadSafeEndpoint(name, endpoint) - } - - override def setupThreadSafeEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { @volatile var endpointRef: AkkaRpcEndpointRef = null // Use lazy because the Actor needs to use `endpointRef`. // So `actorRef` should be created after assigning `endpointRef`. diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7227fa9da4317..917cce1f9686c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -23,14 +23,10 @@ import java.util.concurrent.{TimeUnit, Executors} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.control.NonFatal -import akka.pattern.ask -import akka.util.Timeout - import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.TaskMetrics @@ -165,11 +161,8 @@ class DAGScheduler( taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) blockManagerId: BlockManagerId): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) - implicit val timeout = Timeout(600 seconds) - - Await.result( - blockManagerMaster.driverActor ? BlockManagerHeartbeat(blockManagerId), - timeout.duration).asInstanceOf[Boolean] + blockManagerMaster.driverEndpoint.askWithReply[Boolean]( + BlockManagerHeartbeat(blockManagerId), 600 seconds) } // Called by TaskScheduler when an executor fails. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9bf74f4be198d..70364cea62a80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{SerializableBuffer, Utils} private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable @@ -41,6 +42,7 @@ private[spark] object CoarseGrainedClusterMessages { // Executors to driver case class RegisterExecutor( executorId: String, + executorRef: RpcEndpointRef, hostPort: String, cores: Int, logUrls: Map[String, String]) @@ -70,6 +72,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage + case class SetupDriver(driver: RpcEndpointRef) extends CoarseGrainedClusterMessage + // Exchanged between the driver and the AM in Yarn client mode case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String) extends CoarseGrainedClusterMessage @@ -77,7 +81,7 @@ private[spark] object CoarseGrainedClusterMessages { // Messages exchanged between the driver and the cluster manager for executor allocation // In Yarn mode, these are exchanged between the driver and the AM - case object RegisterClusterManager extends CoarseGrainedClusterMessage + case class RegisterClusterManager(am: RpcEndpointRef) extends CoarseGrainedClusterMessage // Request executors by specifying the new total number of executors desired // This includes executors already pending or running diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 5d258d9da4d1a..4c49da87af9dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -17,20 +17,16 @@ package org.apache.spark.scheduler.cluster +import java.util.concurrent.{TimeUnit, Executors} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.Await -import scala.concurrent.duration._ - -import akka.actor._ -import akka.pattern.ask -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.rpc._ import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -41,7 +37,7 @@ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Ut * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -49,7 +45,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Total number of executors that are currently registered var totalRegisteredExecutors = new AtomicInteger(0) val conf = scheduler.sc.conf - private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. @@ -71,48 +66,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] - class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { + class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) + extends ThreadSafeRpcEndpoint with Logging { override protected def log = CoarseGrainedSchedulerBackend.this.log - private val addressToExecutorId = new HashMap[Address, String] - override def preStart() { - // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + private val addressToExecutorId = new HashMap[RpcAddress, String] + + private val reviveThread = + Executors.newSingleThreadScheduledExecutor(Utils.namedThreadFactory("driver-revive-thread")) + override def onStart() { // Periodically revive offers to allow delay scheduling to work val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) - import context.dispatcher - context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) - } - - def receiveWithLogging: PartialFunction[Any, Unit] = { - case RegisterExecutor(executorId, hostPort, cores, logUrls) => - Utils.checkHostPort(hostPort, "Host port expected " + hostPort) - if (executorDataMap.contains(executorId)) { - sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) - } else { - logInfo("Registered executor: " + sender + " with ID " + executorId) - sender ! RegisteredExecutor - - addressToExecutorId(sender.path.address) = executorId - totalCoreCount.addAndGet(cores) - totalRegisteredExecutors.addAndGet(1) - val (host, _) = Utils.parseHostPort(hostPort) - val data = new ExecutorData(sender, sender.path.address, host, cores, cores, logUrls) - // This must be synchronized because variables mutated - // in this block are read when requesting executors - CoarseGrainedSchedulerBackend.this.synchronized { - executorDataMap.put(executorId, data) - if (numPendingExecutors > 0) { - numPendingExecutors -= 1 - logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") - } - } - listenerBus.post( - SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) - makeOffers() + reviveThread.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ReviveOffers)) } + }, 0, reviveInterval, TimeUnit.MILLISECONDS) + } + override def receive: PartialFunction[Any, Unit] = { case StatusUpdate(executorId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { @@ -133,33 +106,58 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste case KillTask(taskId, executorId, interruptThread) => executorDataMap.get(executorId) match { case Some(executorInfo) => - executorInfo.executorActor ! KillTask(taskId, executorId, interruptThread) + executorInfo.executorEndpoint.send(KillTask(taskId, executorId, interruptThread)) case None => // Ignoring the task kill since the executor is not registered. logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") } + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterExecutor(executorId, executorRef, hostPort, cores, logUrls) => + Utils.checkHostPort(hostPort, "Host port expected " + hostPort) + if (executorDataMap.contains(executorId)) { + context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + } else { + logInfo("Registered executor: " + executorRef + " with ID " + executorId) + context.reply(RegisteredExecutor) + + addressToExecutorId(executorRef.address) = executorId + totalCoreCount.addAndGet(cores) + totalRegisteredExecutors.addAndGet(1) + val (host, _) = Utils.parseHostPort(hostPort) + val data = new ExecutorData(executorRef, executorRef.address, host, cores, cores, logUrls) + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap.put(executorId, data) + if (numPendingExecutors > 0) { + numPendingExecutors -= 1 + logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") + } + } + listenerBus.post( + SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) + makeOffers() + } case StopDriver => - sender ! true - context.stop(self) + context.reply(true) + stop() case StopExecutors => logInfo("Asking each executor to shut down") for ((_, executorData) <- executorDataMap) { - executorData.executorActor ! StopExecutor + executorData.executorEndpoint.send(StopExecutor) } - sender ! true + context.reply(true) case RemoveExecutor(executorId, reason) => removeExecutor(executorId, reason) - sender ! true - - case DisassociatedEvent(_, address, _) => - addressToExecutorId.get(address).foreach(removeExecutor(_, - "remote Akka client disassociated")) + context.reply(true) case RetrieveSparkProps => - sender ! sparkProperties + context.reply(sparkProperties) } // Make fake resource offers on all executors @@ -169,6 +167,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste }.toSeq)) } + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, + "remote Rpc client disassociated")) + } + // Make fake resource offers on just one executor def makeOffers(executorId: String) { val executorData = executorDataMap(executorId) @@ -199,7 +202,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK - executorData.executorActor ! LaunchTask(new SerializableBuffer(serializedTask)) + executorData.executorEndpoint.send(LaunchTask(new SerializableBuffer(serializedTask))) } } } @@ -223,9 +226,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste case None => logError(s"Asked to remove non-existent executor $executorId") } } + + override def onStop() { + reviveThread.shutdownNow() + } } - var driverActor: ActorRef = null + var driverEndpoint: RpcEndpointRef = null val taskIdsOnSlave = new HashMap[String, HashSet[String]] override def start() { @@ -236,16 +243,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } } // TODO (prashant) send conf instead of properties - driverActor = actorSystem.actorOf( - Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) + driverEndpoint = rpcEnv.setupEndpoint( + CoarseGrainedSchedulerBackend.ENDPOINT_NAME, new DriverEndpoint(rpcEnv, properties)) } def stopExecutors() { try { - if (driverActor != null) { + if (driverEndpoint != null) { logInfo("Shutting down all executors") - val future = driverActor.ask(StopExecutors)(timeout) - Await.ready(future, timeout) + driverEndpoint.askWithReply[Boolean](StopExecutors) } } catch { case e: Exception => @@ -256,22 +262,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste override def stop() { stopExecutors() try { - if (driverActor != null) { - val future = driverActor.ask(StopDriver)(timeout) - Await.ready(future, timeout) + if (driverEndpoint != null) { + driverEndpoint.askWithReply[Boolean](StopDriver) } } catch { case e: Exception => - throw new SparkException("Error stopping standalone scheduler's driver actor", e) + throw new SparkException("Error stopping standalone scheduler's driver endpoint", e) } } override def reviveOffers() { - driverActor ! ReviveOffers + driverEndpoint.send(ReviveOffers) } override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - driverActor ! KillTask(taskId, executorId, interruptThread) + driverEndpoint.send(KillTask(taskId, executorId, interruptThread)) } override def defaultParallelism(): Int = { @@ -281,11 +286,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { try { - val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout) - Await.ready(future, timeout) + driverEndpoint.askWithReply[Boolean](RemoveExecutor(executorId, reason)) } catch { case e: Exception => - throw new SparkException("Error notifying standalone scheduler's driver actor", e) + throw new SparkException("Error notifying standalone scheduler's driver endpoint", e) } } @@ -391,5 +395,5 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } private[spark] object CoarseGrainedSchedulerBackend { - val ACTOR_NAME = "CoarseGrainedScheduler" + val ENDPOINT_NAME = "CoarseGrainedScheduler" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 5e571efe76720..26e72c0bff38d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,20 +17,20 @@ package org.apache.spark.scheduler.cluster -import akka.actor.{Address, ActorRef} +import org.apache.spark.rpc.{RpcEndpointRef, RpcAddress} /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. * - * @param executorActor The ActorRef representing this executor + * @param executorEndpoint The ActorRef representing this executor * @param executorAddress The network address of this executor * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor */ private[cluster] class ExecutorData( - val executorActor: ActorRef, - val executorAddress: Address, + val executorEndpoint: RpcEndpointRef, + val executorAddress: RpcAddress, override val executorHost: String, var freeCores: Int, override val totalCores: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 06786a59524e7..0324c9dab910b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -19,16 +19,16 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.spark.rpc.RpcAddress import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.AkkaUtils private[spark] class SimrSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, driverFilePath: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with Logging { val tmpPath = new Path(driverFilePath + "_tmp") @@ -39,12 +39,9 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), - SparkEnv.driverActorSystemName, - sc.conf.get("spark.driver.host"), - sc.conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, + RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) val fs = FileSystem.get(conf) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ffd4825705755..7eb3fdc19b5b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,17 +19,18 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.Semaphore +import org.apache.spark.rpc.RpcAddress import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String]) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with AppClientListener with Logging { @@ -48,12 +49,9 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), - SparkEnv.driverActorSystemName, - conf.get("spark.driver.host"), - conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, + RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val args = Seq( "--driver-url", driverUrl, "--executor-id", "{{EXECUTOR_ID}}", diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 5a38ad9f2b12c..f72566c370a6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -19,10 +19,8 @@ package org.apache.spark.scheduler.cluster import scala.concurrent.{Future, ExecutionContext} -import akka.actor.{Actor, ActorRef, Props} -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils @@ -37,7 +35,7 @@ import scala.util.control.NonFatal private[spark] abstract class YarnSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 @@ -45,10 +43,8 @@ private[spark] abstract class YarnSchedulerBackend( protected var totalExpectedExecutors = 0 - private val yarnSchedulerActor: ActorRef = - actorSystem.actorOf( - Props(new YarnSchedulerActor), - name = YarnSchedulerBackend.ACTOR_NAME) + private val yarnSchedulerEndpoint = rpcEnv.setupEndpoint( + YarnSchedulerBackend.ENDPOINT_NAME, new YarnSchedulerEndpoint(rpcEnv)) private implicit val askTimeout = AkkaUtils.askTimeout(sc.conf) @@ -57,16 +53,14 @@ private[spark] abstract class YarnSchedulerBackend( * This includes executors already pending or running. */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - AkkaUtils.askWithReply[Boolean]( - RequestExecutors(requestedTotal), yarnSchedulerActor, askTimeout) + yarnSchedulerEndpoint.askWithReply[Boolean](RequestExecutors(requestedTotal)) } /** * Request that the ApplicationMaster kill the specified executors. */ override def doKillExecutors(executorIds: Seq[String]): Boolean = { - AkkaUtils.askWithReply[Boolean]( - KillExecutors(executorIds), yarnSchedulerActor, askTimeout) + yarnSchedulerEndpoint.askWithReply[Boolean](KillExecutors(executorIds)) } override def sufficientResourcesRegistered(): Boolean = { @@ -96,64 +90,71 @@ private[spark] abstract class YarnSchedulerBackend( } /** - * An actor that communicates with the ApplicationMaster. + * An [[RpcEndpoint]] that communicates with the ApplicationMaster. */ - private class YarnSchedulerActor extends Actor { - private var amActor: Option[ActorRef] = None - - implicit val askAmActorExecutor = ExecutionContext.fromExecutor( - Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-executor")) + private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) + extends ThreadSafeRpcEndpoint with Logging { + private var amEndpoint: Option[RpcEndpointRef] = None - override def preStart(): Unit = { - // Listen for disassociation events - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } + private val askAmThreadPool = + Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") + implicit val askAmExecutor = ExecutionContext.fromExecutor(askAmThreadPool) override def receive: PartialFunction[Any, Unit] = { - case RegisterClusterManager => - logInfo(s"ApplicationMaster registered as $sender") - amActor = Some(sender) + case RegisterClusterManager(am) => + logInfo(s"ApplicationMaster registered as $am") + amEndpoint = Some(am) + + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) + + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case r: RequestExecutors => - amActor match { - case Some(actor) => - val driverActor = sender + amEndpoint match { + case Some(am) => Future { - driverActor ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + context.reply(am.askWithReply[Boolean](r)) } onFailure { - case NonFatal(e) => logError(s"Sending $r to AM was unsuccessful", e) + case NonFatal(e) => + logError(s"Sending $r to AM was unsuccessful", e) + context.sendFailure(e) } case None => logWarning("Attempted to request executors before the AM has registered!") - sender ! false + context.reply(false) } case k: KillExecutors => - amActor match { - case Some(actor) => - val driverActor = sender + amEndpoint match { + case Some(am) => Future { - driverActor ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + context.reply(am.askWithReply[Boolean](k)) } onFailure { - case NonFatal(e) => logError(s"Sending $k to AM was unsuccessful", e) + case NonFatal(e) => + logError(s"Sending $k to AM was unsuccessful", e) + context.sendFailure(e) } case None => logWarning("Attempted to kill executors before the AM has registered!") - sender ! false + context.reply(false) } - case AddWebUIFilter(filterName, filterParams, proxyBase) => - addWebUIFilter(filterName, filterParams, proxyBase) - sender ! true + } - case d: DisassociatedEvent => - if (amActor.isDefined && sender == amActor.get) { - logWarning(s"ApplicationMaster has disassociated: $d") - } + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (amEndpoint.exists(_.address == remoteAddress)) { + logWarning(s"ApplicationMaster has disassociated: $remoteAddress") + } + } + + override def onStop(): Unit ={ + askAmThreadPool.shutdownNow() } } } private[spark] object YarnSchedulerBackend { - val ACTOR_NAME = "YarnScheduler" + val ENDPOINT_NAME = "YarnScheduler" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index e13de0f46ef89..b037a4966ced0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -47,7 +47,7 @@ private[spark] class CoarseMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with MScheduler with Logging { @@ -148,7 +148,7 @@ private[spark] class CoarseMesosSchedulerBackend( SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val uri = conf.get("spark.executor.uri", null) if (uri == null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index eb3f999b5b375..70a477a6895cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -18,17 +18,14 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer +import java.util.concurrent.{Executors, TimeUnit} -import scala.concurrent.duration._ -import scala.language.postfixOps - -import akka.actor.{Actor, ActorRef, Props} - +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.util.Utils import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.util.ActorLogReceive private case class ReviveOffers() @@ -39,17 +36,19 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private case class StopExecutor() /** - * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on - * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend + * Calls to LocalBackend are all serialized through LocalEndpoint. Using an RpcEndpoint makes the + * calls on LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend * and the TaskSchedulerImpl. */ -private[spark] class LocalActor( +private[spark] class LocalEndpoint( + override val rpcEnv: RpcEnv, scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, private val totalCores: Int) - extends Actor with ActorLogReceive with Logging { + extends ThreadSafeRpcEndpoint with Logging { - import context.dispatcher // to use Akka's scheduler.scheduleOnce() + private val reviveThread = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("local-revive-thread")) private var freeCores = totalCores @@ -59,7 +58,7 @@ private[spark] class LocalActor( private val executor = new Executor( localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receive: PartialFunction[Any, Unit] = { case ReviveOffers => reviveOffers() @@ -87,9 +86,17 @@ private[spark] class LocalActor( } if (tasks.isEmpty && scheduler.activeTaskSets.nonEmpty) { // Try to reviveOffer after 1 second, because scheduler may wait for locality timeout - context.system.scheduler.scheduleOnce(1000 millis, self, ReviveOffers) + reviveThread.schedule(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ReviveOffers)) + } + }, 1000, TimeUnit.MILLISECONDS) } } + + override def onStop(): Unit = { + reviveThread.shutdownNow() + } } /** @@ -101,31 +108,30 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: extends SchedulerBackend with ExecutorBackend { private val appId = "local-" + System.currentTimeMillis - var localActor: ActorRef = null + var localEndpoint: RpcEndpointRef = null override def start() { - localActor = SparkEnv.get.actorSystem.actorOf( - Props(new LocalActor(scheduler, this, totalCores)), - "LocalBackendActor") + localEndpoint = SparkEnv.get.rpcEnv.setupEndpoint( + "LocalBackendEndpoint", new LocalEndpoint(SparkEnv.get.rpcEnv, scheduler, this, totalCores)) } override def stop() { - localActor ! StopExecutor + localEndpoint.send(StopExecutor) } override def reviveOffers() { - localActor ! ReviveOffers + localEndpoint.send(ReviveOffers) } override def defaultParallelism(): Int = scheduler.conf.getInt("spark.default.parallelism", totalCores) override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - localActor ! KillTask(taskId, interruptThread) + localEndpoint.send(KillTask(taskId, interruptThread)) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { - localActor ! StatusUpdate(taskId, state, serializedData) + localEndpoint.send(StatusUpdate(taskId, state, serializedData)) } override def applicationId(): String = appId diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index fc31296f4deb3..1aa0ef18de118 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -26,7 +26,6 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ @@ -37,6 +36,7 @@ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo +import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager @@ -64,7 +64,7 @@ private[spark] class BlockResult( */ private[spark] class BlockManager( executorId: String, - actorSystem: ActorSystem, + rpcEnv: RpcEnv, val master: BlockManagerMaster, defaultSerializer: Serializer, maxMemory: Long, @@ -136,9 +136,9 @@ private[spark] class BlockManager( // Whether to compress shuffle output temporarily spilled to disk private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - private val slaveActor = actorSystem.actorOf( - Props(new BlockManagerSlaveActor(this, mapOutputTracker)), - name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) + private val slaveEndpoint = rpcEnv.setupEndpoint( + "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, + new BlockManagerSlaveEndpoint(rpcEnv, this, mapOutputTracker)) // Pending re-registration action being executed asynchronously or null if none is pending. // Accesses should synchronize on asyncReregisterLock. @@ -167,7 +167,7 @@ private[spark] class BlockManager( */ def this( execId: String, - actorSystem: ActorSystem, + rpcEnv: RpcEnv, master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, @@ -176,7 +176,7 @@ private[spark] class BlockManager( blockTransferService: BlockTransferService, securityManager: SecurityManager, numUsableCores: Int) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), + this(execId, rpcEnv, master, serializer, BlockManager.getMaxMemory(conf), conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager, numUsableCores) } @@ -186,7 +186,7 @@ private[spark] class BlockManager( * where it is only learned after registration with the TaskScheduler). * * This method initializes the BlockTransferService and ShuffleClient, registers with the - * BlockManagerMaster, starts the BlockManagerWorker actor, and registers with a local shuffle + * BlockManagerMaster, starts the BlockManagerWorker endpoint, and registers with a local shuffle * service if configured. */ def initialize(appId: String): Unit = { @@ -202,7 +202,7 @@ private[spark] class BlockManager( blockManagerId } - master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) // Register Executors' configuration with the local shuffle service, if one should exist. if (externalShuffleServiceEnabled && !blockManagerId.isDriver) { @@ -265,7 +265,7 @@ private[spark] class BlockManager( def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") - master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) reportAllBlocks() } @@ -1215,7 +1215,7 @@ private[spark] class BlockManager( shuffleClient.close() } diskBlockManager.stop() - actorSystem.stop(slaveActor) + rpcEnv.stop(slaveEndpoint) blockInfo.clear() memoryStore.clear() diskStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 061964826f08b..ceacf043029f3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -20,35 +20,31 @@ package org.apache.spark.storage import scala.concurrent.{Await, Future} import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor._ - +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] class BlockManagerMaster( - var driverActor: ActorRef, + var driverEndpoint: RpcEndpointRef, conf: SparkConf, isDriver: Boolean) extends Logging { - private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) - private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) - - val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" val timeout = AkkaUtils.askTimeout(conf) - /** Remove a dead executor from the driver actor. This is only called on the driver side. */ + /** Remove a dead executor from the driver endpoint. This is only called on the driver side. */ def removeExecutor(execId: String) { tell(RemoveExecutor(execId)) logInfo("Removed " + execId + " successfully in removeExecutor") } /** Register the BlockManager's id with the driver. */ - def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + def registerBlockManager( + blockManagerId: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef): Unit = { logInfo("Trying to register BlockManager") - tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) + tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) logInfo("Registered BlockManager") } @@ -59,7 +55,7 @@ class BlockManagerMaster( memSize: Long, diskSize: Long, tachyonSize: Long): Boolean = { - val res = askDriverWithReply[Boolean]( + val res = driverEndpoint.askWithReply[Boolean]( UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) logDebug(s"Updated info of block $blockId") res @@ -67,12 +63,12 @@ class BlockManagerMaster( /** Get locations of the blockId from the driver */ def getLocations(blockId: BlockId): Seq[BlockManagerId] = { - askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId)) + driverEndpoint.askWithReply[Seq[BlockManagerId]](GetLocations(blockId)) } /** Get locations of multiple blockIds from the driver */ def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { - askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) + driverEndpoint.askWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } /** @@ -85,11 +81,11 @@ class BlockManagerMaster( /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { - askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) + driverEndpoint.askWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) } - def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { - askDriverWithReply[Option[(String, Int)]](GetActorSystemHostPortForExecutor(executorId)) + def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { + driverEndpoint.askWithReply[Option[(String, Int)]](GetRpcHostPortForExecutor(executorId)) } /** @@ -97,12 +93,12 @@ class BlockManagerMaster( * blocks that the driver knows about. */ def removeBlock(blockId: BlockId) { - askDriverWithReply(RemoveBlock(blockId)) + driverEndpoint.askWithReply[Boolean](RemoveBlock(blockId)) } /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) + val future = driverEndpoint.askWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) future.onFailure { case e: Exception => logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}") @@ -114,7 +110,7 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given shuffle. */ def removeShuffle(shuffleId: Int, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + val future = driverEndpoint.askWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) future.onFailure { case e: Exception => logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}") @@ -126,7 +122,7 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given broadcast. */ def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Int]]]( + val future = driverEndpoint.askWithReply[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) future.onFailure { case e: Exception => @@ -145,11 +141,11 @@ class BlockManagerMaster( * amount of remaining memory. */ def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { - askDriverWithReply[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + driverEndpoint.askWithReply[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) } def getStorageStatus: Array[StorageStatus] = { - askDriverWithReply[Array[StorageStatus]](GetStorageStatus) + driverEndpoint.askWithReply[Array[StorageStatus]](GetStorageStatus) } /** @@ -165,11 +161,12 @@ class BlockManagerMaster( askSlaves: Boolean = true): Map[BlockManagerId, BlockStatus] = { val msg = GetBlockStatus(blockId, askSlaves) /* - * To avoid potential deadlocks, the use of Futures is necessary, because the master actor + * To avoid potential deadlocks, the use of Futures is necessary, because the master endpoint * should not block on waiting for a block manager, which can in turn be waiting for the - * master actor for a response to a prior message. + * master endpoint for a response to a prior message. */ - val response = askDriverWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + val response = driverEndpoint. + askWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) val (blockManagerIds, futures) = response.unzip val result = Await.result(Future.sequence(futures), timeout) if (result == null) { @@ -193,33 +190,28 @@ class BlockManagerMaster( filter: BlockId => Boolean, askSlaves: Boolean): Seq[BlockId] = { val msg = GetMatchingBlockIds(filter, askSlaves) - val future = askDriverWithReply[Future[Seq[BlockId]]](msg) + val future = driverEndpoint.askWithReply[Future[Seq[BlockId]]](msg) Await.result(future, timeout) } - /** Stop the driver actor, called only on the Spark driver node */ + /** Stop the driver endpoint, called only on the Spark driver node */ def stop() { - if (driverActor != null && isDriver) { + if (driverEndpoint != null && isDriver) { tell(StopBlockManagerMaster) - driverActor = null + driverEndpoint = null logInfo("BlockManagerMaster stopped") } } - /** Send a one-way message to the master actor, to which we expect it to reply with true. */ + /** Send a one-way message to the master endpoint, to which we expect it to reply with true. */ private def tell(message: Any) { - if (!askDriverWithReply[Boolean](message)) { - throw new SparkException("BlockManagerMasterActor returned false, expected true.") + if (!driverEndpoint.askWithReply[Boolean](message)) { + throw new SparkException("BlockManagerMasterEndpoint returned false, expected true.") } } - /** - * Send a message to the driver actor and get its result within a default timeout, or - * throw a SparkException if this fails. - */ - private def askDriverWithReply[T](message: Any): T = { - AkkaUtils.askWithReply(message, driverActor, AKKA_RETRY_ATTEMPTS, AKKA_RETRY_INTERVAL_MS, - timeout) - } +} +private[spark] object BlockManagerMaster { + val DRIVER_ENDPOINT_NAME = "BlockManagerMaster" } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 5b5328016124e..28c73a7d543ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -21,25 +21,26 @@ import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.collection.JavaConversions._ -import scala.concurrent.Future -import scala.concurrent.duration._ +import scala.concurrent.{ExecutionContext, Future} -import akka.actor.{Actor, ActorRef} -import akka.pattern.ask - -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, ThreadSafeRpcEndpoint} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} +import org.apache.spark.util.Utils /** - * BlockManagerMasterActor is an actor on the master node to track statuses of - * all slaves' block managers. + * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses + * of all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) - extends Actor with ActorLogReceive with Logging { +class BlockManagerMasterEndpoint( + override val rpcEnv: RpcEnv, + val isLocal: Boolean, + conf: SparkConf, + listenerBus: LiveListenerBus) + extends ThreadSafeRpcEndpoint with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] @@ -50,68 +51,67 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val akkaTimeout = AkkaUtils.askTimeout(conf) + private val askThreadPool = Utils.newDaemonCachedThreadPool("block-manager-ask-thread-pool") + private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool) - override def receiveWithLogging: PartialFunction[Any, Unit] = { - case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => - register(blockManagerId, maxMemSize, slaveActor) - sender ! true + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint) => + register(blockManagerId, maxMemSize, slaveEndpoint) + context.reply(true) case UpdateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => - sender ! updateBlockInfo( - blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) + context.reply(updateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize)) case GetLocations(blockId) => - sender ! getLocations(blockId) + context.reply(getLocations(blockId)) case GetLocationsMultipleBlockIds(blockIds) => - sender ! getLocationsMultipleBlockIds(blockIds) + context.reply(getLocationsMultipleBlockIds(blockIds)) case GetPeers(blockManagerId) => - sender ! getPeers(blockManagerId) + context.reply(getPeers(blockManagerId)) - case GetActorSystemHostPortForExecutor(executorId) => - sender ! getActorSystemHostPortForExecutor(executorId) + case GetRpcHostPortForExecutor(executorId) => + context.reply(getRpcHostPortForExecutor(executorId)) case GetMemoryStatus => - sender ! memoryStatus + context.reply(memoryStatus) case GetStorageStatus => - sender ! storageStatus + context.reply(storageStatus) case GetBlockStatus(blockId, askSlaves) => - sender ! blockStatus(blockId, askSlaves) + context.reply(blockStatus(blockId, askSlaves)) case GetMatchingBlockIds(filter, askSlaves) => - sender ! getMatchingBlockIds(filter, askSlaves) + context.reply(getMatchingBlockIds(filter, askSlaves)) case RemoveRdd(rddId) => - sender ! removeRdd(rddId) + context.reply(removeRdd(rddId)) case RemoveShuffle(shuffleId) => - sender ! removeShuffle(shuffleId) + context.reply(removeShuffle(shuffleId)) case RemoveBroadcast(broadcastId, removeFromDriver) => - sender ! removeBroadcast(broadcastId, removeFromDriver) + context.reply(removeBroadcast(broadcastId, removeFromDriver)) case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) - sender ! true + context.reply(true) case RemoveExecutor(execId) => removeExecutor(execId) - sender ! true + context.reply(true) case StopBlockManagerMaster => - sender ! true - context.stop(self) + context.reply(true) + stop() case BlockManagerHeartbeat(blockManagerId) => - sender ! heartbeatReceived(blockManagerId) + context.reply(heartbeatReceived(blockManagerId)) - case other => - logWarning("Got unknown message: " + other) } private def removeRdd(rddId: Int): Future[Seq[Int]] = { @@ -129,22 +129,20 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. // The dispatcher is used as an implicit argument into the Future sequence construction. - import context.dispatcher val removeMsg = RemoveRdd(rddId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + bm.slaveEndpoint.sendWithReply[Int](removeMsg) }.toSeq ) } private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { - // Nothing to do in the BlockManagerMasterActor data structures - import context.dispatcher + // Nothing to do in the BlockManagerMasterEndpoint data structures val removeMsg = RemoveShuffle(shuffleId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] + bm.slaveEndpoint.sendWithReply[Boolean](removeMsg) }.toSeq ) } @@ -155,14 +153,13 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus * from the executors, but not from the driver. */ private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { - import context.dispatcher val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) val requiredBlockManagers = blockManagerInfo.values.filter { info => removeFromDriver || !info.blockManagerId.isDriver } Future.sequence( requiredBlockManagers.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + bm.slaveEndpoint.sendWithReply[Int](removeMsg) }.toSeq ) } @@ -217,7 +214,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Remove the block from the slave's BlockManager. // Doesn't actually wait for a confirmation and the message might get lost. // If message loss becomes frequent, we should add retry logic here. - blockManager.get.slaveActor.ask(RemoveBlock(blockId))(akkaTimeout) + blockManager.get.slaveEndpoint.sendWithReply[Boolean](RemoveBlock(blockId)) } } } @@ -247,17 +244,16 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private def blockStatus( blockId: BlockId, askSlaves: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { - import context.dispatcher val getBlockStatus = GetBlockStatus(blockId) /* - * Rather than blocking on the block status query, master actor should simply return + * Rather than blocking on the block status query, master endpoint should simply return * Futures to avoid potential deadlocks. This can arise if there exists a block manager - * that is also waiting for this master actor's response to a previous message. + * that is also waiting for this master endpoint's response to a previous message. */ blockManagerInfo.values.map { info => val blockStatusFuture = if (askSlaves) { - info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo[Option[BlockStatus]] + info.slaveEndpoint.sendWithReply[Option[BlockStatus]](getBlockStatus) } else { Future { info.getStatus(blockId) } } @@ -276,13 +272,12 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private def getMatchingBlockIds( filter: BlockId => Boolean, askSlaves: Boolean): Future[Seq[BlockId]] = { - import context.dispatcher val getMatchingBlockIds = GetMatchingBlockIds(filter) Future.sequence( blockManagerInfo.values.map { info => val future = if (askSlaves) { - info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo[Seq[BlockId]] + info.slaveEndpoint.sendWithReply[Seq[BlockId]](getMatchingBlockIds) } else { Future { info.blocks.keys.filter(filter).toSeq } } @@ -291,7 +286,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus ).map(_.flatten.toSeq) } - private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + private def register(id: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef) { val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -308,7 +303,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockManagerIdByExecutor(id.executorId) = id blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveActor) + id, System.currentTimeMillis(), maxMemSize, slaveEndpoint) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) } @@ -379,19 +374,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } /** - * Returns the hostname and port of an executor's actor system, based on the Akka address of its - * BlockManagerSlaveActor. + * Returns the hostname and port of an executor, based on the [[RpcEnv]] address of its + * [[BlockManagerSlaveEndpoint]]. */ - private def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { + private def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { for ( blockManagerId <- blockManagerIdByExecutor.get(executorId); - info <- blockManagerInfo.get(blockManagerId); - host <- info.slaveActor.path.address.host; - port <- info.slaveActor.path.address.port + info <- blockManagerInfo.get(blockManagerId) ) yield { - (host, port) + (info.slaveEndpoint.address.host, info.slaveEndpoint.address.port) } } + + override def onStop(): Unit = { + askThreadPool.shutdownNow() + } } @DeveloperApi @@ -412,7 +409,7 @@ private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, val maxMem: Long, - val slaveActor: ActorRef) + val slaveEndpoint: RpcEndpointRef) extends Logging { private var _lastSeenMs: Long = timeMs diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 48247453edef0..f89d8d7493f7c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -19,8 +19,7 @@ package org.apache.spark.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} -import akka.actor.ActorRef - +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils private[spark] object BlockManagerMessages { @@ -52,7 +51,7 @@ private[spark] object BlockManagerMessages { case class RegisterBlockManager( blockManagerId: BlockManagerId, maxMemSize: Long, - sender: ActorRef) + sender: RpcEndpointRef) extends ToBlockManagerMaster case class UpdateBlockInfo( @@ -92,7 +91,7 @@ private[spark] object BlockManagerMessages { case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class GetActorSystemHostPortForExecutor(executorId: String) extends ToBlockManagerMaster + case class GetRpcHostPortForExecutor(executorId: String) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index 52fb896c4e21f..8980fa8eb70e2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -17,41 +17,43 @@ package org.apache.spark.storage -import scala.concurrent.Future - -import akka.actor.{ActorRef, Actor} +import scala.concurrent.{ExecutionContext, Future} +import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} +import org.apache.spark.util.Utils import org.apache.spark.{Logging, MapOutputTracker, SparkEnv} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.ActorLogReceive /** - * An actor to take commands from the master to execute options. For example, + * An RpcEndpoint to take commands from the master to execute options. For example, * this is used to remove blocks from the slave's BlockManager. */ private[storage] -class BlockManagerSlaveActor( +class BlockManagerSlaveEndpoint( + override val rpcEnv: RpcEnv, blockManager: BlockManager, mapOutputTracker: MapOutputTracker) - extends Actor with ActorLogReceive with Logging { + extends RpcEndpoint with Logging { - import context.dispatcher + private val asyncThreadPool = + Utils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool") + private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool) // Operations that involve removing blocks may be slow and should be done asynchronously - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RemoveBlock(blockId) => - doAsync[Boolean]("removing block " + blockId, sender) { + doAsync[Boolean]("removing block " + blockId, context) { blockManager.removeBlock(blockId) true } case RemoveRdd(rddId) => - doAsync[Int]("removing RDD " + rddId, sender) { + doAsync[Int]("removing RDD " + rddId, context) { blockManager.removeRdd(rddId) } case RemoveShuffle(shuffleId) => - doAsync[Boolean]("removing shuffle " + shuffleId, sender) { + doAsync[Boolean]("removing shuffle " + shuffleId, context) { if (mapOutputTracker != null) { mapOutputTracker.unregisterShuffle(shuffleId) } @@ -59,30 +61,34 @@ class BlockManagerSlaveActor( } case RemoveBroadcast(broadcastId, _) => - doAsync[Int]("removing broadcast " + broadcastId, sender) { + doAsync[Int]("removing broadcast " + broadcastId, context) { blockManager.removeBroadcast(broadcastId, tellMaster = true) } case GetBlockStatus(blockId, _) => - sender ! blockManager.getStatus(blockId) + context.reply(blockManager.getStatus(blockId)) case GetMatchingBlockIds(filter, _) => - sender ! blockManager.getMatchingBlockIds(filter) + context.reply(blockManager.getMatchingBlockIds(filter)) } - private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) { + private def doAsync[T](actionMessage: String, context: RpcCallContext)(body: => T) { val future = Future { logDebug(actionMessage) body } future.onSuccess { case response => logDebug("Done " + actionMessage + ", response is " + response) - responseActor ! response - logDebug("Sent response: " + response + " to " + responseActor) + context.reply(response) + logDebug("Sent response: " + response + " to " + context.sender) } future.onFailure { case t: Throwable => logError("Error in " + actionMessage, t) - responseActor ! null.asInstanceOf[T] + context.sendFailure(t) } } + + override def onStop(): Unit = { + asyncThreadPool.shutdownNow() + } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 7c85e28679f1d..0fdfaf300e95d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1214,6 +1214,16 @@ private[spark] object Utils extends Logging { } } + /** Executes the given block. Log non-fatal errors if any, and only throw fatal errors */ + def tryLogNonFatalError(block: => Unit) { + try { + block + } catch { + case NonFatal(t) => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + } + } + /** * Execute a block of code, then a finally block, but if exceptions happen in * the finally block, do not suppress the original exception. diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala new file mode 100644 index 0000000000000..0fd570e5297d9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import org.scalatest.FunSuite +import org.mockito.Mockito.{mock, spy, verify, when} +import org.mockito.Matchers +import org.mockito.Matchers._ + +import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.util.RpcUtils +import org.scalatest.concurrent.Eventually._ + +class HeartbeatReceiverSuite extends FunSuite with LocalSparkContext { + + test("HeartbeatReceiver") { + sc = spy(new SparkContext("local[2]", "test")) + val scheduler = mock(classOf[TaskScheduler]) + when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(true) + when(sc.taskScheduler).thenReturn(scheduler) + + val heartbeatReceiver = new HeartbeatReceiver(sc) + sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver).send(TaskSchedulerIsSet) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(heartbeatReceiver.scheduler != null) + } + val receiverRef = RpcUtils.makeDriverRef("heartbeat", sc.conf, sc.env.rpcEnv) + + val metrics = new TaskMetrics + val blockManagerId = BlockManagerId("executor-1", "localhost", 12345) + val response = receiverRef.askWithReply[HeartbeatResponse]( + Heartbeat("executor-1", Array(1L -> metrics), blockManagerId)) + + verify(scheduler).executorHeartbeatReceived( + Matchers.eq("executor-1"), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + assert(false === response.reregisterBlockManager) + } + + test("HeartbeatReceiver re-register") { + sc = spy(new SparkContext("local[2]", "test")) + val scheduler = mock(classOf[TaskScheduler]) + when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(false) + when(sc.taskScheduler).thenReturn(scheduler) + + val heartbeatReceiver = new HeartbeatReceiver(sc) + sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver).send(TaskSchedulerIsSet) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(heartbeatReceiver.scheduler != null) + } + val receiverRef = RpcUtils.makeDriverRef("heartbeat", sc.conf, sc.env.rpcEnv) + + val metrics = new TaskMetrics + val blockManagerId = BlockManagerId("executor-1", "localhost", 12345) + val response = receiverRef.askWithReply[HeartbeatResponse]( + Heartbeat("executor-1", Array(1L -> metrics), blockManagerId)) + + verify(scheduler).executorHeartbeatReceived( + Matchers.eq("executor-1"), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + assert(true === response.reregisterBlockManager) + } +} diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index e07bdb9637575..4f19c4f2110d2 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -311,7 +311,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { } test("self: call in onStop") { - @volatile var e: Throwable = null + @volatile var selfOption: Option[RpcEndpointRef] = null val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { override val rpcEnv = env @@ -321,20 +321,18 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { } override def onStop(): Unit = { - self + selfOption = Option(self) } override def onError(cause: Throwable): Unit = { - e = cause } }) env.stop(endpointRef) eventually(timeout(5 seconds), interval(10 millis)) { - // Calling `self` in `onStop` is invalid - assert(e != null) - assert(e.getMessage.contains("Cannot find RpcEndpointRef")) + // Calling `self` in `onStop` will return null, so selfOption will be None + assert(selfOption == None) } } @@ -342,7 +340,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it for(i <- 0 until 100) { @volatile var result = 0 - val endpointRef = env.setupThreadSafeEndpoint(s"receive-in-sequence-$i", new RpcEndpoint { + val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint { override val rpcEnv = env override def receive = { @@ -475,7 +473,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { test("network events") { val events = new mutable.ArrayBuffer[(Any, Any)] with mutable.SynchronizedBuffer[(Any, Any)] - env.setupThreadSafeEndpoint("network-events", new RpcEndpoint { + env.setupEndpoint("network-events", new ThreadSafeRpcEndpoint { override val rpcEnv = env override def receive = { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index c2903c8597997..b4de90b65d545 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -22,11 +22,11 @@ import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps -import akka.actor.{ActorSystem, Props} import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.network.BlockTransferService import org.apache.spark.network.nio.NioBlockTransferService @@ -34,13 +34,12 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.{AkkaUtils, SizeEstimator} /** Testsuite that tests block replication in BlockManager */ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAndAfter { private val conf = new SparkConf(false) - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -61,7 +60,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) - val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + val store = new BlockManager(name, rpcEnv, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) store.initialize("app-id") allStores += store @@ -69,12 +68,10 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd } before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.authenticate", "false") - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") @@ -83,18 +80,17 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) allStores.clear() } after { allStores.foreach { _.stop() } allStores.clear() - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null master = null } @@ -262,7 +258,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, + val failableStore = new BlockManager("failable-store", rpcEnv, master, serializer, 10000, conf, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ecd1cba5b5abe..283090e3bdb1f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,24 +19,18 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays -import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps -import akka.actor._ -import akka.pattern.ask -import akka.util.Timeout - import org.mockito.Mockito.{mock, when} - import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.executor.DataReadMethod import org.apache.spark.network.nio.NioBlockTransferService @@ -53,7 +47,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) @@ -72,28 +66,25 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) - val manager = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + val manager = new BlockManager(name, rpcEnv, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) manager.initialize("app-id") manager } override def beforeEach(): Unit = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -108,9 +99,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store2.stop() store2 = null } - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null master = null } @@ -357,10 +348,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - implicit val timeout = Timeout(30, TimeUnit.SECONDS) - val reregister = !Await.result( - master.driverActor ? BlockManagerHeartbeat(store.blockManagerId), - timeout.duration).asInstanceOf[Boolean] + val reregister = !master.driverEndpoint.askWithReply[Boolean]( + BlockManagerHeartbeat(store.blockManagerId)) assert(reregister == true) } @@ -785,7 +774,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach test("block store put failure") { // Use Java serializer so we can create an unserializable error. val transfer = new NioBlockTransferService(conf, securityMgr) - store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, actorSystem, master, + store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 18a477f92094d..ef4873de2f5a9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -24,20 +24,20 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps -import akka.actor.{ActorSystem, Props} import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util._ -import org.apache.spark.util.{AkkaUtils, ManualClock, Utils} +import org.apache.spark.util.{ManualClock, Utils} import WriteAheadLogBasedBlockHandler._ import WriteAheadLogSuite._ @@ -54,22 +54,19 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche val manualClock = new ManualClock val blockManagerSize = 10000000 - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var blockManagerMaster: BlockManagerMaster = null var blockManager: BlockManager = null var tempDirectory: File = null before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem - conf.set("spark.driver.port", boundPort.toString) + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) + conf.set("spark.driver.port", rpcEnv.address.port.toString) - blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) - blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, + blockManager = new BlockManager("bm", rpcEnv, blockManagerMaster, serializer, blockManagerSize, conf, mapOutputTracker, shuffleManager, new NioBlockTransferService(conf, securityMgr), securityMgr, 0) blockManager.initialize("app-id") @@ -87,9 +84,9 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche blockManagerMaster.stop() blockManagerMaster = null } - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null Utils.deleteRecursively(tempDirectory) } 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 455554eea0597..24a1e02795218 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 @@ -24,22 +24,20 @@ import java.lang.reflect.InvocationTargetException import java.net.{Socket, URL} import java.util.concurrent.atomic.AtomicReference -import akka.actor._ -import akka.remote._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.spark.rpc._ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.SparkException import org.apache.spark.deploy.{PythonRunner, SparkHadoopUtil} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.YarnSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, - SignalLogger, Utils} +import org.apache.spark.util._ /** * Common application master functionality for Spark on Yarn. @@ -72,8 +70,8 @@ private[spark] class ApplicationMaster( @volatile private var allocator: YarnAllocator = _ // Fields used in client mode. - private var actorSystem: ActorSystem = null - private var actor: ActorRef = _ + private var rpcEnv: RpcEnv = null + private var amEndpoint: RpcEndpointRef = _ // Fields used in cluster mode. private val sparkContextRef = new AtomicReference[SparkContext](null) @@ -240,22 +238,21 @@ private[spark] class ApplicationMaster( } /** - * Create an actor that communicates with the driver. + * Create an [[RpcEndpoint]] that communicates with the driver. * * In cluster mode, the AM and the driver belong to same process - * so the AM actor need not monitor lifecycle of the driver. + * so the AMEndpoint need not monitor lifecycle of the driver. */ - private def runAMActor( + private def runAMEndpoint( host: String, port: String, isClusterMode: Boolean): Unit = { - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), + val driverEndpont = rpcEnv.setupEndpointRef( SparkEnv.driverActorSystemName, - host, - port, - YarnSchedulerBackend.ACTOR_NAME) - actor = actorSystem.actorOf(Props(new AMActor(driverUrl, isClusterMode)), name = "YarnAM") + RpcAddress(host, port.toInt), + YarnSchedulerBackend.ENDPOINT_NAME) + amEndpoint = + rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpont, isClusterMode)) } private def runDriver(securityMgr: SecurityManager): Unit = { @@ -272,8 +269,8 @@ private[spark] class ApplicationMaster( ApplicationMaster.EXIT_SC_NOT_INITED, "Timed out waiting for SparkContext.") } else { - actorSystem = sc.env.actorSystem - runAMActor( + rpcEnv = sc.env.rpcEnv + runAMEndpoint( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port"), isClusterMode = true) @@ -283,8 +280,7 @@ private[spark] class ApplicationMaster( } private def runExecutorLauncher(securityMgr: SecurityManager): Unit = { - actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityMgr)._1 + rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, 0, sparkConf, securityMgr) waitForSparkDriver() addAmIpFilter() registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) @@ -431,7 +427,7 @@ private[spark] class ApplicationMaster( sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - runAMActor(driverHost, driverPort.toString, isClusterMode = false) + runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false) } /** Add the Yarn IP filter that is required for properly securing the UI. */ @@ -443,7 +439,7 @@ private[spark] class ApplicationMaster( System.setProperty("spark.ui.filters", amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } else { - actor ! AddWebUIFilter(amFilter, params.toMap, proxyBase) + amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) } } @@ -505,44 +501,29 @@ private[spark] class ApplicationMaster( } /** - * An actor that communicates with the driver's scheduler backend. + * An [[RpcEndpoint]] that communicates with the driver's scheduler backend. */ - private class AMActor(driverUrl: String, isClusterMode: Boolean) extends Actor { - var driver: ActorSelection = _ - - override def preStart(): Unit = { - logInfo("Listen to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - // Send a hello message to establish the connection, after which - // we can monitor Lifecycle Events. - driver ! "Hello" - driver ! RegisterClusterManager - // In cluster mode, the AM can directly monitor the driver status instead - // of trying to deduce it from the lifecycle of the driver's actor - if (!isClusterMode) { - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } + private class AMEndpoint( + override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean) + extends RpcEndpoint with Logging { + + override def onStart(): Unit = { + driver.send(RegisterClusterManager(self)) } override def receive: PartialFunction[Any, Unit] = { - case x: DisassociatedEvent => - logInfo(s"Driver terminated or disconnected! Shutting down. $x") - // In cluster mode, do not rely on the disassociated event to exit - // This avoids potentially reporting incorrect exit codes if the driver fails - if (!isClusterMode) { - finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) - } - case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") - driver ! x + driver.send(x) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RequestExecutors(requestedTotal) => Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") } - sender ! true + context.reply(true) case KillExecutors(executorIds) => logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.") @@ -550,7 +531,16 @@ private[spark] class ApplicationMaster( case Some(a) => executorIds.foreach(a.killExecutor) case None => logWarning("Container allocator is not ready to kill executors yet.") } - sender ! true + context.reply(true) + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") + // In cluster mode, do not rely on the disassociated event to exit + // This avoids potentially reporting incorrect exit codes if the driver fails + if (!isClusterMode) { + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + } } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index c98763e15b58f..b8f42dadcb464 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -112,7 +112,7 @@ private[yarn] class YarnAllocator( SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) // For testing private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true) From acffc43455d7b3e4000be4ff0175b8ea19cd280b Mon Sep 17 00:00:00 2001 From: lewuathe Date: Sun, 5 Apr 2015 16:13:31 -0700 Subject: [PATCH 649/817] [SPARK-6262][MLLIB]Implement missing methods for MultivariateStatisticalSummary Add below methods in pyspark for MultivariateStatisticalSummary - normL1 - normL2 Author: lewuathe Closes #5359 from Lewuathe/SPARK-6262 and squashes the following commits: cbe439e [lewuathe] Implement missing methods for MultivariateStatisticalSummary --- python/pyspark/mllib/stat/_statistics.py | 6 ++++++ python/pyspark/mllib/tests.py | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 218ac148ca992..1d83e9d483f8e 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -49,6 +49,12 @@ def max(self): def min(self): return self.call("min").toArray() + def normL1(self): + return self.call("normL1").toArray() + + def normL2(self): + return self.call("normL2").toArray() + class Statistics(object): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index dd3b66ce67457..47dad7d12e4e4 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -357,6 +357,12 @@ def test_col_with_different_rdds(self): summary = Statistics.colStats(data) self.assertEqual(10, summary.count()) + def test_col_norms(self): + data = RandomRDDs.normalVectorRDD(self.sc, 1000, 10, 10) + summary = Statistics.colStats(data) + self.assertEqual(10, len(summary.normL1())) + self.assertEqual(10, len(summary.normL2())) + class VectorUDTTests(PySparkTestCase): From 0b5d028a93b7d5adb148fbf3a576257bb3a6d8cb Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 5 Apr 2015 21:57:15 -0700 Subject: [PATCH 650/817] [SPARK-6602][Core] Update MapOutputTrackerMasterActor to MapOutputTrackerMasterEndpoint This is the second PR for [SPARK-6602]. It updated MapOutputTrackerMasterActor and its unit tests. cc rxin Author: zsxwing Closes #5371 from zsxwing/rpc-rewrite-part2 and squashes the following commits: fcf3816 [zsxwing] Fix the code style 4013a22 [zsxwing] Add doc for uncaught exceptions in RpcEnv 93c6c20 [zsxwing] Add an example of UnserializableException and add ErrorMonitor to monitor errors from Akka 134fe7b [zsxwing] Update MapOutputTrackerMasterActor to MapOutputTrackerMasterEndpoint --- .../org/apache/spark/MapOutputTracker.scala | 61 +++--- .../scala/org/apache/spark/SparkEnv.scala | 18 +- .../scala/org/apache/spark/rpc/RpcEnv.scala | 4 +- .../apache/spark/rpc/akka/AkkaRpcEnv.scala | 19 +- .../apache/spark/MapOutputTrackerSuite.scala | 100 +++++---- .../org/apache/spark/rpc/RpcEnvSuite.scala | 33 ++- .../apache/spark/util/AkkaUtilsSuite.scala | 198 ++++++++---------- 7 files changed, 221 insertions(+), 212 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5718951451afc..d65c94e410662 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -21,13 +21,11 @@ import java.io._ import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.{HashSet, HashMap, Map} -import scala.concurrent.Await +import scala.collection.mutable.{HashSet, Map} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag -import akka.actor._ -import akka.pattern.ask - +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, RpcEndpoint} import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.BlockManagerId @@ -38,14 +36,15 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -/** Actor class for MapOutputTrackerMaster */ -private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) - extends Actor with ActorLogReceive with Logging { +/** RpcEndpoint class for MapOutputTrackerMaster */ +private[spark] class MapOutputTrackerMasterEndpoint( + override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf) + extends RpcEndpoint with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - override def receiveWithLogging: PartialFunction[Any, Unit] = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => - val hostPort = sender.path.address.hostPort + val hostPort = context.sender.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) val mapOutputStatuses = tracker.getSerializedMapOutputStatuses(shuffleId) val serializedSize = mapOutputStatuses.size @@ -53,19 +52,19 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster val msg = s"Map output statuses were $serializedSize bytes which " + s"exceeds spark.akka.frameSize ($maxAkkaFrameSize bytes)." - /* For SPARK-1244 we'll opt for just logging an error and then throwing an exception. - * Note that on exception the actor will just restart. A bigger refactoring (SPARK-1239) - * will ultimately remove this entire code path. */ + /* For SPARK-1244 we'll opt for just logging an error and then sending it to the sender. + * A bigger refactoring (SPARK-1239) will ultimately remove this entire code path. */ val exception = new SparkException(msg) logError(msg, exception) - throw exception + context.sendFailure(exception) + } else { + context.reply(mapOutputStatuses) } - sender ! mapOutputStatuses case StopMapOutputTracker => - logInfo("MapOutputTrackerActor stopped!") - sender ! true - context.stop(self) + logInfo("MapOutputTrackerMasterEndpoint stopped!") + context.reply(true) + stop() } } @@ -75,12 +74,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster * (driver and executor) use different HashMap to store its metadata. */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { - private val timeout = AkkaUtils.askTimeout(conf) - private val retryAttempts = AkkaUtils.numRetries(conf) - private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) - /** Set to the MapOutputTrackerActor living on the driver. */ - var trackerActor: ActorRef = _ + /** Set to the MapOutputTrackerMasterEndpoint living on the driver. */ + var trackerEndpoint: RpcEndpointRef = _ /** * This HashMap has different behavior for the driver and the executors. @@ -105,12 +101,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging private val fetching = new HashSet[Int] /** - * Send a message to the trackerActor and get its result within a default timeout, or + * Send a message to the trackerEndpoint and get its result within a default timeout, or * throw a SparkException if this fails. */ - protected def askTracker(message: Any): Any = { + protected def askTracker[T: ClassTag](message: Any): T = { try { - AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout) + trackerEndpoint.askWithReply[T](message) } catch { case e: Exception => logError("Error communicating with MapOutputTracker", e) @@ -118,9 +114,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } - /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ + /** Send a one-way message to the trackerEndpoint, to which we expect it to reply with true. */ protected def sendTracker(message: Any) { - val response = askTracker(message) + val response = askTracker[Boolean](message) if (response != true) { throw new SparkException( "Error reply received from MapOutputTracker. Expecting true, got " + response.toString) @@ -157,11 +153,10 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging if (fetchedStatuses == null) { // We won the race to fetch the output locs; do so - logInfo("Doing the fetch; tracker actor = " + trackerActor) + logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint) // This try-finally prevents hangs due to timeouts: try { - val fetchedBytes = - askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]] + val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId)) fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes) logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) @@ -328,7 +323,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) override def stop() { sendTracker(StopMapOutputTracker) mapStatuses.clear() - trackerActor = null + trackerEndpoint = null metadataCleaner.cancel() cachedSerializedStatuses.clear() } @@ -350,6 +345,8 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private[spark] object MapOutputTracker extends Logging { + val ENDPOINT_NAME = "MapOutputTracker" + // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will // generally be pretty compressible because many map outputs will be on the same hostname. diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 55be0a59fedd9..0171488e09562 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -24,7 +24,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.util.Properties -import akka.actor._ import com.google.common.collect.MapMaker import org.apache.spark.annotation.DeveloperApi @@ -41,7 +40,7 @@ import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinato import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ -import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} +import org.apache.spark.util.{RpcUtils, Utils} /** * :: DeveloperApi :: @@ -286,15 +285,6 @@ object SparkEnv extends Logging { val closureSerializer = instantiateClassFromConf[Serializer]( "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") - def registerOrLookup(name: String, newActor: => Actor): ActorRef = { - if (isDriver) { - logInfo("Registering " + name) - actorSystem.actorOf(Props(newActor), name = name) - } else { - AkkaUtils.makeDriverRef(name, conf, actorSystem) - } - } - def registerOrLookupEndpoint( name: String, endpointCreator: => RpcEndpoint): RpcEndpointRef = { @@ -314,9 +304,9 @@ object SparkEnv extends Logging { // Have to assign trackerActor after initialization as MapOutputTrackerActor // requires the MapOutputTracker itself - mapOutputTracker.trackerActor = registerOrLookup( - "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + mapOutputTracker.trackerEndpoint = registerOrLookupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint( + rpcEnv, mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) // Let the user specify short names for shuffle managers val shortShuffleMgrNames = Map( diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index d47e41abcfa50..e259867c14040 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -30,7 +30,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} /** * An RPC environment. [[RpcEndpoint]]s need to register itself with a name to [[RpcEnv]] to * receives messages. Then [[RpcEnv]] will process messages sent from [[RpcEndpointRef]] or remote - * nodes, and deliver them to corresponding [[RpcEndpoint]]s. + * nodes, and deliver them to corresponding [[RpcEndpoint]]s. For uncaught exceptions caught by + * [[RpcEnv]], [[RpcEnv]] will use [[RpcCallContext.sendFailure]] to send exceptions back to the + * sender, or logging them if no such sender or `NotSerializableException`. * * [[RpcEnv]] also provides some methods to retrieve [[RpcEndpointRef]]s given name or uri. */ diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala index 9e06147dff1ed..652e52f2b2e73 100644 --- a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala @@ -17,16 +17,16 @@ package org.apache.spark.rpc.akka -import java.net.URI import java.util.concurrent.ConcurrentHashMap -import scala.concurrent.{Await, Future} +import scala.concurrent.Future import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address} +import akka.event.Logging.Error import akka.pattern.{ask => akkaAsk} import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent} import org.apache.spark.{SparkException, Logging, SparkConf} @@ -242,10 +242,25 @@ private[spark] class AkkaRpcEnvFactory extends RpcEnvFactory { def create(config: RpcEnvConfig): RpcEnv = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( config.name, config.host, config.port, config.conf, config.securityManager) + actorSystem.actorOf(Props(classOf[ErrorMonitor]), "ErrorMonitor") new AkkaRpcEnv(actorSystem, config.conf, boundPort) } } +/** + * Monitor errors reported by Akka and log them. + */ +private[akka] class ErrorMonitor extends Actor with ActorLogReceive with Logging { + + override def preStart(): Unit = { + context.system.eventStream.subscribe(self, classOf[Error]) + } + + override def receiveWithLogging: Actor.Receive = { + case Error(cause: Throwable, _, _, message: String) => logError(message, cause) + } +} + private[akka] class AkkaRpcEndpointRef( @transient defaultAddress: RpcAddress, @transient _actorRef: => ActorRef, diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index ccfe0678cb1c3..6295d34be5ca9 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -17,34 +17,37 @@ package org.apache.spark -import scala.concurrent.Await - -import akka.actor._ -import akka.testkit.TestActorRef +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, isA} import org.scalatest.FunSuite +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite { private val conf = new SparkConf + def createRpcEnv(name: String, host: String = "localhost", port: Int = 0, + securityManager: SecurityManager = new SecurityManager(conf)): RpcEnv = { + RpcEnv.create(name, host, port, conf, securityManager) + } + test("master start and stop") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register shuffle and fetch") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) assert(tracker.containsShuffle(10)) val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) @@ -57,13 +60,14 @@ class MapOutputTrackerSuite extends FunSuite { assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), (BlockManagerId("b", "hostB", 1000), size10000))) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register and unregister shuffle") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) @@ -78,14 +82,14 @@ class MapOutputTrackerSuite extends FunSuite { assert(tracker.getServerStatuses(10, 0).isEmpty) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register shuffle and unregister map output and fetch") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) @@ -104,25 +108,21 @@ class MapOutputTrackerSuite extends FunSuite { intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) } tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, - securityManager = new SecurityManager(conf)) + val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, - securityManager = new SecurityManager(conf)) + val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() @@ -147,8 +147,8 @@ class MapOutputTrackerSuite extends FunSuite { masterTracker.stop() slaveTracker.stop() - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch below akka frame size") { @@ -157,19 +157,24 @@ class MapOutputTrackerSuite extends FunSuite { newConf.set("spark.akka.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) - val actorSystem = ActorSystem("test") - val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) - val masterActor = actorRef.underlyingActor + val rpcEnv = createRpcEnv("spark") + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) // Frame size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) - masterActor.receive(GetMapOutputStatuses(10)) + val sender = mock(classOf[RpcEndpointRef]) + when(sender.address).thenReturn(RpcAddress("localhost", 12345)) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.sender).thenReturn(sender) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(10)) + verify(rpcCallContext).reply(any()) + verify(rpcCallContext, never()).sendFailure(any()) // masterTracker.stop() // this throws an exception - actorSystem.shutdown() + rpcEnv.shutdown() } test("remote fetch exceeds akka frame size") { @@ -178,12 +183,11 @@ class MapOutputTrackerSuite extends FunSuite { newConf.set("spark.akka.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) - val actorSystem = ActorSystem("test") - val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) - val masterActor = actorRef.underlyingActor + val rpcEnv = createRpcEnv("test") + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) - // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. + // Frame size should be ~1.1MB, and MapOutputTrackerMasterEndpoint should throw exception. // Note that the size is hand-selected here because map output statuses are compressed before // being sent. masterTracker.registerShuffle(20, 100) @@ -191,9 +195,15 @@ class MapOutputTrackerSuite extends FunSuite { masterTracker.registerMapOutput(20, i, new CompressedMapStatus( BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } - intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } + val sender = mock(classOf[RpcEndpointRef]) + when(sender.address).thenReturn(RpcAddress("localhost", 12345)) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.sender).thenReturn(sender) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) + verify(rpcCallContext, never()).reply(any()) + verify(rpcCallContext).sendFailure(isA(classOf[SparkException])) // masterTracker.stop() // this throws an exception - actorSystem.shutdown() + rpcEnv.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 4f19c4f2110d2..5a734ec5ba5ec 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -514,10 +514,35 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { ("onDisconnected", remoteAddress))) } } -} -case object Start + test("sendWithReply: unserializable error") { + env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint { + override val rpcEnv = env -case class Ping(id: Int) + override def receiveAndReply(context: RpcCallContext) = { + case msg: String => context.sendFailure(new UnserializableException) + } + }) -case class Pong(id: Int) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "sendWithReply-unserializable-error") + try { + val f = rpcEndpointRef.sendWithReply[String]("hello") + intercept[TimeoutException] { + Await.result(f, 1 seconds) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + +} + +class UnserializableClass + +class UnserializableException extends Exception { + private val unserializableField = new UnserializableClass +} diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 6250d50fb7036..bec79fc4dc8f7 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -19,14 +19,11 @@ package org.apache.spark.util import java.util.concurrent.TimeoutException -import scala.concurrent.Await -import scala.util.{Failure, Try} - -import akka.actor._ - +import akka.actor.ActorNotFound import org.scalatest.FunSuite import org.apache.spark._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.SSLSampleConfigs._ @@ -39,39 +36,37 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro test("remote fetch security bad password") { val conf = new SparkConf + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf + badconf.set("spark.rpc", "akka") badconf.set("spark.authenticate", "true") badconf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = conf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, conf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security off") { @@ -81,28 +76,24 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = badconf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, badconf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -120,8 +111,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security pass") { @@ -131,15 +122,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") @@ -148,13 +138,10 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(securityManagerGood.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = goodconf, securityManager = securityManagerGood) + val slaveRpcEnv =RpcEnv.create("spark-slave", hostname, 0, goodconf, securityManagerGood) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() @@ -170,47 +157,45 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security off client") { val conf = new SparkConf + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf + badconf.set("spark.rpc", "akka") badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === false) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = badconf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, badconf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch ssl on") { @@ -218,26 +203,22 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slaves", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -255,8 +236,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } @@ -267,28 +248,24 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() slaveConf.set("spark.authenticate", "true") slaveConf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === true) @@ -305,45 +282,43 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch ssl on and security enabled - bad credentials") { val conf = sparkSSLConfig() + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() + slaveConf.set("spark.rpc", "akka") slaveConf.set("spark.authenticate", "true") slaveConf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } @@ -352,35 +327,30 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - val result = Try(Await.result(selection.resolveOne(timeout * 2), timeout)) - - result match { - case Failure(ex: ActorNotFound) => - case Failure(ex: TimeoutException) => - case r => fail(s"$r is neither Failure(ActorNotFound) nor Failure(TimeoutException)") + try { + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + fail("should receive either ActorNotFound or TimeoutException") + } catch { + case e: ActorNotFound => + case e: TimeoutException => } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } } From 49f38824a4770fc9017e6cc9b1803c4543b0c081 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Mon, 6 Apr 2015 10:11:20 +0100 Subject: [PATCH 651/817] [SPARK-6673] spark-shell.cmd can't start in Windows even when spark was built added equivalent script to load-spark-env.sh Author: Masayoshi TSUZUKI Closes #5328 from tsudukim/feature/SPARK-6673 and squashes the following commits: aaefb19 [Masayoshi TSUZUKI] removed dust. be3405e [Masayoshi TSUZUKI] [SPARK-6673] spark-shell.cmd can't start in Windows even when spark was built --- bin/load-spark-env.cmd | 59 ++++++++++++++++++++++++++++++++++++++++++ bin/pyspark2.cmd | 3 +-- bin/run-example2.cmd | 3 +-- bin/spark-class2.cmd | 3 +-- 4 files changed, 62 insertions(+), 6 deletions(-) create mode 100644 bin/load-spark-env.cmd diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd new file mode 100644 index 0000000000000..36d932c453b6f --- /dev/null +++ b/bin/load-spark-env.cmd @@ -0,0 +1,59 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script loads spark-env.cmd if it exists, and ensures it is only loaded once. +rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's +rem conf/ subdirectory. + +if [%SPARK_ENV_LOADED%] == [] ( + set SPARK_ENV_LOADED=1 + + if not [%SPARK_CONF_DIR%] == [] ( + set user_conf_dir=%SPARK_CONF_DIR% + ) else ( + set user_conf_dir=%~dp0..\..\conf + ) + + call :LoadSparkEnv +) + +rem Setting SPARK_SCALA_VERSION if not already set. + +set ASSEMBLY_DIR2=%SPARK_HOME%/assembly/target/scala-2.11 +set ASSEMBLY_DIR1=%SPARK_HOME%/assembly/target/scala-2.10 + +if [%SPARK_SCALA_VERSION%] == [] ( + + if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( + echo "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." + echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd." + exit 1 + ) + if exist %ASSEMBLY_DIR2% ( + set SPARK_SCALA_VERSION=2.11 + ) else ( + set SPARK_SCALA_VERSION=2.10 + ) +) +exit /b 0 + +:LoadSparkEnv +if exist "%user_conf_dir%\spark-env.cmd" ( + call "%user_conf_dir%\spark-env.cmd" +) diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 4f5eb5e20614d..09b4149c2a439 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -20,8 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Figure out which Python to use. if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index b49d0dcb4ff2d..c3e0221fb62e3 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -25,8 +25,7 @@ set FWDIR=%~dp0..\ rem Export this as SPARK_HOME set SPARK_HOME=%FWDIR% -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Test that an argument was given if not "x%1"=="x" goto arg_given diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4ce727bc99128..4b3401d745f2a 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -20,8 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Test that an argument was given if "x%1"=="x" ( From 9fe41252198df71f4629843d363db8c83f36440c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 6 Apr 2015 10:18:56 +0100 Subject: [PATCH 652/817] SPARK-6569 [STREAMING] Down-grade same-offset message in Kafka streaming to INFO Reduce "is the same as ending offset" message to INFO level per JIRA discussion Author: Sean Owen Closes #5366 from srowen/SPARK-6569 and squashes the following commits: 8a5b992 [Sean Owen] Reduce "is the same as ending offset" message to INFO level per JIRA discussion --- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 4a83b715fa89d..a0b8a0c565210 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -86,7 +86,7 @@ class KafkaRDD[ val part = thePart.asInstanceOf[KafkaRDDPartition] assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) if (part.fromOffset == part.untilOffset) { - log.warn(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + log.info(s"Beginning offset ${part.fromOffset} is the same as ending offset " + s"skipping ${part.topic} ${part.partition}") Iterator.empty } else { From 30363ede8635f2548e444697dbcf60a795b61a84 Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Mon, 6 Apr 2015 13:15:01 -0700 Subject: [PATCH 653/817] [MLlib] [SPARK-6713] Iterators in columnSimilarities for mapPartitionsWithIndex Use Iterators in columnSimilarities to allow mapPartitionsWithIndex to spill to disk. This could happen in a dense and large column - this way Spark can spill the pairs onto disk instead of building all the pairs before handing them to Spark. Another PR coming to update documentation. Author: Reza Zadeh Closes #5364 from rezazadeh/optmemsim and squashes the following commits: 47c90ba [Reza Zadeh] Iterators in columnSimilarities for flatMap --- .../mllib/linalg/distributed/RowMatrix.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 961111507f2c2..9a89a6f3a515f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -531,7 +531,6 @@ class RowMatrix( val rand = new XORShiftRandom(indx) val scaled = new Array[Double](p.size) iter.flatMap { row => - val buf = new ListBuffer[((Int, Int), Double)]() row match { case SparseVector(size, indices, values) => val nnz = indices.size @@ -540,8 +539,9 @@ class RowMatrix( scaled(k) = values(k) / q(indices(k)) k += 1 } - k = 0 - while (k < nnz) { + + Iterator.tabulate (nnz) { k => + val buf = new ListBuffer[((Int, Int), Double)]() val i = indices(k) val iVal = scaled(k) if (iVal != 0 && rand.nextDouble() < p(i)) { @@ -555,8 +555,8 @@ class RowMatrix( l += 1 } } - k += 1 - } + buf + }.flatten case DenseVector(values) => val n = values.size var i = 0 @@ -564,8 +564,8 @@ class RowMatrix( scaled(i) = values(i) / q(i) i += 1 } - i = 0 - while (i < n) { + Iterator.tabulate (n) { i => + val buf = new ListBuffer[((Int, Int), Double)]() val iVal = scaled(i) if (iVal != 0 && rand.nextDouble() < p(i)) { var j = i + 1 @@ -577,10 +577,9 @@ class RowMatrix( j += 1 } } - i += 1 - } + buf + }.flatten } - buf } }.reduceByKey(_ + _).map { case ((i, j), sim) => MatrixEntry(i.toLong, j.toLong, sim) From e40ea8742a8771ecd46b182f45b5fcd8bd6dd725 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Mon, 6 Apr 2015 18:00:51 -0700 Subject: [PATCH 654/817] [Minor] [SQL] [SPARK-6729] Minor fix for DriverQuirks get The function uses .substring(0, X), which will trigger OutOfBoundsException if string length is less than X. A better way to do this is to use startsWith, which won't error out in this case. Author: Volodymyr Lyubinets Closes #5378 from vlyubin/quirks and squashes the following commits: 504e8e0 [Volodymyr Lyubinets] Minor fix for DriverQuirks get --- .../main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala index 1704be7fcbd30..0feabc4282f4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala @@ -49,9 +49,9 @@ private[sql] object DriverQuirks { * Fetch the DriverQuirks class corresponding to a given database url. */ def get(url: String): DriverQuirks = { - if (url.substring(0, 10).equals("jdbc:mysql")) { + if (url.startsWith("jdbc:mysql")) { new MySQLQuirks() - } else if (url.substring(0, 15).equals("jdbc:postgresql")) { + } else if (url.startsWith("jdbc:postgresql")) { new PostgresQuirks() } else { new NoQuirks() From a0846c4b635eac8d8637c83d490177f881952d27 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 6 Apr 2015 23:33:16 -0700 Subject: [PATCH 655/817] [SPARK-6716] Change SparkContext.DRIVER_IDENTIFIER from to driver Currently, the driver's executorId is set to ``. This choice of ID was present in older Spark versions, but it has started to cause problems now that executorIds are used in more contexts, such as Ganglia metric names or driver thread-dump links the web UI. The angle brackets must be escaped when embedding this ID in XML or as part of URLs and this has led to multiple problems: - https://issues.apache.org/jira/browse/SPARK-6484 - https://issues.apache.org/jira/browse/SPARK-4313 The simplest solution seems to be to change this id to something that does not contain any special characters, such as `driver`. I'm not sure whether we can perform this change in a patch release, since this ID may be considered a stable API by metrics users, but it's probably okay to do this in a major release as long as we document it in the release notes. Author: Josh Rosen Closes #5372 from JoshRosen/driver-id-fix and squashes the following commits: 42d3c10 [Josh Rosen] Clarify comment 0c5d04b [Josh Rosen] Add backwards-compatibility in BlockManagerId.isDriver 7ff12e0 [Josh Rosen] Change SparkContext.DRIVER_IDENTIFIER from to driver --- .../main/scala/org/apache/spark/SparkContext.scala | 12 +++++++++++- .../org/apache/spark/storage/BlockManagerId.scala | 5 ++++- .../org/apache/spark/storage/BlockManagerSuite.scala | 6 ++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 942c5975ece6d..3f1a7dd99d635 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1901,7 +1901,17 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" - private[spark] val DRIVER_IDENTIFIER = "" + /** + * Executor id for the driver. In earlier versions of Spark, this was ``, but this was + * changed to `driver` because the angle brackets caused escaping issues in URLs and XML (see + * SPARK-6716 for more details). + */ + private[spark] val DRIVER_IDENTIFIER = "driver" + + /** + * Legacy version of DRIVER_IDENTIFIER, retained for backwards-compatibility. + */ + private[spark] val LEGACY_DRIVER_IDENTIFIER = "" // The following deprecated objects have already been copied to `object AccumulatorParam` to // make the compiler find them automatically. They are duplicate codes only for backward diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index a6f1ebf325a7c..69ac37511e730 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -60,7 +60,10 @@ class BlockManagerId private ( def port: Int = port_ - def isDriver: Boolean = { executorId == SparkContext.DRIVER_IDENTIFIER } + def isDriver: Boolean = { + executorId == SparkContext.DRIVER_IDENTIFIER || + executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER + } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 283090e3bdb1f..6dc5bc4cb08c4 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -139,6 +139,12 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(id2_.eq(id1), "Deserialized id2 is not the same object as original id1") } + test("BlockManagerId.isDriver() backwards-compatibility with legacy driver ids (SPARK-6716)") { + assert(BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "XXX", 1).isDriver) + assert(BlockManagerId(SparkContext.LEGACY_DRIVER_IDENTIFIER, "XXX", 1).isDriver) + assert(!BlockManagerId("notADriverIdentifier", "XXX", 1).isDriver) + } + test("master + 1 manager interaction") { store = makeBlockManager(20000) val a1 = new Array[Byte](4000) From 6f0d55d76f758d217fd18ffa0ccf273d7ab0377b Mon Sep 17 00:00:00 2001 From: Matt Aasted Date: Mon, 6 Apr 2015 23:50:48 -0700 Subject: [PATCH 656/817] [SPARK-6636] Use public DNS hostname everywhere in spark_ec2.py The spark_ec2.py script uses public_dns_name everywhere in the script except for testing ssh availability, which is done using the public ip address of the instances. This breaks the script for users who are deploying the cluster with a private-network-only security group. The fix is to use public_dns_name in the remaining place. Author: Matt Aasted Closes #5302 from aasted/master and squashes the following commits: 60cf6ee [Matt Aasted] [SPARK-6636] Use public DNS hostname everywhere in spark_ec2.py --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5507a9c5a4733..879a52cef8ff0 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -809,7 +809,7 @@ def is_cluster_ssh_available(cluster_instances, opts): Check if SSH is available on all the instances in a cluster. """ for i in cluster_instances: - if not is_ssh_available(host=i.ip_address, opts=opts): + if not is_ssh_available(host=i.public_dns_name, opts=opts): return False else: return True From ae980eb41c00b5f1f64c650f267b884e864693f0 Mon Sep 17 00:00:00 2001 From: Sasaki Toru Date: Tue, 7 Apr 2015 01:55:32 -0700 Subject: [PATCH 657/817] [SPARK-6736][GraphX][Doc]Example of Graph#aggregateMessages has error Example of Graph#aggregateMessages has error. Since aggregateMessages is a method of Graph, It should be written "rawGraph.aggregateMessages" Author: Sasaki Toru Closes #5388 from sasakitoa/aggregateMessagesExample and squashes the following commits: b1d631b [Sasaki Toru] Example of Graph#aggregateMessages has error --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 8494d06b1cdb7..36dc7b0f86c89 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -409,7 +409,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * {{{ * val rawGraph: Graph[_, _] = Graph.textFile("twittergraph") * val inDeg: RDD[(VertexId, Int)] = - * aggregateMessages[Int](ctx => ctx.sendToDst(1), _ + _) + * rawGraph.aggregateMessages[Int](ctx => ctx.sendToDst(1), _ + _) * }}} * * @note By expressing computation at the edge level we achieve From b65bad65c3500475b974ca0219f218eef296db2c Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 7 Apr 2015 08:36:25 -0500 Subject: [PATCH 658/817] [SPARK-3591][YARN]fire and forget for YARN cluster mode https://issues.apache.org/jira/browse/SPARK-3591 The output after this patch: >doggie153:/opt/oss/spark-1.3.0-bin-hadoop2.4/bin # ./spark-submit --class org.apache.spark.examples.SparkPi --master yarn-cluster ../lib/spark-examples*.jar 15/03/31 21:15:25 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable 15/03/31 21:15:25 INFO RMProxy: Connecting to ResourceManager at doggie153/10.177.112.153:8032 15/03/31 21:15:25 INFO Client: Requesting a new application from cluster with 4 NodeManagers 15/03/31 21:15:25 INFO Client: Verifying our application has not requested more than the maximum memory capability of the cluster (8192 MB per container) 15/03/31 21:15:25 INFO Client: Will allocate AM container, with 896 MB memory including 384 MB overhead 15/03/31 21:15:25 INFO Client: Setting up container launch context for our AM 15/03/31 21:15:25 INFO Client: Preparing resources for our AM container 15/03/31 21:15:26 INFO Client: Uploading resource file:/opt/oss/spark-1.3.0-bin-hadoop2.4/lib/spark-assembly-1.4.0-SNAPSHOT-hadoop2.4.1.jar -> hdfs://doggie153:9000/user/root/.sparkStaging/application_1427257505534_0016/spark-assembly-1.4.0-SNAPSHOT-hadoop2.4.1.jar 15/03/31 21:15:27 INFO Client: Uploading resource file:/opt/oss/spark-1.3.0-bin-hadoop2.4/lib/spark-examples-1.3.0-hadoop2.4.0.jar -> hdfs://doggie153:9000/user/root/.sparkStaging/application_1427257505534_0016/spark-examples-1.3.0-hadoop2.4.0.jar 15/03/31 21:15:28 INFO Client: Setting up the launch environment for our AM container 15/03/31 21:15:28 INFO SecurityManager: Changing view acls to: root 15/03/31 21:15:28 INFO SecurityManager: Changing modify acls to: root 15/03/31 21:15:28 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(root); users with modify permissions: Set(root) 15/03/31 21:15:28 INFO Client: Submitting application 16 to ResourceManager 15/03/31 21:15:28 INFO YarnClientImpl: Submitted application application_1427257505534_0016 15/03/31 21:15:28 INFO Client: ... waiting before polling ResourceManager for application state 15/03/31 21:15:33 INFO Client: ... polling ResourceManager for application state 15/03/31 21:15:33 INFO Client: Application report for application_1427257505534_0016 (state: RUNNING) 15/03/31 21:15:33 INFO Client: client token: N/A diagnostics: N/A ApplicationMaster host: doggie157 ApplicationMaster RPC port: 0 queue: default start time: 1427807728307 final status: UNDEFINED tracking URL: http://doggie153:8088/proxy/application_1427257505534_0016/ user: root /cc andrewor14 Author: WangTaoTheTonic Closes #5297 from WangTaoTheTonic/SPARK-3591 and squashes the following commits: c76d232 [WangTaoTheTonic] wrap lines 16c90a8 [WangTaoTheTonic] move up lines to avoid duplicate fea390d [WangTaoTheTonic] log failed/killed report, style and comment be1cc2e [WangTaoTheTonic] reword f0bc54f [WangTaoTheTonic] minor: expose appid in excepiton messages ba9b22b [WangTaoTheTonic] wrong config name e1a4013 [WangTaoTheTonic] revert to the old version and do some robust 19706c0 [WangTaoTheTonic] add a config to control whether to forget 0cbdce8 [WangTaoTheTonic] fire and forget for YARN cluster mode --- .../org/apache/spark/deploy/Client.scala | 2 +- .../deploy/rest/StandaloneRestClient.scala | 2 +- docs/running-on-yarn.md | 9 ++ .../org/apache/spark/deploy/yarn/Client.scala | 83 +++++++++++-------- 4 files changed, 61 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 65238af2caa24..8d13b2a2cd4f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -89,7 +89,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) /* Find out driver status then exit the JVM */ def pollAndReportStatus(driverId: String) { - println(s"... waiting before polling master for driver state") + println("... waiting before polling master for driver state") Thread.sleep(5000) println("... polling master for driver state") val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index a3539e44bd2f9..b8fd406fb6f9a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -245,7 +245,7 @@ private[deploy] class StandaloneRestClient extends Logging { } } else { val failMessage = Option(submitResponse.message).map { ": " + _ }.getOrElse("") - logError("Application submission failed" + failMessage) + logError(s"Application submission failed$failMessage") } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d9f3eb2b74b18..b7e68d4f71714 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -196,6 +196,15 @@ Most of the configs are the same for Spark on YARN as for other deployment modes It should be no larger than the global number of max attempts in the YARN configuration. + + spark.yarn.submit.waitAppCompletion + true + + In YARN cluster mode, controls whether the client waits to exit until the application completes. + If set to true, the client process will stay alive reporting the application's status. + Otherwise, the client process will exit after submission. + + # Launching Spark on YARN diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 61f8fc3f5a014..79d55a09eb671 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -66,6 +66,8 @@ private[spark] class Client( private val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() private val isClusterMode = args.isClusterMode + private val fireAndForget = isClusterMode && + !sparkConf.getBoolean("spark.yarn.submit.waitAppCompletion", true) def stop(): Unit = yarnClient.stop() @@ -564,31 +566,13 @@ private[spark] class Client( if (logApplicationReport) { logInfo(s"Application report for $appId (state: $state)") - val details = Seq[(String, String)]( - ("client token", getClientToken(report)), - ("diagnostics", report.getDiagnostics), - ("ApplicationMaster host", report.getHost), - ("ApplicationMaster RPC port", report.getRpcPort.toString), - ("queue", report.getQueue), - ("start time", report.getStartTime.toString), - ("final status", report.getFinalApplicationStatus.toString), - ("tracking URL", report.getTrackingUrl), - ("user", report.getUser) - ) - - // Use more loggable format if value is null or empty - val formattedDetails = details - .map { case (k, v) => - val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") - s"\n\t $k: $newValue" } - .mkString("") // If DEBUG is enabled, log report details every iteration // Otherwise, log them every time the application changes state if (log.isDebugEnabled) { - logDebug(formattedDetails) + logDebug(formatReportDetails(report)) } else if (lastState != state) { - logInfo(formattedDetails) + logInfo(formatReportDetails(report)) } } @@ -609,24 +593,57 @@ private[spark] class Client( throw new SparkException("While loop is depleted! This should never happen...") } + private def formatReportDetails(report: ApplicationReport): String = { + val details = Seq[(String, String)]( + ("client token", getClientToken(report)), + ("diagnostics", report.getDiagnostics), + ("ApplicationMaster host", report.getHost), + ("ApplicationMaster RPC port", report.getRpcPort.toString), + ("queue", report.getQueue), + ("start time", report.getStartTime.toString), + ("final status", report.getFinalApplicationStatus.toString), + ("tracking URL", report.getTrackingUrl), + ("user", report.getUser) + ) + + // Use more loggable format if value is null or empty + details.map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" + }.mkString("") + } + /** - * Submit an application to the ResourceManager and monitor its state. - * This continues until the application has exited for any reason. + * Submit an application to the ResourceManager. + * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive + * reporting the application's status until the application has exited for any reason. + * Otherwise, the client process will exit after submission. * If the application finishes with a failed, killed, or undefined status, * throw an appropriate SparkException. */ def run(): Unit = { - val (yarnApplicationState, finalApplicationStatus) = monitorApplication(submitApplication()) - if (yarnApplicationState == YarnApplicationState.FAILED || - finalApplicationStatus == FinalApplicationStatus.FAILED) { - throw new SparkException("Application finished with failed status") - } - if (yarnApplicationState == YarnApplicationState.KILLED || - finalApplicationStatus == FinalApplicationStatus.KILLED) { - throw new SparkException("Application is killed") - } - if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { - throw new SparkException("The final status of application is undefined") + val appId = submitApplication() + if (fireAndForget) { + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + logInfo(s"Application report for $appId (state: $state)") + logInfo(formatReportDetails(report)) + if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { + throw new SparkException(s"Application $appId finished with status: $state") + } + } else { + val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId) + if (yarnApplicationState == YarnApplicationState.FAILED || + finalApplicationStatus == FinalApplicationStatus.FAILED) { + throw new SparkException(s"Application $appId finished with failed status") + } + if (yarnApplicationState == YarnApplicationState.KILLED || + finalApplicationStatus == FinalApplicationStatus.KILLED) { + throw new SparkException(s"Application $appId is killed") + } + if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { + throw new SparkException(s"The final status of application $appId is undefined") + } } } } From 7162ecf88624615c78a332de482f5defd297e415 Mon Sep 17 00:00:00 2001 From: Vinod K C Date: Tue, 7 Apr 2015 10:42:08 -0700 Subject: [PATCH 659/817] [SPARK-6733][ Scheduler]Added scala.language.existentials Author: Vinod K C Closes #5384 from vinodkc/Suppression_Scala_existential_code and squashes the following commits: 82a3a1f [Vinod K C] Added scala.language.existentials --- .../src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 1 + .../test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 917cce1f9686c..c82ae4baa3630 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.duration._ +import scala.language.existentials import scala.language.postfixOps import scala.util.control.NonFatal diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 29d4ec5f85c1e..fc7349330cf86 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -22,6 +22,7 @@ import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.scalatest.FunSuite From 2c32bef1790dac6f77ef9674f6106c2e24ea0338 Mon Sep 17 00:00:00 2001 From: sksamuel Date: Tue, 7 Apr 2015 10:43:22 -0700 Subject: [PATCH 660/817] Replace use of .size with .length for Arrays Invoking .size on arrays is valid, but requires an implicit conversion to SeqLike. This incurs a compile time overhead and more importantly a runtime overhead, as the Array must be wrapped before the method can be invoked. For example, the difference in generated byte code is: public int withSize(); Code: 0: getstatic #23 // Field scala/Predef$.MODULE$:Lscala/Predef$; 3: aload_0 4: invokevirtual #25 // Method array:()[I 7: invokevirtual #29 // Method scala/Predef$.intArrayOps:([I)Lscala/collection/mutable/ArrayOps; 10: invokeinterface #34, 1 // InterfaceMethod scala/collection/mutable/ArrayOps.size:()I 15: ireturn public int withLength(); Code: 0: aload_0 1: invokevirtual #25 // Method array:()[I 4: arraylength 5: ireturn Author: sksamuel Closes #5376 from sksamuel/master and squashes the following commits: 77ec261 [sksamuel] Replace use of .size with .length for Arrays. --- .../apache/spark/network/nio/Connection.scala | 2 +- .../apache/spark/rdd/AsyncRDDActions.scala | 10 ++++----- .../scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../org/apache/spark/rdd/CartesianRDD.scala | 4 ++-- .../org/apache/spark/rdd/CheckpointRDD.scala | 2 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 4 ++-- .../org/apache/spark/rdd/CoalescedRDD.scala | 2 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 4 ++-- .../spark/rdd/OrderedRDDFunctions.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++++++++---------- .../apache/spark/rdd/RDDCheckpointData.scala | 6 ++--- .../org/apache/spark/rdd/SubtractedRDD.scala | 2 +- .../scala/org/apache/spark/rdd/UnionRDD.scala | 6 ++--- .../spark/rdd/ZippedPartitionsRDD.scala | 4 ++-- .../apache/spark/rdd/ZippedWithIndexRDD.scala | 2 +- .../org/apache/spark/storage/RDDInfo.scala | 2 +- .../apache/spark/ui/ConsoleProgressBar.scala | 4 ++-- .../apache/spark/util/collection/BitSet.scala | 2 +- 19 files changed, 42 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 04eb2bf9ba4ab..6b898bd4bfc1b 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -181,7 +181,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, buffer.get(bytes) bytes.foreach(x => print(x + " ")) buffer.position(curPosition) - print(" (" + bytes.size + ")") + print(" (" + bytes.length + ")") } def printBuffer(buffer: ByteBuffer, position: Int, length: Int) { diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 646df283ac069..3406a7e97e368 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -45,7 +45,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } result }, - Range(0, self.partitions.size), + Range(0, self.partitions.length), (index: Int, data: Long) => totalCount.addAndGet(data), totalCount.get()) } @@ -54,8 +54,8 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi * Returns a future for retrieving all elements of this RDD. */ def collectAsync(): FutureAction[Seq[T]] = { - val results = new Array[Array[T]](self.partitions.size) - self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size), + val results = new Array[Array[T]](self.partitions.length) + self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.length), (index, data) => results(index) = data, results.flatten.toSeq) } @@ -111,7 +111,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi */ def foreachAsync(f: T => Unit): FutureAction[Unit] = { val cleanF = self.context.clean(f) - self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size), + self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.length), (index, data) => Unit, Unit) } @@ -119,7 +119,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi * Applies a function f to each partition of this RDD. */ def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = { - self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size), + self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.length), (index, data) => Unit, Unit) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index fffa1911f5bc2..71578d1210fde 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -36,7 +36,7 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds override def getPartitions: Array[Partition] = { assertValid() - (0 until blockIds.size).map(i => { + (0 until blockIds.length).map(i => { new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] }).toArray } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 9059eb13bb5d8..c1d6971787572 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -53,11 +53,11 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( extends RDD[Pair[T, U]](sc, Nil) with Serializable { - val numPartitionsInRdd2 = rdd2.partitions.size + val numPartitionsInRdd2 = rdd2.partitions.length override def getPartitions: Array[Partition] = { // create the cross product split - val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size) + val array = new Array[Partition](rdd1.partitions.length * rdd2.partitions.length) for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) { val idx = s1.index * numPartitionsInRdd2 + s2.index array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 760c0fa3ac96a..0d130dd4c7a60 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -49,7 +49,7 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) if (fs.exists(cpath)) { val dirContents = fs.listStatus(cpath).map(_.getPath) val partitionFiles = dirContents.filter(_.getName.startsWith("part-")).map(_.toString).sorted - val numPart = partitionFiles.size + val numPart = partitionFiles.length if (numPart > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || ! partitionFiles(numPart-1).endsWith(CheckpointRDD.splitIdToFile(numPart-1)))) { throw new SparkException("Invalid checkpoint directory: " + checkpointPath) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 07398a6fa62f6..7021a339e879b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -99,7 +99,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def getPartitions: Array[Partition] = { val array = new Array[Partition](part.numPartitions) - for (i <- 0 until array.size) { + for (i <- 0 until array.length) { // Each CoGroupPartition will have a dependency per contributing RDD array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => // Assume each RDD contributed a single dependency, and get it @@ -120,7 +120,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] - val numRdds = split.deps.size + val numRdds = split.deps.length // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 5117ccfabfcc2..0c1b02c07d09f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -166,7 +166,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: // determines the tradeoff between load-balancing the partitions sizes and their locality // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * prev.partitions.size).toInt + val slack = (balanceSlack * prev.partitions.length).toInt var noLocality = true // if true if no preferredLocations exists for parent RDD diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 71e6e300fec5f..29ca3e9c4bd04 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -70,7 +70,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new MeanEvaluator(self.partitions.size, confidence) + val evaluator = new MeanEvaluator(self.partitions.length, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } @@ -81,7 +81,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new SumEvaluator(self.partitions.size, confidence) + val evaluator = new SumEvaluator(self.partitions.length, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 6fdfdb734d1b8..6afe50161dacd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -56,7 +56,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * order of the keys). */ // TODO: this currently doesn't work on P other than Tuple2! - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.length) : RDD[(K, V)] = { val part = new RangePartitioner(numPartitions, self, ascending) 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 bf1303d39592d..05351ba4ff76b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -823,7 +823,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * RDD will be <= us. */ def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = - subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) + subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length))) /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ddbfd5624e741..d80d94a588346 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -316,7 +316,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(): RDD[T] = distinct(partitions.size) + def distinct(): RDD[T] = distinct(partitions.length) /** * Return a new RDD that has exactly numPartitions partitions. @@ -488,7 +488,7 @@ abstract class RDD[T: ClassTag]( def sortBy[K]( f: (T) => K, ascending: Boolean = true, - numPartitions: Int = this.partitions.size) + numPartitions: Int = this.partitions.length) (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = this.keyBy[K](f) .sortByKey(ascending, numPartitions) @@ -852,7 +852,7 @@ abstract class RDD[T: ClassTag]( * RDD will be <= us. */ def subtract(other: RDD[T]): RDD[T] = - subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) + subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.length))) /** * Return an RDD with the elements from `this` that are not in `other`. @@ -986,14 +986,14 @@ abstract class RDD[T: ClassTag]( combOp: (U, U) => U, depth: Int = 2): U = { require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") - if (partitions.size == 0) { + if (partitions.length == 0) { return Utils.clone(zeroValue, context.env.closureSerializer.newInstance()) } val cleanSeqOp = context.clean(seqOp) val cleanCombOp = context.clean(combOp) val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it))) - var numPartitions = partiallyAggregated.partitions.size + var numPartitions = partiallyAggregated.partitions.length val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation. while (numPartitions > scale + numPartitions / scale) { @@ -1026,7 +1026,7 @@ abstract class RDD[T: ClassTag]( } result } - val evaluator = new CountEvaluator(partitions.size, confidence) + val evaluator = new CountEvaluator(partitions.length, confidence) sc.runApproximateJob(this, countElements, evaluator, timeout) } @@ -1061,7 +1061,7 @@ abstract class RDD[T: ClassTag]( } map } - val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence) + val evaluator = new GroupedCountEvaluator[T](partitions.length, confidence) sc.runApproximateJob(this, countPartition, evaluator, timeout) } @@ -1140,7 +1140,7 @@ abstract class RDD[T: ClassTag]( * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ def zipWithUniqueId(): RDD[(T, Long)] = { - val n = this.partitions.size.toLong + val n = this.partitions.length.toLong this.mapPartitionsWithIndex { case (k, iter) => iter.zipWithIndex.map { case (item, i) => (item, i * n + k) @@ -1243,7 +1243,7 @@ abstract class RDD[T: ClassTag]( queue ++= util.collection.Utils.takeOrdered(items, num)(ord) Iterator.single(queue) } - if (mapRDDs.partitions.size == 0) { + if (mapRDDs.partitions.length == 0) { Array.empty } else { mapRDDs.reduce { (queue1, queue2) => @@ -1489,7 +1489,7 @@ abstract class RDD[T: ClassTag]( } // The first RDD in the dependency stack has no parents, so no need for a +- def firstDebugString(rdd: RDD[_]): Seq[String] = { - val partitionStr = "(" + rdd.partitions.size + ")" + val partitionStr = "(" + rdd.partitions.length + ")" val leftOffset = (partitionStr.length - 1) / 2 val nextPrefix = (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset)) @@ -1499,7 +1499,7 @@ abstract class RDD[T: ClassTag]( } ++ debugChildren(rdd, nextPrefix) } def shuffleDebugString(rdd: RDD[_], prefix: String = "", isLastChild: Boolean): Seq[String] = { - val partitionStr = "(" + rdd.partitions.size + ")" + val partitionStr = "(" + rdd.partitions.length + ")" val leftOffset = (partitionStr.length - 1) / 2 val thisPrefix = prefix.replaceAll("\\|\\s+$", "") val nextPrefix = ( diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index f67e5f1857979..6afd63d537d75 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -94,10 +94,10 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) new SerializableWritable(rdd.context.hadoopConfiguration)) rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) - if (newRDD.partitions.size != rdd.partitions.size) { + if (newRDD.partitions.length != rdd.partitions.length) { throw new SparkException( - "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.size + ") has different " + - "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") + "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.length + ") has different " + + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.length + ")") } // Change the dependencies and partitions of the RDD diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index c27f435eb9c5a..e9d745588ee9a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -76,7 +76,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def getPartitions: Array[Partition] = { val array = new Array[Partition](part.numPartitions) - for (i <- 0 until array.size) { + for (i <- 0 until array.length) { // Each CoGroupPartition will depend on rdd1 and rdd2 array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => dependencies(j) match { diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 4239e7e22af89..3986645350a82 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -63,7 +63,7 @@ class UnionRDD[T: ClassTag]( extends RDD[T](sc, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.size).sum) + val array = new Array[Partition](rdds.map(_.partitions.length).sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) @@ -76,8 +76,8 @@ class UnionRDD[T: ClassTag]( val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { - deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size) - pos += rdd.partitions.size + deps += new RangeDependency(rdd, 0, pos, rdd.partitions.length) + pos += rdd.partitions.length } deps } diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index d0be304762e1f..a96b6c3d23454 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -52,8 +52,8 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( if (preservesPartitioning) firstParent[Any].partitioner else None override def getPartitions: Array[Partition] = { - val numParts = rdds.head.partitions.size - if (!rdds.forall(rdd => rdd.partitions.size == numParts)) { + val numParts = rdds.head.partitions.length + if (!rdds.forall(rdd => rdd.partitions.length == numParts)) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } Array.tabulate[Partition](numParts) { i => diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index 8c43a559409f2..523aaf2b860b5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -41,7 +41,7 @@ class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, L /** The start index of each partition. */ @transient private val startIndices: Array[Long] = { - val n = prev.partitions.size + val n = prev.partitions.length if (n == 0) { Array[Long]() } else if (n == 1) { diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 0186eb30a1905..034525b56f59c 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -52,6 +52,6 @@ class RDDInfo( private[spark] object RDDInfo { def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel) } } diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 67f572e79314d..77c0bc8b5360a 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -65,7 +65,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { val stageIds = sc.statusTracker.getActiveStageIds() val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1) .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId()) - if (stages.size > 0) { + if (stages.length > 0) { show(now, stages.take(3)) // display at most 3 stages in same time } } @@ -81,7 +81,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { val total = s.numTasks() val header = s"[Stage ${s.stageId()}:" val tailer = s"(${s.numCompletedTasks()} + ${s.numActiveTasks()}) / $total]" - val w = width - header.size - tailer.size + val w = width - header.length - tailer.length val bar = if (w > 0) { val percent = w * s.numCompletedTasks() / total (0 until w).map { i => diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index f79e8e0491ea1..41cb8cfe2afa3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -39,7 +39,7 @@ class BitSet(numBits: Int) extends Serializable { val wordIndex = bitIndex >> 6 // divide by 64 var i = 0 while(i < wordIndex) { words(i) = -1; i += 1 } - if(wordIndex < words.size) { + if(wordIndex < words.length) { // Set the remaining bits (note that the mask could still be zero) val mask = ~(-1L << (bitIndex & 0x3f)) words(wordIndex) |= mask From 12322159147581602978f7f5a6b33b887ef781a1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 Apr 2015 12:37:33 -0700 Subject: [PATCH 661/817] [SPARK-6750] Upgrade ScalaStyle to 0.7. 0.7 fixes a bug that's pretty useful, i.e. inline functions no longer return explicit type definition. Author: Reynold Xin Closes #5399 from rxin/style0.7 and squashes the following commits: 54c41b2 [Reynold Xin] Actually update the version. 09c759c [Reynold Xin] [SPARK-6750] Upgrade ScalaStyle to 0.7. --- project/plugins.sbt | 2 +- project/project/SparkPluginBuild.scala | 16 +------- .../scalastyle/NonASCIICharacterChecker.scala | 39 ------------------- 3 files changed, 2 insertions(+), 55 deletions(-) delete mode 100644 project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala diff --git a/project/plugins.sbt b/project/plugins.sbt index ee45b6a51905e..7096b0d3ee7de 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,7 +19,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") -addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.7.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 8863f272da415..471d00bd8223f 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -24,20 +24,6 @@ import sbt.Keys._ * becomes available for scalastyle sbt plugin. */ object SparkPluginDef extends Build { - lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle, sbtPomReader) - lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) + lazy val root = Project("plugins", file(".")) dependsOn(sbtPomReader) lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git#ignore_artifact_id") - - // There is actually no need to publish this artifact. - def styleSettings = Defaults.defaultSettings ++ Seq ( - name := "spark-style", - organization := "org.apache.spark", - scalaVersion := "2.10.4", - scalacOptions := Seq("-unchecked", "-deprecation"), - libraryDependencies ++= Dependencies.scalaStyle - ) - - object Dependencies { - val scalaStyle = Seq("org.scalastyle" %% "scalastyle" % "0.4.0") - } } diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala deleted file mode 100644 index 3d43c35299555..0000000000000 --- a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala +++ /dev/null @@ -1,39 +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.scalastyle - -import java.util.regex.Pattern - -import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} - -import scalariform.lexer.Token -import scalariform.parser.CompilationUnit - -class NonASCIICharacterChecker extends ScalariformChecker { - val errorKey: String = "non.ascii.character.disallowed" - - override def verify(ast: CompilationUnit): List[ScalastyleError] = { - ast.tokens.filter(hasNonAsciiChars).map(x => PositionError(x.offset)).toList - } - - private def hasNonAsciiChars(x: Token) = - x.rawText.trim.nonEmpty && !Pattern.compile( """\p{ASCII}+""", Pattern.DOTALL) - .matcher(x.text.trim).matches() - -} From 596ba77c5fdca79486396989e549632153055caf Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Tue, 7 Apr 2015 14:29:53 -0700 Subject: [PATCH 662/817] [SPARK-6568] spark-shell.cmd --jars option does not accept the jar that has space in its path escape spaces in the arguments. Author: Masayoshi TSUZUKI Closes #5347 from tsudukim/feature/SPARK-6568 and squashes the following commits: 9180aaf [Masayoshi TSUZUKI] [SPARK-6568] spark-shell.cmd --jars option does not accept the jar that has space in its path --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0fdfaf300e95d..25ae6ee579ab3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1661,7 +1661,7 @@ private[spark] object Utils extends Logging { /** * Format a Windows path such that it can be safely passed to a URI. */ - def formatWindowsPath(path: String): String = path.replace("\\", "/") + def formatWindowsPath(path: String): String = path.replace("\\", "/").replace(" ", "%20") /** * Indicates whether Spark is currently running unit tests. diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 5d93086082189..b7cc84078983a 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -241,6 +241,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assertResolves("C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("C:\\path\\to\\file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("file:/C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) + assertResolves("file:/C:/path to/file.txt", "file:/C:/path%20to/file.txt", testWindows = true) assertResolves("file:///C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } @@ -264,8 +265,9 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assertResolves("hdfs:/jar1,file:/jar2,jar3", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") - assertResolves("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi", testWindows = true) + assertResolves("""hdfs:/jar1,file:/jar2,jar3,C:\pi.py#py.pi,C:\path to\jar4.jar""", + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi,file:/C:/path%20to/jar4.jar", + testWindows = true) } test("nonLocalPaths") { From e6f08fb42fda35952ea8b005170750ae551dc7d9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 7 Apr 2015 14:34:15 -0700 Subject: [PATCH 663/817] Revert "[SPARK-6568] spark-shell.cmd --jars option does not accept the jar that has space in its path" This reverts commit 596ba77c5fdca79486396989e549632153055caf. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- core/src/test/scala/org/apache/spark/util/UtilsSuite.scala | 6 ++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 25ae6ee579ab3..0fdfaf300e95d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1661,7 +1661,7 @@ private[spark] object Utils extends Logging { /** * Format a Windows path such that it can be safely passed to a URI. */ - def formatWindowsPath(path: String): String = path.replace("\\", "/").replace(" ", "%20") + def formatWindowsPath(path: String): String = path.replace("\\", "/") /** * Indicates whether Spark is currently running unit tests. diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index b7cc84078983a..5d93086082189 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -241,7 +241,6 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assertResolves("C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("C:\\path\\to\\file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("file:/C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) - assertResolves("file:/C:/path to/file.txt", "file:/C:/path%20to/file.txt", testWindows = true) assertResolves("file:///C:/path/to/file.txt", "file:/C:/path/to/file.txt", testWindows = true) assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } @@ -265,9 +264,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { assertResolves("hdfs:/jar1,file:/jar2,jar3", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") - assertResolves("""hdfs:/jar1,file:/jar2,jar3,C:\pi.py#py.pi,C:\path to\jar4.jar""", - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi,file:/C:/path%20to/jar4.jar", - testWindows = true) + assertResolves("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi", testWindows = true) } test("nonLocalPaths") { From fc957dc78138e72036dbbadc9a54f155d318c038 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Tue, 7 Apr 2015 14:36:57 -0700 Subject: [PATCH 664/817] [SPARK-6720][MLLIB] PySpark MultivariateStatisticalSummary unit test for normL1... ... and normL2. Add test cases to insufficient unit test for `normL1` and `normL2`. Ref: https://github.com/apache/spark/pull/5359 Author: lewuathe Closes #5374 from Lewuathe/SPARK-6720 and squashes the following commits: 5541b24 [lewuathe] More accurate tests dc5718c [lewuathe] [SPARK-6720] PySpark MultivariateStatisticalSummary unit test for normL1 and normL2 --- python/pyspark/mllib/tests.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 47dad7d12e4e4..61ef398487c0c 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -363,6 +363,13 @@ def test_col_norms(self): self.assertEqual(10, len(summary.normL1())) self.assertEqual(10, len(summary.normL2())) + data2 = self.sc.parallelize(xrange(10)).map(lambda x: Vectors.dense(x)) + summary2 = Statistics.colStats(data2) + self.assertEqual(array([45.0]), summary2.normL1()) + import math + expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, xrange(10)))) + self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) + class VectorUDTTests(PySparkTestCase): From 77bcceb9f01e97cb6f41791f2167b40c4311f701 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Apr 2015 07:00:56 +0800 Subject: [PATCH 665/817] [SPARK-6748] [SQL] Makes QueryPlan.schema a lazy val `DataFrame.collect()` calls `SparkPlan.executeCollect()`, which consists of a single line: ```scala execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() ``` The problem is that, `QueryPlan.schema` is a function. And since 1.3.0, `convertRowToScala` starts returning a `GenericRowWithSchema`. Thus, every `GenericRowWithSchema` instance holds a separate copy of the schema object. Also, YJP profiling result of the following simple micro benchmark (executed in Spark shell) shows that constructing the schema object takes up to ~35% CPU time. ```scala sc.parallelize(1 to 10000000). map(i => (i, s"val_$i")). toDF("key", "value"). saveAsParquetFile("file:///tmp/src.parquet") // Profiling started from this line sqlContext.parquetFile("file:///tmp/src.parquet").collect() ``` [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5398) Author: Cheng Lian Closes #5398 from liancheng/spark-6748 and squashes the following commits: 3159469 [Cheng Lian] Makes QueryPlan.schema a lazy val --- .../scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 02f7c26a8ab6e..7967189cacb24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -150,7 +150,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - def schema: StructType = StructType.fromAttributes(output) + lazy val schema: StructType = StructType.fromAttributes(output) /** Returns the output schema in the tree format. */ def schemaString: String = schema.treeString From c83e03948b184ffb3a9418fecc4d2c26ae33b057 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 7 Apr 2015 16:18:55 -0700 Subject: [PATCH 666/817] [SPARK-6737] Fix memory leak in OutputCommitCoordinator This patch fixes a memory leak in the DAGScheduler, which caused us to leak a map entry per submitted stage. The problem is that the OutputCommitCoordinator needs to be informed when stages end in order to remove entries from its `authorizedCommitters` map, but the DAGScheduler only called it in one of the four code paths that are used to mark stages as completed. This patch fixes this issue by consolidating the processing of stage completion into a new `markStageAsFinished` method and updates DAGSchedulerSuite's `assertDataStructuresEmpty` assertion to also check the OutputCommitCoordinator data structures. I've also added a comment at the top of DAGScheduler so that we remember to update this test when adding new data structures. Author: Josh Rosen Closes #5397 from JoshRosen/SPARK-6737 and squashes the following commits: af3b02f [Josh Rosen] Consolidate stage completion handling code in a single method. e96ce3a [Josh Rosen] Consolidate stage completion handling code in a single method. 3052aea [Josh Rosen] Comment update 7896899 [Josh Rosen] Fix SPARK-6737 by informing OutputCommitCoordinator of all stage end events. 4ead1dc [Josh Rosen] Add regression tests for SPARK-6737 --- .../apache/spark/scheduler/DAGScheduler.scala | 63 ++++++++++--------- .../scheduler/OutputCommitCoordinator.scala | 7 +++ .../spark/scheduler/DAGSchedulerSuite.scala | 1 + 3 files changed, 42 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c82ae4baa3630..c912520fded3b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -50,6 +50,10 @@ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task * a small number of times before cancelling the whole stage. * + * Here's a checklist to use when making or reviewing changes to this class: + * + * - When adding a new data structure, update `DAGSchedulerSuite.assertDataStructuresEmpty` to + * include the new structure. This will help to catch memory leaks. */ private[spark] class DAGScheduler( @@ -111,6 +115,8 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] + private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator + // A closure serializer that we reuse. // This is only safe because DAGScheduler runs in a single thread. private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() @@ -128,8 +134,6 @@ class DAGScheduler( private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) - private val outputCommitCoordinator = env.outputCommitCoordinator - // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessLoop.post(BeginEvent(task, taskInfo)) @@ -710,9 +714,10 @@ class DAGScheduler( // cancelling the stages because if the DAG scheduler is stopped, the entire application // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" - runningStages.foreach { stage => - stage.latestInfo.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + // The `toArray` here is necessary so that we don't iterate over `runningStages` while + // mutating it. + runningStages.toArray.foreach { stage => + markStageAsFinished(stage, Some(stageFailedMessage)) } listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error))) } @@ -887,10 +892,9 @@ class DAGScheduler( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } else { - // Because we posted SparkListenerStageSubmitted earlier, we should post - // SparkListenerStageCompleted here in case there are no tasks to run. - outputCommitCoordinator.stageEnd(stage.id) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + // Because we posted SparkListenerStageSubmitted earlier, we should mark + // the stage as completed here in case there are no tasks to run + markStageAsFinished(stage, None) val debugString = stage match { case stage: ShuffleMapStage => @@ -902,7 +906,6 @@ class DAGScheduler( s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" } logDebug(debugString) - runningStages -= stage } } @@ -968,22 +971,6 @@ class DAGScheduler( } val stage = stageIdToStage(task.stageId) - - def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None): Unit = { - val serviceTime = stage.latestInfo.submissionTime match { - case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) - case _ => "Unknown" - } - if (errorMessage.isEmpty) { - logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stage.latestInfo.completionTime = Some(clock.getTimeMillis()) - } else { - stage.latestInfo.stageFailed(errorMessage.get) - logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) - } - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) - runningStages -= stage - } event.reason match { case Success => listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, @@ -1099,7 +1086,6 @@ class DAGScheduler( logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + s"due to a fetch failure from $mapStage (${mapStage.name})") markStageAsFinished(failedStage, Some(failureMessage)) - runningStages -= failedStage } if (disallowStageRetryForTest) { @@ -1215,6 +1201,26 @@ class DAGScheduler( submitWaitingStages() } + /** + * Marks a stage as finished and removes it from the list of running stages. + */ + private def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None): Unit = { + val serviceTime = stage.latestInfo.submissionTime match { + case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) + case _ => "Unknown" + } + if (errorMessage.isEmpty) { + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) + stage.latestInfo.completionTime = Some(clock.getTimeMillis()) + } else { + stage.latestInfo.stageFailed(errorMessage.get) + logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + } + outputCommitCoordinator.stageEnd(stage.id) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + runningStages -= stage + } + /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. @@ -1264,8 +1270,7 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - stage.latestInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + markStageAsFinished(stage, Some(failureReason)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 9e29fd13821dc..7c184b1dcb308 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -59,6 +59,13 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { private val authorizedCommittersByStage: CommittersByStageMap = mutable.Map() private type CommittersByStageMap = mutable.Map[StageId, mutable.Map[PartitionId, TaskAttemptId]] + /** + * Returns whether the OutputCommitCoordinator's internal data structures are all empty. + */ + def isEmpty: Boolean = { + authorizedCommittersByStage.isEmpty + } + /** * Called by tasks to ask whether they can commit their output to HDFS. * diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 63360a0f189a3..eb759f0807a17 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -783,6 +783,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) + assert(scheduler.outputCommitCoordinator.isEmpty) } // Nothing in this test should break if the task info's fields are null, but From d138aa8ee23f4450242da3ac70a493229a90c76b Mon Sep 17 00:00:00 2001 From: Omede Firouz Date: Tue, 7 Apr 2015 23:36:31 -0400 Subject: [PATCH 667/817] [SPARK-6705][MLLIB] Add fit intercept api to ml logisticregression I have the fit intercept enabled by default for logistic regression, I wonder what others think here. I understand that it enables allocation by default which is undesirable, but one needs to have a very strong reason for not having an intercept term enabled so it is the safer default from a statistical sense. Explicitly modeling the intercept by adding a column of all 1s does not work. I believe the reason is that since the API for LogisticRegressionWithLBFGS forces column normalization, and a column of all 1s has 0 variance so dividing by 0 kills it. Author: Omede Firouz Closes #5301 from oefirouz/addIntercept and squashes the following commits: 9f1286b [Omede Firouz] [SPARK-6705][MLLIB] Add fitInterceptTerm to LogisticRegression 1d6bd6f [Omede Firouz] [SPARK-6705][MLLIB] Add a fit intercept term to ML LogisticRegression 9963509 [Omede Firouz] [MLLIB] Add fitIntercept to LogisticRegression 2257fca [Omede Firouz] [MLLIB] Add fitIntercept param to logistic regression 329c1e2 [Omede Firouz] [MLLIB] Add fit intercept term bd9663c [Omede Firouz] [MLLIB] Add fit intercept api to ml logisticregression --- .../spark/ml/classification/LogisticRegression.scala | 8 ++++++-- .../org/apache/spark/ml/param/sharedParams.scala | 12 ++++++++++++ .../ml/classification/LogisticRegressionSuite.scala | 9 +++++++++ 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 49c00f77480e8..34625745dd0a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams - with HasRegParam with HasMaxIter with HasThreshold + with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold /** @@ -55,6 +55,9 @@ class LogisticRegression /** @group setParam */ def setMaxIter(value: Int): this.type = set(maxIter, value) + /** @group setParam */ + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) @@ -67,7 +70,8 @@ class LogisticRegression } // Train model - val lr = new LogisticRegressionWithLBFGS + val lr = new LogisticRegressionWithLBFGS() + .setIntercept(paramMap(fitIntercept)) lr.optimizer .setRegParam(paramMap(regParam)) .setNumIterations(paramMap(maxIter)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 5d660d1e151a7..0739fdbfcbaae 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -106,6 +106,18 @@ private[ml] trait HasProbabilityCol extends Params { def getProbabilityCol: String = get(probabilityCol) } +private[ml] trait HasFitIntercept extends Params { + /** + * param for fitting the intercept term, defaults to true + * @group param + */ + val fitIntercept: BooleanParam = + new BooleanParam(this, "fitIntercept", "indicates whether to fit an intercept term", Some(true)) + + /** @group getParam */ + def getFitIntercept: Boolean = get(fitIntercept) +} + private[ml] trait HasThreshold extends Params { /** * param for threshold in (binary) prediction diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index b3d1bfcfbee0f..35d8c2e16c6cd 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -46,6 +46,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(lr.getPredictionCol == "prediction") assert(lr.getRawPredictionCol == "rawPrediction") assert(lr.getProbabilityCol == "probability") + assert(lr.getFitIntercept == true) val model = lr.fit(dataset) model.transform(dataset) .select("label", "probability", "prediction", "rawPrediction") @@ -55,6 +56,14 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(model.getPredictionCol == "prediction") assert(model.getRawPredictionCol == "rawPrediction") assert(model.getProbabilityCol == "probability") + assert(model.intercept !== 0.0) + } + + test("logistic regression doesn't fit intercept when fitIntercept is off") { + val lr = new LogisticRegression + lr.setFitIntercept(false) + val model = lr.fit(dataset) + assert(model.intercept === 0.0) } test("logistic regression with setters") { From 8d2a36c0fdfbea9f58271ef6aeb89bb79b22cf62 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 7 Apr 2015 22:40:42 -0700 Subject: [PATCH 668/817] [SPARK-6754] Remove unnecessary TaskContextHelper The TaskContextHelper was originally necessary because TaskContext was written in Java, which does not have a way to specify that classes are package-private, so TaskContextHelper existed to work around this. Now that TaskContext has been re-written in Scala, this class is no longer necessary. rxin can you look at this? It looks like you missed this bit of cleanup when you moved TaskContext from Java to Scala in #4324 cc ScrapCodes and pwendell who added this originally. Author: Kay Ousterhout Closes #5402 from kayousterhout/SPARK-6754 and squashes the following commits: f089800 [Kay Ousterhout] [SPARK-6754] Remove unnecessary TaskContextHelper --- .../org/apache/spark/TaskContextHelper.scala | 29 ------------------- .../apache/spark/scheduler/DAGScheduler.scala | 4 +-- .../org/apache/spark/scheduler/Task.scala | 6 ++-- 3 files changed, 5 insertions(+), 34 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/TaskContextHelper.scala diff --git a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala b/core/src/main/scala/org/apache/spark/TaskContextHelper.scala deleted file mode 100644 index 4636c4600a01a..0000000000000 --- a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -/** - * This class exists to restrict the visibility of TaskContext setters. - */ -private [spark] object TaskContextHelper { - - def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) - - def unset(): Unit = TaskContext.unset() - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c912520fded3b..508fe7b3303ca 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -645,13 +645,13 @@ class DAGScheduler( val split = rdd.partitions(job.partitions(0)) val taskContext = new TaskContextImpl(job.finalStage.id, job.partitions(0), taskAttemptId = 0, attemptNumber = 0, runningLocally = true) - TaskContextHelper.setTaskContext(taskContext) + TaskContext.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() - TaskContextHelper.unset() + TaskContext.unset() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 4d9f940813b8e..8b592867ee31d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import org.apache.spark.{TaskContextHelper, TaskContextImpl, TaskContext} +import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream @@ -54,7 +54,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(taskAttemptId: Long, attemptNumber: Int): T = { context = new TaskContextImpl(stageId = stageId, partitionId = partitionId, taskAttemptId = taskAttemptId, attemptNumber = attemptNumber, runningLocally = false) - TaskContextHelper.setTaskContext(context) + TaskContext.setTaskContext(context) context.taskMetrics.setHostname(Utils.localHostName()) taskThread = Thread.currentThread() if (_killed) { @@ -64,7 +64,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex runTask(context) } finally { context.markTaskCompleted() - TaskContextHelper.unset() + TaskContext.unset() } } From 15e0d2bd1304db62fad286c1bb687e87c361e16c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 00:24:59 -0700 Subject: [PATCH 669/817] [SPARK-6765] Fix test code style for streaming. So we can turn style checker on for test code. Author: Reynold Xin Closes #5409 from rxin/test-style-streaming and squashes the following commits: 7aea69b [Reynold Xin] [SPARK-6765] Fix test code style for streaming. --- .../flume/FlumePollingStreamSuite.scala | 29 ++++++------ .../streaming/flume/FlumeStreamSuite.scala | 4 +- .../streaming/mqtt/MQTTStreamSuite.scala | 3 +- .../streaming/BasicOperationsSuite.scala | 6 ++- .../spark/streaming/CheckpointSuite.scala | 45 ++++++++++++++----- .../apache/spark/streaming/FailureSuite.scala | 4 +- .../spark/streaming/InputStreamsSuite.scala | 15 ++++--- .../streaming/ReceivedBlockHandlerSuite.scala | 4 +- .../streaming/ReceivedBlockTrackerSuite.scala | 6 ++- .../spark/streaming/ReceiverSuite.scala | 11 ++--- .../streaming/StreamingContextSuite.scala | 5 ++- .../streaming/StreamingListenerSuite.scala | 4 +- .../spark/streaming/TestSuiteBase.scala | 28 +++++++----- .../spark/streaming/UISeleniumSuite.scala | 3 +- .../streaming/WindowOperationsSuite.scala | 4 +- .../WriteAheadLogBackedBlockRDDSuite.scala | 12 +++-- .../scheduler/JobGeneratorSuite.scala | 2 +- .../streaming/util/WriteAheadLogSuite.scala | 2 +- .../spark/streamingtest/ImplicitSuite.scala | 3 +- 19 files changed, 115 insertions(+), 75 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index e04d4088df7dc..2edea9b5b69ba 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -1,21 +1,20 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * 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 + * 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. + * 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.flume import java.net.InetSocketAddress @@ -213,7 +212,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging assert(counter === totalEventsPerChannel * channels.size) } - def assertChannelIsEmpty(channel: MemoryChannel) = { + def assertChannelIsEmpty(channel: MemoryChannel): Unit = { val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 51d273af8da84..39e6754c81dbf 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -151,7 +151,9 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L } /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { + private class CompressionChannelFactory(compressionLevel: Int) + extends NioClientSocketChannelFactory { + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { val encoder = new ZlibEncoder(compressionLevel) pipeline.addFirst("deflater", encoder) diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 24d78ecb3a97d..a19a72c58a705 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -139,7 +139,8 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { msgTopic.publish(message) } catch { case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - Thread.sleep(50) // wait for Spark streaming to consume something from the message queue + // wait for Spark streaming to consume something from the message queue + Thread.sleep(50) } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index cf191715d29d6..87bc20f79c3cd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -171,7 +171,9 @@ class BasicOperationsSuite extends TestSuiteBase { test("flatMapValues") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)), + (s: DStream[String]) => { + s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)) + }, Seq( Seq(("a", 2), ("a", 12), ("b", 1), ("b", 11)), Seq(("", 2), ("", 12)), Seq() ), true ) @@ -474,7 +476,7 @@ class BasicOperationsSuite extends TestSuiteBase { stream.foreachRDD(_ => {}) // Dummy output stream ssc.start() Thread.sleep(2000) - def getInputFromSlice(fromMillis: Long, toMillis: Long) = { + def getInputFromSlice(fromMillis: Long, toMillis: Long): Set[Int] = { stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 91a2b2bba461d..54c30440a6e8d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -43,7 +43,7 @@ class CheckpointSuite extends TestSuiteBase { var ssc: StreamingContext = null - override def batchDuration = Milliseconds(500) + override def batchDuration: Duration = Milliseconds(500) override def beforeFunction() { super.beforeFunction() @@ -72,7 +72,7 @@ class CheckpointSuite extends TestSuiteBase { val input = (1 to 10).map(_ => Seq("a")).toSeq val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Int], state: Option[Int]) => { - Some((values.sum + state.getOrElse(0))) + Some(values.sum + state.getOrElse(0)) } st.map(x => (x, 1)) .updateStateByKey(updateFunc) @@ -199,7 +199,12 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ), (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _), - Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), Seq() ), 3 ) } @@ -212,7 +217,8 @@ class CheckpointSuite extends TestSuiteBase { val n = 10 val w = 4 val input = (1 to n).map(_ => Seq("a")).toSeq - val output = Seq(Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4))) + val output = Seq( + Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4))) val operation = (st: DStream[String]) => { st.map(x => (x, 1)) .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration) @@ -236,7 +242,13 @@ class CheckpointSuite extends TestSuiteBase { classOf[TextOutputFormat[Text, IntWritable]]) output }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -259,7 +271,13 @@ class CheckpointSuite extends TestSuiteBase { classOf[NewTextOutputFormat[Text, IntWritable]]) output }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -298,7 +316,13 @@ class CheckpointSuite extends TestSuiteBase { output } }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -533,7 +557,8 @@ class CheckpointSuite extends TestSuiteBase { * Advances the manual clock on the streaming scheduler by given number of batches. * It also waits for the expected amount of time for each batch. */ - def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { + def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = + { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] logInfo("Manual clock before advancing = " + clock.getTimeMillis()) for (i <- 1 to numBatches.toInt) { @@ -543,7 +568,7 @@ class CheckpointSuite extends TestSuiteBase { logInfo("Manual clock after advancing = " + clock.getTimeMillis()) Thread.sleep(batchDuration.milliseconds) - val outputStream = ssc.graph.getOutputStreams.filter { dstream => + val outputStream = ssc.graph.getOutputStreams().filter { dstream => dstream.isInstanceOf[TestOutputStreamWithPartitions[V]] }.head.asInstanceOf[TestOutputStreamWithPartitions[V]] outputStream.output.map(_.flatten) @@ -552,4 +577,4 @@ class CheckpointSuite extends TestSuiteBase { private object CheckpointSuite extends Serializable { var batchThreeShouldBlockIndefinitely: Boolean = true -} \ No newline at end of file +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 26435d8515815..0c4c06534a693 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -29,9 +29,9 @@ class FailureSuite extends TestSuiteBase with Logging { val directory = Utils.createTempDir() val numBatches = 30 - override def batchDuration = Milliseconds(1000) + override def batchDuration: Duration = Milliseconds(1000) - override def useManualClock = false + override def useManualClock: Boolean = false override def afterFunction() { Utils.deleteRecursively(directory) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7ed6320a3d0bc..e6ac4975c5e68 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -52,7 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) + def output: ArrayBuffer[String] = outputBuffer.flatMap(x => x) outputStream.register() ssc.start() @@ -164,7 +164,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] val outputStream = new TestOutputStream(countStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) + def output: ArrayBuffer[Long] = outputBuffer.flatMap(x => x) outputStream.register() ssc.start() @@ -196,7 +196,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val queueStream = ssc.queueStream(queue, oneAtATime = true) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output = outputBuffer.filter(_.size > 0) + def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) outputStream.register() ssc.start() @@ -204,7 +204,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq("1", "2", "3", "4", "5") val expectedOutput = input.map(Seq(_)) - //Thread.sleep(1000) + val inputIterator = input.toIterator for (i <- 0 until input.size) { // Enqueue more than 1 item per tick but they should dequeue one at a time @@ -239,7 +239,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val queueStream = ssc.queueStream(queue, oneAtATime = false) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output = outputBuffer.filter(_.size > 0) + def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) outputStream.register() ssc.start() @@ -352,7 +352,8 @@ class TestServer(portToBind: Int = 0) extends Logging { logInfo("New connection") try { clientSocket.setTcpNoDelay(true) - val outputStream = new BufferedWriter(new OutputStreamWriter(clientSocket.getOutputStream)) + val outputStream = new BufferedWriter( + new OutputStreamWriter(clientSocket.getOutputStream)) while(clientSocket.isConnected) { val msg = queue.poll(100, TimeUnit.MILLISECONDS) @@ -384,7 +385,7 @@ class TestServer(portToBind: Int = 0) extends Logging { def stop() { servingThread.interrupt() } - def port = serverSocket.getLocalPort + def port: Int = serverSocket.getLocalPort } /** This is a receiver to test multiple threads inserting data using block generator */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index ef4873de2f5a9..c090eaec2928d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -96,7 +96,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data @@ -120,7 +120,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 42fad769f0c1a..b63b37d9f9cef 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -228,7 +228,8 @@ class ReceivedBlockTrackerSuite * Get all the data written in the given write ahead log files. By default, it will read all * files in the test log directory. */ - def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles): Seq[ReceivedBlockTrackerLogEvent] = { + def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles) + : Seq[ReceivedBlockTrackerLogEvent] = { logFiles.flatMap { file => new WriteAheadLogReader(file, hadoopConf).toSeq }.map { byteBuffer => @@ -244,7 +245,8 @@ class ReceivedBlockTrackerSuite } /** Create batch allocation object from the given info */ - def createBatchAllocation(time: Long, blockInfos: Seq[ReceivedBlockInfo]): BatchAllocationEvent = { + def createBatchAllocation(time: Long, blockInfos: Seq[ReceivedBlockInfo]) + : BatchAllocationEvent = { BatchAllocationEvent(time, AllocatedBlocks(Map((streamId -> blockInfos)))) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index aa20ad0b5374e..10c35cba8dc53 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -308,7 +308,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { val errors = new ArrayBuffer[Throwable] /** Check if all data structures are clean */ - def isAllEmpty = { + def isAllEmpty: Boolean = { singles.isEmpty && byteBuffers.isEmpty && iterators.isEmpty && arrayBuffers.isEmpty && errors.isEmpty } @@ -320,24 +320,21 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { def pushBytes( bytes: ByteBuffer, optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { byteBuffers += bytes } def pushIterator( iterator: Iterator[_], optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { iterators += iterator } def pushArrayBuffer( arrayBuffer: ArrayBuffer[_], optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { arrayBuffers += arrayBuffer } 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 2e5005ef6ff14..d1bbf39dc7897 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -213,7 +213,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 SlowTestReceiver.receivedAllRecords = false - //Create test receiver that sleeps in onStop() + // Create test receiver that sleeps in onStop() val totalNumRecords = 15 val recordsPerSecond = 1 val input = ssc.receiverStream(new SlowTestReceiver(totalNumRecords, recordsPerSecond)) @@ -370,7 +370,8 @@ object TestReceiver { } /** Custom receiver for testing whether a slow receiver can be shutdown gracefully or not */ -class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { +class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) + extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { var receivingThreadOption: Option[Thread] = None diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index f52562b0a0f73..852e8bb71d4f6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -38,8 +38,8 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { // To make sure that the processing start and end times in collected // information are different for successive batches - override def batchDuration = Milliseconds(100) - override def actuallyWait = true + override def batchDuration: Duration = Milliseconds(100) + override def actuallyWait: Boolean = true test("batch info reporting") { val ssc = setupStreams(input, operation) 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 3565d621e8a6c..c3cae8aeb6d15 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -53,8 +53,9 @@ class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], val selectedInput = if (index < input.size) input(index) else Seq[T]() // lets us test cases where RDDs are not created - if (selectedInput == null) + if (selectedInput == null) { return None + } val rdd = ssc.sc.makeRDD(selectedInput, numPartitions) logInfo("Created RDD " + rdd.id + " with " + selectedInput) @@ -104,7 +105,9 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], output.clear() } - def toTestOutputStream = new TestOutputStream[T](this.parent, this.output.map(_.flatten)) + def toTestOutputStream: TestOutputStream[T] = { + new TestOutputStream[T](this.parent, this.output.map(_.flatten)) + } } /** @@ -148,34 +151,34 @@ class BatchCounter(ssc: StreamingContext) { trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Name of the framework for Spark context - def framework = this.getClass.getSimpleName + def framework: String = this.getClass.getSimpleName // Master for Spark context - def master = "local[2]" + def master: String = "local[2]" // Batch duration - def batchDuration = Seconds(1) + def batchDuration: Duration = Seconds(1) // Directory where the checkpoint data will be saved - lazy val checkpointDir = { + lazy val checkpointDir: String = { val dir = Utils.createTempDir() logDebug(s"checkpointDir: $dir") dir.toString } // Number of partitions of the input parallel collections created for testing - def numInputPartitions = 2 + def numInputPartitions: Int = 2 // Maximum time to wait before the test times out - def maxWaitTimeMillis = 10000 + def maxWaitTimeMillis: Int = 10000 // Whether to use manual clock or not - def useManualClock = true + def useManualClock: Boolean = true // Whether to actually wait in real time before changing manual clock - def actuallyWait = false + def actuallyWait: Boolean = false - //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things. + // A SparkConf to use in tests. Can be modified before calling setupStreams to configure things. val conf = new SparkConf() .setMaster(master) .setAppName(framework) @@ -346,7 +349,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Wait until expected number of output items have been generated val startTime = System.currentTimeMillis() - while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + while (output.size < numExpectedOutput && + System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput) ssc.awaitTerminationOrTimeout(50) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 87a0395efbf2a..998426ebb82e5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -32,7 +32,8 @@ import org.apache.spark._ /** * Selenium tests for the Spark Web UI. */ -class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { +class UISeleniumSuite + extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { implicit var webDriver: WebDriver = _ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index a5d2bb2fde16c..c39ad05f41520 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.storage.StorageLevel class WindowOperationsSuite extends TestSuiteBase { - override def maxWaitTimeMillis = 20000 // large window tests can sometimes take longer + override def maxWaitTimeMillis: Int = 20000 // large window tests can sometimes take longer - override def batchDuration = Seconds(1) // making sure its visible in this class + override def batchDuration: Duration = Seconds(1) // making sure its visible in this class val largerSlideInput = Seq( Seq(("a", 1)), diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index 7a6a2f3e577dd..c3602a5b73732 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -28,10 +28,13 @@ import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBloc import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} import org.apache.spark.util.Utils -class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { +class WriteAheadLogBackedBlockRDDSuite + extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { + val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) + val hadoopConf = new Configuration() var sparkContext: SparkContext = null @@ -86,7 +89,8 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log * @param testStoreInBM Test whether blocks read from log are stored back into block manager */ - private def testRDD(numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { + private def testRDD( + numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { val numBlocks = numPartitionsInBM + numPartitionsInWAL val data = Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50)) @@ -110,7 +114,7 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w "Unexpected blocks in BlockManager" ) - // Make sure that the right `numPartitionsInWAL` blocks are in write ahead logs, and other are not + // Make sure that the right `numPartitionsInWAL` blocks are in WALs, and other are not require( segments.takeRight(numPartitionsInWAL).forall(s => new File(s.path.stripPrefix("file://")).exists()), @@ -152,6 +156,6 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w } private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = { - Array.fill(count)(new WriteAheadLogFileSegment("random", 0l, 0)) + Array.fill(count)(new WriteAheadLogFileSegment("random", 0L, 0)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala index 4150b60635ed6..7865b06c2e3c2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -90,7 +90,7 @@ class JobGeneratorSuite extends TestSuiteBase { val receiverTracker = ssc.scheduler.receiverTracker // Get the blocks belonging to a batch - def getBlocksOfBatch(batchTime: Long) = { + def getBlocksOfBatch(batchTime: Long): Seq[ReceivedBlockInfo] = { receiverTracker.getBlocksOfBatchAndStream(Time(batchTime), inputStream.id) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8335659667f22..a3919c43b95b4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -291,7 +291,7 @@ object WriteAheadLogSuite { manager } - /** Read data from a segments of a log file directly and return the list of byte buffers.*/ + /** Read data from a segments of a log file directly and return the list of byte buffers. */ def readDataManually(segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { segments.map { segment => val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) diff --git a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala index d0bf328f2b74d..d66750463033a 100644 --- a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala @@ -25,7 +25,8 @@ package org.apache.spark.streamingtest */ class ImplicitSuite { - // We only want to test if `implict` works well with the compiler, so we don't need a real DStream. + // We only want to test if `implicit` works well with the compiler, + // so we don't need a real DStream. def mockDStream[T]: org.apache.spark.streaming.dstream.DStream[T] = null def testToPairDStreamFunctions(): Unit = { From f7e21dd1ec4541be54eb01d8b15cfcc6714feed0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 10:14:52 -0700 Subject: [PATCH 670/817] [SPARK-6506] [pyspark] Do not try to retrieve SPARK_HOME when not needed... .... In particular, this makes pyspark in yarn-cluster mode fail unless SPARK_HOME is set, when it's not really needed. Author: Marcelo Vanzin Closes #5405 from vanzin/SPARK-6506 and squashes the following commits: e184507 [Marcelo Vanzin] [SPARK-6506] [pyspark] Do not try to retrieve SPARK_HOME when not needed. --- python/pyspark/java_gateway.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0a16cbd8bff62..2a5e84a7dfdb4 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -29,11 +29,10 @@ def launch_gateway(): - SPARK_HOME = os.environ["SPARK_HOME"] - if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: + SPARK_HOME = os.environ["SPARK_HOME"] # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" From 9d44ddce1d1e19011026605549c37d0db6d6afa1 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 8 Apr 2015 10:26:45 -0700 Subject: [PATCH 671/817] [SPARK-6753] Clone SparkConf in ShuffleSuite tests Prior to this change, the unit test for SPARK-3426 did not clone the original SparkConf, which meant that that test did not use the options set by suites that subclass ShuffleSuite.scala. This commit fixes that problem. JoshRosen would be great if you could take a look at this, since you wrote this test originally. Author: Kay Ousterhout Closes #5401 from kayousterhout/SPARK-6753 and squashes the following commits: 368c540 [Kay Ousterhout] [SPARK-6753] Clone SparkConf in ShuffleSuite tests --- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index f57921b768310..30b6184c77839 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -242,14 +242,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex shuffleSpillCompress <- Set(true, false); shuffleCompress <- Set(true, false) ) { - val conf = new SparkConf() + val myConf = conf.clone() .setAppName("test") .setMaster("local") .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) .set("spark.shuffle.compress", shuffleCompress.toString) .set("spark.shuffle.memoryFraction", "0.001") resetSparkContext() - sc = new SparkContext(conf) + sc = new SparkContext(myConf) try { sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect() } catch { From 8d812f9986f2edf420a18ca822711c9765f480e2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 11:31:48 -0700 Subject: [PATCH 672/817] [SPARK-6765] Fix test code style for graphx. So we can turn style checker on for test code. Author: Reynold Xin Closes #5410 from rxin/test-style-graphx and squashes the following commits: 89e253a [Reynold Xin] [SPARK-6765] Fix test code style for graphx. --- .../org/apache/spark/graphx/GraphSuite.scala | 71 ++++++++++--------- .../spark/graphx/LocalSparkContext.scala | 2 +- .../apache/spark/graphx/VertexRDDSuite.scala | 26 +++---- .../graphx/lib/ConnectedComponentsSuite.scala | 18 ++--- .../spark/graphx/lib/PageRankSuite.scala | 33 ++++----- .../StronglyConnectedComponentsSuite.scala | 23 +++--- 6 files changed, 88 insertions(+), 85 deletions(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 8d15150458d26..a570e4ed75fc3 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -38,12 +38,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { val doubleRing = ring ++ ring val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) assert(graph.edges.count() === doubleRing.size) - assert(graph.edges.collect.forall(e => e.attr == 1)) + assert(graph.edges.collect().forall(e => e.attr == 1)) // uniqueEdges option should uniquify edges and store duplicate count in edge attributes val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) assert(uniqueGraph.edges.count() === ring.size) - assert(uniqueGraph.edges.collect.forall(e => e.attr == 2)) + assert(uniqueGraph.edges.collect().forall(e => e.attr == 2)) } } @@ -64,7 +64,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.collect().map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } @@ -75,15 +75,17 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === - (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) + assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect().toSet + === (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) } } test("partitionBy") { withSpark { sc => - def mkGraph(edges: List[(Long, Long)]) = Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) - def nonemptyParts(graph: Graph[Int, Int]) = { + def mkGraph(edges: List[(Long, Long)]): Graph[Int, Int] = { + Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + } + def nonemptyParts(graph: Graph[Int, Int]): RDD[List[Edge[Int]]] = { graph.edges.partitionsRDD.mapPartitions { iter => Iterator(iter.next()._2.iterator.toList) }.filter(_.nonEmpty) @@ -102,7 +104,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1) // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into // the same partition - assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) + assert( + nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) // partitionBy(EdgePartition2D) puts identical edges in the same partition assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1) @@ -140,10 +143,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val g = Graph( sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))), sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2)) - assert(g.triplets.collect.map(_.toTuple).toSet === + assert(g.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) val gPart = g.partitionBy(EdgePartition2D) - assert(gPart.triplets.collect.map(_.toTuple).toSet === + assert(gPart.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) } } @@ -154,10 +157,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = starGraph(sc, n) // mapVertices preserving type val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") - assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) + assert(mappedVAttrs.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) // mapVertices changing type val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) - assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) + assert(mappedVAttrs2.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -177,12 +180,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { // Trigger initial vertex replication graph0.triplets.foreach(x => {}) // Change type of replicated vertices, but preserve erased type - val graph1 = graph0.mapVertices { - case (vid, integerOpt) => integerOpt.map((x: java.lang.Integer) => (x.toDouble): java.lang.Double) + val graph1 = graph0.mapVertices { case (vid, integerOpt) => + integerOpt.map((x: java.lang.Integer) => x.toDouble: java.lang.Double) } // Access replicated vertices, exposing the erased type val graph2 = graph1.mapTriplets(t => t.srcAttr.get) - assert(graph2.edges.map(_.attr).collect.toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) + assert(graph2.edges.map(_.attr).collect().toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) } } @@ -202,7 +205,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect.toSet === + assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect().toSet === (1L to n).map(x => Edge(0, x, "vv")).toSet) } } @@ -211,7 +214,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) + assert(star.reverse.outDegrees.collect().toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -221,7 +224,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) val graph = Graph(vertices, edges).reverse val result = graph.mapReduceTriplets[Int](et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(result.collect.toSet === Set((1L, 2))) + assert(result.collect().toSet === Set((1L, 2))) } } @@ -237,7 +240,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) // And 4 edges. - assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) + assert(subgraph.edges.map(_.copy()).collect().toSet === + (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } @@ -273,9 +277,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 to n).flatMap(x => List((0: VertexId, x: VertexId), (0: VertexId, x: VertexId))), 1), "v") val star2 = doubleStar.groupEdges { (a, b) => a} - assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === - star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) - assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) + assert(star2.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect().toSet === star.vertices.collect().toSet) } } @@ -300,21 +304,23 @@ class GraphSuite extends FunSuite with LocalSparkContext { throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) } Iterator((et.srcId, 1)) - }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect().toSet assert(numEvenNeighbors === (1 to n).map(x => (x: VertexId, n / 2)).toSet) // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x+1) % n: VertexId)), 3) + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x + 1) % n: VertexId)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() - val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => + newOpt.getOrElse(old) + } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set if (et.srcId % 2 != 1) { throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) } Iterator((et.dstId, 1)) - }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect().toSet assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexId, 1)).toSet) } @@ -340,17 +346,18 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type - val reverseStarDegrees = - reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { + (vid, a, bOpt) => bOpt.getOrElse(0) + } val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), - (a: Int, b: Int) => a + b).collect.toSet + (a: Int, b: Int) => a + b).collect().toSet assert(neighborDegreeSums === Set((0: VertexId, n)) ++ (1 to n).map(x => (x: VertexId, 0))) // outerJoinVertices preserving type val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } val newReverseStar = reverseStar.outerJoinVertices(messages) { (vid, a, bOpt) => a + bOpt.getOrElse("") } - assert(newReverseStar.vertices.map(_._2).collect.toSet === + assert(newReverseStar.vertices.map(_._2).collect().toSet === (0 to n).map(x => "v%d".format(x)).toSet) } } @@ -361,7 +368,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) val graph = Graph(verts, edges) val triplets = graph.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)) - .collect.toSet + .collect().toSet assert(triplets === Set((1: VertexId, 2: VertexId, "a", "b"), (2: VertexId, 1: VertexId, "b", "a"))) } @@ -417,7 +424,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val neighborAttrSums = graph.mapReduceTriplets[Int]( et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) + assert(neighborAttrSums.collect().toSet === Set((0: VertexId, n))) } finally { sc.stop() } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index a3e28efc75a98..d2ad9be555770 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkContext */ trait LocalSparkContext { /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ - def withSpark[T](f: SparkContext => T) = { + def withSpark[T](f: SparkContext => T): T = { val conf = new SparkConf() GraphXUtils.registerKryoClasses(conf) val sc = new SparkContext("local", "test", conf) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index c9443d11c76cf..d0a7198d691d7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { - def vertices(sc: SparkContext, n: Int) = { + private def vertices(sc: SparkContext, n: Int) = { VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) } @@ -52,7 +52,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() val vertexB = VertexRDD(sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1))).cache() val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) } } @@ -62,7 +62,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB: RDD[(VertexId, Int)] = sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1)).cache() val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) } } @@ -72,7 +72,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB = VertexRDD(sc.parallelize(50 until 100, 2).map(i => (i.toLong, 1))) assert(vertexA.partitions.size != vertexB.partitions.size) val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 50).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 50).toSet) } } @@ -106,7 +106,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) assert(vertexA.partitions.size != vertexB.partitions.size) val vertexC = vertexA.diff(vertexB) - assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + assert(vertexC.map(_._1).collect().toSet === (8 until 16).toSet) } } @@ -116,11 +116,11 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD - assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) // leftJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) } } @@ -134,7 +134,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => old - newOpt.getOrElse(0) } - assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect().toSet == (1 to 99 by 2).toSet) } } @@ -144,11 +144,11 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD - assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) // innerJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } @@ -161,7 +161,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => old - newVal } - assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect().toSet == (0 to 98 by 2).toSet) } } @@ -171,7 +171,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n) val messageTargets = (0 to n) ++ (0 to n by 2) val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) - assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect.toSet === + assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect().toSet === (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) } } @@ -183,7 +183,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) // test merge function - assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + assert(rdd.collect().toSet == Set((0L, 0), (1L, 3), (2L, 9))) } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 3915be15b3434..4cc30a96408f8 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -32,7 +32,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -42,7 +42,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -50,8 +50,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) val ccGraph = twoChains.connectedComponents() @@ -73,12 +73,12 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse val ccGraph = twoChains.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if (id < 10) { assert(cc === 0) @@ -120,9 +120,9 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { // Build the initial Graph val graph = Graph(users, relationships, defaultUser) val ccGraph = graph.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { - assert(cc == 0) + assert(cc === 0) } } } // end of toy connected components diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index fc491ae327c2a..95804b07b1db0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -19,15 +19,12 @@ package org.apache.spark.graphx.lib import org.scalatest.FunSuite -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ + object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double): Seq[(VertexId, Double)] = { val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) val outDegree = Array.fill(nRows * nCols)(0) // Convert row column address into vertex ids (row major order) @@ -35,13 +32,13 @@ object GridPageRank { // Make the grid graph for (r <- 0 until nRows; c <- 0 until nCols) { val ind = sub2ind(r,c) - if (r+1 < nRows) { + if (r + 1 < nRows) { outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind + inNbrs(sub2ind(r + 1,c)) += ind } - if (c+1 < nCols) { + if (c + 1 < nCols) { outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind + inNbrs(sub2ind(r,c + 1)) += ind } } // compute the pagerank @@ -64,7 +61,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum + .map { case (id, error) => error }.sum() } test("Star PageRank") { @@ -80,12 +77,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum + }.map { case (vid, test) => test }.sum() assert(notMatching === 0) val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + val p = math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) + val correct = (vid > 0 && pr == resetProb) || (vid == 0L && p < 1.0E-5) if (!correct) 1 else 0 } assert(staticErrors.sum === 0) @@ -95,8 +92,6 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } } // end of test Star PageRank - - test("Grid PageRank") { withSpark { sc => val rows = 10 @@ -109,18 +104,18 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() + val referenceRanks = VertexRDD( + sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) } } // end of Grid PageRank - test("Chain PageRank") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() val resetProb = 0.15 val tol = 0.0001 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala index df54aa37cad68..1f658c371ffcf 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -34,8 +34,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) val sccGraph = graph.stronglyConnectedComponents(5) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(id === scc) } } } @@ -45,8 +45,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(0L == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(0L === scc) } } } @@ -60,13 +60,14 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - if (id < 3) - assert(0L == scc) - else if (id < 6) - assert(3L == scc) - else - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + if (id < 3) { + assert(0L === scc) + } else if (id < 6) { + assert(3L === scc) + } else { + assert(id === scc) + } } } } From 66159c35010af35098dd1ec75475bb5d4d0fd6ca Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 11:32:44 -0700 Subject: [PATCH 673/817] [SPARK-6765] Fix test code style for mllib. So we can turn style checker on for test code. Author: Reynold Xin Closes #5411 from rxin/test-style-mllib and squashes the following commits: d8a2569 [Reynold Xin] [SPARK-6765] Fix test code style for mllib. --- .../org/apache/spark/ml/param/TestParams.scala | 2 +- .../mllib/classification/NaiveBayesSuite.scala | 2 +- .../StreamingLogisticRegressionSuite.scala | 2 +- .../spark/mllib/clustering/KMeansSuite.scala | 10 +++++++--- .../spark/mllib/clustering/LDASuite.scala | 4 ++-- .../clustering/StreamingKMeansSuite.scala | 4 ++-- .../spark/mllib/random/RandomRDDsSuite.scala | 5 +++-- .../spark/mllib/recommendation/ALSSuite.scala | 3 +++ .../regression/RidgeRegressionSuite.scala | 2 +- .../StreamingLinearRegressionSuite.scala | 2 +- .../apache/spark/mllib/util/TestingUtils.scala | 14 +++++++------- .../spark/mllib/util/TestingUtilsSuite.scala | 18 +++++++++++------- 12 files changed, 40 insertions(+), 28 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 1a65883d78a71..ce52f2f230085 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -28,7 +28,7 @@ class TestParams extends Params { def setInputCol(value: String): this.type = { set(inputCol, value); this } def getInputCol: String = get(inputCol) - override def validate(paramMap: ParamMap) = { + override def validate(paramMap: ParamMap): Unit = { val m = this.paramMap ++ paramMap require(m(maxIter) >= 0) require(m.contains(inputCol)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index f9fe3e006ccb8..ea89b17b7c08f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -102,7 +102,7 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { def validateModelFit( piData: Array[Double], thetaData: Array[Array[Double]], - model: NaiveBayesModel) = { + model: NaiveBayesModel): Unit = { def closeFit(d1: Double, d2: Double, precision: Double): Boolean = { (d1 - d2).abs <= precision } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index d50c43d439187..5683b55e8500a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 // Test if we can accurately learn B for Y = logistic(BX) on streaming data test("parameter accuracy") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 7bf250eb5a383..0f2b26d462ad2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -199,9 +199,13 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext { test("k-means|| initialization") { case class VectorWithCompare(x: Vector) extends Ordered[VectorWithCompare] { - @Override def compare(that: VectorWithCompare): Int = { - if(this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > - that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) -1 else 1 + override def compare(that: VectorWithCompare): Int = { + if (this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > + that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) { + -1 + } else { + 1 + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 302d751eb8a94..15de10fd13a19 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} +import org.apache.spark.mllib.linalg.{Vector, DenseMatrix, Matrix, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -141,7 +141,7 @@ private[clustering] object LDASuite { (terms.toArray, termWeights.toArray) } - def tinyCorpus = Array( + def tinyCorpus: Array[(Long, Vector)] = Array( Vectors.dense(1, 3, 0, 2, 8), Vectors.dense(0, 2, 1, 0, 4), Vectors.dense(2, 3, 12, 3, 1), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index 850c9fce507cd..f90025d535e45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.random.XORShiftRandom class StreamingKMeansSuite extends FunSuite with TestSuiteBase { - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 test("accuracy for single center and equivalence to grand average") { // set parameters @@ -59,7 +59,7 @@ class StreamingKMeansSuite extends FunSuite with TestSuiteBase { // estimated center from streaming should exactly match the arithmetic mean of all data points // because the decay factor is set to 1.0 val grandMean = - input.flatten.map(x => x.toBreeze).reduce(_+_) / (numBatches * numPoints).toDouble + input.flatten.map(x => x.toBreeze).reduce(_ + _) / (numBatches * numPoints).toDouble assert(model.latestModel().clusterCenters(0) ~== Vectors.dense(grandMean.toArray) absTol 1E-5) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 6395188a0842a..63f2ea916d457 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -181,7 +181,8 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) - val exponential = RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) + val exponential = + RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) testGeneratedVectorRDD(exponential, rows, cols, parts, exponentialMean, exponentialMean, 0.1) val gamma = RandomRDDs.gammaVectorRDD(sc, gammaShape, gammaScale, rows, cols, parts, seed) @@ -197,7 +198,7 @@ private[random] class MockDistro extends RandomDataGenerator[Double] { // This allows us to check that each partition has a different seed override def nextValue(): Double = seed.toDouble - override def setSeed(seed: Long) = this.seed = seed + override def setSeed(seed: Long): Unit = this.seed = seed override def copy(): MockDistro = new MockDistro } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 8775c0ca9df84..b3798940ddc38 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -203,6 +203,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { * @param numProductBlocks number of product blocks to partition products into * @param negativeFactors whether the generated user/product factors can have negative entries */ + // scalastyle:off def testALS( users: Int, products: Int, @@ -216,6 +217,8 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { numUserBlocks: Int = -1, numProductBlocks: Int = -1, negativeFactors: Boolean = true) { + // scalastyle:on + val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights, negativeFactors) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 43d61151e2471..d6c93cc0e49cd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -35,7 +35,7 @@ private object RidgeRegressionSuite { class RidgeRegressionSuite extends FunSuite with MLlibTestSparkContext { - def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { + def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = { predictions.zip(input).map { case (prediction, expected) => (prediction - expected.label) * (prediction - expected.label) }.reduceLeft(_ + _) / predictions.size diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 24fd8df691817..26604dbe6c1ef 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 20000 + override def maxWaitTimeMillis: Int = 20000 // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index e957fa5d25f4c..352193a67860c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -95,16 +95,16 @@ object TestingUtils { /** * Comparison using absolute tolerance. */ - def absTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(AbsoluteErrorComparison, - x, eps, ABS_TOL_MSG) + def absTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(AbsoluteErrorComparison, x, eps, ABS_TOL_MSG) /** * Comparison using relative tolerance. */ - def relTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(RelativeErrorComparison, - x, eps, REL_TOL_MSG) + def relTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(RelativeErrorComparison, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareVectorRightSide( @@ -166,7 +166,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareMatrixRightSide( @@ -229,7 +229,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index b0ecb33c28483..59e6c778806f4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -88,16 +88,20 @@ class TestingUtilsSuite extends FunSuite { assert(!(17.8 ~= 17.59 absTol 0.2)) // Comparisons of numbers very close to zero, and both side of zeros - assert(Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - - assert(-Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert( + -Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) } test("Comparing vectors using relative error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) @@ -130,7 +134,7 @@ class TestingUtilsSuite extends FunSuite { test("Comparing vectors using absolute error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) From 6ada4f6f52cf1d992c7ab0c32318790cf08b0a0d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 8 Apr 2015 13:31:45 -0700 Subject: [PATCH 674/817] [SPARK-6781] [SQL] use sqlContext in python shell Use `sqlContext` in PySpark shell, make it consistent with SQL programming guide. `sqlCtx` is also kept for compatibility. Author: Davies Liu Closes #5425 from davies/sqlCtx and squashes the following commits: af67340 [Davies Liu] sqlCtx -> sqlContext 15a278f [Davies Liu] use sqlContext in python shell --- docs/ml-guide.md | 2 +- docs/sql-programming-guide.md | 4 +- .../spark/examples/sql/JavaSparkSQL.java | 20 ++--- .../ml/simple_text_classification_pipeline.py | 2 +- .../src/main/python/mllib/dataset_example.py | 6 +- python/pyspark/ml/classification.py | 4 +- python/pyspark/ml/feature.py | 4 +- python/pyspark/shell.py | 6 +- python/pyspark/sql/context.py | 79 +++++++++---------- python/pyspark/sql/dataframe.py | 6 +- python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/types.py | 4 +- 12 files changed, 69 insertions(+), 70 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index c08c76d226713..771a07183e26f 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -493,7 +493,7 @@ from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.sql import Row, SQLContext sc = SparkContext(appName="SimpleTextClassificationPipeline") -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4441d6a000a02..663f656883721 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1642,7 +1642,7 @@ moved into the udf object in `SQLContext`.
    {% highlight java %} -sqlCtx.udf.register("strLen", (s: String) => s.length()) +sqlContext.udf.register("strLen", (s: String) => s.length()) {% endhighlight %}
    @@ -1650,7 +1650,7 @@ sqlCtx.udf.register("strLen", (s: String) => s.length())
    {% highlight java %} -sqlCtx.udf().register("strLen", (String s) -> { s.length(); }); +sqlContext.udf().register("strLen", (String s) -> { s.length(); }); {% endhighlight %}
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index dee794840a3e1..8159ffbe2d269 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -55,7 +55,7 @@ public void setAge(int age) { public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); JavaSparkContext ctx = new JavaSparkContext(sparkConf); - SQLContext sqlCtx = new SQLContext(ctx); + SQLContext sqlContext = new SQLContext(ctx); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -74,11 +74,11 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - DataFrame schemaPeople = sqlCtx.createDataFrame(people, Person.class); + DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - DataFrame teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -99,12 +99,12 @@ public String call(Row row) { // Read in the parquet file created above. // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a DataFrame. - DataFrame parquetFile = sqlCtx.parquetFile("people.parquet"); + DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers2 = - sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -120,7 +120,7 @@ public String call(Row row) { // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a DataFrame from the file(s) pointed by path - DataFrame peopleFromJsonFile = sqlCtx.jsonFile(path); + DataFrame peopleFromJsonFile = sqlContext.jsonFile(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. @@ -133,8 +133,8 @@ public String call(Row row) { // Register this DataFrame as a table. peopleFromJsonFile.registerTempTable("people"); - // SQL statements can be run by using the sql methods provided by sqlCtx. - DataFrame teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + // SQL statements can be run by using the sql methods provided by sqlContext. + DataFrame teenagers3 = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrame and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -151,7 +151,7 @@ public String call(Row row) { List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - DataFrame peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd()); + DataFrame peopleFromJsonRDD = sqlContext.jsonRDD(anotherPeopleRDD.rdd()); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -164,7 +164,7 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - DataFrame peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); + DataFrame peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index d281f4fa44282..c73edb7fd6b20 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -33,7 +33,7 @@ if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index b5a70db2b9a3c..fcbf56cbf0c52 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -44,19 +44,19 @@ def summarize(dataset): print >> sys.stderr, "Usage: dataset_example.py " exit(-1) sc = SparkContext(appName="DatasetExample") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) if len(sys.argv) == 2: input = sys.argv[1] else: input = "data/mllib/sample_libsvm_data.txt" points = MLUtils.loadLibSVMFile(sc, input) - dataset0 = sqlCtx.inferSchema(points).setName("dataset0").cache() + dataset0 = sqlContext.inferSchema(points).setName("dataset0").cache() summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) print "Save dataset as a Parquet file to %s." % tempdir dataset0.saveAsParquetFile(tempdir) print "Load it back and summarize it again." - dataset1 = sqlCtx.parquetFile(tempdir).setName("dataset1").cache() + dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4ff7463498cce..7f42de531f3b4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -91,9 +91,9 @@ class LogisticRegressionModel(JavaModel): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 433b4fb5d22bf..1cfcd019dfb18 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -117,9 +117,9 @@ def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 1a02fece9c5a5..81aa970a32f76 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -53,9 +53,9 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = HiveContext(sc) + sqlCtx = sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = SQLContext(sc) + sqlCtx = sqlContext = SQLContext(sc) print("""Welcome to ____ __ @@ -68,7 +68,7 @@ platform.python_version(), platform.python_build()[0], platform.python_build()[1])) -print("SparkContext available as sc, %s available as sqlCtx." % sqlCtx.__class__.__name__) +print("SparkContext available as sc, %s available as sqlContext." % sqlContext.__class__.__name__) if add_files is not None: print("Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead") diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index c2d81ba804110..93e2d176a5b6f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -37,12 +37,12 @@ __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] -def _monkey_patch_RDD(sqlCtx): +def _monkey_patch_RDD(sqlContext): def toDF(self, schema=None, sampleRatio=None): """ Converts current :class:`RDD` into a :class:`DataFrame` - This is a shorthand for ``sqlCtx.createDataFrame(rdd, schema, sampleRatio)`` + This is a shorthand for ``sqlContext.createDataFrame(rdd, schema, sampleRatio)`` :param schema: a StructType or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring @@ -51,7 +51,7 @@ def toDF(self, schema=None, sampleRatio=None): >>> rdd.toDF().collect() [Row(name=u'Alice', age=1)] """ - return sqlCtx.createDataFrame(self, schema, sampleRatio) + return sqlContext.createDataFrame(self, schema, sampleRatio) RDD.toDF = toDF @@ -75,13 +75,13 @@ def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. >>> from datetime import datetime - >>> sqlCtx = SQLContext(sc) + >>> sqlContext = SQLContext(sc) >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") - >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, @@ -133,18 +133,18 @@ def registerFunction(self, name, f, returnType=StringType()): :param samplingRatio: lambda function :param returnType: a :class:`DataType` object - >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) - >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlContext.sql("SELECT stringLengthString('test')").collect() [Row(c0=u'4')] >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + >>> sqlContext.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + >>> sqlContext.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) @@ -229,26 +229,26 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): :param samplingRatio: the sample ratio of rows used for inferring >>> l = [('Alice', 1)] - >>> sqlCtx.createDataFrame(l).collect() + >>> sqlContext.createDataFrame(l).collect() [Row(_1=u'Alice', _2=1)] - >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect() + >>> sqlContext.createDataFrame(l, ['name', 'age']).collect() [Row(name=u'Alice', age=1)] >>> d = [{'name': 'Alice', 'age': 1}] - >>> sqlCtx.createDataFrame(d).collect() + >>> sqlContext.createDataFrame(d).collect() [Row(age=1, name=u'Alice')] >>> rdd = sc.parallelize(l) - >>> sqlCtx.createDataFrame(rdd).collect() + >>> sqlContext.createDataFrame(rdd).collect() [Row(_1=u'Alice', _2=1)] - >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) + >>> df = sqlContext.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name=u'Alice', age=1)] >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) - >>> df2 = sqlCtx.createDataFrame(person) + >>> df2 = sqlContext.createDataFrame(person) >>> df2.collect() [Row(name=u'Alice', age=1)] @@ -256,11 +256,11 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): >>> schema = StructType([ ... StructField("name", StringType(), True), ... StructField("age", IntegerType(), True)]) - >>> df3 = sqlCtx.createDataFrame(rdd, schema) + >>> df3 = sqlContext.createDataFrame(rdd, schema) >>> df3.collect() [Row(name=u'Alice', age=1)] - >>> sqlCtx.createDataFrame(df.toPandas()).collect() # doctest: +SKIP + >>> sqlContext.createDataFrame(df.toPandas()).collect() # doctest: +SKIP [Row(name=u'Alice', age=1)] """ if isinstance(data, DataFrame): @@ -316,7 +316,7 @@ def registerDataFrameAsTable(self, df, tableName): Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") """ if (df.__class__ is DataFrame): self._ssql_ctx.registerDataFrameAsTable(df._jdf, tableName) @@ -330,7 +330,7 @@ def parquetFile(self, *paths): >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -352,7 +352,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): >>> shutil.rmtree(jsonFile) >>> with open(jsonFile, 'w') as f: ... f.writelines(jsonStrings) - >>> df1 = sqlCtx.jsonFile(jsonFile) + >>> df1 = sqlContext.jsonFile(jsonFile) >>> df1.printSchema() root |-- field1: long (nullable = true) @@ -365,7 +365,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): ... StructField("field2", StringType()), ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))]))]) - >>> df2 = sqlCtx.jsonFile(jsonFile, schema) + >>> df2 = sqlContext.jsonFile(jsonFile, schema) >>> df2.printSchema() root |-- field2: string (nullable = true) @@ -386,11 +386,11 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): If the schema is provided, applies the given schema to this JSON dataset. Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. - >>> df1 = sqlCtx.jsonRDD(json) + >>> df1 = sqlContext.jsonRDD(json) >>> df1.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) - >>> df2 = sqlCtx.jsonRDD(json, df1.schema) + >>> df2 = sqlContext.jsonRDD(json, df1.schema) >>> df2.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) @@ -400,7 +400,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))])) ... ]) - >>> df3 = sqlCtx.jsonRDD(json, schema) + >>> df3 = sqlContext.jsonRDD(json, schema) >>> df3.first() Row(field2=u'row1', field3=Row(field5=None)) """ @@ -480,8 +480,8 @@ def createExternalTable(self, tableName, path=None, source=None, def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ @@ -490,8 +490,8 @@ def sql(self, sqlQuery): def table(self, tableName): """Returns the specified table as a :class:`DataFrame`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.table("table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -505,8 +505,8 @@ def tables(self, dbName=None): The returned DataFrame has two columns: ``tableName`` and ``isTemporary`` (a column with :class:`BooleanType` indicating if a table is a temporary one or not). - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.tables() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.tables() >>> df2.filter("tableName = 'table1'").first() Row(tableName=u'table1', isTemporary=True) """ @@ -520,10 +520,10 @@ def tableNames(self, dbName=None): If ``dbName`` is not specified, the current database will be used. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> "table1" in sqlCtx.tableNames() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> "table1" in sqlContext.tableNames() True - >>> "table1" in sqlCtx.tableNames("db") + >>> "table1" in sqlContext.tableNames("db") True """ if dbName is None: @@ -578,11 +578,11 @@ def _get_hive_ctx(self): class UDFRegistration(object): """Wrapper for user-defined function registration.""" - def __init__(self, sqlCtx): - self.sqlCtx = sqlCtx + def __init__(self, sqlContext): + self.sqlContext = sqlContext def register(self, name, f, returnType=StringType()): - return self.sqlCtx.registerFunction(name, f, returnType) + return self.sqlContext.registerFunction(name, f, returnType) register.__doc__ = SQLContext.registerFunction.__doc__ @@ -595,13 +595,12 @@ def _test(): globs = pyspark.sql.context.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['rdd'] = rdd = sc.parallelize( [Row(field1=1, field2="row1"), Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) - _monkey_patch_RDD(sqlCtx) globs['df'] = rdd.toDF() jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c30326ebd133e..ef91a9c4f522d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -110,7 +110,7 @@ def saveAsParquetFile(self, path): >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df2.collect()) == sorted(df.collect()) True """ @@ -123,7 +123,7 @@ def registerTempTable(self, name): that was used to create this :class:`DataFrame`. >>> df.registerTempTable("people") - >>> df2 = sqlCtx.sql("select * from people") + >>> df2 = sqlContext.sql("select * from people") >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -1180,7 +1180,7 @@ def _test(): globs = pyspark.sql.dataframe.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 146ba6f3e0d98..daeb6916b58bc 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -161,7 +161,7 @@ def _test(): globs = pyspark.sql.functions.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 45eb8b945dcb0..7e0124b13671b 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -434,7 +434,7 @@ def _parse_datatype_json_string(json_string): >>> def check_datatype(datatype): ... pickled = pickle.loads(pickle.dumps(datatype)) ... assert datatype == pickled - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) + ... scala_datatype = sqlContext._ssql_ctx.parseDataType(datatype.json()) ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) ... assert datatype == python_datatype >>> for cls in _all_primitive_types.values(): @@ -1237,7 +1237,7 @@ def _test(): globs = pyspark.sql.types.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['ExamplePoint'] = ExamplePoint globs['ExamplePointUDT'] = ExamplePointUDT (failure_count, test_count) = doctest.testmod( From 2f482d706b9d38820472c3152dbd1612c98729bd Mon Sep 17 00:00:00 2001 From: Tijo Thomas Date: Wed, 8 Apr 2015 13:42:29 -0700 Subject: [PATCH 675/817] [SPARK-6767][SQL] Fixed Query DSL error in spark sql Readme Fixed the following error query.where('key > 30).select(avg('key)).collect() :43: error: value > is not a member of Symbol query.where('key > 30).select(avg('key)).collect() Author: Tijo Thomas Closes #5415 from tijoparacka/ERROR_SQL_DATAFRAME_EXAMPLE and squashes the following commits: 234751e [Tijo Thomas] Fixed Query DSL error in spark sql Readme --- sql/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/README.md b/sql/README.md index fbb3200a3a4b4..237620e3fa808 100644 --- a/sql/README.md +++ b/sql/README.md @@ -56,6 +56,6 @@ res2: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,v You can also build further queries on top of these `DataFrames` using the query DSL. ``` -scala> query.where('key > 30).select(avg('key)).collect() +scala> query.where(query("key") > 30).select(avg(query("key"))).collect() res3: Array[org.apache.spark.sql.Row] = Array([274.79025423728814]) ``` From 86403f5525782bc9656ab11790f7020baa6b2c1f Mon Sep 17 00:00:00 2001 From: Michelangelo D'Agostino Date: Wed, 8 Apr 2015 16:48:45 -0400 Subject: [PATCH 676/817] [SPARK-5242]: Add --private-ips flag to EC2 script The `spark_ec2.py` script currently references the `ip_address` and `public_dns_name` attributes of an instance. On private networks, these fields aren't set, so we have problems. This PR introduces a `--private-ips` flag that instead refers to the `private_ip_address` attribute in both cases. Author: Michelangelo D'Agostino Closes #5244 from mdagost/ec2_private_nets and squashes the following commits: b684c67 [Michelangelo D'Agostino] STY: A few python lint changes. a4a2eac [Michelangelo D'Agostino] ENH: Fix IP's typo and refactor conditional logic into functions. c004604 [Michelangelo D'Agostino] ENH: Add --private-ips flag. --- ec2/spark_ec2.py | 64 +++++++++++++++++++++++++++++++++++------------- 1 file changed, 47 insertions(+), 17 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 879a52cef8ff0..0c1f24761d0de 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -282,6 +282,10 @@ def parse_args(): parser.add_option( "--vpc-id", default=None, help="VPC to launch instances in") + parser.add_option( + "--private-ips", action="store_true", default=False, + help="Use private IPs for instances rather than public if VPC/subnet " + + "requires that.") (opts, args) = parser.parse_args() if len(args) != 2: @@ -707,7 +711,7 @@ def get_instances(group_names): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): - master = master_nodes[0].public_dns_name + master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: print "Generating cluster's SSH key on master..." key_setup = """ @@ -719,8 +723,9 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) print "Transferring cluster's SSH key to slaves..." for slave in slave_nodes: - print slave.public_dns_name - ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) + slave_address = get_dns_name(slave, opts.private_ips) + print slave_address + ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone', 'tachyon'] @@ -809,7 +814,8 @@ def is_cluster_ssh_available(cluster_instances, opts): Check if SSH is available on all the instances in a cluster. """ for i in cluster_instances: - if not is_ssh_available(host=i.public_dns_name, opts=opts): + dns_name = get_dns_name(i, opts.private_ips) + if not is_ssh_available(host=dns_name, opts=opts): return False else: return True @@ -923,7 +929,7 @@ def get_num_disks(instance_type): # # root_dir should be an absolute path to the directory with the files we want to deploy. def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) num_disks = get_num_disks(opts.instance_type) hdfs_data_dirs = "/mnt/ephemeral-hdfs/data" @@ -948,10 +954,12 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): print "Deploying Spark via git hash; Tachyon won't be set up" modules = filter(lambda x: x != "tachyon", modules) + master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] + slave_addresses = [get_dns_name(i, opts.private_ips) for i in slave_nodes] template_vars = { - "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), + "master_list": '\n'.join(master_addresses), "active_master": active_master, - "slave_list": '\n'.join([i.public_dns_name for i in slave_nodes]), + "slave_list": '\n'.join(slave_addresses), "cluster_url": cluster_url, "hdfs_data_dirs": hdfs_data_dirs, "mapred_local_dirs": mapred_local_dirs, @@ -1011,7 +1019,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # # root_dir should be an absolute path. def deploy_user_files(root_dir, opts, master_nodes): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) command = [ 'rsync', '-rv', '-e', stringify_command(ssh_command(opts)), @@ -1122,6 +1130,20 @@ def get_partition(total, num_partitions, current_partitions): return num_slaves_this_zone +# Gets the IP address, taking into account the --private-ips flag +def get_ip_address(instance, private_ips=False): + ip = instance.ip_address if not private_ips else \ + instance.private_ip_address + return ip + + +# Gets the DNS name, taking into account the --private-ips flag +def get_dns_name(instance, private_ips=False): + dns = instance.public_dns_name if not private_ips else \ + instance.private_ip_address + return dns + + def real_main(): (opts, action, cluster_name) = parse_args() @@ -1230,7 +1252,7 @@ def real_main(): if any(master_nodes + slave_nodes): print "The following instances will be terminated:" for inst in master_nodes + slave_nodes: - print "> %s" % inst.public_dns_name + print "> %s" % get_dns_name(inst, opts.private_ips) print "ALL DATA ON ALL NODES WILL BE LOST!!" msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) @@ -1294,13 +1316,17 @@ def real_main(): elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - master = master_nodes[0].public_dns_name - print "Logging into master " + master + "..." - proxy_opt = [] - if opts.proxy_port is not None: - proxy_opt = ['-D', opts.proxy_port] - subprocess.check_call( - ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) + if not master_nodes[0].public_dns_name and not opts.private_ips: + print "Master has no public DNS name. Maybe you meant to specify " \ + "--private-ips?" + else: + master = get_dns_name(master_nodes[0], opts.private_ips) + print "Logging into master " + master + "..." + proxy_opt = [] + if opts.proxy_port is not None: + proxy_opt = ['-D', opts.proxy_port] + subprocess.check_call( + ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) elif action == "reboot-slaves": response = raw_input( @@ -1318,7 +1344,11 @@ def real_main(): elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print master_nodes[0].public_dns_name + if not master_nodes[0].public_dns_name and not opts.private_ips: + print "Master has no public DNS name. Maybe you meant to specify " \ + "--private-ips?" + else: + print get_dns_name(master_nodes[0], opts.private_ips) elif action == "stop": response = raw_input( From 55a92ef34c0b57b6e379523d5d79baa05392de37 Mon Sep 17 00:00:00 2001 From: unknown Date: Wed, 8 Apr 2015 13:56:42 -0700 Subject: [PATCH 677/817] [SPARK-4346][SPARK-3596][YARN] Commonize the monitor logic 1. YarnClientSchedulerBack.asyncMonitorApplication use Client.monitorApplication so that commonize the monitor logic 2. Support changing the yarn client monitor interval, see #5292 3. More details see discussion on https://github.com/apache/spark/pull/3143 Author: unknown Author: Sephiroth-Lin Closes #5305 from Sephiroth-Lin/SPARK-4346_3596 and squashes the following commits: 47c0014 [unknown] Edit conflicts 52b29fe [unknown] Interrupt thread when we call stop() d4298a1 [unknown] Unused, don't push aaacb42 [Sephiroth-Lin] don't wrap the entire block in the try ee2b2fd [Sephiroth-Lin] update 6483a2a [unknown] Catch exception 6b47ff7 [unknown] Update code 568f46f [unknown] YarnClientSchedulerBack.asyncMonitorApplication should be common with Client.monitorApplication --- .../org/apache/spark/deploy/yarn/Client.scala | 10 +++++- .../cluster/YarnClientSchedulerBackend.scala | 32 ++++++------------- 2 files changed, 18 insertions(+), 24 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 79d55a09eb671..7219852c0a752 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} @@ -561,7 +562,14 @@ private[spark] class Client( var lastState: YarnApplicationState = null while (true) { Thread.sleep(interval) - val report = getApplicationReport(appId) + val report: ApplicationReport = + try { + getApplicationReport(appId) + } catch { + case e: ApplicationNotFoundException => + logError(s"Application $appId not found.") + return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED) + } val state = report.getYarnApplicationState if (logApplicationReport) { diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 8abdc26b43806..407dc1ac4d37d 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -34,7 +34,7 @@ private[spark] class YarnClientSchedulerBackend( private var client: Client = null private var appId: ApplicationId = null - @volatile private var stopping: Boolean = false + private var monitorThread: Thread = null /** * Create a Yarn client to submit an application to the ResourceManager. @@ -57,7 +57,8 @@ private[spark] class YarnClientSchedulerBackend( client = new Client(args, conf) appId = client.submitApplication() waitForApplication() - asyncMonitorApplication() + monitorThread = asyncMonitorApplication() + monitorThread.start() } /** @@ -123,34 +124,19 @@ private[spark] class YarnClientSchedulerBackend( * If the application has exited for any reason, stop the SparkContext. * This assumes both `client` and `appId` have already been set. */ - private def asyncMonitorApplication(): Unit = { + private def asyncMonitorApplication(): Thread = { assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - while (!stopping) { - var state: YarnApplicationState = null - try { - val report = client.getApplicationReport(appId) - state = report.getYarnApplicationState() - } catch { - case e: ApplicationNotFoundException => - state = YarnApplicationState.KILLED - } - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.KILLED || - state == YarnApplicationState.FAILED) { - logError(s"Yarn application has already exited with state $state!") - sc.stop() - stopping = true - } - Thread.sleep(1000L) - } + val (state, _) = client.monitorApplication(appId, logApplicationReport = false) + logError(s"Yarn application has already exited with state $state!") + sc.stop() Thread.currentThread().interrupt() } } t.setName("Yarn application state monitor") t.setDaemon(true) - t.start() + t } /** @@ -158,7 +144,7 @@ private[spark] class YarnClientSchedulerBackend( */ override def stop() { assert(client != null, "Attempted to stop this scheduler before starting it!") - stopping = true + monitorThread.interrupt() super.stop() client.stop() logInfo("Stopped") From 9418280547f962eaf309bfff9986cdd848409643 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 Apr 2015 13:57:01 -0700 Subject: [PATCH 678/817] [SQL][minor] remove duplicated resolveGetField and update comment It's after https://github.com/apache/spark/pull/5189 Author: Wenchen Fan Closes #5304 from cloud-fan/tmp and squashes the following commits: c58c9b3 [Wenchen Fan] remove duplicated code and update comment --- .../sql/catalyst/analysis/Analyzer.scala | 32 +------------------ .../catalyst/plans/logical/LogicalPlan.scala | 13 +++----- 2 files changed, 6 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 119cb9c3a4400..b3aba4f68ddf9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -293,7 +293,7 @@ class Analyzer( logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => - resolveGetField(child, fieldName) + q.resolveGetField(child, fieldName, resolver) } } @@ -313,36 +313,6 @@ class Analyzer( */ protected def containsStar(exprs: Seq[Expression]): Boolean = exprs.exists(_.collect { case _: Star => true }.nonEmpty) - - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - protected def resolveGetField(expr: Expression, fieldName: String): Expression = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 2e9f3aa4ec4ad..d8f5858f5033e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -205,11 +205,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => try { - - // The foldLeft adds UnresolvedGetField for every remaining parts of the name, - // and aliased it with the last part of the name. - // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. - // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias + // The foldLeft adds GetFields for every remaining parts of the identifier, + // and aliases it with the last part of the identifier. + // For example, consider "a.b.c", where "a" is resolved to an existing attribute. + // Then this will add GetField("c", GetField("b", a)), and alias // the final expression as "c". val fieldExprs = nestedFields.foldLeft(a: Expression)(resolveGetField(_, _, resolver)) val aliasName = nestedFields.last @@ -234,10 +233,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** * Returns the resolved `GetField`, and report error if no desired field or over one * desired fields are found. - * - * TODO: this code is duplicated from Analyzer and should be refactored to avoid this. */ - protected def resolveGetField( + def resolveGetField( expr: Expression, fieldName: String, resolver: Resolver): Expression = { From 7d7384c781ea72e1eabab3daca2e237e3b0fc666 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Wed, 8 Apr 2015 18:42:34 -0700 Subject: [PATCH 679/817] [SPARK-6451][SQL] supported code generation for CombineSum Author: Venkata Ramana Gollamudi Closes #5138 from gvramana/sum_fix_codegen and squashes the following commits: 95f5fe4 [Venkata Ramana Gollamudi] rebase merge changes 12f45a5 [Venkata Ramana Gollamudi] Combined and added code generations tests as per comment d6a76ac [Venkata Ramana Gollamudi] added support for codegeneration for CombineSum and tests --- .../sql/execution/GeneratedAggregate.scala | 44 ++++++++- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 92 ++++++++++++++++++- 3 files changed, 133 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index a8018b9213f2b..861a2c21ad9a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -99,7 +99,10 @@ case class GeneratedAggregate( // but really, common sub expression elimination would be better.... val zero = Cast(Literal(0), calcType) val updateFunction = Coalesce( - Add(Coalesce(currentSum :: zero :: Nil), Cast(expr, calcType)) :: currentSum :: Nil) + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType) + ) :: currentSum :: zero :: Nil) val result = expr.dataType match { case DecimalType.Fixed(_, _) => @@ -109,6 +112,45 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + case cs @ CombineSum(expr) => + val calcType = expr.dataType + expr.dataType match { + case DecimalType.Fixed(_, _) => + DecimalType.Unlimited + case _ => + expr.dataType + } + + val currentSum = AttributeReference("currentSum", calcType, nullable = true)() + val initialValue = Literal.create(null, calcType) + + // Coalasce avoids double calculation... + // but really, common sub expression elimination would be better.... + val zero = Cast(Literal(0), calcType) + // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its + // UnscaledValue will be null if and only if x is null; helps with Average on decimals + val actualExpr = expr match { + case UnscaledValue(e) => e + case _ => expr + } + // partial sum result can be null only when no input rows present + val updateFunction = If( + IsNotNull(actualExpr), + Coalesce( + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType)) :: currentSum :: zero :: Nil), + currentSum) + + val result = + expr.dataType match { + case DecimalType.Fixed(_, _) => + Cast(currentSum, cs.dataType) + case _ => currentSum + } + + AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + case a @ Average(expr) => val calcType = expr.dataType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f754fa770d1b5..23f7e5609414b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -155,7 +155,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 87e7cf8c8af9f..1ad92a3941187 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -102,14 +103,99 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT ABS(2.5)"), Row(2.5)) } - + test("aggregation with codegen") { val originalValue = conf.codegenEnabled setConf(SQLConf.CODEGEN_ENABLED, "true") - sql("SELECT key FROM testData GROUP BY key").collect() + // Prepare a table that we can group some rows. + table("testData") + .unionAll(table("testData")) + .unionAll(table("testData")) + .registerTempTable("testData3x") + + def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = { + val df = sql(sqlText) + // First, check if we have GeneratedAggregate. + var hasGeneratedAgg = false + df.queryExecution.executedPlan.foreach { + case generatedAgg: GeneratedAggregate => hasGeneratedAgg = true + case _ => + } + if (!hasGeneratedAgg) { + fail( + s""" + |Codegen is enabled, but query $sqlText does not have GeneratedAggregate in the plan. + |${df.queryExecution.simpleString} + """.stripMargin) + } + // Then, check results. + checkAnswer(df, expectedResults) + } + + // Just to group rows. + testCodeGen( + "SELECT key FROM testData3x GROUP BY key", + (1 to 100).map(Row(_))) + // COUNT + testCodeGen( + "SELECT key, count(value) FROM testData3x GROUP BY key", + (1 to 100).map(i => Row(i, 3))) + testCodeGen( + "SELECT count(key) FROM testData3x", + Row(300) :: Nil) + // COUNT DISTINCT ON int + testCodeGen( + "SELECT value, count(distinct key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 1))) + testCodeGen( + "SELECT count(distinct key) FROM testData3x", + Row(100) :: Nil) + // SUM + testCodeGen( + "SELECT value, sum(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 3 * i))) + testCodeGen( + "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x", + Row(5050 * 3, 5050 * 3.0) :: Nil) + // AVERAGE + testCodeGen( + "SELECT value, avg(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT avg(key) FROM testData3x", + Row(50.5) :: Nil) + // MAX + testCodeGen( + "SELECT value, max(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT max(key) FROM testData3x", + Row(100) :: Nil) + // Some combinations. + testCodeGen( + """ + |SELECT + | value, + | sum(key), + | max(key), + | avg(key), + | count(key), + | count(distinct key) + |FROM testData3x + |GROUP BY value + """.stripMargin, + (1 to 100).map(i => Row(i.toString, i*3, i, i, 3, 1))) + testCodeGen( + "SELECT max(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 50.5, 300, 100) :: Nil) + // Aggregate with Code generation handling all null values + testCodeGen( + "SELECT sum('a'), avg('a'), count(null) FROM testData", + Row(0, null, 0) :: Nil) + + dropTempTable("testData3x") setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } - test("Add Parser of SQL COALESCE()") { checkAnswer( sql("""SELECT COALESCE(1, 2)"""), From 891ada5be1e7fdd796380e2626d80843f2ef6017 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Apr 2015 18:47:39 -0700 Subject: [PATCH 680/817] [SPARK-6696] [SQL] Adds HiveContext.refreshTable to PySpark [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5349) Author: Cheng Lian Closes #5349 from liancheng/py-refresh-table and squashes the following commits: 004bec0 [Cheng Lian] Adds HiveContext.refreshTable to PySpark --- python/pyspark/sql/context.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 93e2d176a5b6f..e8529a8f8e3a4 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -574,6 +574,15 @@ def _ssql_ctx(self): def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) + def refreshTable(self, tableName): + """Invalidate and refresh all the cached the metadata of the given + table. For performance reasons, Spark SQL or the external data source + library it uses might cache certain metadata about a table, such as the + location of blocks. When those change outside of Spark SQL, users should + call this function to invalidate the cache. + """ + self._ssql_ctx.refreshTable(tableName) + class UDFRegistration(object): """Wrapper for user-defined function registration.""" From 1b2aab8d5b9cc2ff702506038bd71aa8debe7ca0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 20:35:29 -0700 Subject: [PATCH 681/817] [SPARK-6765] Fix test code style for SQL So we can turn style checker on for test code. Author: Reynold Xin Closes #5412 from rxin/test-style-sql and squashes the following commits: 9098a31 [Reynold Xin] One more compilation error ... 8c7250a [Reynold Xin] Fix compilation. 82d0944 [Reynold Xin] Indentation. 0b03fbb [Reynold Xin] code review. f2f4348 [Reynold Xin] oops. ef4ec48 [Reynold Xin] Hive module. 7e0db5e [Reynold Xin] sql module 04ec7ac [Reynold Xin] catalyst module --- .../sql/catalyst/DistributionSuite.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 10 +- .../analysis/HiveTypeCoercionSuite.scala | 8 +- .../ExpressionEvaluationSuite.scala | 134 ++++++++++++------ .../optimizer/ConstantFoldingSuite.scala | 51 ++++--- .../optimizer/FilterPushdownSuite.scala | 3 +- .../catalyst/optimizer/OptimizeInSuite.scala | 2 +- .../spark/sql/catalyst/plans/PlanTest.scala | 5 +- .../sql/catalyst/plans/SameResultSuite.scala | 2 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 8 +- .../apache/spark/sql/CachedTableSuite.scala | 3 +- .../org/apache/spark/sql/DataFrameSuite.scala | 3 +- .../org/apache/spark/sql/QueryTest.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 30 ++-- .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../spark/sql/UserDefinedTypeSuite.scala | 2 +- .../sql/columnar/ColumnarTestUtils.scala | 4 +- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- .../TestCompressibleColumnBuilder.scala | 2 +- .../sql/execution/debug/DebuggingSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 98 ++++++------- .../org/apache/spark/sql/json/JsonSuite.scala | 17 ++- .../spark/sql/parquet/ParquetIOSuite.scala | 2 +- .../sql/parquet/ParquetSchemaSuite.scala | 2 + .../spark/sql/sources/DDLTestSuite.scala | 8 +- .../spark/sql/sources/FilteredScanSuite.scala | 3 +- .../spark/sql/sources/PrunedScanSuite.scala | 5 +- .../spark/sql/sources/SaveLoadSuite.scala | 2 +- .../spark/sql/sources/TableScanSuite.scala | 9 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 33 +++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 57 ++++++-- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../execution/BigDataBenchmarkSuite.scala | 12 +- .../hive/execution/HiveComparisonTest.scala | 27 ++-- .../hive/execution/HiveQueryFileTest.scala | 11 +- .../sql/hive/execution/HiveQuerySuite.scala | 13 +- .../hive/execution/HiveResolutionSuite.scala | 3 +- .../sql/hive/execution/HiveSerDeSuite.scala | 3 +- .../execution/HiveTypeCoercionSuite.scala | 6 +- .../sql/hive/execution/HiveUdfSuite.scala | 16 ++- .../sql/hive/execution/PruningSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../apache/spark/sql/hive/parquetSuites.scala | 7 +- 45 files changed, 395 insertions(+), 234 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 46b2250aab231..ea82cd2622de9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -30,7 +30,7 @@ class DistributionSuite extends FunSuite { inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if (inputPartitioning.satisfies(requiredDistribution) != satisfied) + if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" |== Input Partitioning == @@ -40,6 +40,7 @@ class DistributionSuite extends FunSuite { |== Does input partitioning satisfy required distribution? == |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) + } } test("HashPartitioning is the output partitioning") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index ee7b14c7a157c..6e3d6b9263e86 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -27,6 +27,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import scala.collection.immutable + class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) @@ -41,10 +43,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { } - def caseSensitiveAnalyze(plan: LogicalPlan) = + def caseSensitiveAnalyze(plan: LogicalPlan): Unit = caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer(plan)) - def caseInsensitiveAnalyze(plan: LogicalPlan) = + def caseInsensitiveAnalyze(plan: LogicalPlan): Unit = caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer(plan)) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) @@ -147,7 +149,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { name: String, plan: LogicalPlan, errorMessages: Seq[String], - caseSensitive: Boolean = true) = { + caseSensitive: Boolean = true): Unit = { test(name) { val error = intercept[AnalysisException] { if(caseSensitive) { @@ -202,7 +204,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { case class UnresolvedTestPlan() extends LeafNode { override lazy val resolved = false - override def output = Nil + override def output: Seq[Attribute] = Nil } errorTest( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 70aef1cac421a..fcd745f43cfbf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -96,7 +96,9 @@ class HiveTypeCoercionSuite extends PlanTest { widenTest(StringType, TimestampType, None) // ComplexType - widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, + MapType(IntegerType, StringType, false), + Some(MapType(IntegerType, StringType, false))) widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) widenTest(StringType, MapType(IntegerType, StringType, true), None) widenTest(ArrayType(IntegerType), StructType(Seq()), None) @@ -113,7 +115,9 @@ class HiveTypeCoercionSuite extends PlanTest { // Remove superflous boolean -> boolean casts. ruleTest(Cast(Literal(true), BooleanType), Literal(true)) // Stringify boolean when casting to string. - ruleTest(Cast(Literal(false), StringType), If(Literal(false), Literal("true"), Literal("false"))) + ruleTest( + Cast(Literal(false), StringType), + If(Literal(false), Literal("true"), Literal("false"))) } test("coalesce casts") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 3dbefa40d2808..d2b1090a0cdd5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -82,10 +82,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte]) } + // scalastyle:off /** * Checks for three-valued-logic. Based on: * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 - * I.e. in flat cpo "False -> Unknown -> True", OR is lowest upper bound, AND is greatest lower bound. + * I.e. in flat cpo "False -> Unknown -> True", + * OR is lowest upper bound, + * AND is greatest lower bound. * p q p OR q p AND q p = q * True True True True True * True False True False False @@ -102,7 +105,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { * False True * Unknown Unknown */ - + // scalastyle:on val notTrueTable = (true, false) :: (false, true) :: @@ -165,7 +168,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation( + In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), + true) } test("Divide") { @@ -180,7 +185,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("Remainder") { @@ -195,7 +201,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("INSET") { @@ -264,7 +271,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation(Literal.create(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) + checkEvaluation(Literal.create(null, StringType) like regEx, null, + new GenericRow(Array[Any]("bc%"))) } test("RLIKE literal Regular Expression") { @@ -507,8 +515,10 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("array casting") { - val array = Literal.create(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) - val array_notNull = Literal.create(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) + val array = Literal.create(Seq("123", "abc", "", null), + ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "abc", ""), + ArrayType(StringType, containsNull = false)) { val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) @@ -765,7 +775,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) + checkEvaluation( + If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) checkEvaluation(If(c3, c1, c2), "^Ba*n", row) checkEvaluation(If(c4, c2, c1), "^Ba*n", row) checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) @@ -842,18 +853,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) checkEvaluation(GetItem(Literal.create(null, typeMap), Literal("aa")), null, row) - checkEvaluation(GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) checkEvaluation(GetItem(Literal.create(null, typeArray), Literal(1)), null, row) - checkEvaluation(GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal.create(null, IntegerType)), null, row) - def quickBuildGetField(expr: Expression, fieldName: String) = { + def quickBuildGetField(expr: Expression, fieldName: String): StructGetField = { expr.dataType match { case StructType(fields) => val field = fields.find(_.name == fieldName).get @@ -861,7 +874,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } } - def quickResolve(u: UnresolvedGetField) = quickBuildGetField(u.child, u.fieldName) + def quickResolve(u: UnresolvedGetField): StructGetField = { + quickBuildGetField(u.child, u.fieldName) + } checkEvaluation(quickBuildGetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) checkEvaluation(quickBuildGetField(Literal.create(null, typeS), "a"), null, row) @@ -872,7 +887,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { ) assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) + assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable + === false) assert(quickBuildGetField(Literal.create(null, typeS), "a").nullable === true) assert(quickBuildGetField(Literal.create(null, typeS_notNullable), "a").nullable === true) @@ -896,7 +912,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Add(c1, c2), 3, row) checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation(Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(-c1, -1, row) checkEvaluation(c1 + c2, 3, row) @@ -919,7 +936,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Add(c1, c2), 3.1, row) checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) - checkEvaluation(Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) + checkEvaluation( + Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) checkEvaluation(-c1, -1.1, row) checkEvaluation(c1 + c2, 3.1, row) @@ -942,7 +960,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(LessThan(c1, c2), true, row) checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation(LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 < c2, true, row) checkEvaluation(c1 <= c2, true, row) @@ -985,54 +1004,84 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val s = 'a.string.at(0) // substring from zero position with less-than-full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) // substring from zero position with full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) // substring from zero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), + "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), + "example", row) // substring from nonzero position with less-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), "xa", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), + "xa", row) // substring from nonzero position with full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), + "xample", row) // substring from nonzero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), + "xample", row) // zero-length substring (within string bounds) - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), + "", row) // zero-length substring (beyond string bounds) - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + "", row) // substring(null, _, _) -> null - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + null, new GenericRow(Array[Any](null))) // substring(_, null, _) -> null - checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), null, row) + checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), + null, row) // substring(_, _, null) -> null - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), + null, + row) // 2-arg substring from zero position - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) // 2-arg substring from nonzero position - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "xample", row) + checkEvaluation( + Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "xample", + row) val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === false) - assert(Substring(s_notNull, Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === true) + assert( + Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === false) + assert(Substring(s_notNull, + Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, + Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -1065,17 +1114,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(BitwiseAnd(c1, c4), null, row) checkEvaluation(BitwiseAnd(c1, c2), 0, row) checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseOr(c1, c4), null, row) checkEvaluation(BitwiseOr(c1, c2), 3, row) checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseXor(c1, c4), null, row) checkEvaluation(BitwiseXor(c1, c2), 3, row) checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseNot(c4), null, row) checkEvaluation(BitwiseNot(c1), -2, row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index a0efe9e2e7f6b..4396bd0dda9a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -176,40 +176,39 @@ class ConstantFoldingSuite extends PlanTest { } test("Constant folding test: expressions have null literals") { - val originalQuery = - testRelation - .select( - IsNull(Literal(null)) as 'c1, - IsNotNull(Literal(null)) as 'c2, + val originalQuery = testRelation.select( + IsNull(Literal(null)) as 'c1, + IsNotNull(Literal(null)) as 'c2, - GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, - GetItem(Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, - UnresolvedGetField( - Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), - "a") as 'c5, + GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + GetItem( + Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, + UnresolvedGetField( + Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), + "a") as 'c5, - UnaryMinus(Literal.create(null, IntegerType)) as 'c6, - Cast(Literal(null), IntegerType) as 'c7, - Not(Literal.create(null, BooleanType)) as 'c8, + UnaryMinus(Literal.create(null, IntegerType)) as 'c6, + Cast(Literal(null), IntegerType) as 'c7, + Not(Literal.create(null, BooleanType)) as 'c8, - Add(Literal.create(null, IntegerType), 1) as 'c9, - Add(1, Literal.create(null, IntegerType)) as 'c10, + Add(Literal.create(null, IntegerType), 1) as 'c9, + Add(1, Literal.create(null, IntegerType)) as 'c10, - EqualTo(Literal.create(null, IntegerType), 1) as 'c11, - EqualTo(1, Literal.create(null, IntegerType)) as 'c12, + EqualTo(Literal.create(null, IntegerType), 1) as 'c11, + EqualTo(1, Literal.create(null, IntegerType)) as 'c12, - Like(Literal.create(null, StringType), "abc") as 'c13, - Like("abc", Literal.create(null, StringType)) as 'c14, + Like(Literal.create(null, StringType), "abc") as 'c13, + Like("abc", Literal.create(null, StringType)) as 'c14, - Upper(Literal.create(null, StringType)) as 'c15, + Upper(Literal.create(null, StringType)) as 'c15, - Substring(Literal.create(null, StringType), 0, 1) as 'c16, - Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, + Substring(Literal.create(null, StringType), 0, 1) as 'c16, + Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, + Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, - Contains(Literal.create(null, StringType), "abc") as 'c19, - Contains("abc", Literal.create(null, StringType)) as 'c20 - ) + Contains(Literal.create(null, StringType), "abc") as 'c19, + Contains("abc", Literal.create(null, StringType)) as 'c20 + ) val optimized = Optimize(originalQuery.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 55c6766520a1e..1448098c770aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -432,7 +432,8 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { z.join(x.join(y)) - .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && + ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) } val optimized = Optimize(originalQuery.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 233e329cb2038..966bc9ada1e6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -52,7 +52,7 @@ class OptimizeInSuite extends PlanTest { val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2)) + .where(InSet(UnresolvedAttribute("a"), HashSet[Any]() + 1 + 2)) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 129d091ca03e3..e7cafcc96de87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -45,12 +45,13 @@ class PlanTest extends FunSuite { protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizeExprIds(plan1) val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) + if (normalized1 != normalized2) { fail( s""" |== FAIL: Plans do not match === |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) + """.stripMargin) + } } /** Fails the test if the two expressions do not match */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index 11e6831b24768..1273921f6394c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -32,7 +32,7 @@ class SameResultSuite extends FunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) - def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true) = { + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true): Unit = { val aAnalyzed = a.analyze val bAnalyzed = b.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 274f3ede0045c..4eb8708335dcf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -25,12 +25,12 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { - def children = optKey.toSeq - def nullable = true - def dataType = NullType + def children: Seq[Expression] = optKey.toSeq + def nullable: Boolean = true + def dataType: NullType = NullType override lazy val resolved = true type EvaluatedType = Any - def eval(input: Row) = null.asInstanceOf[Any] + def eval(input: Row): Any = null.asInstanceOf[Any] } class TreeNodeSuite extends FunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c240f2be955ca..f7b5f08beb92f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -92,7 +92,8 @@ class CachedTableSuite extends QueryTest { test("too big for memory") { val data = "*" * 10000 - sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF().registerTempTable("bigData") + sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() + .registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) table("bigData").unpersist(blocking = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 1db0cf7daac03..f5df8c6a59f10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -329,8 +329,9 @@ class DataFrameSuite extends QueryTest { checkAnswer( decimalData.agg(avg('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0))) + // non-partial checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), // non-partial + decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9b4dd6c620fec..9a81fc5d72819 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -67,7 +67,7 @@ class QueryTest extends PlanTest { checkAnswer(df, Seq(expectedAnswer)) } - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) { test(sqlString) { checkAnswer(sqlContext.sql(sqlString), expectedAnswer) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1ad92a3941187..1392b4819131b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -268,7 +268,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')"), + """ + |SELECT time FROM timestamps + |WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002') + """.stripMargin), Seq(Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")), Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002")))) @@ -334,7 +337,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row("1")) } - def sortTest() = { + def sortTest(): Unit = { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"), Seq(Row(1,1), Row(1,2), Row(2,1), Row(2,2), Row(3,1), Row(3,2))) @@ -413,7 +416,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("from follow multiple brackets") { checkAnswer(sql( - "select key from ((select * from testData limit 1) union all (select * from testData limit 1)) x limit 1"), + """ + |select key from ((select * from testData limit 1) + | union all (select * from testData limit 1)) x limit 1 + """.stripMargin), Row(1) ) @@ -423,7 +429,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ) checkAnswer(sql( - "select key from (select * from testData limit 1 union all select * from testData limit 1) x limit 1"), + """ + |select key from + | (select * from testData limit 1 union all select * from testData limit 1) x + | limit 1 + """.stripMargin), Row(1) ) } @@ -470,7 +480,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Seq(Row(1, 0), Row(2, 1))) checkAnswer( - sql("SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"), + sql( + """ + |SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3 + """.stripMargin), Row(2, 1, 2, 2, 1)) } @@ -1083,7 +1096,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3483 Special chars in column names") { - val data = sparkContext.parallelize(Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) + val data = sparkContext.parallelize( + Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") sql("SELECT `key?number1` FROM records") } @@ -1168,8 +1182,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-6145: ORDER BY test for nested fields") { - jsonRDD(sparkContext.makeRDD( - """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder") + jsonRDD(sparkContext.makeRDD("""{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) + .registerTempTable("nestedOrder") checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 17e923ca48502..3fa00fd9d0ccb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -80,7 +80,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) + new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectData") @@ -103,7 +103,8 @@ class ScalaReflectionRelationSuite extends FunSuite { val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectOptionalData") - assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) + assert(sql("SELECT * FROM reflectOptionalData").collect().head === + Row.fromSeq(Seq.fill(7)(null))) } // Equality is broken for Arrays, so we test that separately. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index fe618e0e8e767..902da5c3ba6d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -63,7 +63,7 @@ private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { } } - override def userClass = classOf[MyDenseVector] + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] private[spark] override def asNullable: MyDenseVectorUDT = this } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index c7a40845db16c..b301818a008e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types.{Decimal, DataType, NativeType} object ColumnarTestUtils { - def makeNullRow(length: Int) = { + def makeNullRow(length: Int): GenericMutableRow = { val row = new GenericMutableRow(length) (0 until length).foreach(row.setNullAt) row @@ -93,7 +93,7 @@ object ColumnarTestUtils { def makeUniqueValuesAndSingleValueRows[T <: NativeType]( columnType: NativeColumnType[T], - count: Int) = { + count: Int): (Seq[T#JvmType], Seq[GenericMutableRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index bb305355276bf..a0702144f942c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -31,7 +31,8 @@ class TestNullableColumnAccessor[T <: DataType, JvmType]( with NullableColumnAccessor object TestNullableColumnAccessor { - def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = { + def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) + : TestNullableColumnAccessor[T, JvmType] = { // Skips the column type ID buffer.getInt() new TestNullableColumnAccessor(buffer, columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 75a47498683f4..3a5605d2335d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -27,7 +27,8 @@ class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T with NullableColumnBuilder object TestNullableColumnBuilder { - def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = { + def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) + : TestNullableColumnBuilder[T, JvmType] = { val builder = new TestNullableColumnBuilder(columnType) builder.initialize(initialSize) builder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 0b18b4119268f..fc8ff3b41d0e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -35,7 +35,7 @@ object TestCompressibleColumnBuilder { def apply[T <: NativeType]( columnStats: ColumnStats, columnType: NativeColumnType[T], - scheme: CompressionScheme) = { + scheme: CompressionScheme): TestCompressibleColumnBuilder[T] = { val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) builder.initialize(0, "", useCompression = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 4e9472c60249e..358d8cf06e463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -30,4 +30,4 @@ class DebuggingSuite extends FunSuite { test("DataFrame.typeCheck()") { testData.typeCheck() } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 592ed4b23b7d3..3596b183d4328 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -45,10 +45,12 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { conn = DriverManager.getConnection(url, properties) conn.prepareStatement("create schema test").executeUpdate() - conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() + conn.prepareStatement( + "create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() - conn.prepareStatement("insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() + conn.prepareStatement( + "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() conn.commit() sql( @@ -132,25 +134,25 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT *") { - assert(sql("SELECT * FROM foobar").collect().size == 3) + assert(sql("SELECT * FROM foobar").collect().size === 3) } test("SELECT * WHERE (simple predicates)") { - assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size == 1) - assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size == 1) - assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size == 2) - assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size == 2) + assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size === 2) } test("SELECT * WHERE (quoted strings)") { - assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size == 1) + assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size === 1) } test("SELECT first field") { val names = sql("SELECT NAME FROM foobar").collect().map(x => x.getString(0)).sortWith(_ < _) - assert(names.size == 3) + assert(names.size === 3) assert(names(0).equals("fred")) assert(names(1).equals("joe 'foo' \"bar\"")) assert(names(2).equals("mary")) @@ -158,10 +160,10 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { test("SELECT second field") { val ids = sql("SELECT THEID FROM foobar").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("SELECT * partitioned") { @@ -169,46 +171,46 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT WHERE (simple predicates) partitioned") { - assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size == 1) + assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size === 1) } test("SELECT second field partitioned") { val ids = sql("SELECT THEID FROM parts").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("Basic API") { - assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect().size === 3) } test("Partitioning via JDBCPartitioningInfo API") { assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3) - .collect.size == 3) + .collect.size === 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect().size === 3) } test("H2 integral types") { val rows = sql("SELECT * FROM inttypes WHERE A IS NOT NULL").collect() - assert(rows.size == 1) - assert(rows(0).getInt(0) == 1) - assert(rows(0).getBoolean(1) == false) - assert(rows(0).getInt(2) == 3) - assert(rows(0).getInt(3) == 4) - assert(rows(0).getLong(4) == 1234567890123L) + assert(rows.size === 1) + assert(rows(0).getInt(0) === 1) + assert(rows(0).getBoolean(1) === false) + assert(rows(0).getInt(2) === 3) + assert(rows(0).getInt(3) === 4) + assert(rows(0).getLong(4) === 1234567890123L) } test("H2 null entries") { val rows = sql("SELECT * FROM inttypes WHERE A IS NULL").collect() - assert(rows.size == 1) + assert(rows.size === 1) assert(rows(0).isNullAt(0)) assert(rows(0).isNullAt(1)) assert(rows(0).isNullAt(2)) @@ -230,27 +232,27 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = sql("SELECT * FROM timetypes").collect() val cal = new GregorianCalendar(java.util.Locale.ROOT) cal.setTime(rows(0).getAs[java.sql.Timestamp](0)) - assert(cal.get(Calendar.HOUR_OF_DAY) == 12) - assert(cal.get(Calendar.MINUTE) == 34) - assert(cal.get(Calendar.SECOND) == 56) + assert(cal.get(Calendar.HOUR_OF_DAY) === 12) + assert(cal.get(Calendar.MINUTE) === 34) + assert(cal.get(Calendar.SECOND) === 56) cal.setTime(rows(0).getAs[java.sql.Timestamp](1)) - assert(cal.get(Calendar.YEAR) == 1996) - assert(cal.get(Calendar.MONTH) == 0) - assert(cal.get(Calendar.DAY_OF_MONTH) == 1) + assert(cal.get(Calendar.YEAR) === 1996) + assert(cal.get(Calendar.MONTH) === 0) + assert(cal.get(Calendar.DAY_OF_MONTH) === 1) cal.setTime(rows(0).getAs[java.sql.Timestamp](2)) - assert(cal.get(Calendar.YEAR) == 2002) - assert(cal.get(Calendar.MONTH) == 1) - assert(cal.get(Calendar.DAY_OF_MONTH) == 20) - assert(cal.get(Calendar.HOUR) == 11) - assert(cal.get(Calendar.MINUTE) == 22) - assert(cal.get(Calendar.SECOND) == 33) - assert(rows(0).getAs[java.sql.Timestamp](2).getNanos == 543543543) + assert(cal.get(Calendar.YEAR) === 2002) + assert(cal.get(Calendar.MONTH) === 1) + assert(cal.get(Calendar.DAY_OF_MONTH) === 20) + assert(cal.get(Calendar.HOUR) === 11) + assert(cal.get(Calendar.MINUTE) === 22) + assert(cal.get(Calendar.SECOND) === 33) + assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543543) } test("H2 floating-point types") { val rows = sql("SELECT * FROM flttypes").collect() - assert(rows(0).getDouble(0) == 1.00000000000000022) // Yes, I meant ==. - assert(rows(0).getDouble(1) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. + assert(rows(0).getDouble(1) === 1.00000011920928955) // Yes, I meant ==. assert(rows(0).getAs[BigDecimal](2) .equals(new BigDecimal("123456789012345.54321543215432100000"))) } @@ -264,7 +266,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { | user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) val rows = sql("SELECT * FROM hack").collect() - assert(rows(0).getDouble(0) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000011920928955) // Yes, I meant ==. // For some reason, H2 computes this square incorrectly... assert(math.abs(rows(0).getDouble(1) - 1.00000023841859331) < 1e-12) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 706c966ee05f5..1fe0b76c00be3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -380,8 +380,10 @@ class JsonSuite extends QueryTest { sql("select * from jsonTable"), Row("true", 11L, null, 1.1, "13.1", "str1") :: Row("12", null, new java.math.BigDecimal("21474836470.9"), null, null, "true") :: - Row("false", 21474836470L, new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: - Row(null, 21474836570L, new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil + Row("false", 21474836470L, + new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: + Row(null, 21474836570L, + new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil ) // Number and Boolean conflict: resolve the type as number in this query. @@ -404,7 +406,8 @@ class JsonSuite extends QueryTest { // Widening to DecimalType checkAnswer( sql("select num_num_2 + 1.2 from jsonTable where num_num_2 > 1.1"), - Row(new java.math.BigDecimal("21474836472.1")) :: Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil + Row(new java.math.BigDecimal("21474836472.1")) :: + Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil ) // Widening to DoubleType @@ -913,8 +916,10 @@ class JsonSuite extends QueryTest { df1.registerTempTable("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() + // scalastyle:off assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") + // scalastyle:on val schema2 = StructType( StructField("f1", StructType( @@ -968,7 +973,8 @@ class JsonSuite extends QueryTest { // Access elements of a BigInteger array (we use DecimalType internally). checkAnswer( - sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from complexTable"), + sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] " + + " from complexTable"), Row(new java.math.BigDecimal("922337203685477580700"), new java.math.BigDecimal("-922337203685477580800"), null) ) @@ -1008,7 +1014,8 @@ class JsonSuite extends QueryTest { // Access elements of an array field of a struct. checkAnswer( - sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from complexTable"), + sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] " + + "from complexTable"), Row(5, null) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 203bc79f153dd..97c0f439acf13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -218,7 +218,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } test("compression codec") { - def compressionCodecFor(path: String) = { + def compressionCodecFor(path: String): String = { val codecs = ParquetTypesConverter .readMetaData(new Path(path), Some(configuration)) .getBlocks diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 61f1cf347ab0f..c964b6d984557 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -180,10 +180,12 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { val caseClassString = "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))" + // scalastyle:off val jsonString = """ |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]} """.stripMargin + // scalastyle:on val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString) val fromJson = ParquetTypesConverter.convertFromString(jsonString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 54af50c6e10ad..3f24a497390c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -31,7 +32,7 @@ class DDLScanSource extends RelationProvider { case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, new MetadataBuilder().putString("comment", "test comment").build()), @@ -57,8 +58,9 @@ case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLConte )) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to). - map(e => Row(s"people$e", e * 2)) + override def buildScan(): RDD[Row] = { + sqlContext.sparkContext.parallelize(from to to).map(e => Row(s"people$e", e * 2)) + } } class DDLTestSuite extends DataSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 773bd1602d5e5..cb5e5147ff189 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -41,7 +42,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL StructField("b", IntegerType, nullable = false) :: StructField("c", StringType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 08fb5380dc026..6a1ddf2f8e98b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -34,12 +35,12 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo extends BaseRelation with PrunedScan { - override def schema = + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: StructField("b", IntegerType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String]) = { + override def buildScan(requiredColumns: Array[String]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 43bc8eb2d11a7..cb287ba85c1f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -114,4 +114,4 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { message.contains("Append mode is not supported"), "We should complain that 'Append mode is not supported' for JSON source.") } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 7928600ac2fb5..60c8c00bda4d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import java.sql.{Timestamp, Date} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -35,10 +36,10 @@ class SimpleScanSource extends RelationProvider { case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) + override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) } class AllDataTypesScanSource extends SchemaRelationProvider { @@ -57,9 +58,9 @@ case class AllDataTypesScan( extends BaseRelation with TableScan { - override def schema = userSpecifiedSchema + override def schema: StructType = userSpecifiedSchema - override def buildScan() = { + override def buildScan(): RDD[Row] = { sqlContext.sparkContext.parallelize(from to to).map { i => Row( s"str_$i", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index 968557c9c4686..d960a30e00738 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -136,7 +136,7 @@ class ErrorPositionSuite extends QueryTest with BeforeAndAfter { * @param query the query to analyze * @param token a unique token in the string that should be indicated by the exception */ - def positionTest(name: String, query: String, token: String) = { + def positionTest(name: String, query: String, token: String): Unit = { def parseTree = Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index c482c6de8a736..2a7374cc172b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -116,21 +116,20 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { - dt1.zip(dt2).map { - case (dd1, dd2) => - assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info + dt1.zip(dt2).foreach { case (dd1, dd2) => + assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info } } def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { - row1.zip(row2).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2).foreach { case (r1, r2) => + checkValue(r1, r2) } } def checkValues(row1: Seq[Any], row2: Row): Unit = { - row1.zip(row2.toSeq).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2.toSeq).foreach { case (r1, r2) => + checkValue(r1, r2) } } @@ -141,7 +140,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { assert(r1.compare(r2) === 0) case (r1: Array[Byte], r2: Array[Byte]) if r1 != null && r2 != null && r1.length == r2.length => - r1.zip(r2).map { case (b1, b2) => assert(b1 === b2) } + r1.zip(r2).foreach { case (b1, b2) => assert(b1 === b2) } case (r1, r2) => assert(r1 === r2) } } @@ -166,7 +165,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val constantData = constantExprs.map(_.eval()) val constantNullData = constantData.map(_ => null) val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType)) - val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) + val constantNullWritableOIs = + constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) checkValues(constantData, constantData.zip(constantWritableOIs).map { case (d, oi) => unwrap(wrap(d, oi), oi) @@ -202,7 +202,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { case (t, idx) => StructField(s"c_$idx", t) }) - checkValues(row, unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) + checkValues(row, + unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) } @@ -212,8 +213,10 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = row(0) :: row(0) :: Nil checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } test("wrap / unwrap Map Type") { @@ -222,7 +225,9 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = Map(row(0) -> row(1)) checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 8011952e0d535..ecb990e8aac91 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -115,11 +115,36 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { test("SPARK-4203:random partition directory order") { sql("CREATE TABLE tmp_table (key int, value string)") val tmpDir = Utils.createTempDir() - sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='3') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='4') SELECT 'blarr' FROM tmp_table") + sql( + s""" + |CREATE TABLE table_with_partition(c1 string) + |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='1') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='2') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='3') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='4') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) def listFolders(path: File, acc: List[String]): List[List[String]] = { val dir = path.listFiles() val folders = dir.filter(_.isDirectory).toList @@ -196,34 +221,42 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { testData.registerTempTable("testData") val testDatawithNull = TestHive.sparkContext.parallelize( - (1 to 10).map(i => ThreeCloumntable(i, i.toString,null))).toDF() + (1 to 10).map(i => ThreeCloumntable(i, i.toString, null))).toDF() val tmpDir = Utils.createTempDir() - sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") + sql( + s""" + |CREATE TABLE table_with_partition(key int,value string) + |PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (ds='1') SELECT key,value FROM testData + """.stripMargin) // test schema the same between partition and table sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) // test difference type of field sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) // add column to table sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)") checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "), - testDatawithNull.collect.toSeq + testDatawithNull.collect().toSeq ) // change column name to table sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT") checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) sql("DROP TABLE table_with_partition") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index ccd0e5aa51f95..00a69de9e4262 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -142,7 +142,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { after: () => Unit, query: String, expectedAnswer: Seq[Row], - ct: ClassTag[_]) = { + ct: ClassTag[_]): Unit = { before() var df = sql(query) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 42a82c1fbf5c7..a3f5921a0cb23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.hive.test.TestHive._ class BigDataBenchmarkSuite extends HiveComparisonTest { val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( "rankings", @@ -63,7 +64,7 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { | searchWord STRING, | duration INT) | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," - | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "uservisits").getCanonicalPath}" + | STORED AS TEXTFILE LOCATION "$userVisitPath" """.stripMargin.cmd), TestTable( "documents", @@ -83,7 +84,10 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") createQueryTest("query2", - "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") + """ + |SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits + |GROUP BY SUBSTR(sourceIP, 1, 10) + """.stripMargin) createQueryTest("query3", """ @@ -113,8 +117,8 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { |CREATE TABLE url_counts_total AS | SELECT SUM(count) AS totalCount, destpage | FROM url_counts_partial GROUP BY destpage - |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic - |-- given different input splits. + |-- The following queries run, but generate different results in HIVE + |-- likely because the UDF is not deterministic given different input splits. |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial |-- SELECT COUNT(*) FROM url_counts_partial |-- SELECT * FROM url_counts_partial diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a5ec312ee430c..027056d4b865f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -255,8 +255,9 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") .filterNot(_ contains "hive.exec.post.hooks") - if (allQueries != queryList) + if (allQueries != queryList) { logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -305,13 +306,16 @@ abstract class HiveComparisonTest try { // Hooks often break the harness and don't really affect our test anyway, don't // even try running them. - if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) + if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) { sys.error("hive exec hooks not supported for tests.") + } - logWarning(s"Running query ${i+1}/${queryList.size} with hive.") + logWarning(s"Running query ${i + 1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { - case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _: ExplainCommand => + // No need to execute EXPLAIN queries as we don't check the output. + Nil case _ => TestHive.runSqlHive(queryString) } @@ -394,21 +398,24 @@ abstract class HiveComparisonTest case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => if (System.getProperty("spark.hive.canarytest") != null) { - // When we encounter an error we check to see if the environment is still okay by running a simple query. - // If this fails then we halt testing since something must have gone seriously wrong. + // When we encounter an error we check to see if the environment is still + // okay by running a simple query. If this fails then we halt testing since + // something must have gone seriously wrong. try { new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") - // The testing setup traps exits so wait here for a long time so the developer can see when things started - // to go wrong. + logError(s"FATAL ERROR: Canary query threw $e This implies that the " + + "testing environment has likely been corrupted.") + // The testing setup traps exits so wait here for a long time so the developer + // can see when things started to go wrong. Thread.sleep(1000000) } } - // If the canary query didn't fail then the environment is still okay, so just throw the original exception. + // If the canary query didn't fail then the environment is still okay, + // so just throw the original exception. throw originalException } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 02518d516261b..f7b37dae0a5f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. * - * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles that should be included. - * Additionally, there is support for whitelisting and blacklisting tests as development progresses. + * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles + * that should be included. Additionally, there is support for whitelisting and blacklisting + * tests as development progresses. */ abstract class HiveQueryFileTest extends HiveComparisonTest { /** A list of tests deemed out of scope and thus completely disregarded */ @@ -54,15 +55,17 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { logDebug(s"Blacklisted test skipped $testCaseName") - } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { + } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || + runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString) } else { // Only output warnings for the built in whitelist as this clutters the output when the user // trying to execute a single test from the commandline. - if(System.getProperty(whiteListProperty) == null && !runAll) + if (System.getProperty(whiteListProperty) == null && !runAll) { ignore(testCaseName) {} + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index de140fc72a2c3..af781a502e9f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -37,7 +37,8 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(a: Int, b: String) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault @@ -237,7 +238,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } createQueryTest("modulus", - "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") + "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), " + + "(101 / 2) % 10 FROM src LIMIT 1") test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") @@ -309,7 +311,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { "SELECT * FROM src a JOIN src b ON a.key = b.key") createQueryTest("small.cartesian", - "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN (SELECT key FROM src WHERE key = 2) b") + "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN " + + "(SELECT key FROM src WHERE key = 2) b") createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") @@ -457,6 +460,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view3", "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX") + // scalastyle:off createQueryTest("lateral view4", """ |create table src_lv1 (key string, value string); @@ -466,6 +470,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |insert overwrite table src_lv1 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX |insert overwrite table src_lv2 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX """.stripMargin) + // scalastyle:on createQueryTest("lateral view5", "FROM src SELECT explode(array(key+3, key+4))") @@ -584,7 +589,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - def isExplanation(result: DataFrame) = { + def isExplanation(result: DataFrame): Boolean = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } explanation.contains("== Physical Plan ==") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index f4440e5b7846a..8ad3627504229 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -25,7 +25,8 @@ case class Nested(a: Int, B: Int) case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveResolutionSuite extends HiveComparisonTest { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 7486bfa82b00b..d05e11fcf281b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -26,8 +26,9 @@ import org.apache.spark.sql.hive.test.TestHive */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - override def beforeAll() = { + override def beforeAll(): Unit = { TestHive.cacheTables = false + super.beforeAll() } createQueryTest( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index ab0e0443c7faa..f0f04f8c73fb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -35,8 +35,10 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { val nullVal = "null" baseTypes.init.foreach { i => - createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") - createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + createQueryTest(s"case when then $i else $nullVal end ", + s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", + s"SELECT case when true then $nullVal else $i end FROM src limit 1") } test("[SPARK-2210] boolean cast on boolean value should be removed") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index d7c5d1a25a82b..7f49eac490572 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -123,9 +123,10 @@ class HiveUdfSuite extends QueryTest { IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() testData.registerTempTable("integerTable") - sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") + val udfName = classOf[UDFIntegerToString].getName + sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") checkAnswer( - sql("SELECT testUDFIntegerToString(i) FROM integerTable"), //.collect(), + sql("SELECT testUDFIntegerToString(i) FROM integerTable"), Seq(Row("1"), Row("2"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFIntegerToString") @@ -141,7 +142,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( - sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), //.collect(), + sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), Seq(Row(0), Row(2), Row(13))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListListInt") @@ -156,7 +157,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( - sql("SELECT testUDFListString(l) FROM listStringTable"), //.collect(), + sql("SELECT testUDFListString(l) FROM listStringTable"), Seq(Row("a,b,c"), Row("d,e"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListString") @@ -170,7 +171,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") checkAnswer( - sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), //.collect(), + sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), Seq(Row("hello world"), Row("hello goodbye"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUdf") @@ -187,7 +188,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( - sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), //.collect(), + sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), Seq(Row("0, 0"), Row("2, 2"), Row("13, 13"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList") @@ -247,7 +248,8 @@ class PairUdf extends GenericUDF { override def initialize(p1: Array[ObjectInspector]): ObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( Seq("id", "value"), - Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, PrimitiveObjectInspectorFactory.javaIntObjectInspector) + Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.javaIntObjectInspector) ) override def evaluate(args: Array[DeferredObject]): AnyRef = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8474d850c9c6c..067b577f1560e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -143,7 +143,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { sql: String, expectedOutputColumns: Seq[String], expectedScannedColumns: Seq[String], - expectedPartValues: Seq[Seq[String]]) = { + expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 817b9dcb8f505..7811bd2e9effb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -422,7 +422,7 @@ class SQLQuerySuite extends QueryTest { } test("resolve udtf with single alias") { - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") @@ -435,7 +435,7 @@ class SQLQuerySuite extends QueryTest { // is not in a valid state (cannot be executed). Because of this bug, the analysis rule of // PreInsertionCasts will actually start to work before ImplicitGenerate and then // generates an invalid query plan. - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val originalConf = getConf("spark.sql.hive.convertCTAS", "false") setConf("spark.sql.hive.convertCTAS", "false") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 5f71e1bbc2d2e..d5dd0bf58e702 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -887,7 +886,11 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll test(s"SPARK-5775 read struct from $table") { checkAnswer( - sql(s"SELECT p, structField.intStructField, structField.stringStructField FROM $table WHERE p = 1"), + sql( + s""" + |SELECT p, structField.intStructField, structField.stringStructField + |FROM $table WHERE p = 1 + """.stripMargin), (1 to 10).map(i => Row(1, i, f"${i}_string"))) } From 2fe0a1aaeebbf7f60bd4130847d738c29f1e3d53 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 8 Apr 2015 22:45:40 -0700 Subject: [PATCH 682/817] [SPARK-5654] Integrate SparkR This pull requests integrates SparkR, an R frontend for Spark. The SparkR package contains both RDD and DataFrame APIs in R and is integrated with Spark's submission scripts to work on different cluster managers. Some integration points that would be great to get feedback on: 1. Build procedure: SparkR requires R to be installed on the machine to be built. Right now we have a new Maven profile `-PsparkR` that can be used to enable SparkR builds 2. YARN cluster mode: The R package that is built needs to be present on the driver and all the worker nodes during execution. The R package location is currently set using SPARK_HOME, but this might not work on YARN cluster mode. The SparkR package represents the work of many contributors and attached below is a list of people along with areas they worked on edwardt (edwart) - Documentation improvements Felix Cheung (felixcheung) - Documentation improvements Hossein Falaki (falaki) - Documentation improvements Chris Freeman (cafreeman) - DataFrame API, Programming Guide Todd Gao (7c00) - R worker Internals Ryan Hafen (hafen) - SparkR Internals Qian Huang (hqzizania) - RDD API Hao Lin (hlin09) - RDD API, Closure cleaner Evert Lammerts (evertlammerts) - DataFrame API Davies Liu (davies) - DataFrame API, R worker internals, Merging with Spark Yi Lu (lythesia) - RDD API, Worker internals Matt Massie (massie) - Jenkins build Harihar Nahak (hnahak87) - SparkR examples Oscar Olmedo (oscaroboto) - Spark configuration Antonio Piccolboni (piccolbo) - SparkR examples, Namespace bug fixes Dan Putler (dputler) - Dataframe API, SparkR Install Guide Ashutosh Raina (ashutoshraina) - Build improvements Josh Rosen (joshrosen) - Travis CI build Sun Rui (sun-rui)- RDD API, JVM Backend, Shuffle improvements Shivaram Venkataraman (shivaram) - RDD API, JVM Backend, Worker Internals Zongheng Yang (concretevitamin) - RDD API, Pipelined RDDs, Examples and EC2 guide Author: Shivaram Venkataraman Author: Shivaram Venkataraman Author: Zongheng Yang Author: cafreeman Author: Shivaram Venkataraman Author: Davies Liu Author: Davies Liu Author: hlin09 Author: Sun Rui Author: lythesia Author: oscaroboto Author: Antonio Piccolboni Author: root Author: edwardt Author: hqzizania Author: dputler Author: Todd Gao Author: Chris Freeman Author: Felix Cheung Author: Hossein Author: Evert Lammerts Author: Felix Cheung Author: felixcheung Author: Ryan Hafen Author: Ashutosh Raina Author: Oscar Olmedo Author: Josh Rosen Author: Yi Lu Author: Harihar Nahak Closes #5096 from shivaram/R and squashes the following commits: da64742 [Davies Liu] fix Date serialization 59266d1 [Davies Liu] check exclusive of primary-py-file and primary-r-file 55808e4 [Davies Liu] fix tests 5581c75 [Davies Liu] update author of SparkR f731b48 [Shivaram Venkataraman] Only run SparkR tests if R is installed 64eda24 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R d7c3f22 [Shivaram Venkataraman] Address code review comments Changes include 1. Adding SparkR docs to API docs generated 2. Style fixes in SparkR scala files 3. Clean up of shell scripts and explanation of install-dev.sh 377151f [Shivaram Venkataraman] Merge remote-tracking branch 'apache/master' into R eb5da53 [Shivaram Venkataraman] Merge pull request #3 from davies/R2 a18ff5c [Davies Liu] Update sparkR.R 5133f3a [Shivaram Venkataraman] Merge pull request #7 from hqzizania/R3 940b631 [hqzizania] [SPARKR-92] Phase 2: implement sum(rdd) 0e788c0 [Shivaram Venkataraman] Merge pull request #5 from hlin09/doc-fix 3487461 [hlin09] Add tests log in .gitignore. 1d1802e [Shivaram Venkataraman] Merge pull request #4 from felixcheung/r-require 11981b7 [felixcheung] Update R to fail early if SparkR package is missing c300e08 [Davies Liu] remove duplicated file b045701 [Davies Liu] Merge branch 'remote_r' into R 19c9368 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r f8fa8af [Davies Liu] mute logging when start/stop context e7104b6 [Davies Liu] remove ::: in SparkR a1777eb [Davies Liu] move rules into R/.gitignore e88b649 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 6e20e71 [Davies Liu] address comments b433817 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R a1cedad [Shivaram Venkataraman] Merge pull request #228 from felixcheung/doc e089151 [Davies Liu] Merge pull request #225 from sun-rui/SPARKR-154_2 463e28c [Davies Liu] Merge pull request #2 from shivaram/doc-fixes bc2d6d8 [Shivaram Venkataraman] Remove arg from sparkR.stop and update docs d425363 [Shivaram Venkataraman] Some doc fixes for column, generics, group 1f1a7e0 [Shivaram Venkataraman] Some fixes to DataFrame, RDD, SQLContext docs 104ad4e [Shivaram Venkataraman] Check the right env in exists cf5cd99 [Shivaram Venkataraman] Remove unused numCols argument 85a50ec [Shivaram Venkataraman] Merge pull request #226 from RevolutionAnalytics/master 3eacfc0 [Davies Liu] fix flaky test 733380d [Davies Liu] update R examples (remove master from args) b21a0da [Davies Liu] Merge pull request #1 from shivaram/log4j-tests a1493d7 [Shivaram Venkataraman] Address comments e1f83ab [Shivaram Venkataraman] Send Spark INFO logs to a file in SparkR tests 58276f5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 52cc92d [Shivaram Venkataraman] Add license to create-docs.sh 6ff5ea2 [Shivaram Venkataraman] Add instructions to generate docs 1f478c5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 02b4833 [Shivaram Venkataraman] Add a script to generate R docs (Rd, html) Also fix some issues with our documentation d6d3729 [Davies Liu] enable spark and pyspark tests 0e5a83f [Davies Liu] fix code style afd8a77 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R d87a181 [Davies Liu] fix flaky tests 7100fb9 [Shivaram Venkataraman] Fix libPaths in README bdf3a14 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 05e7375 [Davies Liu] sort generics b44e371 [Shivaram Venkataraman] Include RStudio instructions in README 855537f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 9fb6af3 [Davies Liu] mark R classes/objects are private 423ea3c [Shivaram Venkataraman] Ignore unknown jobj in cleanup 974e4ea [Davies Liu] fix flaky test 410ec18 [Davies Liu] fix zipRDD() tests d8b24fc [Davies Liu] disable spark and python tests temporary ce3ca62 [Davies Liu] fix license check 7da0049 [Davies Liu] fix build 2892e29 [Davies Liu] support R in YARN cluster ebd4d07 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 38cbf59 [Davies Liu] fix test of zipRDD() 756ece0 [Shivaram Venkataraman] Update README remove outdated TODO d436f26 [Davies Liu] add missing files 40d193a [Shivaram Venkataraman] Merge pull request #224 from sun-rui/SPARKR-224-new 1a16cd6 [Davies Liu] rm PROJECT_HOME 56670ef [Davies Liu] rm man page ba4b80b [Davies Liu] Merge branch 'remote_r' into R f04080c [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r 028cbfb [Davies Liu] fix exit code of sparkr unit test 42d8b4c [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R ef26015 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R a1870e8 [Shivaram Venkataraman] Merge pull request #214 from sun-rui/SPARKR-156_3 cb6e5e3 [Shivaram Venkataraman] Add scripts to start SparkR on windows 8030847 [Shivaram Venkataraman] Set windows file separators, install dirs 05afef0 [Shivaram Venkataraman] Only stop backend JVM if R launched it 95d2de3 [Davies Liu] fix spark-submit with R scripot baefd9e [Shivaram Venkataraman] Make bin/sparkR use spark-submit As a part of this move the R initialization functions into first.R and first-submit.R d6f2bdd [Shivaram Venkataraman] Fix run-tests path ea90fab [Davies Liu] fix spark-submit with R path and sparkR -h 0e2412c [Davies Liu] fix bin/sparkR 9f6aa1f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 479e3fe [Davies Liu] change println() to logging 52ca6e5 [Shivaram Venkataraman] Add missing comma 716b16f [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 2d235d4 [Shivaram Venkataraman] Build SparkR with Maven profile aae881b [Davies Liu] fix rat ff776aa [Shivaram Venkataraman] Fix style e4f1937 [Shivaram Venkataraman] Remove DFC example f7b6936 [Davies Liu] remove Spark prefix for class 043959e [Davies Liu] cleanup ba53b09 [Davies Liu] support R in spark-submit f403b4a [Davies Liu] rm .travis.yml c4a5bdf [Davies Liu] run sparkr tests in Spark e8fc7ca [Davies Liu] fix .gitignore 35e5755 [Davies Liu] reduce size of example data 50bff63 [Davies Liu] add LICENSE header for R sources facb6e0 [Davies Liu] add .gitignore for .o, .so, .Rd 18e5eed [Davies Liu] update docs 0a0e632 [Davies Liu] move sparkR into bin/ a76472f [Davies Liu] fix path of assembly jar df3eeea [Davies Liu] move R/examples into examples/src/main/r 3415cc7 [Davies Liu] move Scala source into core/ and sql/ 180fc9c [Davies Liu] move scala 014d253 [Davies Liu] delete man pages 49a8133 [Davies Liu] Merge branch 'remote_r' into R 44994c2 [Davies Liu] Moved files to R/ 2fc553f [Shivaram Venkataraman] Merge pull request #222 from davies/column2 b043876 [Davies Liu] fix test 5e610cb [Davies Liu] add more API for Column 6f95d49 [Shivaram Venkataraman] Merge pull request #221 from shivaram/sparkr-stop-start 3214c6d [Shivaram Venkataraman] Merge pull request #217 from hlin09/cleanClosureFix f5d3355 [Shivaram Venkataraman] Merge pull request #218 from davies/merge 70f620c [Davies Liu] address comments 4b1628d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into merge 3139325 [Shivaram Venkataraman] Merge pull request #212 from davies/toDF 6122e0e [Davies Liu] handle NULL bc2ff38 [Davies Liu] handle NULL 7f5e70c [Davies Liu] Update SerDe.scala 46454e4 [Davies Liu] address comments dd52cbc [Shivaram Venkataraman] Merge pull request #220 from shivaram/sparkr-utils-include 662938a [Shivaram Venkataraman] Include utils before SparkR for `head` to work Before this change calling `head` on a DataFrame would not work from the sparkR script as utils would be loaded after SparkR and placed ahead in the search list. This change requires utils to be loaded before SparkR 1bc2998 [Shivaram Venkataraman] Merge pull request #179 from evertlammerts/sparkr-sql 7695d36 [Evert Lammerts] added tests 8190127 [Evert Lammerts] fixed parquetFile signature d8c8fcc [Shivaram Venkataraman] Merge pull request #219 from shivaram/sparkr-build-final 963c7ee [Davies Liu] Merge branch 'master' into merge 8bff523 [Shivaram Venkataraman] Remove staging repo now that 1.3 is released e52258f [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into toDF 05b9126 [Shivaram Venkataraman] Merge pull request #215 from davies/agg 8e1497d [Davies Liu] Update DataFrame.R 72adb14 [Davies Liu] Update SQLContext.R 66cc92a [Davies Liu] address commets 55c38bc [Shivaram Venkataraman] Merge pull request #216 from davies/select2 3e0555d [Shivaram Venkataraman] Merge pull request #193 from davies/daemon 0467474 [Davies Liu] add more selecter for DataFrame 9a6be74 [Davies Liu] include grouping columns in agg() e87bb98 [Davies Liu] improve comment and logging a6dc435 [Davies Liu] remove dependency of jsonlite 26a3621 [Davies Liu] support date.frame and Date/Time 4e4908a [Davies Liu] createDataFrame from rdd 5757b95 [Shivaram Venkataraman] Merge pull request #196 from davies/die 90f2692 [Shivaram Venkataraman] Merge pull request #211 from hlin09/generics 8583968 [Davies Liu] readFully() 46cea3d [Davies Liu] retry 01aa5ee [Davies Liu] add config for using daemon, refactor ff948db [hlin09] Remove missingOrInteger. ecdfda1 [hlin09] Remove duplication. 411b751 [Davies Liu] make RStudio happy 8f8813f [Davies Liu] switch back to use parallel 6bccbbf [hlin09] Move roxygen doc back to implementation. ffd6e8e [Shivaram Venkataraman] Merge pull request #210 from hlin09/hlin09 471c794 [hlin09] Move getJRDD and broadcast's value to 00-generic.R. 89b886d [hlin09] Move setGeneric() to 00-generics.R. 97dde1a [hlin09] Add a test for access operators. 09ff163 [Shivaram Venkataraman] Merge pull request #204 from cafreeman/sparkr-sql 15a713f [cafreeman] Fix example for `dropTempTable` dc1291b [hlin09] Add checks for namespace access operators in cleanClosure. b4c0b2e [Davies Liu] use fork package 3db5649 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql 789be97 [Shivaram Venkataraman] Merge pull request #207 from shivaram/err-remove e60578a [cafreeman] update tests to guarantee row order 5eec6fc [Shivaram Venkataraman] Merge pull request #206 from sun-rui/SPARKR-156_2 3f7aed6 [Sun Rui] Fix minor typos in the function description. a8cebf0 [Shivaram Venkataraman] Remove print statement in SparkRBackendHandler This print statement is noisy for SQL methods which have multiple APIs (like loadDF). We already have a better error message when no valid methods are found 5e3a576 [Sun Rui] Fix indentation. f3d99a6 [Sun Rui] [SPARKR-156] phase 2: implement zipWithIndex() of the RDD class. a582810 [cafreeman] Merge branch 'dfMethods' into sparkr-sql 7a5d6fd [cafreeman] `withColumn` and `withColumnRenamed` c5fa3b9 [cafreeman] New `select` method bcb0bf5 [Shivaram Venkataraman] Merge pull request #180 from davies/group 9dd6a5a [Davies Liu] Update SparkRBackendHandler.scala e6fb8d8 [Davies Liu] improve logging 428a99a [Davies Liu] remove test, catch exception fef99de [cafreeman] `intersect`, `subtract`, `unionAll` befbd32 [cafreeman] `insertInto` 9d01bcd [cafreeman] `dropTempTable` d8c1c09 [Davies Liu] add test to start and stop context multiple times 18c6004 [Shivaram Venkataraman] Merge pull request #201 from sun-rui/SPARKR-156_1 dfb399a [Davies Liu] address comments f06ccec [Sun Rui] Use mapply() instead of for statement. 3c7674f [Davies Liu] Merge branch 'die' of github.com:davies/SparkR-pkg into die ac8a852 [Davies Liu] close monitor connection in sparkR.stop() 4d0fb56 [Shivaram Venkataraman] Merge pull request #203 from shivaram/sparkr-hive-fix 62b0760 [Shivaram Venkataraman] Fix test hive context package name 47a613f [Shivaram Venkataraman] Fix HiveContext package name fb3b139 [Davies Liu] fix tests d0d4626 [Shivaram Venkataraman] Merge pull request #199 from davies/load 8b7fb67 [Davies Liu] fix HiveContext bb46832 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load e9e2a03 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group b875b4f [Davies Liu] fix style de2abfa [Shivaram Venkataraman] Merge pull request #202 from cafreeman/sparkr-sql 3675fcf [cafreeman] Update `explain` and fixed doc for `toJSON` 5fd9575 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load 6fac596 [Davies Liu] support Column expression in agg() f10a24e [Davies Liu] address comments ff8b005 [cafreeman] 'saveAsParquetFile` a5c2887 [cafreeman] fix test 3fab0f8 [cafreeman] `showDF` 779c102 [cafreeman] `isLocal` 68b11cf [cafreeman] `toJSON` 0ac4abc [cafreeman] 'explain` 20242c4 [cafreeman] clean up docs 6a1fe64 [Shivaram Venkataraman] Merge pull request #198 from cafreeman/sparkr-sql 198c130 [Shivaram Venkataraman] Merge pull request #200 from shivaram/sparkr-sql-build 870acd4 [Shivaram Venkataraman] Use rc2 explicitly 8b9a963 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql bc90115 [cafreeman] Fixed docs 3865f39 [Sun Rui] [SPARKR-156] phase 1: implement zipWithUniqueId() of the RDD class. a37fd80 [Davies Liu] Update sparkR.R d18f9d3 [Shivaram Venkataraman] Remove SparkR snapshot build We now have 1.3.0 RC2 on Apache Staging 8de958d [Davies Liu] Update SparkRBackend.scala 4e0becc [Shivaram Venkataraman] Merge pull request #194 from davies/api 197a79b [Davies Liu] add HiveContext (commented) 32aa01d [Shivaram Venkataraman] Merge pull request #191 from felixcheung/doc 5073e07 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load 7918634 [cafreeman] Fix test acea146 [cafreeman] remove extra line 74269f3 [cafreeman] Merge branch 'dfMethods' into sparkr-sql cd7ac8a [Shivaram Venkataraman] Merge pull request #197 from cafreeman/sparkr-sql 494a4dd [cafreeman] update export e14c328 [cafreeman] `selectExpr` 32b37d1 [cafreeman] Fixed indent in `join` test. 2e7b190 [Felix Cheung] small update on yarn deploy mode. 8ff29d6 [Davies Liu] fix tests 12a6db2 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api 294ca4a [cafreeman] `join`, `sort`, and `filter` 4fa6343 [cafreeman] Refactor `join` generic for use with `DataFrame` 3f22c8d [Shivaram Venkataraman] Merge pull request #195 from cafreeman/sparkr-sql 2b6f980 [Davies Liu] shutdown the JVM after R process die e8639c3 [cafreeman] New 1.3 repo and updates to `column.R` ed9a89f [Davies Liu] address comments 03bcf20 [Davies Liu] Merge branch 'group' of github.com:davies/SparkR-pkg into group 39c253d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group 98cc97a [Davies Liu] fix test and docs e2d144a [Felix Cheung] Fixed small typos 3beadcf [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api 06cbc2d [Davies Liu] launch R worker by a daemon 8a676b1 [Shivaram Venkataraman] Merge pull request #188 from davies/column 524c122 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column f798402 [Davies Liu] Update column.R 1d0f2ae [Davies Liu] Update DataFrame.R 03402eb [Felix Cheung] Updates as per feedback on sparkR-submit 76cf2e0 [Shivaram Venkataraman] Merge pull request #192 from cafreeman/sparkr-sql 1955a09 [cafreeman] return object instead of a list of one object f585929 [cafreeman] Fix brackets e998356 [cafreeman] define generic for 'first' in RDD API 71d66a1 [Davies Liu] fix first(0 8ec21af [Davies Liu] fix signature acae527 [Davies Liu] refactor d7b17a4 [Davies Liu] fix approxCountDistinct 7dfe27d [Davies Liu] fix cyclic namespace dependency 8caf5bb [Davies Liu] use S4 methods 5c0bb24 [Felix Cheung] Doc updates: build and running on YARN 773baf0 [Zongheng Yang] Merge pull request #178 from davies/random 862f07c [Shivaram Venkataraman] Merge pull request #190 from shivaram/SPARKR-79 b457833 [Shivaram Venkataraman] Merge pull request #189 from shivaram/stdErrFix f7caeb8 [Davies Liu] Update SparkRBackend.scala 8c4deae [Shivaram Venkataraman] Remove unused function 6e51c7f [Shivaram Venkataraman] Fix stderr redirection on executors 7afa4c9 [Shivaram Venkataraman] Merge pull request #186 from hlin09/funcDep3 4d36ab1 [hlin09] Add tests for broadcast variables. 3f57e56 [hlin09] Fix comments. 7b72487 [hlin09] Fix comments. ae05bf1 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column abb4bb9 [Davies Liu] add Column and expression eb8ac11 [Shivaram Venkataraman] Set Spark version 1.3.0 in Windows build 5c72e73 [Davies Liu] wait atmost 100 seconds e425437 [Shivaram Venkataraman] Merge pull request #177 from lythesia/master a00f502 [lythesia] fix indents 0346e5f [Davies Liu] address comment 6134649 [Shivaram Venkataraman] Merge pull request #187 from cafreeman/sparkr-sql ad0935e [lythesia] minor fixes b0e7f73 [cafreeman] Update `sampleDF` test 7b0d070 [lythesia] keep partitions check 889c265 [cafreeman] numToInt utility function 27dd3a0 [lythesia] modify tests for repartition cad0f0c [cafreeman] Fix docs and indents 2808dcf [cafreeman] Three more DataFrame methods 5ef66fb [Davies Liu] send back the port via temporary file 3b46429 [Davies Liu] Merge branch 'master' of github.com:amplab-extras/SparkR-pkg into random 798f453 [cafreeman] Merge branch 'sparkr-sql' into dev 9aa4acf [Shivaram Venkataraman] Merge pull request #184 from davies/socket 020bce8 [Shivaram Venkataraman] Merge pull request #183 from cafreeman/sparkr-sql 222e06b [cafreeman] Lazy evaluation and formatting changes e776324 [Davies Liu] fix import 211cc15 [cafreeman] Merge branch 'sparkr-sql' into dev 3351afd [hlin09] Replaces getDependencies with cleanClosure, to serialize UDFs to workers. e7c56d6 [lythesia] fix random partition key 50c74b1 [Davies Liu] address comments 083c89f [cafreeman] Remove commented lines an unused import dfa119b [hlin09] Improve the coverage of processClosure. a41c9b9 [cafreeman] Merge branch 'wrapper' into sparkr-sql 1cd714f [cafreeman] Wrapper function docs. db0cd9e [cafreeman] Clean up for wrapper functions 818c19f [cafreeman] Update schema-related functions a57884e [cafreeman] Remove unused import d72e830 [cafreeman] Add wrapper for `StructField` and `StructType` 2ea2ecf [lythesia] use generic arg 09b9512 [hlin09] add docs f4f077c [hlin09] Add recursive cleanClosure for function access. f84ad27 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2 5300766 [Shivaram Venkataraman] Merge pull request #185 from hlin09/hlin09 07aa7c0 [hlin09] Unifies the implementation of lapply with lapplyParitionsWithIndex. f4dbb0b [Davies Liu] use socket in worker 8282c59 [Davies Liu] Update DataFrame.R ba495a8 [Davies Liu] Update NAMESPACE 36dffb3 [cafreeman] Add 'head` and `first` 534a95f [cafreeman] Schema-related methods 64f488d [cafreeman] Cache and Persist Methods 30d71fd [cafreeman] Standardize method arguments for DataFrame methods 785898b [Shivaram Venkataraman] Merge pull request #182 from cafreeman/sparkr-sql 2619003 [Shivaram Venkataraman] Merge pull request #181 from cafreeman/master a9bbe0b [cafreeman] Update existing SparkSQL functions 8c241a3 [cafreeman] Merge with master, include changes to method args 68d6de4 [cafreeman] Fix typos 8d2ec6e [Davies Liu] add sum/max/min/avg/mean 774e687 [Davies Liu] add missing API in SQLContext 1e72b4b [Davies Liu] missing API in SQLContext 3294949 [Chris Freeman] Restore `rdd` argument to `getJRDD` 3a58ebc [Davies Liu] rm unrelated file 8bd93b5 [Davies Liu] fix signature c652b4c [cafreeman] Update method signatures to use generic arg 48c8827 [Davies Liu] update NAMESPACE 84e2d8c [Davies Liu] groupBy and agg() 7c3ddbd [Davies Liu] create jmode in JVM 9465426 [Davies Liu] load and save 982f342 [lythesia] fix numeric issue 7651d84 [lythesia] fix coalesce 4e712e1 [Davies Liu] use random port in backend 041d22b [Shivaram Venkataraman] Merge pull request #172 from cafreeman/sparkr-sql 0d07770 [cafreeman] Added `limit` and updated `take` 301d8e5 [cafreeman] Remove extraneous map functions 0387db2 [cafreeman] Remove colNames 04c4b65 [lythesia] add repartition/coalesce 231deab [cafreeman] Change reserialize to serializeToBytes acf7e1a [cafreeman] Rework the Scala to R DataFrame Conversion 481ae37 [cafreeman] Updated stale comments and standardized arg names 21d4a97 [hlin09] Adds cleanClosure to capture the function closures. d24ffb4 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2 8be02de [hlin09] Revert "loop 1-12 test pass." fddb9cc [hlin09] Revert "add docs" f8ef0ab [hlin09] Revert "More docs" 8e4b3da [hlin09] Revert "More docs" 57e005b [hlin09] Revert "fix tests." c10148e [Shivaram Venkataraman] Merge pull request #174 from shivaram/sparkr-runner 910e3be [Shivaram Venkataraman] Add a timeout for initialization Also move sparkRBackend.stop into a finally block bf52b17 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner 08102b0 [Shivaram Venkataraman] Merge pull request #176 from lythesia/master 9c77b20 [Chris Freeman] Merge pull request #2 from shivaram/sparkr-sql 179ab38 [lythesia] add try counts and increase time interval 71a73b2 [Shivaram Venkataraman] Use a getter for serialization mode This change encapsulates the semantics of serialization mode for RDDs inside a getter function. For PipelinedRDDs if a backing JavaRDD is available we use that else we fall back to a default serialization mode 06bf250 [Shivaram Venkataraman] Merge pull request #173 from shivaram/windows-space-fix 88bf97f [Shivaram Venkataraman] Create SparkContext for R shell launch f9268d9 [Shivaram Venkataraman] Fix code review comments e6ad12d [Shivaram Venkataraman] Update comment describing sparkR-submit 17eda4c [Shivaram Venkataraman] Merge pull request #175 from falaki/docfix ba2b72b [Hossein] Spark 1.1.0 is default 4cd7d3f [lythesia] retry backend connection 749e2d0 [Hossein] Updated README bc04cf4 [Shivaram Venkataraman] Use SPARKR_BACKEND_PORT in sparkR.R as default Change SparkRRunner to use EXISTING_SPARKR_BACKEND_PORT to differentiate between the two 22a19ac [Shivaram Venkataraman] Use a semaphore to wait for backend to initalize Also pick a random port to avoid collisions 7f1f0f8 [cafreeman] Move comments to fit 100 char line length 8b84e4e [cafreeman] Make if statements more explicit ce5d5ab [cafreeman] New tests for Union and Object File b063320 [cafreeman] Changed 'serialized' to 'serializedMode' 0981dff [Zongheng Yang] Merge pull request #168 from sun-rui/SPARKR-153_2 86fc639 [Shivaram Venkataraman] Move sparkR-submit into pkg/inst fd8f8a9 [Shivaram Venkataraman] Merge branch 'hqzizania-master' a33dbea [Shivaram Venkataraman] Merge branch 'master' of https://github.com/hqzizania/SparkR-pkg into hqzizania-master 384e6e2 [Shivaram Venkataraman] Merge pull request #171 from hlin09/hlin09 1f5a6ac [hlin09] fixed comments 7f7596a [cafreeman] Additional handling for "row" serialization 8c3b8c5 [cafreeman] Add test for UnionRDD on "row" serialization b1141f8 [cafreeman] Fixed formatting issues. 5db30bf [cafreeman] Changed serialized from bool to string 2f0c0b8 [cafreeman] Add check for serialized type d243dfb [cafreeman] Clean up code 5ff63a2 [cafreeman] Change test from boolean to string 77fec1a [cafreeman] Updated .Rd files 9224989 [cafreeman] Various updates for DataFrame to RRDD 26af62b [cafreeman] DataFrame to RRDD e004481 [cafreeman] Update UnionRDD test 5292be7 [hlin09] Adds support of pipeRDD(). e2a7560 [Shivaram Venkataraman] Merge pull request #170 from cafreeman/sparkr-sql 5d537f4 [cafreeman] Add pairRDD to Description b6fa88e [cafreeman] Updating to current master 0cda231 [Sun Rui] [SPARKR-153] phase 2: implement aggregateByKey() and foldByKey(). 95ee6b4 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner 67fbc60 [Shivaram Venkataraman] Add support for SparkR shell to use spark-submit This ensures that SparkConf options are read in both in batch and interactive modes 2271030 [Shivaram Venkataraman] Merge pull request #167 from sun-rui/removePartionByInRDD 7fcb46a [Sun Rui] Remove partitionBy() in RDD. 52f94c4 [Shivaram Venkataraman] Merge pull request #160 from lythesia/master 59e2d54 [lythesia] merge with upstream 5836650 [Zongheng Yang] Merge pull request #163 from sun-rui/SPARKR-153_1 141723e [Sun Rui] fix comments. f73a07e [Shivaram Venkataraman] Merge pull request #165 from shivaram/sparkr-sql-build 10ffc6d [Shivaram Venkataraman] Set Spark version to 1.3 using staging dependency Also fix the maven build c91ede2 [Shivaram Venkataraman] Merge pull request #164 from hlin09/hlin09 9d335a9 [hlin09] Makes git to ignore Eclipse meta files. 94066bf [Sun Rui] [SPARKR-153] phase 1: implement fold() and aggregate(). 9c391c7 [hqzizania] Merge remote-tracking branch 'upstream/master' 5f29551 [hqzizania] modified: pkg/R/RDD.R modified: pkg/R/context.R d968664 [lythesia] fix comment 7972858 [Shivaram Venkataraman] Merge pull request #159 from sun-rui/SPARKR-150_2 7690878 [lythesia] separate out pair RDD functions f4573c1 [Sun Rui] Use reduce() instead of sortBy().take() to get the ordered elements. 63e62ed [Sun Rui] [SPARKR-150] phase 2: implement takeOrdered() and top(). 050390b [Shivaram Venkataraman] Fix bugs in inferring R file 8398f2e [Shivaram Venkataraman] Add sparkR-submit helper script Also adjust R file path for YARN cluster mode bd6705b [Zongheng Yang] Merge pull request #154 from sun-rui/SPARKR-150 c7964c9 [Sun Rui] Merge with upstream master. 7feac38 [Sun Rui] Use default arguments for sortBy() and sortKeyBy(). de2bfb3 [Sun Rui] Fix minor comments and add more test cases. 0c6e071 [Zongheng Yang] Merge pull request #157 from lythesia/master f5038c0 [lythesia] pull out anonymous functions in groupByKey ba6f044 [lythesia] fixes for reduceByKeyLocally 343b6ab [Oscar Olmedo] Export sparkR.stop Closes #156 from oscaroboto/master 25639cf [Shivaram Venkataraman] Replace tabs with spaces bb25920 [Shivaram Venkataraman] Merge branch 'dputler-master' fd836db [hlin09] fix tests. 24a7f13 [hlin09] More docs a465165 [hlin09] More docs 6ad4fc3 [hlin09] add docs b082a35 [lythesia] add reduceByKeyLocally 7ca6512 [Shivaram Venkataraman] First cut of SparkRRunner 193f5fe [hlin09] loop 1-12 test pass. 345f1b8 [dputler] [SPARKR-195] Implemented project style guidelines for if-else statements 8043559 [Sun Rui] Add a TODO to use binary search in the range partitioner. 91b2fd6 [Sun Rui] Add more test cases. e8ebbe4 [Shivaram Venkataraman] Merge pull request #152 from cafreeman/sparkr-sql 0c53d6c [dputler] Data frames now coerced to lists, and messages issued for a data frame or matrix on how they are parallelized 6d57ec0 [cafreeman] Remove json test file since we're using a temp ac1ef09 [cafreeman] Update registerTempTable test d9da451 [Sun Rui] [SPARKR-150] phase 1: implement sortBy() and sortByKey(). 08ff30b [Shivaram Venkataraman] Merge pull request #153 from hqzizania/master 9767e8e [hqzizania] modified: pkg/man/collect-methods.Rd 5d69f0a [hqzizania] modified: pkg/R/RDD.R 4914091 [hqzizania] modified: pkg/inst/tests/test_rdd.R 742a68b [cafreeman] Update test_sparkRSQL.R a95823e [hqzizania] modified: pkg/R/RDD.R 2d04526 [cafreeman] Formatting fae9bdd [cafreeman] Renamed to SQLUtils.scala 39888ea [Chris Freeman] Update test_sparkSQL.R fce2453 [cafreeman] Updated documentation for SQLContext 13fbf12 [cafreeman] Regenerated .Rd files 51ecf41 [cafreeman] Updated Scala object 30d7337 [cafreeman] Added SparkSQL test 74b3ed6 [cafreeman] Incorporate code feedback 554bda0 [Zongheng Yang] Merge pull request #147 from shivaram/sparkr-ec2-fixes a5f4f8f [cafreeman] Squashed commit of the following: f34bb88 [Shivaram Venkataraman] Remove profiling information from this PR c662f29 [Zongheng Yang] Merge pull request #146 from shivaram/spark-1.2-build 21e9b74 [Zongheng Yang] Merge pull request #145 from lythesia/master 76f6b9e [Shivaram Venkataraman] Merge pull request #149 from hqzizania/master 1c2dbec [lythesia] minor fix for refactoring join code 5b380d3 [hqzizania] modified: pkg/man/combineByKey.Rd modified: pkg/man/groupByKey.Rd modified: pkg/man/partitionBy.Rd modified: pkg/man/reduceByKey.Rd 98794fe [hqzizania] modified: pkg/R/RDD.R b66534d [Zongheng Yang] Merge pull request #144 from shivaram/fix-rd-files 60da1df [Shivaram Venkataraman] Initialize timing variables 179aa75 [Shivaram Venkataraman] Bunch of fixes for longer running jobs 1. Increase the timeout for socket connection to wait for long jobs 2. Add some profiling information in worker.R 3. Put temp file writes before stdin writes in RRDD.scala 06d99f0 [Shivaram Venkataraman] Fix URI to have right number of slashes add97f5 [Shivaram Venkataraman] Use URL encode to create valid URIs for jars 4eec962 [lythesia] refactor join functions 73430c6 [Shivaram Venkataraman] Make SparkR work on paths with spaces on Windows aaf8f47 [Shivaram Venkataraman] Exclude hadoop client from Spark dependency 227ee42 [Zongheng Yang] Merge pull request #141 from shivaram/SPARKR-140 ac5ceb1 [Shivaram Venkataraman] Fix code review comments 32394de [Shivaram Venkataraman] Regenerate Rd files for SparkR This fixes a number of issues in SparkR man pages. The main changes are 1. Don't export or generate docs for PipelineRDD 2. Fix variable names for Filter, count to match base methods 3. Document missing arguments for sparkR.init, print.jobj etc. e157bf6 [Shivaram Venkataraman] Use prev_serialized to track if JRDD is serialized This changes introduces a new variable in PipelineRDD environment to track if the prev_jrdd is serialized or not. 7428a7e [Zongheng Yang] Merge pull request #143 from shivaram/SPARKR-181 7dd1797 [Shivaram Venkataraman] Address code review comments 8f81c45 [Shivaram Venkataraman] Remove roxygen export for PipelinedRDD 0cb90f1 [Zongheng Yang] Merge pull request #142 from shivaram/SPARKR-169 d1c6e6c [Shivaram Venkataraman] Buffer stderr from R and return it on Exception This change buffers the last 100 lines from R process and passes these lines back to the driver if we have an exception. This will help users debug why their tasks failed on the cluster d6c1393 [Shivaram Venkataraman] Suppress warnings from normalizePath a382835 [Shivaram Venkataraman] Fix serialization tracking in pipelined RDDs When creating a pipeline RDD, we need to check if the JavaRDD belonging to the parent is serialized. da39529 [Zongheng Yang] Merge pull request #140 from sun-rui/SPARKR-183 2814caa [Sun Rui] Merge with upstream master. cd2a5b3 [Sun Rui] Add reference to Nagle's algorithm and clean code. 52356b6 [Shivaram Venkataraman] Merge pull request #139 from shivaram/fix-backend-exit 97e5a1f [Sun Rui] [SPARKR-183] Fix the issue that parallelize collect tests are slow. a9f8e8e [Shivaram Venkataraman] Merge pull request #138 from concretevitamin/fix-collect-test 125ae43 [Shivaram Venkataraman] Fix SparkR backend to exit in more cases This change has two fixes 1. When the workspace is saved (from R or RStudio) the backend connection seems to be closed before the finalizer is run. In such cases we reopen the connection and stop the backend 2. With RStudio when R is restarted, there are port-conflicts which appear due to a race condition between the JVM and rsession restart. This change adds a 1 sec sleep to avoid this race. 12c102a [Zongheng Yang] Simplify a unit test. 9c0637a [Zongheng Yang] Merge pull request #137 from shivaram/fix-docs 0df0e18 [Shivaram Venkataraman] Fix documentation for includePackage 7549f88 [Zongheng Yang] Merge pull request #136 from shivaram/man-updates 7edbe46 [Shivaram Venkataraman] Add missing man pages 9cb9567 [Shivaram Venkataraman] Merge pull request #131 from shivaram/rJavaExpt 1fa722e [Shivaram Venkataraman] Rename to SerDe now 2fcb051 [Shivaram Venkataraman] Rename to SerDeJVMR d112cf0 [Shivaram Venkataraman] Style fixes 9fd01cc [Shivaram Venkataraman] Remove unnecessary braces 0881931 [Shivaram Venkataraman] Some more style fixes f00b531 [Shivaram Venkataraman] Address code review comments. Big changes include style fixes throughout for named arguments c09ba05 [Shivaram Venkataraman] Change jobj id to be just an integer Add a new print.jobj that gets the class name and prints it Also add a utility function isInstanceOf be05b16 [Shivaram Venkataraman] Check if context, connection exist before stopping d596a23 [Shivaram Venkataraman] Address code review comments 396e7ac [Shivaram Venkataraman] Changes to make new backend work on Windows This change uses file.path to construct the Java binary path in a OS agnostic way and uses system2 to handle quoting binary paths correctly. Tests pass on Mac OSX and a Windows EC2 instance. e7a4e03 [Shivaram Venkataraman] Remove unused file BACKEND.md 62f380b [Shivaram Venkataraman] Update worker.R to use new deserialization call 8b9c4e6 [Shivaram Venkataraman] Change RDD name, setName to use new backend 6dcd5c5 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into rJavaExpt 0873397 [Shivaram Venkataraman] Refactor java object tracking into a new singleton. Also add comments describing each class 95db964 [Shivaram Venkataraman] Add comments, cleanup new R code bcd4258 [Zongheng Yang] Merge pull request #130 from lythesia/master 74dbc5e [Sun Rui] Match method using parameter types. 7ad4a4d [Sun Rui] Use 1 char to represent types on the backend->client direction. bace887 [Sun Rui] Use an integer count for the backend java object ID because Uniqueness isn't guaranteed by System.identityHashCode(). b38d04f [Sun Rui] Use 1 char to represent types on the client -> backend direction. f88bc68 [lythesia] Merge branch 'master' of github.com:lythesia/SparkR-pkg 71d41f5 [lythesia] add test case for fullOuterJoin eb4f423 [lythesia] --amend cffecc5 [lythesia] add test case for fullOuterJoin a547dd2 [Shivaram Venkataraman] Move classTag, rddRef into newJObject call This avoids them getting eagerly garbage collected 1255391 [Shivaram Venkataraman] Add a finalizer for jobj objects This enables Java objects to be garbage collected on the backend when they are no longer referenced in R. Also rename newJava to newJObject to be more consistent with callJMethod 70fa409 [Sun Rui] Add YARN Conf Dir to the class path when launching the backend. a1108ca [lythesia] add fullOuterJoin in RDD.R 2152727 [Shivaram Venkataraman] Remove empty file cd08bee [Shivaram Venkataraman] Update all functions to use new backend All unit tests pass. 9de49b7 [Shivaram Venkataraman] Add high level calls for methods, constructors Also update BACKEND.md 5a97ea4 [Shivaram Venkataraman] Add jobj S3 class that holds backend refs e071d3e [Shivaram Venkataraman] Change SparkRBackend to use general method calls This change uses a custom protocl + JNI to invoke any method on a given object type. Also update serializers, deserializers to make code more concise 49f0404 [Shivaram Venkataraman] Merge pull request #129 from lythesia/master 7f8cd82 [lythesia] update man 4715ed2 [Yi Lu] Update RDD.R 5a53801 [lythesia] fix name,setName 4f3870b [lythesia] add name,setName in RDD.R 1c25700 [Shivaram Venkataraman] Merge pull request #128 from sun-rui/SPARKR-165 c8507d8 [Sun Rui] [SPARKR-165] IS_SCALAR is not present in R before 3.1 2cff2bd [Sun Rui] Add function to invoke Java method. 7a31da1 [Shivaram Venkataraman] Merge branch 'dputler-master'. Closes #119 0ceba82 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/dputler/SparkR-pkg into dputler-master 735f70c [Shivaram Venkataraman] Merge pull request #125 from 7c00/rawcon fccfe6c [Shivaram Venkataraman] Merge pull request #127 from sun-rui/SPARKR-164 387bd57 [Sun Rui] [SPARKR-164] Temporary files used by SparkR accumulat as time goes on. 5f2268f [Shivaram Venkataraman] Add support to stop backend 5f745c0 [Shivaram Venkataraman] Update notes in backend 22015c1 [Shivaram Venkataraman] Add first cut of SparkR Backend 52821da [Todd Gao] switch the order of packages and function deps d7b0007 [Todd Gao] remove memCompress cb6873e [Shivaram Venkataraman] Merge pull request #126 from sun-rui/SPARKR-147 c5962eb [Todd Gao] further optimize using rawConnection f04c6e0 [Sun Rui] [SPARKR-147] Support multiple directories as input to textFile. b7de604 [Todd Gao] optimize execFunctionDeps loading in worker.R 4d4fc30 [Shivaram Venkataraman] Merge pull request #122 from cafreeman/master b508877 [cafreeman] Update SparkR_IDE_Setup.sh 21ed9d7 [cafreeman] Update build.sbt f73ec16 [cafreeman] Delete SparkR_IDE_Setup_Guide.md d63b026 [cafreeman] Delete SparkR_Quick_Start_Guide.md 6e6cb62 [cafreeman] Update SparkR_IDE_Setup.sh bc6042b [cafreeman] Update build.sbt a8197d5 [cafreeman] Merge remote-tracking branch 'upstream/master' d671564 [Zongheng Yang] Merge pull request #123 from shivaram/jcheck-void 76b8d00 [Zongheng Yang] Merge pull request #124 from shivaram/master b690d58 [Shivaram Venkataraman] Specify how to change Spark versions in README 0fb003d [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into jcheck-void 1c227b4 [Shivaram Venkataraman] Also add a check in context.R 96812b6 [Shivaram Venkataraman] Check for exceptions after void method calls f5c216d [cafreeman] Merge remote-tracking branch 'upstream/master' 90c8933 [Zongheng Yang] Merge pull request #121 from shivaram/fix-sort-order bd0e3b4 [Shivaram Venkataraman] Fix saveAsTextFile test case 2e55f67 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into fix-sort-order f10c607 [Shivaram Venkataraman] Merge pull request #118 from sun-rui/saveAsTextFile 6c9bfc0 [Sun Rui] Merge remote-tracking branch 'SparkR_upstream/master' into saveAsTextFile 6faedbe [cafreeman] Update SparkR_IDE_Setup_Guide.md 57008bc [cafreeman] Update SparkR_IDE_Setup.sh bb1c17d [cafreeman] Update SparkR_IDE_Setup.sh 538bfdb [cafreeman] Update SparkR_Quick_Start_Guide.md 31322c6 [cafreeman] Update SparkR_IDE_Setup.sh ca3f593 [Sun Rui] Refactor RRDD code. df58d95 [cafreeman] Update SparkR_Quick_Start_Guide.md b488c88 [cafreeman] Rename Spark_IDE_Setup.sh to SparkR_IDE_Setup.sh b2545a4 [cafreeman] Added IDE Setup Guide 0ffb5de [cafreeman] Merge branch 'master' of https://github.com/cafreeman/SparkR-pkg bd8fbfb [cafreeman] Merge remote-tracking branch 'upstream/master' 98efa5b [cafreeman] Added Quick Start Guide 3cf88f2 [Shivaram Venkataraman] Sort lists before comparing in unit tests Since Spark doesn't guarantee that shuffle results will always be in the same order, we need to sort the results before comparing for deterministic behavior d621dbc [Shivaram Venkataraman] Merge pull request #120 from sun-rui/objectFile c4a44d7 [Sun Rui] Add @seealso in comments and extract some common code into a function. 724e3a4 [cafreeman] Update Spark_IDE_Setup.sh 8153e5a [Sun Rui] [SPARKR-146] Support read/save object files in SparkR. 17f9909 [cafreeman] Update Spark_IDE_Setup.sh a9eb080 [cafreeman] IDE Shell Script 64d800c [dputler] Merge remote branch 'upstream/master' 1fbdb2e [dputler] Added the ability for the user to specify a text file location throught the use of tilde expansion or just the file name if it is in the working directory. d83c017 [Shivaram Venkataraman] Merge pull request #113 from sun-rui/stringHashCodeInC a7d9cdb [Sun Rui] Fix build on Windows. 7d81b05 [Shivaram Venkataraman] Merge pull request #114 from hlin09/hlin09 47c4bb7 [hlin09] fix reviews a457f7f [Shivaram Venkataraman] Merge pull request #116 from dputler/master 0fa48d1 [Shivaram Venkataraman] Merge pull request #117 from sun-rui/keyBy 85cfeb4 [Sun Rui] [SPARKR-144] Implement saveAsTextFile() in the RDD class. 09083d9 [Sun Rui] Add keyBy() to the RDD class. caad5d7 [dputler] Adding the script to install software on the Cloudera Quick Start VM. dca3d05 [hlin09] Minor fix. ece5f7d [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09 a40874b [hlin09] Use extendible accumulators aggregate the cogroup values. d0347ce [Zongheng Yang] Merge pull request #112 from sun-rui/outer_join 492f76e [Sun Rui] Refine code and add description. ba01358 [Shivaram Venkataraman] Merge pull request #115 from sun-rui/SPARKR-130 5c8e46e [Sun Rui] Fix per the review comments. 7190a2c [Sun Rui] Update comment to add a reference to storage levels. 1da705e [hlin09] Fix the review comments. c4b77be [Sun Rui] [SPARKR-130] Add persist(storageLevel) API to RDD. b424a1a [hlin09] Add function cogroup(). 9770312 [Shivaram Venkataraman] Merge pull request #111 from hlin09/hlin09 cead7df [hlin09] fix review comments. 54f712e [Sun Rui] Implement string hash code in C. 425f0c6 [Sun Rui] Add leftOuterJoin() and rightOuterJoin() to the RDD class. 39509c7 [hlin09] add Rd file for foreach and foreachPartition. 63d6ac7 [hlin09] Adds function foreach() and foreachPartition(). 9c954df [Zongheng Yang] Merge pull request #105 from sun-rui/join c71228d [Sun Rui] Pre-allocate list with fixed length. Add test case for join() using string key. bc3e9f6 [Shivaram Venkataraman] Merge pull request #108 from concretevitamin/take-optimize c06fc90 [Zongheng Yang] Fix: only optimize for unserialized dataset case. d399aeb [Zongheng Yang] Apply size-capping on logical representation instead of physical. e4217dd [Zongheng Yang] Merge pull request #107 from shivaram/master 7952180 [Shivaram Venkataraman] Copy, use getLocalDirs from Spark Utils.scala 08e24c3 [Zongheng Yang] Merge pull request #109 from hlin09/hlin09 97d4e02 [Zongheng Yang] Min() upper-bound size with actual size. bb779bf [hlin09] Rename the filter function to filterRDD to follow the API consistency. Filter() is also kept. ce1661f [Zongheng Yang] Fix slow take(): deserialize only up to necessary # of elements. 4dca9b1 [Shivaram Venkataraman] Merge pull request #106 from hlin09/hlin09 1220d92 [hlin09] Adds function numPartitions(). 2326a65 [Shivaram Venkataraman] Use SPARK_LOCAL_DIRS to create tmp files e119757 [hlin09] Minor fix. 9c24c8b [hlin09] Adds function countByKey(). 48fce67 [hlin09] Adds countByValue(). 6679eef [Sun Rui] Update documentation for join(). 70586b4 [Sun Rui] Add join() to the RDD class. e6fb999 [Zongheng Yang] Merge pull request #103 from shivaram/rlibdir-fix a21f146 [Shivaram Venkataraman] Merge pull request #102 from hlin09/hlin09 32eb619 [Shivaram Venkataraman] Merge pull request #104 from sun-rui/add_keys_values d8692e9 [Sun Rui] Add keys() and values() for the RDD class. 18b9be1 [Shivaram Venkataraman] Allow users to set where SparkR is installed This also adds a warning if somebody tries to call sparkR.init multiple times. a17f135 [hlin09] Adds tests for flatMap and flatMapValues. 4bcf59b [hlin09] Adds function flatMapValues. 4a193ef [Zongheng Yang] Merge pull request #101 from ashutoshraina/master 60d22f2 [Ashutosh Raina] changed sbt version 5400793 [Zongheng Yang] Merge pull request #98 from shivaram/windows-fixes-build 36d61a7 [Shivaram Venkataraman] Merge pull request #97 from hlin09/hlin09 f7d7d89 [hlin09] Remove redundant code in test. 6bbe823 [hlin09] minor style fix. 9b47f3a [Shivaram Venkataraman] Merge pull request #100 from hnahak87/patch-1 7f6e4ea [Harihar Nahak] Update logistic_regression.R a605047 [Shivaram Venkataraman] Merge pull request #99 from hlin09/makefile 323151d [hlin09] Fix yar flag in Makefile to remove build error in Maven. 8911897 [hlin09] Make reserialize() private function in package. 79aee73 [Shivaram Venkataraman] Add notes on how to build SparkR on windows 49a99e7 [Shivaram Venkataraman] Clean up some commented code ddc271b [Shivaram Venkataraman] Only append file:/// to non empty jar paths a53952e [Shivaram Venkataraman] Add windows build scripts 325b179 [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09 daf5040 [hlin09] Add reserialize() before union if two RDDs are not both serialized. 536afb1 [hlin09] Add new function of union(). 7044677 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes d22a02d [Zongheng Yang] Merge pull request #94 from shivaram/windows-fixes-stdin 51924f7 [Shivaram Venkataraman] Merge pull request #90 from oscaroboto/master eb97d85 [Shivaram Venkataraman] Merge pull request #96 from sun-rui/add_clarification_readme 5a128f4 [Sun Rui] Add clarification on setting Spark master when launching the SparkR shell. 187526a [oscaroboto] Update sparkR.R 32c567b [Shivaram Venkataraman] Merge pull request #95 from concretevitamin/master 4cd2d5e [Zongheng Yang] Notes about spark-ec2. 1c28e3b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes 8e8a029 [Zongheng Yang] Merge pull request #92 from shivaram/sparkr-yarn 721043b [Zongheng Yang] Update README.md with YARN instructions. 1681f58 [Shivaram Venkataraman] Use temporary files for input instead of stdin This fixes a bug for Windows where stdin would get truncated b084314 [oscaroboto] removed ... from example 44c93d4 [oscaroboto] Added example to SparkR.R be82dcc [Shivaram Venkataraman] Merge pull request #93 from hlin09/hlin09 868554d [oscaroboto] Update sparkR.R 488ac47 [hlin09] Add generated Rd file of previous added functions, distinct() and mapValues(). b2740ad [hlin09] Add test for filter all elements. Add filter() as alias. 08d3631 [hlin09] Minor style fixes. 2c0e34f [hlin09] Adds function Filter(), which extracts the elements that satisfy a predicate. 5951d3b [Shivaram Venkataraman] Remove SBT plugin 4e70ced [oscaroboto] changed ExecutorEnv to sparkExecutorEnvMap, to make it consistent with sparkEnvirMap 903d18a [oscaroboto] changed executorEnv to sparkExecutorEnvMap, will do the same in R f97346e [oscaroboto] executorEnv to lower-case e 88a524e [oscaroboto] Added LD_LIBRARY_PATH to the ExecutorEnv. This is need so that the nodes can find libjvm.so, or if the master has a different LD_LIBRARY_PATH then the nodes. Make sure to export LD_LIBRARY_PATH that includes the path to libjvm.so in the nodes. 1d208ae [oscaroboto] added the YARN_CONF_DIR to the classpath 8a9b75c [oscaroboto] forgot to change hm and ee inside the for loops 579db58 [Shivaram Venkataraman] Merge pull request #91 from sun-rui/add_max_min 4381efa [Sun Rui] use reduce() to implemement max() and min(). a5459c5 [Shivaram Venkataraman] Consolidate yarn flags 86b04eb [Shivaram Venkataraman] Don't use quotes around yarn bf0797f [Shivaram Venkataraman] Add dependency on spark yarn module af5fe77 [Shivaram Venkataraman] Fix SBT build, add dependency tree plugin 4917607 [Sun Rui] Add maximum() and minimum() API to RDD. 51bbbe4 [Shivaram Venkataraman] Changes to make SparkR work with YARN 9d5e3ab [oscaroboto] a few stylistic changes. Also change vars to sparkEnvirMap and eevars to ExecutorEnv, to match sparkR.R 578f545 [oscaroboto] a few stylistic changes 39eea2f [oscaroboto] Modification to dynamically create a sparkContext with YARN. Added .setExecutorEnv to the sparkConf in createSparkContext within the RRDD object. This modification was made together with sparkR.R 17ec42e [oscaroboto] A modification to dynamically create a sparkContext with YARN. sparkR.R modified to pass custom Jar file names and EnvironmentEnv to the sparkConf. RRDD.scala was also modified to accept the new inputs to creatSparkContext. 624ac9d [Shivaram Venkataraman] Merge pull request #87 from sun-rui/SPARKR-125 4f213db [Shivaram Venkataraman] Merge pull request #89 from sun-rui/SPARKR-108 eb833c5 [Shivaram Venkataraman] Merge pull request #88 from hlin09/hlin09 07bf971 [Sun Rui] [SPARKR-108] Implement map-side reduction for reduceByKey(). 4accba1 [hlin09] Fixes style and adds an optional param 'numPartition' in distinct(). 80d303a [hlin09] typo fixed. e37a9b5 [hlin09] Adds function distinct() and mapValues(). 08dac06 [Sun Rui] [SPARKR-125] Get the iterator of the parent RDD before launching a R worker process in compute() of RRDD/PairwiseRRDD c4ba53c [Shivaram Venkataraman] Merge pull request #85 from edwardt/master 72a9d27 [root] reorder to keep relative ordering the same f3fcb10 [root] fix up build.sbt also to match pom.xml 5ecbe3e [root] Make spark verison configurable in build script per ISSUE122 a44e63d [Shivaram Venkataraman] Merge pull request #84 from sun-rui/SPARKR-94 fbb5663 [Sun Rui] Add {} to one-line functions and add a test case for lookup where no match is found. 95beb4e [Shivaram Venkataraman] Merge pull request #82 from edwardt/master 36776c5 [edwardt] missed one 0.9.0 revert b26deec [Sun Rui] [SPARKR-94] Add a method to get an element of a pair RDD object by key. 1ba256e [edwardt] Keep 0.9.0 and says uses 1.1.0 by default 5380c43 [root] missed one version 21f74da [root] upgrade to spark version 1.1.0 to match lastest merge list ddfcde9 [root] merge 67d067a [Shivaram Venkataraman] Merge pull request #81 from sun-rui/SparkR-117 993868f [Sun Rui] [SPARKR-117] Update Spark dependency to 1.1.0 d20661a [Zongheng Yang] Merge pull request #80 from sun-rui/master 0b2da9f [Sun Rui] Update Rd file and add a test case for mapPartitions. 5879648 [Sun Rui] Add mapPartitions() method to RDD for API consistency. c033461 [Shivaram Venkataraman] Merge pull request #79 from sun-rui/fix-kmeans f62b77e [Sun Rui] Adjust coding style. b40911d [Sun Rui] Fix syntax error in examples/kmeans.R. 5304451 [Shivaram Venkataraman] Merge pull request #78 from sun-rui/master 70ffbfb [Sun Rui] Fix a bug that modifications to build.sbt won't trigger rebuilding. a25696c [Shivaram Venkataraman] Merge pull request #76 from edwardt/addjira b8bbd93 [edwardt] Update README.md 615d930 [edwardt] Update README.md e522e69 [edwardt] Update README.md 03e6ced [edwardt] Update README.md 3007015 [root] don't check in gedit buffer file' c35c9a6 [root] Add where to enter bugs ad feeback 469eae3 [edwardt] Update README.md 61b4a43 [edwardt] Update Makefile (style uniformity) ce3337d [edwardt] Update README.md 7ff68fc [root] Merge branch 'master' of https://github.com/edwardt/SparkR-pkg 16353f5 [root] add links to devtools and install_github 513b9e5 [Shivaram Venkataraman] Merge pull request #72 from edwardt/master 31608a4 [edwardt] Update Makefile (style uniformity) 4ffe146 [root] Makefile: factor out SPARKR_VERSION to reduce potential copy&paste error; cp & rm called with -f in build/clean phase; .gitignore includes checkpoints and unit test log generated by run-tests.sh 715275f [Zongheng Yang] Merge pull request #68 from shivaram/master 90e2083 [Shivaram Venkataraman] Add return type to hasNext 8eb983d [Shivaram Venkataraman] Fix up comment 2206164 [Shivaram Venkataraman] Delete temporary files after they are read This change deletes temporary files used for communication between Rscript and the JVM once they have been completely read. 5881da7 [Zongheng Yang] Merge pull request #67 from shivaram/improve-shuffle 81251e2 [Shivaram Venkataraman] Address code review comments a5f573f [Shivaram Venkataraman] Use a better list append in shuffles This is helpful in scenarios where we have a large number of values in a bucket 388e64d [Shivaram Venkataraman] Merge pull request #55 from RevolutionAnalytics/master e1f95b6 [Zongheng Yang] Merge pull request #65 from concretevitamin/parallelize-fix fc1a71a [Zongheng Yang] Fix that collect(parallelize(sc,1:72,15)) drops elements. b8204c5 [Zongheng Yang] Minor: update a URL in README. 86f30c3 [Antonio Piccolboni] better fix for amplab-extras/SparkR-pkg#53 b3c318d [Antonio Piccolboni] delayed loading to have all namespaces available. f323e97 [Antonio Piccolboni] tentative fix for amplab-extras/SparkR-pkg#53 6f82269 [Zongheng Yang] Merge pull request #48 from shivaram/master 8f433e5 [Shivaram Venkataraman] Move up Hadoop in pom.xml and add back protobufs As Hadoop 1.0.4 doesn't use protobufs, we can't exclude protobufs from Spark always. This change tries to order the dependencies so that the shader first picks up Hadoop's protobufs over Mesos. bfe7e26 [Shivaram Venkataraman] Merge pull request #36 from RevolutionAnalytics/vectorize-examples 059ae41 [Antonio Piccolboni] and more formatting 9dbd531 [Antonio Piccolboni] more formatting per committer request 948738a [Antonio Piccolboni] converted tabs to spaces per project request 49f5f5a [Shivaram Venkataraman] Merge pull request #35 from shivaram/master 3eb5ad3 [Shivaram Venkataraman] on_failure -> after_failure in travis.yml 139bdee [Shivaram Venkataraman] Cache sbt, maven, ivy dependencies 4ebced2 [Shivaram Venkataraman] Merge pull request #34 from shivaram/master 8437061 [Shivaram Venkataraman] Exclude protobuf from Spark dependency in Maven This avoids pulling in multiple versions of protobuf from Mesos and Hadoop. 91aa527 [Antonio Piccolboni] vectorized version, 36s 10 slices 10^6 per slice. The older version takes 30 sec on 1/10th of data. f137a57 [Antonio Piccolboni] for rstudio users 1f7ffb0 [Antonio Piccolboni] implemented using matrices and vectorized calls wherever possible 46b23df [Antonio Piccolboni] replace require with library b15d7db [Antonio Piccolboni] faster parsing 8b7aeb3 [Antonio Piccolboni] 22x speed improvement, 3X mem impovement c5bce07 [Zongheng Yang] Merge pull request #30 from shivaram/string-tests 21fa2d8 [Shivaram Venkataraman] Fix bug where serialized was not changed for RRRD Reason: When an RRDD is created in getJRDD we have converted any possibly unserialized RDD to a serialized RDD. 9d1ea20 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into string-tests 7b9348c [Shivaram Venkataraman] Add tests for partition with string keys Add two tests one with a string array and one from a textFile to test both codepaths aacd726 [Shivaram Venkataraman] Update README with maven proxy instructions 803e62c [Shivaram Venkataraman] Merge pull request #28 from concretevitamin/master 7c093e6 [Zongheng Yang] Use inherits() to test an object's class. 061c591 [Shivaram Venkataraman] Merge pull request #26 from hafen/master 90f9fda [Ryan Hafen] Fix isRdd() to properly check for class 5b10cc7 [Zongheng Yang] Merge pull request #24 from shivaram/master 7014f83 [Shivaram Venkataraman] Remove unused transformers in maven's pom.xml b00cea5 [Shivaram Venkataraman] Add support for a Maven build 11ec9b2 [Shivaram Venkataraman] Merge pull request #12 from concretevitamin/pipelined 6b18a90 [Zongheng Yang] Merge branch 'master' into pipelined 57127b8 [Zongheng Yang] Merge pull request #23 from shivaram/master 1ac3940 [Zongheng Yang] Review feedback. a06fb34 [Zongheng Yang] Remove outdated comment. 0a1fc13 [Shivaram Venkataraman] Fixes for using SparkR with Hadoop2. 1. Exclude ASM, Netty from Hadoop similar to Spark. 2. Concat services files to ensure HDFS filesystems work. 3. Update README with an example 9a1db44 [Zongheng Yang] Merge pull request #22 from shivaram/master e462448 [Shivaram Venkataraman] Use `$` for calling `put` instead of .jrcall ed4559a [Shivaram Venkataraman] Add support for passing Spark environment vars This change creates a new `createSparkContext` method in RRDD as we can't pass Map through rJava. Also use SPARK_MEM in local mode to increase heap size and update the README with some examples. 10228fb [Shivaram Venkataraman] Merge pull request #20 from concretevitamin/digit-ex 1398d9f [Zongheng Yang] Add linear_solver_mnist to examples/. d484c2a [Zongheng Yang] Add tests for actions on PipelinedRDD. d9cb95c [Zongheng Yang] Add setCheckpointDir() to context.R; comment fix. f8bc8a9 [Zongheng Yang] Minor edits per Shivaram's comments. 8cd67f7 [Shivaram Venkataraman] Merge pull request #15 from shivaram/master d4468a9 [Shivaram Venkataraman] Remove trailing comma e2714b8 [Shivaram Venkataraman] Remove Apache Staging repo and update README 334eace [Zongheng Yang] Add a multi-transformation test to benchmark on pipelining. 5650ad7 [Zongheng Yang] Put serialized field inside env for both RDD and PipelinedRDD. 0b9e8bb [Zongheng Yang] First cut at PipelinedRDD. a4c431e [Zongheng Yang] Add `isCheckpointed` field and checkpoint(). dac0795 [Zongheng Yang] Minor inline comment style fix. bfb8e26 [Zongheng Yang] Add isCached field (inside an env) and unpersist(). 295bff6 [Zongheng Yang] Merge pull request #11 from shivaram/master 4cb209c [Shivaram Venkataraman] Search rLibDir in worker before libPaths This ensures we pick up the SparkR intended and not an older version installed on the same machine ef198ff [Zongheng Yang] Merge pull request #10 from shivaram/unit-tests e0557a8 [Shivaram Venkataraman] Update travis to install plyr 8b18bc1 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into unit-tests 4a9ca31 [Shivaram Venkataraman] Use smaller broadcast and plyr instead of Matrix Matrix package takes around 2s to load and slows down unit tests. 21c6a61 [Zongheng Yang] Merge pull request #8 from shivaram/master 08c2947 [Shivaram Venkataraman] Move dev install directory to front of libPaths bda42ee [Shivaram Venkataraman] Merge pull request #7 from JoshRosen/travis cc5f5c0 [Josh Rosen] Add Travis CI integration (using craigcitro/r-travis) b6c864b [Shivaram Venkataraman] Merge pull request #6 from concretevitamin/env-style-fix 4fcef22 [Zongheng Yang] Use one style ($) for accessing names in environments. 8a948c6 [Shivaram Venkataraman] Merge pull request #4 from shivaram/master 24978eb [Shivaram Venkataraman] Update README to use install_github 8899db4 [Shivaram Venkataraman] Update TODO.md 91792de [Shivaram Venkataraman] Update Spark requirements f34f4bf [Shivaram Venkataraman] Check tests for failures and output error msg cd750d3 [Shivaram Venkataraman] Update run-tests to use new path 1877b7c [Shivaram Venkataraman] Unset R_TESTS to make tests work with R CMD check Also silence Akka remoting logs and update Makefile to build on log4j changes e60e18a [Shivaram Venkataraman] Update README to remove Spark installation notes 4450189 [Shivaram Venkataraman] Add Spark 0.9 dependency from Apache Staging Also clean up assembly jar from inst on make clean 5eb2131 [Shivaram Venkataraman] Update repo path in README ec8210e [Shivaram Venkataraman] Remove broadcastId hack as it is public in Spark 9f0e080 [Shivaram Venkataraman] Merge branch 'install-github' 5c88fbd [Shivaram Venkataraman] Add helper script to run tests 77450a1 [Shivaram Venkataraman] Remove dependency on Spark Logging 6cb00d1 [Shivaram Venkataraman] Update README and add helper script install-dev.sh 28346ca [Shivaram Venkataraman] Only normalize if SPARK_HOME is not empty 0fd6571 [Shivaram Venkataraman] Normalize SPARK_HOME before passing it ff96d5c [Shivaram Venkataraman] Pass in SPARK_HOME and jar file path 34c4dce [Shivaram Venkataraman] Move src into pkg and update Makefile This enables the package to be installed using install_github using devtools and automates the build procedure. b25afed [Shivaram Venkataraman] Change package name to edu.berkeley.cs.amplab c691464 [Shivaram Venkataraman] Add Apache 2.0 License file 27a4a4b [Shivaram Venkataraman] Add notes on how to compile roxygen2 docs ca63844 [Shivaram Venkataraman] Add broadcast documentation Also generate documentation for sample, takeSample etc. e4dd976 [Shivaram Venkataraman] Update TODO.md e42d435 [Shivaram Venkataraman] Add support for broadcast variables 6b638e7 [Shivaram Venkataraman] Add the assembly jar to SparkContext bf24e32 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg 43c05ce [Zongheng Yang] Fix a flaky/incorrect test for sampleRDD(). c6a9dfc [Zongheng Yang] Initial port of the kmeans example. 6885581 [Zongheng Yang] Implement element-level sampleRDD() and takeSample() with tests. d3a4987 [Zongheng Yang] Add a test for lapplyPartitionsWithIndex on pairwise RDD. c7899c1 [Zongheng Yang] Add lapplyPartitionsWithIndex, with a test and an alias function. a9a7436 [Shivaram Venkataraman] Add DFC example from Tselil, Benjamin and Jonah fbc5a95 [Zongheng Yang] Implement take() and takeSample(). c4a3409 [Shivaram Venkataraman] Use RDD instead of RRDD dfad3f5 [Zongheng Yang] Add test_utils.R: a unit test for convertJListToRList(). a45227d [Zongheng Yang] Update .gitignore. 238fe6e [Zongheng Yang] Add a unit test for textFile(). a88898b [Zongheng Yang] Rename test_rrd to test_rrdd 10c8baa [Shivaram Venkataraman] Make SparkR work as a standalone package. Changes include: 1. Adding a new `sbt` project that builds RRDD.scala 2. Change the onLoad functions to load the assembly jar for SparkR 3. Set rLibDir in RRDD.scala and worker.R to load things correctly 78adcd8 [Shivaram Venkataraman] Add a gitignore ca6108f [Shivaram Venkataraman] Merge branch 'SparkR-scalacode' of ../SparkR 999bd61 [Shivaram Venkataraman] Update collectPartition in R and use ClassTag c58f63e [Shivaram Venkataraman] Update collectPartition in R and use ClassTag 48265fd [Shivaram Venkataraman] Use new version of collectPartitions in take d4fe086 [Shivaram Venkataraman] Move collectPartitions to JavaRDDLike Also remove numPartitions in JavaRDD and update R code bfecd7b [Shivaram Venkataraman] Scala 2.10 changes 1. Update sparkR script 2. Use classTag instead of classManifest 092a4b3 [Shivaram Venkataraman] Add combineByKey, update TODO ac0d81d [Shivaram Venkataraman] Add more documentation d1dc3fa [Shivaram Venkataraman] Add more documentation c515e3a [Shivaram Venkataraman] Update TODO db56a34 [Shivaram Venkataraman] Add a test case for include package 41cea51 [Shivaram Venkataraman] Ensure all parent environments are serialized. Also add a test case with an inline function a978e84 [Shivaram Venkataraman] Add support to include packages in the worker 12bf8ce [Shivaram Venkataraman] Add support to include packages in the worker fb7e72c [Shivaram Venkataraman] Cleanup TODO 16ac314 [Shivaram Venkataraman] Add documentation for functions in context, sparkR 85b1d25 [Shivaram Venkataraman] Set license to Apache 88f1101 [Shivaram Venkataraman] Add unit test running instructions c40768e [Shivaram Venkataraman] Update TODO 0c7efbf [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions 5880d42 [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions 2dee36c [Shivaram Venkataraman] Remove empty test file a82219b [Shivaram Venkataraman] Update TODOs 5db00dc [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD. f196479 [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD. 987e36f [Shivaram Venkataraman] Add perf todo 0b03265 [Shivaram Venkataraman] Update TODO with testing, docs todo 685aaad [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R. 95b9ddc [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R. 4f00895 [Zongheng Yang] Remove the unnecessary `pairwise' flag in RRDD class. Reasons: 75d36d9 [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD. e3fbd9d [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD. 67a4335 [Zongheng Yang] Add unit test for parallelize() and collect() pairwise data. 100ae65 [Zongheng Yang] Properly parallelize() and collect() pairwise data. cd0a5e2 [Zongheng Yang] Properly parallelize() and collect() pairwise data. aea16c3 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues. 45eb943 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues. 11c893b [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func 82c201a [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func b3bfad2 [Zongheng Yang] Update TODO: take() done. 0e45293 [Zongheng Yang] Add ability to parallelize key-val collections in R. f60406a [Zongheng Yang] Add ability to parallelize key-val collections in R. 7d7fe3b [Zongheng Yang] Re-implement take(): take a partition at a time and append. a054e55 [Zongheng Yang] Fix take() tests(): mode difference. 9de0935 [Zongheng Yang] Implement take() for RRDD. 1e4427e [Zongheng Yang] Implement take() for RRDD. ec3cd67 [Shivaram Venkataraman] Use temp file in Spark to pipe output 417aaed [Shivaram Venkataraman] Use temp file in Spark to pipe output bb0a3c3 [Shivaram Venkataraman] Add conf directory to classpath 9594d8a [Shivaram Venkataraman] Clean up LR example 3b26b58 [Shivaram Venkataraman] Add a list of things to do. cabce68 [Shivaram Venkataraman] Fix warnings from package check fde3f9c [Shivaram Venkataraman] Flatten by default and disable recursive unlist ab2e061 [Shivaram Venkataraman] Create LIB_DIR before installing SparkR package 555220a [Shivaram Venkataraman] Add readme and update Makefile 1319cda [Shivaram Venkataraman] Make standalone programs run with sparkR ae19fa8 [Shivaram Venkataraman] Add support for cache and use `tempfile` 4e89ca4 [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples 25a0bea [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples f50223f [Zongheng Yang] Make parallelize() and collect() use lists. Add a few more tests for them. fc7693f [Zongheng Yang] Refactor and enhance the previously added unit test a little bit. 6de9b81 [Zongheng Yang] Add a simple unit test for parallelize(). 8b95155 [Zongheng Yang] Add testthat skeleton infrastructure ef305bf [Zongheng Yang] parallelize() followed by collect() now work for vectors/lists of strings and numerics (should work for other primitives as well). dc16af4 [Zongheng Yang] Comment: toArray() allocates memory for a copy f50121e [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object. 46eb063 [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object. 6b4938a [Zongheng Yang] parallelize(): a raw can be parallelized by JavaSparkContext and get back JavaRDD 978aa0f [Zongheng Yang] Add parallelize() skeleton: only return serialized slices now 84c1fd2 [Zongheng Yang] Use .jsimplify() to get around generic List's get() type erasure problem f16b891 [Zongheng Yang] Convert a few reflectionc alls to .jcall 1284c13 [Zongheng Yang] WIP on collect(): JavaListToRList() failed with errors. 4c2e516 [Zongheng Yang] Add simple prototype of S4 class RRDD. Make TextFile() returns an RRDD. 82aa17a [Zongheng Yang] Add textFile() 83ce63f [Zongheng Yang] Create a JavaSparkContext and save it in .sparkEnv using sparkR.init() 01cdf0e [Zongheng Yang] Add Makefile for SparkR fc9cae2 [Shivaram Venkataraman] Add skeleton R package --- .gitignore | 2 + .rat-excludes | 2 + R/.gitignore | 6 + R/DOCUMENTATION.md | 12 + R/README.md | 67 + R/WINDOWS.md | 13 + R/create-docs.sh | 46 + R/install-dev.bat | 27 + R/install-dev.sh | 36 + R/log4j.properties | 28 + R/pkg/DESCRIPTION | 35 + R/pkg/NAMESPACE | 182 ++ R/pkg/R/DataFrame.R | 1270 ++++++++++++++ R/pkg/R/RDD.R | 1539 +++++++++++++++++ R/pkg/R/SQLContext.R | 520 ++++++ R/pkg/R/SQLTypes.R | 64 + R/pkg/R/backend.R | 115 ++ R/pkg/R/broadcast.R | 86 + R/pkg/R/client.R | 57 + R/pkg/R/column.R | 199 +++ R/pkg/R/context.R | 225 +++ R/pkg/R/deserialize.R | 184 ++ R/pkg/R/generics.R | 543 ++++++ R/pkg/R/group.R | 132 ++ R/pkg/R/jobj.R | 101 ++ R/pkg/R/pairRDD.R | 789 +++++++++ R/pkg/R/serialize.R | 195 +++ R/pkg/R/sparkR.R | 266 +++ R/pkg/R/utils.R | 467 +++++ R/pkg/R/zzz.R | 21 + R/pkg/inst/profile/general.R | 22 + R/pkg/inst/profile/shell.R | 31 + R/pkg/inst/tests/test_binaryFile.R | 90 + R/pkg/inst/tests/test_binary_function.R | 68 + R/pkg/inst/tests/test_broadcast.R | 48 + R/pkg/inst/tests/test_context.R | 50 + R/pkg/inst/tests/test_includePackage.R | 57 + R/pkg/inst/tests/test_parallelize_collect.R | 109 ++ R/pkg/inst/tests/test_rdd.R | 644 +++++++ R/pkg/inst/tests/test_shuffle.R | 209 +++ R/pkg/inst/tests/test_sparkSQL.R | 695 ++++++++ R/pkg/inst/tests/test_take.R | 67 + R/pkg/inst/tests/test_textFile.R | 162 ++ R/pkg/inst/tests/test_utils.R | 137 ++ R/pkg/inst/worker/daemon.R | 52 + R/pkg/inst/worker/worker.R | 128 ++ R/pkg/src/Makefile | 27 + R/pkg/src/Makefile.win | 27 + R/pkg/src/string_hash_code.c | 49 + R/pkg/tests/run-all.R | 21 + R/run-tests.sh | 39 + bin/sparkR | 39 + bin/sparkR.cmd | 23 + bin/sparkR2.cmd | 26 + core/pom.xml | 51 + .../org/apache/spark/api/r/RBackend.scala | 145 ++ .../apache/spark/api/r/RBackendHandler.scala | 223 +++ .../scala/org/apache/spark/api/r/RRDD.scala | 450 +++++ .../scala/org/apache/spark/api/r/SerDe.scala | 340 ++++ .../org/apache/spark/deploy/RRunner.scala | 92 + .../org/apache/spark/deploy/SparkSubmit.scala | 73 +- .../spark/deploy/SparkSubmitArguments.scala | 8 +- dev/run-tests | 15 + dev/run-tests-codes.sh | 1 + dev/run-tests-jenkins | 2 + docs/README.md | 12 +- docs/_layouts/global.html | 1 + docs/_plugins/copy_api_dirs.rb | 15 +- examples/src/main/r/kmeans.R | 93 + examples/src/main/r/linear_solver_mnist.R | 107 ++ examples/src/main/r/logistic_regression.R | 62 + examples/src/main/r/pi.R | 46 + examples/src/main/r/wordcount.R | 42 + .../spark/launcher/CommandBuilderUtils.java | 8 +- .../launcher/SparkSubmitCommandBuilder.java | 87 +- .../launcher/CommandBuilderUtilsSuite.java | 6 +- pom.xml | 3 + .../org/apache/spark/sql/GroupedData.scala | 2 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 127 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../yarn/ApplicationMasterArguments.scala | 11 + .../org/apache/spark/deploy/yarn/Client.scala | 13 +- .../spark/deploy/yarn/ClientArguments.scala | 11 + 83 files changed, 12043 insertions(+), 55 deletions(-) create mode 100644 R/.gitignore create mode 100644 R/DOCUMENTATION.md create mode 100644 R/README.md create mode 100644 R/WINDOWS.md create mode 100755 R/create-docs.sh create mode 100644 R/install-dev.bat create mode 100755 R/install-dev.sh create mode 100644 R/log4j.properties create mode 100644 R/pkg/DESCRIPTION create mode 100644 R/pkg/NAMESPACE create mode 100644 R/pkg/R/DataFrame.R create mode 100644 R/pkg/R/RDD.R create mode 100644 R/pkg/R/SQLContext.R create mode 100644 R/pkg/R/SQLTypes.R create mode 100644 R/pkg/R/backend.R create mode 100644 R/pkg/R/broadcast.R create mode 100644 R/pkg/R/client.R create mode 100644 R/pkg/R/column.R create mode 100644 R/pkg/R/context.R create mode 100644 R/pkg/R/deserialize.R create mode 100644 R/pkg/R/generics.R create mode 100644 R/pkg/R/group.R create mode 100644 R/pkg/R/jobj.R create mode 100644 R/pkg/R/pairRDD.R create mode 100644 R/pkg/R/serialize.R create mode 100644 R/pkg/R/sparkR.R create mode 100644 R/pkg/R/utils.R create mode 100644 R/pkg/R/zzz.R create mode 100644 R/pkg/inst/profile/general.R create mode 100644 R/pkg/inst/profile/shell.R create mode 100644 R/pkg/inst/tests/test_binaryFile.R create mode 100644 R/pkg/inst/tests/test_binary_function.R create mode 100644 R/pkg/inst/tests/test_broadcast.R create mode 100644 R/pkg/inst/tests/test_context.R create mode 100644 R/pkg/inst/tests/test_includePackage.R create mode 100644 R/pkg/inst/tests/test_parallelize_collect.R create mode 100644 R/pkg/inst/tests/test_rdd.R create mode 100644 R/pkg/inst/tests/test_shuffle.R create mode 100644 R/pkg/inst/tests/test_sparkSQL.R create mode 100644 R/pkg/inst/tests/test_take.R create mode 100644 R/pkg/inst/tests/test_textFile.R create mode 100644 R/pkg/inst/tests/test_utils.R create mode 100644 R/pkg/inst/worker/daemon.R create mode 100644 R/pkg/inst/worker/worker.R create mode 100644 R/pkg/src/Makefile create mode 100644 R/pkg/src/Makefile.win create mode 100644 R/pkg/src/string_hash_code.c create mode 100644 R/pkg/tests/run-all.R create mode 100755 R/run-tests.sh create mode 100755 bin/sparkR create mode 100644 bin/sparkR.cmd create mode 100644 bin/sparkR2.cmd create mode 100644 core/src/main/scala/org/apache/spark/api/r/RBackend.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/RRDD.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/SerDe.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/RRunner.scala create mode 100644 examples/src/main/r/kmeans.R create mode 100644 examples/src/main/r/linear_solver_mnist.R create mode 100644 examples/src/main/r/logistic_regression.R create mode 100644 examples/src/main/r/pi.R create mode 100644 examples/src/main/r/wordcount.R create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala diff --git a/.gitignore b/.gitignore index d162fa9cca994..d54d21b802be8 100644 --- a/.gitignore +++ b/.gitignore @@ -63,6 +63,8 @@ ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml +R-unit-tests.log +R/unit-tests.out # For Hive metastore_db/ diff --git a/.rat-excludes b/.rat-excludes index 8c61e67a0c7d1..8aca5a7f7a967 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -67,3 +67,5 @@ logs .*scalastyle-output.xml .*dependency-reduced-pom.xml known_translations +DESCRIPTION +NAMESPACE diff --git a/R/.gitignore b/R/.gitignore new file mode 100644 index 0000000000000..9a5889ba28b2a --- /dev/null +++ b/R/.gitignore @@ -0,0 +1,6 @@ +*.o +*.so +*.Rd +lib +pkg/man +pkg/html diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md new file mode 100644 index 0000000000000..931d01549b265 --- /dev/null +++ b/R/DOCUMENTATION.md @@ -0,0 +1,12 @@ +# SparkR Documentation + +SparkR documentation is generated using in-source comments annotated using using +`roxygen2`. After making changes to the documentation, to generate man pages, +you can run the following from an R console in the SparkR home directory + + library(devtools) + devtools::document(pkg="./pkg", roclets=c("rd")) + +You can verify if your changes are good by running + + R CMD check pkg/ diff --git a/R/README.md b/R/README.md new file mode 100644 index 0000000000000..a6970e39b55f3 --- /dev/null +++ b/R/README.md @@ -0,0 +1,67 @@ +# R on Spark + +SparkR is an R package that provides a light-weight frontend to use Spark from R. + +### SparkR development + +#### Build Spark + +Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run +``` + build/mvn -DskipTests -Psparkr package +``` + +#### Running sparkR + +You can start using SparkR by launching the SparkR shell with + + ./bin/sparkR + +The `sparkR` script automatically creates a SparkContext with Spark by default in +local mode. To specify the Spark master of a cluster for the automatically created +SparkContext, you can run + + ./bin/sparkR --master "local[2]" + +To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR` + +#### Using SparkR from RStudio + +If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example +``` +# Set this to where Spark is installed +Sys.setenv(SPARK_HOME="/Users/shivaram/spark") +# This line loads SparkR from the installed directory +.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) +library(SparkR) +sc <- sparkR.init(master="local") +``` + +#### Making changes to SparkR + +The [instructions](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for making contributions to Spark also apply to SparkR. +If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes. +Once you have made your changes, please include unit tests for them and run existing unit tests using the `run-tests.sh` script as described below. + +#### Generating documentation + +The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script. + +### Examples, Unit tests + +SparkR comes with several sample programs in the `examples/src/main/r` directory. +To run one of them, use `./bin/sparkR `. For example: + + ./bin/sparkR examples/src/main/r/pi.R local[2] + +You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first): + + R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' + ./R/run-tests.sh + +### Running on YARN +The `./bin/spark-submit` and `./bin/sparkR` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run +``` +export YARN_CONF_DIR=/etc/hadoop/conf +./bin/spark-submit --master yarn examples/src/main/r/pi.R 4 +``` diff --git a/R/WINDOWS.md b/R/WINDOWS.md new file mode 100644 index 0000000000000..3f889c0ca3d1e --- /dev/null +++ b/R/WINDOWS.md @@ -0,0 +1,13 @@ +## Building SparkR on Windows + +To build SparkR on Windows, the following steps are required + +1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to +include Rtools and R in `PATH`. +2. Install +[JDK7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) and set +`JAVA_HOME` in the system environment variables. +3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` +directory in Maven in `PATH`. +4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). +5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package` diff --git a/R/create-docs.sh b/R/create-docs.sh new file mode 100755 index 0000000000000..4194172a2e115 --- /dev/null +++ b/R/create-docs.sh @@ -0,0 +1,46 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Script to create API docs for SparkR +# This requires `devtools` and `knitr` to be installed on the machine. + +# After running this script the html docs can be found in +# $SPARK_HOME/R/pkg/html + +# Figure out where the script is +export FWDIR="$(cd "`dirname "$0"`"; pwd)" +pushd $FWDIR + +# Generate Rd file +Rscript -e 'library(devtools); devtools::document(pkg="./pkg", roclets=c("rd"))' + +# Install the package +./install-dev.sh + +# Now create HTML files + +# knit_rd puts html in current working directory +mkdir -p pkg/html +pushd pkg/html + +Rscript -e 'library(SparkR, lib.loc="../../lib"); library(knitr); knit_rd("SparkR")' + +popd + +popd diff --git a/R/install-dev.bat b/R/install-dev.bat new file mode 100644 index 0000000000000..008a5c668bc45 --- /dev/null +++ b/R/install-dev.bat @@ -0,0 +1,27 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Install development version of SparkR +rem + +set SPARK_HOME=%~dp0.. + +MKDIR %SPARK_HOME%\R\lib + +R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ diff --git a/R/install-dev.sh b/R/install-dev.sh new file mode 100755 index 0000000000000..55ed6f4be1a4a --- /dev/null +++ b/R/install-dev.sh @@ -0,0 +1,36 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This scripts packages the SparkR source files (R and C files) and +# creates a package that can be loaded in R. The package is by default installed to +# $FWDIR/lib and the package can be loaded by using the following command in R: +# +# library(SparkR, lib.loc="$FWDIR/lib") +# +# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory +# to load the SparkR package on the worker nodes. + + +FWDIR="$(cd `dirname $0`; pwd)" +LIB_DIR="$FWDIR/lib" + +mkdir -p $LIB_DIR + +# Install R +R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/ diff --git a/R/log4j.properties b/R/log4j.properties new file mode 100644 index 0000000000000..701adb2a3da1d --- /dev/null +++ b/R/log4j.properties @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=R-unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION new file mode 100644 index 0000000000000..1842b97d43651 --- /dev/null +++ b/R/pkg/DESCRIPTION @@ -0,0 +1,35 @@ +Package: SparkR +Type: Package +Title: R frontend for Spark +Version: 1.4.0 +Date: 2013-09-09 +Author: The Apache Software Foundation +Maintainer: Shivaram Venkataraman +Imports: + methods +Depends: + R (>= 3.0), + methods, +Suggests: + testthat +Description: R frontend for Spark +License: Apache License (== 2.0) +Collate: + 'generics.R' + 'jobj.R' + 'SQLTypes.R' + 'RDD.R' + 'pairRDD.R' + 'column.R' + 'group.R' + 'DataFrame.R' + 'SQLContext.R' + 'broadcast.R' + 'context.R' + 'deserialize.R' + 'serialize.R' + 'sparkR.R' + 'backend.R' + 'client.R' + 'utils.R' + 'zzz.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE new file mode 100644 index 0000000000000..a354cdce74afa --- /dev/null +++ b/R/pkg/NAMESPACE @@ -0,0 +1,182 @@ +#exportPattern("^[[:alpha:]]+") +exportClasses("RDD") +exportClasses("Broadcast") +exportMethods( + "aggregateByKey", + "aggregateRDD", + "cache", + "checkpoint", + "coalesce", + "cogroup", + "collect", + "collectAsMap", + "collectPartition", + "combineByKey", + "count", + "countByKey", + "countByValue", + "distinct", + "Filter", + "filterRDD", + "first", + "flatMap", + "flatMapValues", + "fold", + "foldByKey", + "foreach", + "foreachPartition", + "fullOuterJoin", + "glom", + "groupByKey", + "join", + "keyBy", + "keys", + "length", + "lapply", + "lapplyPartition", + "lapplyPartitionsWithIndex", + "leftOuterJoin", + "lookup", + "map", + "mapPartitions", + "mapPartitionsWithIndex", + "mapValues", + "maximum", + "minimum", + "numPartitions", + "partitionBy", + "persist", + "pipeRDD", + "reduce", + "reduceByKey", + "reduceByKeyLocally", + "repartition", + "rightOuterJoin", + "sampleRDD", + "saveAsTextFile", + "saveAsObjectFile", + "sortBy", + "sortByKey", + "sumRDD", + "take", + "takeOrdered", + "takeSample", + "top", + "unionRDD", + "unpersist", + "value", + "values", + "zipRDD", + "zipWithIndex", + "zipWithUniqueId" + ) + +# S3 methods exported +export( + "textFile", + "objectFile", + "parallelize", + "hashCode", + "includePackage", + "broadcast", + "setBroadcastValue", + "setCheckpointDir" + ) +export("sparkR.init") +export("sparkR.stop") +export("print.jobj") +useDynLib(SparkR, stringHashCode) +importFrom(methods, setGeneric, setMethod, setOldClass) + +# SparkRSQL + +exportClasses("DataFrame") + +exportMethods("columns", + "distinct", + "dtypes", + "explain", + "filter", + "groupBy", + "head", + "insertInto", + "intersect", + "isLocal", + "limit", + "orderBy", + "names", + "printSchema", + "registerTempTable", + "repartition", + "sampleDF", + "saveAsParquetFile", + "saveAsTable", + "saveDF", + "schema", + "select", + "selectExpr", + "show", + "showDF", + "sortDF", + "subtract", + "toJSON", + "toRDD", + "unionAll", + "where", + "withColumn", + "withColumnRenamed") + +exportClasses("Column") + +exportMethods("abs", + "alias", + "approxCountDistinct", + "asc", + "avg", + "cast", + "contains", + "countDistinct", + "desc", + "endsWith", + "getField", + "getItem", + "isNotNull", + "isNull", + "last", + "like", + "lower", + "max", + "mean", + "min", + "rlike", + "sqrt", + "startsWith", + "substr", + "sum", + "sumDistinct", + "upper") + +exportClasses("GroupedData") +exportMethods("agg") + +export("sparkRSQL.init", + "sparkRHive.init") + +export("cacheTable", + "clearCache", + "createDataFrame", + "createExternalTable", + "dropTempTable", + "jsonFile", + "jsonRDD", + "loadDF", + "parquetFile", + "sql", + "table", + "tableNames", + "tables", + "toDF", + "uncacheTable") + +export("print.structType", + "print.structField") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R new file mode 100644 index 0000000000000..feafd56909a67 --- /dev/null +++ b/R/pkg/R/DataFrame.R @@ -0,0 +1,1270 @@ +# +# 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. +# + +# DataFrame.R - DataFrame class and methods implemented in S4 OO classes + +#' @include jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame +#' @description DataFrames can be created using functions like +#' \code{jsonFile}, \code{table} etc. +#' @rdname DataFrame +#' @seealso jsonFile, table +#' +#' @param env An R environment that stores bookkeeping states of the DataFrame +#' @param sdf A Java object reference to the backing Scala DataFrame +#' @export +setClass("DataFrame", + slots = list(env = "environment", + sdf = "jobj")) + +setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { + .Object@env <- new.env() + .Object@env$isCached <- isCached + + .Object@sdf <- sdf + .Object +}) + +#' @rdname DataFrame +#' @export +dataFrame <- function(sdf, isCached = FALSE) { + new("DataFrame", sdf, isCached) +} + +############################ DataFrame Methods ############################################## + +#' Print Schema of a DataFrame +#' +#' Prints out the schema in tree format +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname printSchema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' printSchema(df) +#'} +setMethod("printSchema", + signature(x = "DataFrame"), + function(x) { + schemaString <- callJMethod(schema(x)$jobj, "treeString") + cat(schemaString) + }) + +#' Get schema object +#' +#' Returns the schema of this DataFrame as a structType object. +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname schema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dfSchema <- schema(df) +#'} +setMethod("schema", + signature(x = "DataFrame"), + function(x) { + structType(callJMethod(x@sdf, "schema")) + }) + +#' Explain +#' +#' Print the logical and physical Catalyst plans to the console for debugging. +#' +#' @param x A SparkSQL DataFrame +#' @param extended Logical. If extended is False, explain() only prints the physical plan. +#' @rdname explain +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' explain(df, TRUE) +#'} +setMethod("explain", + signature(x = "DataFrame"), + function(x, extended = FALSE) { + queryExec <- callJMethod(x@sdf, "queryExecution") + if (extended) { + cat(callJMethod(queryExec, "toString")) + } else { + execPlan <- callJMethod(queryExec, "executedPlan") + cat(callJMethod(execPlan, "toString")) + } + }) + +#' isLocal +#' +#' Returns True if the `collect` and `take` methods can be run locally +#' (without any Spark executors). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname isLocal +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' isLocal(df) +#'} +setMethod("isLocal", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "isLocal") + }) + +#' ShowDF +#' +#' Print the first numRows rows of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' @param numRows The number of rows to print. Defaults to 20. +#' +#' @rdname showDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' showDF(df) +#'} +setMethod("showDF", + signature(x = "DataFrame"), + function(x, numRows = 20) { + cat(callJMethod(x@sdf, "showString", numToInt(numRows)), "\n") + }) + +#' show +#' +#' Print the DataFrame column names and types +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname show +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' show(df) +#'} +setMethod("show", "DataFrame", + function(object) { + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste("DataFrame[", s, "]\n", sep = "")) + }) + +#' DataTypes +#' +#' Return all column names and their data types as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname dtypes +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dtypes(df) +#'} +setMethod("dtypes", + signature(x = "DataFrame"), + function(x) { + lapply(schema(x)$fields(), function(f) { + c(f$name(), f$dataType.simpleString()) + }) + }) + +#' Column names +#' +#' Return all column names as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname columns +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' columns(df) +#'} +setMethod("columns", + signature(x = "DataFrame"), + function(x) { + sapply(schema(x)$fields(), function(f) { + f$name() + }) + }) + +#' @rdname columns +#' @export +setMethod("names", + signature(x = "DataFrame"), + function(x) { + columns(x) + }) + +#' Register Temporary Table +#' +#' Registers a DataFrame as a Temporary Table in the SQLContext +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @rdname registerTempTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "json_df") +#' new_df <- sql(sqlCtx, "SELECT * FROM json_df") +#'} +setMethod("registerTempTable", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName) { + callJMethod(x@sdf, "registerTempTable", tableName) + }) + +#' insertInto +#' +#' Insert the contents of a DataFrame into a table registered in the current SQL Context. +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' @param overwrite A logical argument indicating whether or not to overwrite +#' the existing rows in the table. +#' +#' @rdname insertInto +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' df2 <- loadDF(sqlCtx, path2, "parquet") +#' registerTempTable(df, "table1") +#' insertInto(df2, "table1", overwrite = TRUE) +#'} +setMethod("insertInto", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName, overwrite = FALSE) { + callJMethod(x@sdf, "insertInto", tableName, overwrite) + }) + +#' Cache +#' +#' Persist with the default storage level (MEMORY_ONLY). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname cache-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' cache(df) +#'} +setMethod("cache", + signature(x = "DataFrame"), + function(x) { + cached <- callJMethod(x@sdf, "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist +#' +#' Persist this DataFrame with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The DataFrame to persist +#' @rdname persist +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#'} +setMethod("persist", + signature(x = "DataFrame", newLevel = "character"), + function(x, newLevel) { + callJMethod(x@sdf, "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist +#' +#' Mark this DataFrame as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The DataFrame to unpersist +#' @param blocking Whether to block until all blocks are deleted +#' @rdname unpersist-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#' unpersist(df) +#'} +setMethod("unpersist", + signature(x = "DataFrame"), + function(x, blocking = TRUE) { + callJMethod(x@sdf, "unpersist", blocking) + x@env$isCached <- FALSE + x + }) + +#' Repartition +#' +#' Return a new DataFrame that has exactly numPartitions partitions. +#' +#' @param x A SparkSQL DataFrame +#' @param numPartitions The number of partitions to use. +#' @rdname repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- repartition(df, 2L) +#'} +setMethod("repartition", + signature(x = "DataFrame", numPartitions = "numeric"), + function(x, numPartitions) { + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + dataFrame(sdf) + }) + +#' toJSON +#' +#' Convert the rows of a DataFrame into JSON objects and return an RDD where +#' each element contains a JSON string. +#' +#' @param x A SparkSQL DataFrame +#' @return A StringRRDD of JSON objects +#' @rdname tojson +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newRDD <- toJSON(df) +#'} +setMethod("toJSON", + signature(x = "DataFrame"), + function(x) { + rdd <- callJMethod(x@sdf, "toJSON") + jrdd <- callJMethod(rdd, "toJavaRDD") + RDD(jrdd, serializedMode = "string") + }) + +#' saveAsParquetFile +#' +#' Save the contents of a DataFrame as a Parquet file, preserving the schema. Files written out +#' with this method can be read back in as a DataFrame using parquetFile(). +#' +#' @param x A SparkSQL DataFrame +#' @param path The directory where the file is saved +#' @rdname saveAsParquetFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsParquetFile(df, "/tmp/sparkr-tmp/") +#'} +setMethod("saveAsParquetFile", + signature(x = "DataFrame", path = "character"), + function(x, path) { + invisible(callJMethod(x@sdf, "saveAsParquetFile", path)) + }) + +#' Distinct +#' +#' Return a new DataFrame containing the distinct rows in this DataFrame. +#' +#' @param x A SparkSQL DataFrame +#' @rdname distinct +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' distinctDF <- distinct(df) +#'} +setMethod("distinct", + signature(x = "DataFrame"), + function(x) { + sdf <- callJMethod(x@sdf, "distinct") + dataFrame(sdf) + }) + +#' SampleDF +#' +#' Return a sampled subset of this DataFrame using a random seed. +#' +#' @param x A SparkSQL DataFrame +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @rdname sampleDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collect(sampleDF(df, FALSE, 0.5)) +#' collect(sampleDF(df, TRUE, 0.5)) +#'} +setMethod("sampleDF", + # TODO : Figure out how to send integer as java.lang.Long to JVM so + # we can send seed as an argument through callJMethod + signature(x = "DataFrame", withReplacement = "logical", + fraction = "numeric"), + function(x, withReplacement, fraction) { + if (fraction < 0.0) stop(cat("Negative fraction value:", fraction)) + sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction) + dataFrame(sdf) + }) + +#' Count +#' +#' Returns the number of rows in a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname count +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' count(df) +#' } +setMethod("count", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "count") + }) + +#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame. +#' +#' @param x A SparkSQL DataFrame +#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns +#' should be converted to factors. FALSE by default. + +#' @rdname collect-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collected <- collect(df) +#' firstName <- collected[[1]]$name +#' } +setMethod("collect", + signature(x = "DataFrame"), + function(x, stringsAsFactors = FALSE) { + # listCols is a list of raw vectors, one per column + listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) + cols <- lapply(listCols, function(col) { + objRaw <- rawConnection(col) + numRows <- readInt(objRaw) + col <- readCol(objRaw, numRows) + close(objRaw) + col + }) + names(cols) <- columns(x) + do.call(cbind.data.frame, list(cols, stringsAsFactors = stringsAsFactors)) + }) + +#' Limit +#' +#' Limit the resulting DataFrame to the number of rows specified. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return +#' @return A new DataFrame containing the number of rows specified. +#' +#' @rdname limit +#' @export +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' limitedDF <- limit(df, 10) +#' } +setMethod("limit", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + res <- callJMethod(x@sdf, "limit", as.integer(num)) + dataFrame(res) + }) + +# Take the first NUM rows of a DataFrame and return a the results as a data.frame + +#' @rdname take +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' take(df, 2) +#' } +setMethod("take", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + limited <- limit(x, num) + collect(limited) + }) + +#' Head +#' +#' Return the first NUM rows of a DataFrame as a data.frame. If NUM is NULL, +#' then head() returns the first 6 rows in keeping with the current data.frame +#' convention in R. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return. Default is 6. +#' @return A data.frame +#' +#' @rdname head +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' head(df) +#' } +setMethod("head", + signature(x = "DataFrame"), + function(x, num = 6L) { + # Default num is 6L in keeping with R's data.frame convention + take(x, num) + }) + +#' Return the first row of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' first(df) +#' } +setMethod("first", + signature(x = "DataFrame"), + function(x) { + take(x, 1) + }) + +#' toRDD() +#' +#' Converts a Spark DataFrame to an RDD while preserving column names. +#' +#' @param x A Spark DataFrame +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' rdd <- toRDD(df) +#' } +setMethod("toRDD", + signature(x = "DataFrame"), + function(x) { + jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf) + colNames <- callJMethod(x@sdf, "columns") + rdd <- RDD(jrdd, serializedMode = "row") + lapply(rdd, function(row) { + names(row) <- colNames + row + }) + }) + +#' GroupBy +#' +#' Groups the DataFrame using the specified columns, so we can run aggregation on them. +#' +#' @param x a DataFrame +#' @return a GroupedData +#' @seealso GroupedData +#' @rdname DataFrame +#' @export +#' @examples +#' \dontrun{ +#' # Compute the average for all numeric columns grouped by department. +#' avg(groupBy(df, "department")) +#' +#' # Compute the max age and average salary, grouped by department and gender. +#' agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max") +#' } +setMethod("groupBy", + signature(x = "DataFrame"), + function(x, ...) { + cols <- list(...) + if (length(cols) >= 1 && class(cols[[1]]) == "character") { + sgd <- callJMethod(x@sdf, "groupBy", cols[[1]], listToSeq(cols[-1])) + } else { + jcol <- lapply(cols, function(c) { c@jc }) + sgd <- callJMethod(x@sdf, "groupBy", listToSeq(jcol)) + } + groupedData(sgd) + }) + +#' Agg +#' +#' Compute aggregates by specifying a list of columns +#' +#' @rdname DataFrame +#' @export +setMethod("agg", + signature(x = "DataFrame"), + function(x, ...) { + agg(groupBy(x), ...) + }) + + +############################## RDD Map Functions ################################## +# All of the following functions mirror the existing RDD map functions, # +# but allow for use with DataFrames by first converting to an RRDD before calling # +# the requested map function. # +################################################################################### + +#' @rdname lapply +setMethod("lapply", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapply(rdd, FUN) + }) + +#' @rdname lapply +setMethod("map", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' @rdname flatMap +setMethod("flatMap", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + flatMap(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("lapplyPartition", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapplyPartition(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("mapPartitions", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' @rdname foreach +setMethod("foreach", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreach(rdd, func) + }) + +#' @rdname foreach +setMethod("foreachPartition", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreachPartition(rdd, func) + }) + + +############################## SELECT ################################## + +getColumn <- function(x, c) { + column(callJMethod(x@sdf, "col", c)) +} + +#' @rdname select +setMethod("$", signature(x = "DataFrame"), + function(x, name) { + getColumn(x, name) + }) + +setMethod("$<-", signature(x = "DataFrame"), + function(x, name, value) { + stopifnot(class(value) == "Column") + cols <- columns(x) + if (name %in% cols) { + cols <- lapply(cols, function(c) { + if (c == name) { + alias(value, name) + } else { + col(c) + } + }) + nx <- select(x, cols) + } else { + nx <- withColumn(x, name, value) + } + x@sdf <- nx@sdf + x + }) + +#' @rdname select +setMethod("[[", signature(x = "DataFrame"), + function(x, i) { + if (is.numeric(i)) { + cols <- columns(x) + i <- cols[[i]] + } + getColumn(x, i) + }) + +#' @rdname select +setMethod("[", signature(x = "DataFrame", i = "missing"), + function(x, i, j, ...) { + if (is.numeric(j)) { + cols <- columns(x) + j <- cols[j] + } + if (length(j) > 1) { + j <- as.list(j) + } + select(x, j) + }) + +#' Select +#' +#' Selects a set of columns with names or Column expressions. +#' @param x A DataFrame +#' @param col A list of columns or single Column or name +#' @return A new DataFrame with selected columns +#' @export +#' @rdname select +#' @examples +#' \dontrun{ +#' select(df, "*") +#' select(df, "col1", "col2") +#' select(df, df$name, df$age + 1) +#' select(df, c("col1", "col2")) +#' select(df, list(df$name, df$age + 1)) +#' # Columns can also be selected using `[[` and `[` +#' df[[2]] == df[["age"]] +#' df[,2] == df[,"age"] +#' # Similar to R data frames columns can also be selected using `$` +#' df$age +#' } +setMethod("select", signature(x = "DataFrame", col = "character"), + function(x, col, ...) { + sdf <- callJMethod(x@sdf, "select", col, toSeq(...)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", signature(x = "DataFrame", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(jcols)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", + signature(x = "DataFrame", col = "list"), + function(x, col) { + cols <- lapply(col, function(c) { + if (class(c)== "Column") { + c@jc + } else { + col(c)@jc + } + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(cols)) + dataFrame(sdf) + }) + +#' SelectExpr +#' +#' Select from a DataFrame using a set of SQL expressions. +#' +#' @param x A DataFrame to be selected from. +#' @param expr A string containing a SQL expression +#' @param ... Additional expressions +#' @return A DataFrame +#' @rdname selectExpr +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' selectExpr(df, "col1", "(col2 * 5) as newCol") +#' } +setMethod("selectExpr", + signature(x = "DataFrame", expr = "character"), + function(x, expr, ...) { + exprList <- list(expr, ...) + sdf <- callJMethod(x@sdf, "selectExpr", listToSeq(exprList)) + dataFrame(sdf) + }) + +#' WithColumn +#' +#' Return a new DataFrame with the specified column added. +#' +#' @param x A DataFrame +#' @param colName A string containing the name of the new column. +#' @param col A Column expression. +#' @return A DataFrame with the new column added. +#' @rdname withColumn +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumn(df, "newCol", df$col1 * 5) +#' } +setMethod("withColumn", + signature(x = "DataFrame", colName = "character", col = "Column"), + function(x, colName, col) { + select(x, x$"*", alias(col, colName)) + }) + +#' WithColumnRenamed +#' +#' Rename an existing column in a DataFrame. +#' +#' @param x A DataFrame +#' @param existingCol The name of the column you want to change. +#' @param newCol The new column name. +#' @return A DataFrame with the column name changed. +#' @rdname withColumnRenamed +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumnRenamed(df, "col1", "newCol1") +#' } +setMethod("withColumnRenamed", + signature(x = "DataFrame", existingCol = "character", newCol = "character"), + function(x, existingCol, newCol) { + cols <- lapply(columns(x), function(c) { + if (c == existingCol) { + alias(col(c), newCol) + } else { + col(c) + } + }) + select(x, cols) + }) + +setClassUnion("characterOrColumn", c("character", "Column")) + +#' SortDF +#' +#' Sort a DataFrame by the specified column(s). +#' +#' @param x A DataFrame to be sorted. +#' @param col Either a Column object or character vector indicating the field to sort on +#' @param ... Additional sorting fields +#' @return A DataFrame where all elements are sorted. +#' @rdname sortDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' sortDF(df, df$col1) +#' sortDF(df, "col1") +#' sortDF(df, asc(df$col1), desc(abs(df$col2))) +#' } +setMethod("sortDF", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col, ...) { + if (class(col) == "character") { + sdf <- callJMethod(x@sdf, "sort", col, toSeq(...)) + } else if (class(col) == "Column") { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "sort", listToSeq(jcols)) + } + dataFrame(sdf) + }) + +#' @rdname sortDF +#' @export +setMethod("orderBy", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col) { + sortDF(x, col) + }) + +#' Filter +#' +#' Filter the rows of a DataFrame according to a given condition. +#' +#' @param x A DataFrame to be sorted. +#' @param condition The condition to sort on. This may either be a Column expression +#' or a string containing a SQL statement +#' @return A DataFrame containing only the rows that meet the condition. +#' @rdname filter +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' filter(df, "col1 > 0") +#' filter(df, df$col2 != "abcdefg") +#' } +setMethod("filter", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + if (class(condition) == "Column") { + condition <- condition@jc + } + sdf <- callJMethod(x@sdf, "filter", condition) + dataFrame(sdf) + }) + +#' @rdname filter +#' @export +setMethod("where", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + filter(x, condition) + }) + +#' Join +#' +#' Join two DataFrames based on the given join expression. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a +#' Column expression. If joinExpr is omitted, join() wil perform a Cartesian join +#' @param joinType The type of join to perform. The following join types are available: +#' 'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'. The default joinType is "inner". +#' @return A DataFrame containing the result of the join operation. +#' @rdname join +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' join(df1, df2) # Performs a Cartesian +#' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression +#' join(df1, df2, df1$col1 == df2$col2, "right_outer") +#' } +setMethod("join", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y, joinExpr = NULL, joinType = NULL) { + if (is.null(joinExpr)) { + sdf <- callJMethod(x@sdf, "join", y@sdf) + } else { + if (class(joinExpr) != "Column") stop("joinExpr must be a Column") + if (is.null(joinType)) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc) + } else { + if (joinType %in% c("inner", "outer", "left_outer", "right_outer", "semijoin")) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) + } else { + stop("joinType must be one of the following types: ", + "'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'") + } + } + } + dataFrame(sdf) + }) + +#' UnionAll +#' +#' Return a new DataFrame containing the union of rows in this DataFrame +#' and another DataFrame. This is equivalent to `UNION ALL` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the union. +#' @rdname unionAll +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' unioned <- unionAll(df, df2) +#' } +setMethod("unionAll", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + unioned <- callJMethod(x@sdf, "unionAll", y@sdf) + dataFrame(unioned) + }) + +#' Intersect +#' +#' Return a new DataFrame containing rows only in both this DataFrame +#' and another DataFrame. This is equivalent to `INTERSECT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the intersect. +#' @rdname intersect +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' intersectDF <- intersect(df, df2) +#' } +setMethod("intersect", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + intersected <- callJMethod(x@sdf, "intersect", y@sdf) + dataFrame(intersected) + }) + +#' Subtract +#' +#' Return a new DataFrame containing rows in this DataFrame +#' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the subtract operation. +#' @rdname subtract +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' subtractDF <- subtract(df, df2) +#' } +setMethod("subtract", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + subtracted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(subtracted) + }) + +#' Save the contents of the DataFrame to a data source +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param path A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveDF", + signature(df = "DataFrame", path = 'character', source = 'character', + mode = 'character'), + function(df, path = NULL, source = NULL, mode = "append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] = path + } + callJMethod(df@sdf, "save", source, jmode, options) + }) + + +#' saveAsTable +#' +#' Save the contents of the DataFrame to a data source as a table +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param tableName A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveAsTable", + signature(df = "DataFrame", tableName = 'character', source = 'character', + mode = 'character'), + function(df, tableName, source = NULL, mode="append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) + }) + diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R new file mode 100644 index 0000000000000..604ad03c407b9 --- /dev/null +++ b/R/pkg/R/RDD.R @@ -0,0 +1,1539 @@ +# +# 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. +# + +# RDD in R implemented in S4 OO system. + +setOldClass("jobj") + +#' @title S4 class that represents an RDD +#' @description RDD can be created using functions like +#' \code{parallelize}, \code{textFile} etc. +#' @rdname RDD +#' @seealso parallelize, textFile +#' +#' @slot env An R environment that stores bookkeeping states of the RDD +#' @slot jrdd Java object reference to the backing JavaRDD +#' to an RDD +#' @export +setClass("RDD", + slots = list(env = "environment", + jrdd = "jobj")) + +setClass("PipelinedRDD", + slots = list(prev = "RDD", + func = "function", + prev_jrdd = "jobj"), + contains = "RDD") + +setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, + isCached, isCheckpointed) { + # Check that RDD constructor is using the correct version of serializedMode + stopifnot(class(serializedMode) == "character") + stopifnot(serializedMode %in% c("byte", "string", "row")) + # RDD has three serialization types: + # byte: The RDD stores data serialized in R. + # string: The RDD stores data as strings. + # row: The RDD stores the serialized rows of a DataFrame. + + # We use an environment to store mutable states inside an RDD object. + # Note that R's call-by-value semantics makes modifying slots inside an + # object (passed as an argument into a function, such as cache()) difficult: + # i.e. one needs to make a copy of the RDD object and sets the new slot value + # there. + + # The slots are inheritable from superclass. Here, both `env' and `jrdd' are + # inherited from RDD, but only the former is used. + .Object@env <- new.env() + .Object@env$isCached <- isCached + .Object@env$isCheckpointed <- isCheckpointed + .Object@env$serializedMode <- serializedMode + + .Object@jrdd <- jrdd + .Object +}) + +setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { + .Object@env <- new.env() + .Object@env$isCached <- FALSE + .Object@env$isCheckpointed <- FALSE + .Object@env$jrdd_val <- jrdd_val + if (!is.null(jrdd_val)) { + # This tracks the serialization mode for jrdd_val + .Object@env$serializedMode <- prev@env$serializedMode + } + + .Object@prev <- prev + + isPipelinable <- function(rdd) { + e <- rdd@env + !(e$isCached || e$isCheckpointed) + } + + if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { + # This transformation is the first in its stage: + .Object@func <- func + .Object@prev_jrdd <- getJRDD(prev) + .Object@env$prev_serializedMode <- prev@env$serializedMode + # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD + # prev_serializedMode is used during the delayed computation of JRDD in getJRDD + } else { + pipelinedFunc <- function(split, iterator) { + func(split, prev@func(split, iterator)) + } + .Object@func <- pipelinedFunc + .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline + # Get the serialization mode of the parent RDD + .Object@env$prev_serializedMode <- prev@env$prev_serializedMode + } + + .Object +}) + +#' @rdname RDD +#' @export +#' +#' @param jrdd Java object reference to the backing JavaRDD +#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +#' stores strings, and "row" if the RDD stores the rows of a DataFrame +#' @param isCached TRUE if the RDD is cached +#' @param isCheckpointed TRUE if the RDD has been checkpointed +RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, + isCheckpointed = FALSE) { + new("RDD", jrdd, serializedMode, isCached, isCheckpointed) +} + +PipelinedRDD <- function(prev, func) { + new("PipelinedRDD", prev, func, NULL) +} + +# Return the serialization mode for an RDD. +setGeneric("getSerializedMode", function(rdd, ...) { standardGeneric("getSerializedMode") }) +# For normal RDDs we can directly read the serializedMode +setMethod("getSerializedMode", signature(rdd = "RDD"), function(rdd) rdd@env$serializedMode ) +# For pipelined RDDs if jrdd_val is set then serializedMode should exist +# if not we return the defaultSerialization mode of "byte" as we don't know the serialization +# mode at this point in time. +setMethod("getSerializedMode", signature(rdd = "PipelinedRDD"), + function(rdd) { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$serializedMode) + } else { + return("byte") + } + }) + +# The jrdd accessor function. +setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) +setMethod("getJRDD", signature(rdd = "PipelinedRDD"), + function(rdd, serializedMode = "byte") { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$jrdd_val) + } + + computeFunc <- function(split, part) { + rdd@func(split, part) + } + + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + serializedFuncArr <- serialize(computeFunc, connection = NULL) + + prev_jrdd <- rdd@prev_jrdd + + if (serializedMode == "string") { + rddRef <- newJObject("org.apache.spark.api.r.StringRRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } else { + rddRef <- newJObject("org.apache.spark.api.r.RRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } + # Save the serialization flag after we create a RRDD + rdd@env$serializedMode <- serializedMode + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val + }) + +setValidity("RDD", + function(object) { + jrdd <- getJRDD(object) + cls <- callJMethod(jrdd, "getClass") + className <- callJMethod(cls, "getName") + if (grep("spark.api.java.*RDD*", className) == 1) { + TRUE + } else { + paste("Invalid RDD class ", className) + } + }) + + +############ Actions and Transformations ############ + +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} +#' @rdname cache-methods +#' @aliases cache,RDD-method +setMethod("cache", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persist(rdd, "MEMORY_AND_DISK") +#'} +#' @rdname persist +#' @aliases persist,RDD-method +setMethod("persist", + signature(x = "RDD", newLevel = "character"), + function(x, newLevel) { + callJMethod(getJRDD(x), "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The RDD to unpersist +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersist(rdd) # rdd@@env$isCached == FALSE +#'} +#' @rdname unpersist-methods +#' @aliases unpersist,RDD-method +setMethod("unpersist", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "unpersist") + x@env$isCached <- FALSE + x + }) + +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param x The RDD to checkpoint +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoints") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} +#' @rdname checkpoint-methods +#' @aliases checkpoint,RDD-method +setMethod("checkpoint", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + callJMethod(jrdd, "checkpoint") + x@env$isCheckpointed <- TRUE + x + }) + +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' numPartitions(rdd) # 2L +#'} +#' @rdname numPartitions +#' @aliases numPartitions,RDD-method +setMethod("numPartitions", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + partitions <- callJMethod(jrdd, "splits") + callJMethod(partitions, "size") + }) + +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collect(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} +#' @rdname collect-methods +#' @aliases collect,RDD-method +setMethod("collect", + signature(x = "RDD"), + function(x, flatten = TRUE) { + # Assumes a pairwise RDD is backed by a JavaPairRDD. + collected <- callJMethod(getJRDD(x), "collect") + convertJListToRList(collected, flatten, + serializedMode = getSerializedMode(x)) + }) + + +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) +#' @rdname collect-methods +#' @aliases collectPartition,integer,RDD-method +setMethod("collectPartition", + signature(x = "RDD", partitionId = "integer"), + function(x, partitionId) { + jPartitionsList <- callJMethod(getJRDD(x), + "collectPartitions", + as.list(as.integer(partitionId))) + + jList <- jPartitionsList[[1]] + convertJListToRList(jList, flatten = TRUE, + serializedMode = getSerializedMode(x)) + }) + +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +#' @rdname collect-methods +#' @aliases collectAsMap,RDD-method +setMethod("collectAsMap", + signature(x = "RDD"), + function(x) { + pairList <- collect(x) + map <- new.env() + lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) + as.list(map) + }) + +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' count(rdd) # 10 +#' length(rdd) # Same as count +#'} +#' @rdname count +#' @aliases count,RDD-method +setMethod("count", + signature(x = "RDD"), + function(x) { + countPartition <- function(part) { + as.integer(length(part)) + } + valsRDD <- lapplyPartition(x, countPartition) + vals <- collect(valsRDD) + sum(as.integer(vals)) + }) + +#' Return the number of elements in the RDD +#' @export +#' @rdname count +setMethod("length", + signature(x = "RDD"), + function(x) { + count(x) + }) + +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} +#' @rdname countByValue +#' @aliases countByValue,RDD-method +setMethod("countByValue", + signature(x = "RDD"), + function(x) { + ones <- lapply(x, function(item) { list(item, 1L) }) + collect(reduceByKey(ones, `+`, numPartitions(x))) + }) + +#' Apply a function to all elements +#' +#' This function creates a new RDD by applying the given transformation to all +#' elements of the given RDD +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @rdname lapply +#' @aliases lapply +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +#' collect(multiplyByTwo) # 2,4,6... +#'} +setMethod("lapply", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(split, iterator) { + lapply(iterator, FUN) + } + lapplyPartitionsWithIndex(X, func) + }) + +#' @rdname lapply +#' @aliases map,RDD,function-method +setMethod("map", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' Flatten results after apply a function to all elements +#' +#' This function return a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collect(multiplyByTwo) # 2,20,4,40,6,60... +#'} +#' @rdname flatMap +#' @aliases flatMap,RDD,function-method +setMethod("flatMap", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + partitionFunc <- function(part) { + unlist( + lapply(part, FUN), + recursive = F + ) + } + lapplyPartition(X, partitionFunc) + }) + +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collect(partitionSum) # 15, 40 +#'} +#' @rdname lapplyPartition +#' @aliases lapplyPartition,RDD,function-method +setMethod("lapplyPartition", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) + }) + +#' mapPartitions is the same as lapplyPartition. +#' +#' @rdname lapplyPartition +#' @aliases mapPartitions,RDD,function-method +setMethod("mapPartitions", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { +#' split * Reduce("+", part) }) +#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} +#' @rdname lapplyPartitionsWithIndex +#' @aliases lapplyPartitionsWithIndex,RDD,function-method +setMethod("lapplyPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + FUN <- cleanClosure(FUN) + closureCapturingFunc <- function(split, part) { + FUN(split, part) + } + PipelinedRDD(X, closureCapturingFunc) + }) + +#' @rdname lapplyPartitionsWithIndex +#' @aliases mapPartitionsWithIndex,RDD,function-method +setMethod("mapPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, FUN) + }) + +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} +#' @rdname filterRDD +#' @aliases filterRDD,RDD,function-method +setMethod("filterRDD", + signature(x = "RDD", f = "function"), + function(x, f) { + filter.func <- function(part) { + Filter(f, part) + } + lapplyPartition(x, filter.func) + }) + +#' @rdname filterRDD +#' @aliases Filter +setMethod("Filter", + signature(f = "function", x = "RDD"), + function(f, x) { + filterRDD(x, f) + }) + +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param x The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} +#' @rdname reduce +#' @aliases reduce,RDD,ANY-method +setMethod("reduce", + signature(x = "RDD", func = "ANY"), + function(x, func) { + + reducePartition <- function(part) { + Reduce(func, part) + } + + partitionList <- collect(lapplyPartition(x, reducePartition), + flatten = FALSE) + Reduce(func, partitionList) + }) + +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} +#' @rdname maximum +#' @aliases maximum,RDD +setMethod("maximum", + signature(x = "RDD"), + function(x) { + reduce(x, max) + }) + +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} +#' @rdname minimum +#' @aliases minimum,RDD +setMethod("minimum", + signature(x = "RDD"), + function(x) { + reduce(x, min) + }) + +#' Add up the elements in an RDD. +#' +#' @param x The RDD to add up the elements in +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' sumRDD(rdd) # 55 +#'} +#' @rdname sumRDD +#' @aliases sumRDD,RDD +setMethod("sumRDD", + signature(x = "RDD"), + function(x) { + reduce(x, "+") + }) + +#' Applies a function to all elements in an RDD, and force evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} +#' @rdname foreach +#' @aliases foreach,RDD,function-method +setMethod("foreach", + signature(x = "RDD", func = "function"), + function(x, func) { + partition.func <- function(x) { + lapply(x, func) + NULL + } + invisible(collect(mapPartitions(x, partition.func))) + }) + +#' Applies a function to each partition in an RDD, and force evaluation. +#' +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} +#' @rdname foreach +#' @aliases foreachPartition,RDD,function-method +setMethod("foreachPartition", + signature(x = "RDD", func = "function"), + function(x, func) { + invisible(collect(mapPartitions(x, func))) + }) + +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' take(rdd, 2L) # list(1, 2) +#'} +#' @rdname take +#' @aliases take,RDD,numeric-method +setMethod("take", + signature(x = "RDD", num = "numeric"), + function(x, num) { + resList <- list() + index <- -1 + jrdd <- getJRDD(x) + numPartitions <- numPartitions(x) + + # TODO(shivaram): Collect more than one partition based on size + # estimates similar to the scala version of `take`. + while (TRUE) { + index <- index + 1 + + if (length(resList) >= num || index >= numPartitions) + break + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + size <- num - length(resList) + # elems is capped to have at most `size` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = size, + serializedMode = getSerializedMode(x)) + # TODO: Check if this append is O(n^2)? + resList <- append(resList, elems) + } + resList + }) + +#' First +#' +#' Return the first element of an RDD +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' first(rdd) +#' } +setMethod("first", + signature(x = "RDD"), + function(x) { + take(x, 1)[[1]] + }) + +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#'} +#' @rdname distinct +#' @aliases distinct,RDD-method +setMethod("distinct", + signature(x = "RDD"), + function(x, numPartitions = SparkR::numPartitions(x)) { + identical.mapped <- lapply(x, function(x) { list(x, NULL) }) + reduced <- reduceByKey(identical.mapped, + function(x, y) { x }, + numPartitions) + resRDD <- lapply(reduced, function(x) { x[[1]] }) + resRDD + }) + +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split +#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} +#' @rdname sampleRDD +#' @aliases sampleRDD,RDD +setMethod("sampleRDD", + signature(x = "RDD", withReplacement = "logical", + fraction = "numeric", seed = "integer"), + function(x, withReplacement, fraction, seed) { + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(split, part) { + set.seed(seed) + res <- vector("list", length(part)) + len <- 0 + + # Discards some random values to ensure each partition has a + # different random seed. + runif(split) + + for (elem in part) { + if (withReplacement) { + count <- rpois(1, fraction) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < fraction) { + len <- len + 1 + res[[len]] <- elem + } + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. + if (len > 0) + res[1:len] + else + list() + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) + +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} +#' @rdname takeSample +#' @aliases takeSample,RDD +setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", + num = "integer", seed = "integer"), + function(x, withReplacement, num, seed) { + # This function is ported from RDD.scala. + fraction <- 0.0 + total <- 0 + multiplier <- 3.0 + initialCount <- count(x) + maxSelected <- 0 + MAXINT <- .Machine$integer.max + + if (num < 0) + stop(paste("Negative number of elements requested")) + + if (initialCount > MAXINT - 1) { + maxSelected <- MAXINT - 1 + } else { + maxSelected <- initialCount + } + + if (num > initialCount && !withReplacement) { + total <- maxSelected + fraction <- multiplier * (maxSelected + 1) / initialCount + } else { + total <- num + fraction <- multiplier * (num + 1) / initialCount + } + + set.seed(seed) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + # If the first sample didn't turn out large enough, keep trying to + # take samples; this shouldn't happen often because we use a big + # multiplier for thei initial size + while (length(samples) < total) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + + # TODO(zongheng): investigate if this call is an in-place shuffle? + sample(samples)[1:total] + }) + +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} +#' @rdname keyBy +#' @aliases keyBy,RDD +setMethod("keyBy", + signature(x = "RDD", func = "function"), + function(x, func) { + apply.func <- function(x) { + list(func(x), x) + } + lapply(x, apply.func) + }) + +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso coalesce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} +#' @rdname repartition +#' @aliases repartition,RDD +setMethod("repartition", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions) { + coalesce(x, numToInt(numPartitions), TRUE) + }) + +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso repartition +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} +#' @rdname coalesce +#' @aliases coalesce,RDD +setMethod("coalesce", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions, shuffle = FALSE) { + numPartitions <- numToInt(numPartitions) + if (shuffle || numPartitions > SparkR::numPartitions(x)) { + func <- function(s, part) { + set.seed(s) # split as seed + start <- as.integer(sample(numPartitions, 1) - 1) + lapply(seq_along(part), + function(i) { + pos <- (start + i) %% numPartitions + list(pos, part[[i]]) + }) + } + shuffled <- lapplyPartitionsWithIndex(x, func) + repartitioned <- partitionBy(shuffled, numPartitions) + values(repartitioned) + } else { + jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) + RDD(jrdd) + } + }) + +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved +#' @seealso objectFile +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsObjectFile +#' @aliases saveAsObjectFile,RDD +setMethod("saveAsObjectFile", + signature(x = "RDD", path = "character"), + function(x, path) { + # If serializedMode == "string" we need to serialize the data before saving it since + # objectFile() assumes serializedMode == "byte". + if (getSerializedMode(x) != "byte") { + x <- serializeToBytes(x) + } + # Return nothing + invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) + }) + +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the splits of the text file are saved +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsTextFile +#' @aliases saveAsTextFile,RDD +setMethod("saveAsTextFile", + signature(x = "RDD", path = "character"), + function(x, path) { + func <- function(str) { + toString(str) + } + stringRdd <- lapply(x, func) + # Return nothing + invisible( + callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) + }) + +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +#' @rdname sortBy +#' @aliases sortBy,RDD,RDD-method +setMethod("sortBy", + signature(x = "RDD", func = "function"), + function(x, func, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + values(sortByKey(keyBy(x, func), ascending, numPartitions)) + }) + +# Helper function to get first N elements from an RDD in the specified order. +# Param: +# x An RDD. +# num Number of elements to return. +# ascending A flag to indicate whether the sorting is ascending or descending. +# Return: +# A list of the first N elements from the RDD in the specified order. +# +takeOrderedElem <- function(x, num, ascending = TRUE) { + if (num <= 0L) { + return(list()) + } + + partitionFunc <- function(part) { + if (num < length(part)) { + # R limitation: order works only on primitive types! + ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) + list(part[ord[1:num]]) + } else { + list(part) + } + } + + reduceFunc <- function(elems, part) { + newElems <- append(elems, part) + # R limitation: order works only on primitive types! + ord <- order(unlist(newElems, recursive = FALSE), decreasing = !ascending) + newElems[ord[1:num]] + } + + newRdd <- mapPartitions(x, partitionFunc) + reduce(newRdd, reduceFunc) +} + +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +#' @rdname takeOrdered +#' @aliases takeOrdered,RDD,RDD-method +setMethod("takeOrdered", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num) + }) + +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +#' @rdname top +#' @aliases top,RDD,RDD-method +setMethod("top", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num, FALSE) + }) + +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +#' @rdname fold +#' @aliases fold,RDD,RDD-method +setMethod("fold", + signature(x = "RDD", zeroValue = "ANY", op = "ANY"), + function(x, zeroValue, op) { + aggregateRDD(x, zeroValue, op, op) + }) + +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4)) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +#' @rdname aggregateRDD +#' @aliases aggregateRDD,RDD,RDD-method +setMethod("aggregateRDD", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), + function(x, zeroValue, seqOp, combOp) { + partitionFunc <- function(part) { + Reduce(seqOp, part, zeroValue) + } + + partitionList <- collect(lapplyPartition(x, partitionFunc), + flatten = FALSE) + Reduce(combOp, partitionList, zeroValue) + }) + +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. +#' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +#' @rdname pipeRDD +#' @aliases pipeRDD,RDD,character-method +setMethod("pipeRDD", + signature(x = "RDD", command = "character"), + function(x, command, env = list()) { + func <- function(part) { + trim.trailing.func <- function(x) { + sub("[\r\n]*$", "", toString(x)) + } + input <- unlist(lapply(part, trim.trailing.func)) + res <- system2(command, stdout = TRUE, input = input, env = env) + lapply(res, trim.trailing.func) + } + lapplyPartition(x, func) + }) + +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. +#' @rdname name +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} +#' @rdname name +#' @aliases name,RDD +setMethod("name", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "name") + }) + +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +#' @rdname setName +#' @aliases setName,RDD +setMethod("setName", + signature(x = "RDD", name = "character"), + function(x, name) { + callJMethod(getJRDD(x), "setName", name) + x + }) + +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithIndex +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +#' @rdname zipWithUniqueId +#' @aliases zipWithUniqueId,RDD +setMethod("zipWithUniqueId", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + + partitionFunc <- function(split, part) { + mapply( + function(item, index) { + list(item, (index - 1) * n + split) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithUniqueId +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +#' @rdname zipWithIndex +#' @aliases zipWithIndex,RDD +setMethod("zipWithIndex", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + if (n > 1) { + nums <- collect(lapplyPartition(x, + function(part) { + list(length(part)) + })) + startIndices <- Reduce("+", nums, accumulate = TRUE) + } + + partitionFunc <- function(split, part) { + if (split == 0) { + startIndex <- 0 + } else { + startIndex <- startIndices[[split]] + } + + mapply( + function(item, index) { + list(item, index - 1 + startIndex) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Coalesce all elements within each partition of an RDD into a list. +#' +#' @param x An RDD. +#' @return An RDD created by coalescing all elements within +#' each partition into a list. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, as.list(1:4), 2L) +#' collect(glom(rdd)) +#' # list(list(1, 2), list(3, 4)) +#'} +#' @rdname glom +#' @aliases glom,RDD +setMethod("glom", + signature(x = "RDD"), + function(x) { + partitionFunc <- function(part) { + list(part) + } + + lapplyPartition(x, partitionFunc) + }) + +############ Binary Functions ############# + +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} +#' @rdname unionRDD +#' @aliases unionRDD,RDD,RDD-method +setMethod("unionRDD", + signature(x = "RDD", y = "RDD"), + function(x, y) { + if (getSerializedMode(x) == getSerializedMode(y)) { + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, getSerializedMode(x)) + } else { + # One of the RDDs is not serialized, we need to serialize it first. + if (getSerializedMode(x) != "byte") x <- serializeToBytes(x) + if (getSerializedMode(y) != "byte") y <- serializeToBytes(y) + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, "byte") + } + union.rdd + }) + +#' Zip an RDD with another RDD. +#' +#' Zips this RDD with another one, returning key-value pairs with the +#' first element in each RDD second element in each RDD, etc. Assumes +#' that the two RDDs have the same number of partitions and the same +#' number of elements in each partition (e.g. one was made through +#' a map on the other). +#' +#' @param x An RDD to be zipped. +#' @param other Another RDD to be zipped. +#' @return An RDD zipped from the two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 0:4) +#' rdd2 <- parallelize(sc, 1000:1004) +#' collect(zipRDD(rdd1, rdd2)) +#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#'} +#' @rdname zipRDD +#' @aliases zipRDD,RDD +setMethod("zipRDD", + signature(x = "RDD", other = "RDD"), + function(x, other) { + n1 <- numPartitions(x) + n2 <- numPartitions(other) + if (n1 != n2) { + stop("Can only zip RDDs which have the same number of partitions.") + } + + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # check if corresponding partitions of both RDDs have the same number of elements. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + part[[length(part) + 1]] <- length(part) + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + + zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) + # The zippedRDD's elements are of scala Tuple2 type. The serialized + # flag Here is used for the elements inside the tuples. + serializerMode <- getSerializedMode(x) + zippedRDD <- RDD(zippedJRDD, serializerMode) + + partitionFunc <- function(split, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # check if corresponding partitions of both RDDs have the same number of elements. + if (lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + keys <- list() + values <- list() + } + } else { + # Keys, values must have same length here, because this has + # been validated inside the JavaRDD.zip() function. + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { + list(k, v) + }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(zippedRDD, partitionFunc) + }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R new file mode 100644 index 0000000000000..930ada22f4c38 --- /dev/null +++ b/R/pkg/R/SQLContext.R @@ -0,0 +1,520 @@ +# +# 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. +# + +# SQLcontext.R: SQLContext-driven functions + +#' infer the SQL type +infer_type <- function(x) { + if (is.null(x)) { + stop("can not infer type from NULL") + } + + # class of POSIXlt is c("POSIXlt" "POSIXt") + type <- switch(class(x)[[1]], + integer = "integer", + character = "string", + logical = "boolean", + double = "double", + numeric = "double", + raw = "binary", + list = "array", + environment = "map", + Date = "date", + POSIXlt = "timestamp", + POSIXct = "timestamp", + stop(paste("Unsupported type for DataFrame:", class(x)))) + + if (type == "map") { + stopifnot(length(x) > 0) + key <- ls(x)[[1]] + list(type = "map", + keyType = "string", + valueType = infer_type(get(key, x)), + valueContainsNull = TRUE) + } else if (type == "array") { + stopifnot(length(x) > 0) + names <- names(x) + if (is.null(names)) { + list(type = "array", elementType = infer_type(x[[1]]), containsNull = TRUE) + } else { + # StructType + types <- lapply(x, infer_type) + fields <- lapply(1:length(x), function(i) { + list(name = names[[i]], type = types[[i]], nullable = TRUE) + }) + list(type = "struct", fields = fields) + } + } else if (length(x) > 1) { + list(type = "array", elementType = type, containsNull = TRUE) + } else { + type + } +} + +#' dump the schema into JSON string +tojson <- function(x) { + if (is.list(x)) { + names <- names(x) + if (!is.null(names)) { + items <- lapply(names, function(n) { + safe_n <- gsub('"', '\\"', n) + paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') + }) + d <- paste(items, collapse = ', ') + paste('{', d, '}', sep = '') + } else { + l <- paste(lapply(x, tojson), collapse = ', ') + paste('[', l, ']', sep = '') + } + } else if (is.character(x)) { + paste('"', x, '"', sep = '') + } else if (is.logical(x)) { + if (x) "true" else "false" + } else { + stop(paste("unexpected type:", class(x))) + } +} + +#' Create a DataFrame from an RDD +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param sqlCtx A SQLContext +#' @param data An RDD or list or data.frame +#' @param schema a list of column names or named list (StructType), optional +#' @return an DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- createDataFrame(sqlCtx, rdd) +#' } + +# TODO(davies): support sampling and infer type from NA +createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { + if (is.data.frame(data)) { + # get the names of columns, they will be put into RDD + schema <- names(data) + n <- nrow(data) + m <- ncol(data) + # get rid of factor type + dropFactor <- function(x) { + if (is.factor(x)) { + as.character(x) + } else { + x + } + } + data <- lapply(1:n, function(i) { + lapply(1:m, function(j) { dropFactor(data[i,j]) }) + }) + } + if (is.list(data)) { + sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlCtx) + rdd <- parallelize(sc, data) + } else if (inherits(data, "RDD")) { + rdd <- data + } else { + stop(paste("unexpected type:", class(data))) + } + + if (is.null(schema) || is.null(names(schema))) { + row <- first(rdd) + names <- if (is.null(schema)) { + names(row) + } else { + as.list(schema) + } + if (is.null(names)) { + names <- lapply(1:length(row), function(x) { + paste("_", as.character(x), sep = "") + }) + } + + # SPAKR-SQL does not support '.' in column name, so replace it with '_' + # TODO(davies): remove this once SPARK-2775 is fixed + names <- lapply(names, function(n) { + nn <- gsub("[.]", "_", n) + if (nn != n) { + warning(paste("Use", nn, "instead of", n, " as column name")) + } + nn + }) + + types <- lapply(row, infer_type) + fields <- lapply(1:length(row), function(i) { + list(name = names[[i]], type = types[[i]], nullable = TRUE) + }) + schema <- list(type = "struct", fields = fields) + } + + stopifnot(class(schema) == "list") + stopifnot(schema$type == "struct") + stopifnot(class(schema$fields) == "list") + schemaString <- tojson(schema) + + jrdd <- getJRDD(lapply(rdd, function(x) x), "row") + srdd <- callJMethod(jrdd, "rdd") + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", + srdd, schemaString, sqlCtx) + dataFrame(sdf) +} + +#' toDF +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param x An RDD +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- toDF(rdd) +#' } + +setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) + +setMethod("toDF", signature(x = "RDD"), + function(x, ...) { + sqlCtx <- if (exists(".sparkRHivesc", envir = .sparkREnv)) { + get(".sparkRHivesc", envir = .sparkREnv) + } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + get(".sparkRSQLsc", envir = .sparkREnv) + } else { + stop("no SQL context available") + } + createDataFrame(sqlCtx, x, ...) + }) + +#' Create a DataFrame from a JSON file. +#' +#' Loads a JSON file (one object per line), returning the result as a DataFrame +#' It goes through the entire dataset once to determine the schema. +#' +#' @param sqlCtx SQLContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' } + +jsonFile <- function(sqlCtx, path) { + # Allow the user to have a more flexible definiton of the text file path + path <- normalizePath(path) + # Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + sdf <- callJMethod(sqlCtx, "jsonFile", path) + dataFrame(sdf) +} + + +#' JSON RDD +#' +#' Loads an RDD storing one JSON object per string as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param rdd An RDD of JSON string +#' @param schema A StructType object to use as schema +#' @param samplingRatio The ratio of simpling used to infer the schema +#' @return A DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- texFile(sc, "path/to/json") +#' df <- jsonRDD(sqlCtx, rdd) +#' } + +# TODO: support schema +jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) { + rdd <- serializeToString(rdd) + if (is.null(schema)) { + sdf <- callJMethod(sqlCtx, "jsonRDD", callJMethod(getJRDD(rdd), "rdd"), samplingRatio) + dataFrame(sdf) + } else { + stop("not implemented") + } +} + + +#' Create a DataFrame from a Parquet file. +#' +#' Loads a Parquet file, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param ... Path(s) of parquet file(s) to read. +#' @return DataFrame +#' @export + +# TODO: Implement saveasParquetFile and write examples for both +parquetFile <- function(sqlCtx, ...) { + # Allow the user to have a more flexible definiton of the text file path + paths <- lapply(list(...), normalizePath) + sdf <- callJMethod(sqlCtx, "parquetFile", paths) + dataFrame(sdf) +} + +#' SQL Query +#' +#' Executes a SQL query using Spark, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param sqlQuery A character vector containing the SQL query +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- sql(sqlCtx, "SELECT * FROM table") +#' } + +sql <- function(sqlCtx, sqlQuery) { + sdf <- callJMethod(sqlCtx, "sql", sqlQuery) + dataFrame(sdf) +} + + +#' Create a DataFrame from a SparkSQL Table +#' +#' Returns the specified Table as a DataFrame. The Table must have already been registered +#' in the SQLContext. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The SparkSQL Table to convert to a DataFrame. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- table(sqlCtx, "table") +#' } + +table <- function(sqlCtx, tableName) { + sdf <- callJMethod(sqlCtx, "table", tableName) + dataFrame(sdf) +} + + +#' Tables +#' +#' Returns a DataFrame containing names of tables in the given database. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tables(sqlCtx, "hive") +#' } + +tables <- function(sqlCtx, databaseName = NULL) { + jdf <- if (is.null(databaseName)) { + callJMethod(sqlCtx, "tables") + } else { + callJMethod(sqlCtx, "tables", databaseName) + } + dataFrame(jdf) +} + + +#' Table Names +#' +#' Returns the names of tables in the given database as an array. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a list of table names +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tableNames(sqlCtx, "hive") +#' } + +tableNames <- function(sqlCtx, databaseName = NULL) { + if (is.null(databaseName)) { + callJMethod(sqlCtx, "tableNames") + } else { + callJMethod(sqlCtx, "tableNames", databaseName) + } +} + + +#' Cache Table +#' +#' Caches the specified table in-memory. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being cached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' cacheTable(sqlCtx, "table") +#' } + +cacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "cacheTable", tableName) +} + +#' Uncache Table +#' +#' Removes the specified table from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being uncached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' uncacheTable(sqlCtx, "table") +#' } + +uncacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "uncacheTable", tableName) +} + +#' Clear Cache +#' +#' Removes all cached tables from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @examples +#' \dontrun{ +#' clearCache(sqlCtx) +#' } + +clearCache <- function(sqlCtx) { + callJMethod(sqlCtx, "clearCache") +} + +#' Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the SparkSQL table to be dropped. +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' registerTempTable(df, "table") +#' dropTempTable(sqlCtx, "table") +#' } + +dropTempTable <- function(sqlCtx, tableName) { + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + callJMethod(sqlCtx, "dropTempTable", tableName) +} + +#' Load an DataFrame +#' +#' Returns the dataset in a data source as a DataFrame +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- load(sqlCtx, "path/to/file.json", source = "json") +#' } + +loadDF <- function(sqlCtx, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "load", source, options) + dataFrame(sdf) +} + +#' Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns the DataFrame associated with the external table. +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName A name of the table +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- sparkRSQL.createExternalTable(sqlCtx, "myjson", path="path/to/json", source="json") +#' } + +createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options) + dataFrame(sdf) +} diff --git a/R/pkg/R/SQLTypes.R b/R/pkg/R/SQLTypes.R new file mode 100644 index 0000000000000..962fba5b3cf03 --- /dev/null +++ b/R/pkg/R/SQLTypes.R @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Utility functions for handling SparkSQL DataTypes. + +# Handler for StructType +structType <- function(st) { + obj <- structure(new.env(parent = emptyenv()), class = "structType") + obj$jobj <- st + obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) } + obj +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + fieldsList <- lapply(x$fields(), function(i) { i$print() }) + print(fieldsList) +} + +# Handler for StructField +structField <- function(sf) { + obj <- structure(new.env(parent = emptyenv()), class = "structField") + obj$jobj <- sf + obj$name <- function() { callJMethod(sf, "name") } + obj$dataType <- function() { callJMethod(sf, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(sf, "nullable") } + obj$print <- function() { paste("StructField(", + paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "), + ")", sep = "") } + obj +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat(x$print()) +} diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R new file mode 100644 index 0000000000000..2fb6fae55f28c --- /dev/null +++ b/R/pkg/R/backend.R @@ -0,0 +1,115 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Methods to call into SparkRBackend. + + +# Returns TRUE if object is an instance of given class +isInstanceOf <- function(jobj, className) { + stopifnot(class(jobj) == "jobj") + cls <- callJStatic("java.lang.Class", "forName", className) + callJMethod(cls, "isInstance", jobj) +} + +# Call a Java method named methodName on the object +# specified by objId. objId should be a "jobj" returned +# from the SparkRBackend. +callJMethod <- function(objId, methodName, ...) { + stopifnot(class(objId) == "jobj") + if (!isValidJobj(objId)) { + stop("Invalid jobj ", objId$id, + ". If SparkR was restarted, Spark operations need to be re-executed.") + } + invokeJava(isStatic = FALSE, objId$id, methodName, ...) +} + +# Call a static method on a specified className +callJStatic <- function(className, methodName, ...) { + invokeJava(isStatic = TRUE, className, methodName, ...) +} + +# Create a new object of the specified class name +newJObject <- function(className, ...) { + invokeJava(isStatic = TRUE, className, methodName = "", ...) +} + +# Remove an object from the SparkR backend. This is done +# automatically when a jobj is garbage collected. +removeJObject <- function(objId) { + invokeJava(isStatic = TRUE, "SparkRHandler", "rm", objId) +} + +isRemoveMethod <- function(isStatic, objId, methodName) { + isStatic == TRUE && objId == "SparkRHandler" && methodName == "rm" +} + +# Invoke a Java method on the SparkR backend. Users +# should typically use one of the higher level methods like +# callJMethod, callJStatic etc. instead of using this. +# +# isStatic - TRUE if the method to be called is static +# objId - String that refers to the object on which method is invoked +# Should be a jobj id for non-static methods and the classname +# for static methods +# methodName - name of method to be invoked +invokeJava <- function(isStatic, objId, methodName, ...) { + if (!exists(".sparkRCon", .sparkREnv)) { + stop("No connection to backend found. Please re-run sparkR.init") + } + + # If this isn't a removeJObject call + if (!isRemoveMethod(isStatic, objId, methodName)) { + objsToRemove <- ls(.toRemoveJobjs) + if (length(objsToRemove) > 0) { + sapply(objsToRemove, + function(e) { + removeJObject(e) + }) + rm(list = objsToRemove, envir = .toRemoveJobjs) + } + } + + + rc <- rawConnection(raw(0), "r+") + + writeBoolean(rc, isStatic) + writeString(rc, objId) + writeString(rc, methodName) + + args <- list(...) + writeInt(rc, length(args)) + writeArgs(rc, args) + + # Construct the whole request message to send it once, + # avoiding write-write-read pattern in case of Nagle's algorithm. + # Refer to http://en.wikipedia.org/wiki/Nagle%27s_algorithm for the details. + bytesToSend <- rawConnectionValue(rc) + close(rc) + rc <- rawConnection(raw(0), "r+") + writeInt(rc, length(bytesToSend)) + writeBin(bytesToSend, rc) + requestMessage <- rawConnectionValue(rc) + close(rc) + + conn <- get(".sparkRCon", .sparkREnv) + writeBin(requestMessage, conn) + + # TODO: check the status code to output error information + returnStatus <- readInt(conn) + stopifnot(returnStatus == 0) + readObject(conn) +} diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R new file mode 100644 index 0000000000000..583fa2e7fdcfd --- /dev/null +++ b/R/pkg/R/broadcast.R @@ -0,0 +1,86 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# S4 class representing Broadcast variables + +# Hidden environment that holds values for broadcast variables +# This will not be serialized / shipped by default +.broadcastNames <- new.env() +.broadcastValues <- new.env() +.broadcastIdToName <- new.env() + +#' @title S4 class that represents a Broadcast variable +#' @description Broadcast variables can be created using the broadcast +#' function from a \code{SparkContext}. +#' @rdname broadcast-class +#' @seealso broadcast +#' +#' @param id Id of the backing Spark broadcast variable +#' @export +setClass("Broadcast", slots = list(id = "character")) + +#' @rdname broadcast-class +#' @param value Value of the broadcast variable +#' @param jBroadcastRef reference to the backing Java broadcast object +#' @param objName name of broadcasted object +#' @export +Broadcast <- function(id, value, jBroadcastRef, objName) { + .broadcastValues[[id]] <- value + .broadcastNames[[as.character(objName)]] <- jBroadcastRef + .broadcastIdToName[[id]] <- as.character(objName) + new("Broadcast", id = id) +} + +#' @description +#' \code{value} can be used to get the value of a broadcast variable inside +#' a distributed function. +#' +#' @param bcast The broadcast variable to get +#' @rdname broadcast +#' @aliases value,Broadcast-method +setMethod("value", + signature(bcast = "Broadcast"), + function(bcast) { + if (exists(bcast@id, envir = .broadcastValues)) { + get(bcast@id, envir = .broadcastValues) + } else { + NULL + } + }) + +#' Internal function to set values of a broadcast variable. +#' +#' This function is used internally by Spark to set the value of a broadcast +#' variable on workers. Not intended for use outside the package. +#' +#' @rdname broadcast-internal +#' @seealso broadcast, value + +#' @param bcastId The id of broadcast variable to set +#' @param value The value to be set +#' @export +setBroadcastValue <- function(bcastId, value) { + bcastIdStr <- as.character(bcastId) + .broadcastValues[[bcastIdStr]] <- value +} + +#' Helper function to clear the list of broadcast variables we know about +#' Should be called when the SparkR JVM backend is shutdown +clearBroadcastVariables <- function() { + bcasts <- ls(.broadcastNames) + rm(list = bcasts, envir = .broadcastNames) +} diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R new file mode 100644 index 0000000000000..1281c41213e32 --- /dev/null +++ b/R/pkg/R/client.R @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Client code to connect to SparkRBackend + +# Creates a SparkR client connection object +# if one doesn't already exist +connectBackend <- function(hostname, port, timeout = 6000) { + if (exists(".sparkRcon", envir = .sparkREnv)) { + if (isOpen(.sparkREnv[[".sparkRCon"]])) { + cat("SparkRBackend client connection already exists\n") + return(get(".sparkRcon", envir = .sparkREnv)) + } + } + + con <- socketConnection(host = hostname, port = port, server = FALSE, + blocking = TRUE, open = "wb", timeout = timeout) + + assign(".sparkRCon", con, envir = .sparkREnv) + con +} + +launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts) { + if (.Platform$OS.type == "unix") { + sparkSubmitBinName = "spark-submit" + } else { + sparkSubmitBinName = "spark-submit.cmd" + } + + if (sparkHome != "") { + sparkSubmitBin <- file.path(sparkHome, "bin", sparkSubmitBinName) + } else { + sparkSubmitBin <- sparkSubmitBinName + } + + if (jars != "") { + jars <- paste("--jars", jars) + } + + combinedArgs <- paste(jars, sparkSubmitOpts, args, sep = " ") + cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n") + invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) +} diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R new file mode 100644 index 0000000000000..e196305186b9a --- /dev/null +++ b/R/pkg/R/column.R @@ -0,0 +1,199 @@ +# +# 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. +# + +# Column Class + +#' @include generics.R jobj.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame column +#' @description The column class supports unary, binary operations on DataFrame columns + +#' @rdname column +#' +#' @param jc reference to JVM DataFrame column +#' @export +setClass("Column", + slots = list(jc = "jobj")) + +setMethod("initialize", "Column", function(.Object, jc) { + .Object@jc <- jc + .Object +}) + +column <- function(jc) { + new("Column", jc) +} + +col <- function(x) { + column(callJStatic("org.apache.spark.sql.functions", "col", x)) +} + +#' @rdname show +setMethod("show", "Column", + function(object) { + cat("Column", callJMethod(object@jc, "toString"), "\n") + }) + +operators <- list( + "+" = "plus", "-" = "minus", "*" = "multiply", "/" = "divide", "%%" = "mod", + "==" = "equalTo", ">" = "gt", "<" = "lt", "!=" = "notEqual", "<=" = "leq", ">=" = "geq", + # we can not override `&&` and `||`, so use `&` and `|` instead + "&" = "and", "|" = "or" #, "!" = "unary_$bang" +) +column_functions1 <- c("asc", "desc", "isNull", "isNotNull") +column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains") +functions <- c("min", "max", "sum", "avg", "mean", "count", "abs", "sqrt", + "first", "last", "lower", "upper", "sumDistinct") + +createOperator <- function(op) { + setMethod(op, + signature(e1 = "Column"), + function(e1, e2) { + jc <- if (missing(e2)) { + if (op == "-") { + callJMethod(e1@jc, "unary_$minus") + } else { + callJMethod(e1@jc, operators[[op]]) + } + } else { + if (class(e2) == "Column") { + e2 <- e2@jc + } + callJMethod(e1@jc, operators[[op]], e2) + } + column(jc) + }) +} + +createColumnFunction1 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + column(callJMethod(x@jc, name)) + }) +} + +createColumnFunction2 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x, data) { + if (class(data) == "Column") { + data <- data@jc + } + jc <- callJMethod(x@jc, name, data) + column(jc) + }) +} + +createStaticFunction <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", name, x@jc) + column(jc) + }) +} + +createMethods <- function() { + for (op in names(operators)) { + createOperator(op) + } + for (name in column_functions1) { + createColumnFunction1(name) + } + for (name in column_functions2) { + createColumnFunction2(name) + } + for (x in functions) { + createStaticFunction(x) + } +} + +createMethods() + +#' alias +#' +#' Set a new name for a column +setMethod("alias", + signature(object = "Column"), + function(object, data) { + if (is.character(data)) { + column(callJMethod(object@jc, "as", data)) + } else { + stop("data should be character") + } + }) + +#' An expression that returns a substring. +#' +#' @param start starting position +#' @param stop ending position +setMethod("substr", signature(x = "Column"), + function(x, start, stop) { + jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1)) + column(jc) + }) + +#' Casts the column to a different data type. +#' @examples +#' \dontrun{ +#' cast(df$age, "string") +#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) +#' } +setMethod("cast", + signature(x = "Column"), + function(x, dataType) { + if (is.character(dataType)) { + column(callJMethod(x@jc, "cast", dataType)) + } else if (is.list(dataType)) { + json <- tojson(dataType) + jdataType <- callJStatic("org.apache.spark.sql.types.DataType", "fromJson", json) + column(callJMethod(x@jc, "cast", jdataType)) + } else { + stop("dataType should be character or list") + } + }) + +#' Approx Count Distinct +#' +#' Returns the approximate number of distinct items in a group. +#' +setMethod("approxCountDistinct", + signature(x = "Column"), + function(x, rsd = 0.95) { + jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd) + column(jc) + }) + +#' Count Distinct +#' +#' returns the number of distinct items in a group. +#' +setMethod("countDistinct", + signature(x = "Column"), + function(x, ...) { + jcol <- lapply(list(...), function (x) { + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "countDistinct", x@jc, + listToSeq(jcol)) + column(jc) + }) + diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R new file mode 100644 index 0000000000000..2fc0bb294bcce --- /dev/null +++ b/R/pkg/R/context.R @@ -0,0 +1,225 @@ +# +# 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. +# + +# context.R: SparkContext driven functions + +getMinSplits <- function(sc, minSplits) { + if (is.null(minSplits)) { + defaultParallelism <- callJMethod(sc, "defaultParallelism") + minSplits <- min(defaultParallelism, 2) + } + as.integer(minSplits) +} + +#' Create an RDD from a text file. +#' +#' This function reads a text file from HDFS, a local file system (available on all +#' nodes), or any Hadoop-supported file system URI, and creates an +#' RDD of strings from it. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minSplits Minimum number of splits to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD where each item is of type \code{character} +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' lines <- textFile(sc, "myfile.txt") +#'} +textFile <- function(sc, path, minSplits = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "textFile", path, getMinSplits(sc, minSplits)) + # jrdd is of type JavaRDD[String] + RDD(jrdd, "string") +} + +#' Load an RDD saved as a SequenceFile containing serialized objects. +#' +#' The file to be loaded should be one that was previously generated by calling +#' saveAsObjectFile() of the RDD class. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minSplits Minimum number of splits to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD containing serialized R objects. +#' @seealso saveAsObjectFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- objectFile(sc, "myfile") +#'} +objectFile <- function(sc, path, minSplits = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "objectFile", path, getMinSplits(sc, minSplits)) + # Assume the RDD contains serialized R objects. + RDD(jrdd, "byte") +} + +#' Create an RDD from a homogeneous list or vector. +#' +#' This function creates an RDD from a local homogeneous list in R. The elements +#' in the list are split into \code{numSlices} slices and distributed to nodes +#' in the cluster. +#' +#' @param sc SparkContext to use +#' @param coll collection to parallelize +#' @param numSlices number of partitions to create in the RDD +#' @return an RDD created from this collection +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2) +#' # The RDD should contain 10 elements +#' length(rdd) +#'} +parallelize <- function(sc, coll, numSlices = 1) { + # TODO: bound/safeguard numSlices + # TODO: unit tests for if the split works for all primitives + # TODO: support matrix, data frame, etc + if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) { + if (is.data.frame(coll)) { + message(paste("context.R: A data frame is parallelized by columns.")) + } else { + if (is.matrix(coll)) { + message(paste("context.R: A matrix is parallelized by elements.")) + } else { + message(paste("context.R: parallelize() currently only supports lists and vectors.", + "Calling as.list() to coerce coll into a list.")) + } + } + coll <- as.list(coll) + } + + if (numSlices > length(coll)) + numSlices <- length(coll) + + sliceLen <- ceiling(length(coll) / numSlices) + slices <- split(coll, rep(1:(numSlices + 1), each = sliceLen)[1:length(coll)]) + + # Serialize each slice: obtain a list of raws, or a list of lists (slices) of + # 2-tuples of raws + serializedSlices <- lapply(slices, serialize, connection = NULL) + + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", + "createRDDFromArray", sc, serializedSlices) + + RDD(jrdd, "byte") +} + +#' Include this specified package on all workers +#' +#' This function can be used to include a package on all workers before the +#' user's code is executed. This is useful in scenarios where other R package +#' functions are used in a function passed to functions like \code{lapply}. +#' NOTE: The package is assumed to be installed on every node in the Spark +#' cluster. +#' +#' @param sc SparkContext to use +#' @param pkg Package name +#' +#' @export +#' @examples +#'\dontrun{ +#' library(Matrix) +#' +#' sc <- sparkR.init() +#' # Include the matrix library we will be using +#' includePackage(sc, Matrix) +#' +#' generateSparse <- function(x) { +#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +#' } +#' +#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +#' collect(rdd) +#'} +includePackage <- function(sc, pkg) { + pkg <- as.character(substitute(pkg)) + if (exists(".packages", .sparkREnv)) { + packages <- .sparkREnv$.packages + } else { + packages <- list() + } + packages <- c(packages, pkg) + .sparkREnv$.packages <- packages +} + +#' @title Broadcast a variable to all workers +#' +#' @description +#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +#' object for reading it in distributed functions. +#' +#' @param sc Spark Context to use +#' @param object Object to be broadcast +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2, 2L) +#' +#' # Large Matrix object that we want to broadcast +#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +#' randomMatBr <- broadcast(sc, randomMat) +#' +#' # Use the broadcast variable inside the function +#' useBroadcast <- function(x) { +#' sum(value(randomMatBr) * x) +#' } +#' sumRDD <- lapply(rdd, useBroadcast) +#'} +broadcast <- function(sc, object) { + objName <- as.character(substitute(object)) + serializedObj <- serialize(object, connection = NULL) + + jBroadcast <- callJMethod(sc, "broadcast", serializedObj) + id <- as.character(callJMethod(jBroadcast, "id")) + + Broadcast(id, object, jBroadcast, objName) +} + +#' @title Set the checkpoint directory +#' +#' Set the directory under which RDDs are going to be checkpointed. The +#' directory must be a HDFS path if running on a cluster. +#' +#' @param sc Spark Context to use +#' @param dirName Directory path +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "~/checkpoints") +#' rdd <- parallelize(sc, 1:2, 2L) +#' checkpoint(rdd) +#'} +setCheckpointDir <- function(sc, dirName) { + invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) +} diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R new file mode 100644 index 0000000000000..257b435607ce8 --- /dev/null +++ b/R/pkg/R/deserialize.R @@ -0,0 +1,184 @@ +# +# 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. +# + +# Utility functions to deserialize objects from Java. + +# Type mapping from Java to R +# +# void -> NULL +# Int -> integer +# String -> character +# Boolean -> logical +# Double -> double +# Long -> double +# Array[Byte] -> raw +# Date -> Date +# Time -> POSIXct +# +# Array[T] -> list() +# Object -> jobj + +readObject <- function(con) { + # Read type first + type <- readType(con) + readTypedObject(con, type) +} + +readTypedObject <- function(con, type) { + switch (type, + "i" = readInt(con), + "c" = readString(con), + "b" = readBoolean(con), + "d" = readDouble(con), + "r" = readRaw(con), + "D" = readDate(con), + "t" = readTime(con), + "l" = readList(con), + "n" = NULL, + "j" = getJobj(readString(con)), + stop(paste("Unsupported type for deserialization", type))) +} + +readString <- function(con) { + stringLen <- readInt(con) + string <- readBin(con, raw(), stringLen, endian = "big") + rawToChar(string) +} + +readInt <- function(con) { + readBin(con, integer(), n = 1, endian = "big") +} + +readDouble <- function(con) { + readBin(con, double(), n = 1, endian = "big") +} + +readBoolean <- function(con) { + as.logical(readInt(con)) +} + +readType <- function(con) { + rawToChar(readBin(con, "raw", n = 1L)) +} + +readDate <- function(con) { + as.Date(readString(con)) +} + +readTime <- function(con) { + t <- readDouble(con) + as.POSIXct(t, origin = "1970-01-01") +} + +# We only support lists where all elements are of same type +readList <- function(con) { + type <- readType(con) + len <- readInt(con) + if (len > 0) { + l <- vector("list", len) + for (i in 1:len) { + l[[i]] <- readTypedObject(con, type) + } + l + } else { + list() + } +} + +readRaw <- function(con) { + dataLen <- readInt(con) + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readRawLen <- function(con, dataLen) { + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readDeserialize <- function(con) { + # We have two cases that are possible - In one, the entire partition is + # encoded as a byte array, so we have only one value to read. If so just + # return firstData + dataLen <- readInt(con) + firstData <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + + # Else, read things into a list + dataLen <- readInt(con) + if (length(dataLen) > 0 && dataLen > 0) { + data <- list(firstData) + while (length(dataLen) > 0 && dataLen > 0) { + data[[length(data) + 1L]] <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + dataLen <- readInt(con) + } + unlist(data, recursive = FALSE) + } else { + firstData + } +} + +readDeserializeRows <- function(inputCon) { + # readDeserializeRows will deserialize a DataOutputStream composed of + # a list of lists. Since the DOS is one continuous stream and + # the number of rows varies, we put the readRow function in a while loop + # that termintates when the next row is empty. + data <- list() + while(TRUE) { + row <- readRow(inputCon) + if (length(row) == 0) { + break + } + data[[length(data) + 1L]] <- row + } + data # this is a list of named lists now +} + +readRowList <- function(obj) { + # readRowList is meant for use inside an lapply. As a result, it is + # necessary to open a standalone connection for the row and consume + # the numCols bytes inside the read function in order to correctly + # deserialize the row. + rawObj <- rawConnection(obj, "r+") + on.exit(close(rawObj)) + readRow(rawObj) +} + +readRow <- function(inputCon) { + numCols <- readInt(inputCon) + if (length(numCols) > 0 && numCols > 0) { + lapply(1:numCols, function(x) { + obj <- readObject(inputCon) + if (is.null(obj)) { + NA + } else { + obj + } + }) # each row is a list now + } else { + list() + } +} + +# Take a single column as Array[Byte] and deserialize it into an atomic vector +readCol <- function(inputCon, numRows) { + # sapply can not work with POSIXlt + do.call(c, lapply(1:numRows, function(x) { + value <- readObject(inputCon) + # Replace NULL with NA so we can coerce to vectors + if (is.null(value)) NA else value + })) +} diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R new file mode 100644 index 0000000000000..5fb1ccaa84ee2 --- /dev/null +++ b/R/pkg/R/generics.R @@ -0,0 +1,543 @@ +# +# 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. +# + +############ RDD Actions and Transformations ############ + +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' @rdname cache-methods +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname coalesce +#' @seealso repartition +#' @export +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) + +#' @rdname checkpoint-methods +#' @export +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) + +#' @rdname collect-methods +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectPartition", + function(x, partitionId) { + standardGeneric("collectPartition") + }) + +#' @rdname count +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + +#' @rdname countByValue +#' @export +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) + +#' @rdname distinct +#' @export +setGeneric("distinct", function(x, numPartitions = 1L) { standardGeneric("distinct") }) + +#' @rdname filterRDD +#' @export +setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) + +#' @rdname first +#' @export +setGeneric("first", function(x) { standardGeneric("first") }) + +#' @rdname flatMap +#' @export +setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) + +#' @rdname fold +#' @seealso reduce +#' @export +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) + +#' @rdname foreach +#' @export +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + +#' @rdname foreach +#' @export +setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) + +# The jrdd accessor function. +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) + +#' @rdname glom +#' @export +setGeneric("glom", function(x) { standardGeneric("glom") }) + +#' @rdname keyBy +#' @export +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("lapplyPartitionsWithIndex", + function(X, FUN) { + standardGeneric("lapplyPartitionsWithIndex") + }) + +#' @rdname lapply +#' @export +setGeneric("map", function(X, FUN) { standardGeneric("map") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("mapPartitionsWithIndex", + function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) + +#' @rdname maximum +#' @export +setGeneric("maximum", function(x) { standardGeneric("maximum") }) + +#' @rdname minimum +#' @export +setGeneric("minimum", function(x) { standardGeneric("minimum") }) + +#' @rdname sumRDD +#' @export +setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) + +#' @rdname name +#' @export +setGeneric("name", function(x) { standardGeneric("name") }) + +#' @rdname numPartitions +#' @export +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' @rdname pipeRDD +#' @export +setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) + +#' @rdname reduce +#' @export +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) + +#' @rdname repartition +#' @seealso coalesce +#' @export +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) + +#' @rdname sampleRDD +#' @export +setGeneric("sampleRDD", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleRDD") + }) + +#' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) + +#' @rdname saveAsTextFile +#' @export +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) + +#' @rdname setName +#' @export +setGeneric("setName", function(x, name) { standardGeneric("setName") }) + +#' @rdname sortBy +#' @export +setGeneric("sortBy", + function(x, func, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortBy") + }) + +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' @rdname takeOrdered +#' @export +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) + +#' @rdname takeSample +#' @export +setGeneric("takeSample", + function(x, withReplacement, num, seed) { + standardGeneric("takeSample") + }) + +#' @rdname top +#' @export +setGeneric("top", function(x, num) { standardGeneric("top") }) + +#' @rdname unionRDD +#' @export +setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) + +#' @rdname unpersist-methods +#' @export +setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) + +#' @rdname zipRDD +#' @export +setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") }) + +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + + +############ Binary Functions ############# + +#' @rdname countByKey +#' @export +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) + +#' @rdname flatMapValues +#' @export +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + +#' @rdname keys +#' @export +setGeneric("keys", function(x) { standardGeneric("keys") }) + +#' @rdname lookup +#' @export +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) + +#' @rdname mapValues +#' @export +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' @rdname values +#' @export +setGeneric("values", function(x) { standardGeneric("values") }) + + + +############ Shuffle Functions ############ + +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +setGeneric("aggregateByKey", + function(x, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' @rdname cogroup +#' @export +setGeneric("cogroup", + function(..., numPartitions) { + standardGeneric("cogroup") + }, + signature = "...") + +#' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +setGeneric("combineByKey", + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +setGeneric("foldByKey", + function(x, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' @rdname join-methods +#' @export +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' @rdname groupByKey +#' @seealso reduceByKey +#' @export +setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") }) + +#' @rdname join-methods +#' @export +setGeneric("join", function(x, y, ...) { standardGeneric("join") }) + +#' @rdname join-methods +#' @export +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) + +#' @rdname reduceByKey +#' @seealso groupByKey +#' @export +setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")}) + +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +setGeneric("reduceByKeyLocally", + function(x, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' @rdname join-methods +#' @export +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' @rdname sortByKey +#' @export +setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortByKey") +}) + + +################### Broadcast Variable Methods ################# + +#' @rdname broadcast +#' @export +setGeneric("value", function(bcast) { standardGeneric("value") }) + + + +#################### DataFrame Methods ######################## + +#' @rdname schema +#' @export +setGeneric("columns", function(x) {standardGeneric("columns") }) + +#' @rdname schema +#' @export +setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) + +#' @rdname explain +#' @export +setGeneric("explain", function(x, ...) { standardGeneric("explain") }) + +#' @rdname filter +#' @export +setGeneric("filter", function(x, condition) { standardGeneric("filter") }) + +#' @rdname DataFrame +#' @export +setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") }) + +#' @rdname insertInto +#' @export +setGeneric("insertInto", function(x, tableName, ...) { standardGeneric("insertInto") }) + +#' @rdname intersect +#' @export +setGeneric("intersect", function(x, y) { standardGeneric("intersect") }) + +#' @rdname isLocal +#' @export +setGeneric("isLocal", function(x) { standardGeneric("isLocal") }) + +#' @rdname limit +#' @export +setGeneric("limit", function(x, num) {standardGeneric("limit") }) + +#' @rdname sortDF +#' @export +setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) + +#' @rdname schema +#' @export +setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) + +#' @rdname registerTempTable +#' @export +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + +#' @rdname sampleDF +#' @export +setGeneric("sampleDF", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleDF") + }) + +#' @rdname saveAsParquetFile +#' @export +setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) + +#' @rdname saveAsTable +#' @export +setGeneric("saveAsTable", function(df, tableName, source, mode, ...) { + standardGeneric("saveAsTable") +}) + +#' @rdname saveAsTable +#' @export +setGeneric("saveDF", function(df, path, source, mode, ...) { standardGeneric("saveDF") }) + +#' @rdname schema +#' @export +setGeneric("schema", function(x) { standardGeneric("schema") }) + +#' @rdname select +#' @export +setGeneric("select", function(x, col, ...) { standardGeneric("select") } ) + +#' @rdname select +#' @export +setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") }) + +#' @rdname showDF +#' @export +setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) + +#' @rdname sortDF +#' @export +setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) + +#' @rdname subtract +#' @export +setGeneric("subtract", function(x, y) { standardGeneric("subtract") }) + +#' @rdname tojson +#' @export +setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) + +#' @rdname DataFrame +#' @export +setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) + +#' @rdname unionAll +#' @export +setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) + +#' @rdname filter +#' @export +setGeneric("where", function(x, condition) { standardGeneric("where") }) + +#' @rdname withColumn +#' @export +setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn") }) + +#' @rdname withColumnRenamed +#' @export +setGeneric("withColumnRenamed", function(x, existingCol, newCol) { + standardGeneric("withColumnRenamed") }) + + +###################### Column Methods ########################## + +#' @rdname column +#' @export +setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) + +#' @rdname column +#' @export +setGeneric("asc", function(x) { standardGeneric("asc") }) + +#' @rdname column +#' @export +setGeneric("avg", function(x, ...) { standardGeneric("avg") }) + +#' @rdname column +#' @export +setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) + +#' @rdname column +#' @export +setGeneric("contains", function(x, ...) { standardGeneric("contains") }) +#' @rdname column +#' @export +setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) + +#' @rdname column +#' @export +setGeneric("desc", function(x) { standardGeneric("desc") }) + +#' @rdname column +#' @export +setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") }) + +#' @rdname column +#' @export +setGeneric("getField", function(x, ...) { standardGeneric("getField") }) + +#' @rdname column +#' @export +setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) + +#' @rdname column +#' @export +setGeneric("isNull", function(x) { standardGeneric("isNull") }) + +#' @rdname column +#' @export +setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) + +#' @rdname column +#' @export +setGeneric("last", function(x) { standardGeneric("last") }) + +#' @rdname column +#' @export +setGeneric("like", function(x, ...) { standardGeneric("like") }) + +#' @rdname column +#' @export +setGeneric("lower", function(x) { standardGeneric("lower") }) + +#' @rdname column +#' @export +setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) + +#' @rdname column +#' @export +setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") }) + +#' @rdname column +#' @export +setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) + +#' @rdname column +#' @export +setGeneric("upper", function(x) { standardGeneric("upper") }) + diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R new file mode 100644 index 0000000000000..09fc0a7abe48a --- /dev/null +++ b/R/pkg/R/group.R @@ -0,0 +1,132 @@ +# +# 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. +# + +# group.R - GroupedData class and methods implemented in S4 OO classes + +setOldClass("jobj") + +#' @title S4 class that represents a GroupedData +#' @description GroupedDatas can be created using groupBy() on a DataFrame +#' @rdname GroupedData +#' @seealso groupBy +#' +#' @param sgd A Java object reference to the backing Scala GroupedData +#' @export +setClass("GroupedData", + slots = list(sgd = "jobj")) + +setMethod("initialize", "GroupedData", function(.Object, sgd) { + .Object@sgd <- sgd + .Object +}) + +#' @rdname DataFrame +groupedData <- function(sgd) { + new("GroupedData", sgd) +} + + +#' @rdname show +setMethod("show", "GroupedData", + function(object) { + cat("GroupedData\n") + }) + +#' Count +#' +#' Count the number of rows for each group. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @export +#' @examples +#' \dontrun{ +#' count(groupBy(df, "name")) +#' } +setMethod("count", + signature(x = "GroupedData"), + function(x) { + dataFrame(callJMethod(x@sgd, "count")) + }) + +#' Agg +#' +#' Aggregates on the entire DataFrame without groups. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' df2 <- agg(df, = ) +#' df2 <- agg(df, newColName = aggFunction(column)) +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @rdname agg +#' @examples +#' \dontrun{ +#' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)' +#' df2 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum +#' } +setGeneric("agg", function (x, ...) { standardGeneric("agg") }) + +setMethod("agg", + signature(x = "GroupedData"), + function(x, ...) { + cols = list(...) + stopifnot(length(cols) > 0) + if (is.character(cols[[1]])) { + cols <- varargsToEnv(...) + sdf <- callJMethod(x@sgd, "agg", cols) + } else if (class(cols[[1]]) == "Column") { + ns <- names(cols) + if (!is.null(ns)) { + for (n in ns) { + if (n != "") { + cols[[n]] = alias(cols[[n]], n) + } + } + } + jcols <- lapply(cols, function(c) { c@jc }) + # the GroupedData.agg(col, cols*) API does not contain grouping Column + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "aggWithGrouping", + x@sgd, listToSeq(jcols)) + } else { + stop("agg can only support Column or character") + } + dataFrame(sdf) + }) + + +# sum/mean/avg/min/max +methods <- c("sum", "mean", "avg", "min", "max") + +createMethod <- function(name) { + setMethod(name, + signature(x = "GroupedData"), + function(x, ...) { + sdf <- callJMethod(x@sgd, name, toSeq(...)) + dataFrame(sdf) + }) +} + +createMethods <- function() { + for (name in methods) { + createMethod(name) + } +} + +createMethods() + diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R new file mode 100644 index 0000000000000..4180f146b7fbc --- /dev/null +++ b/R/pkg/R/jobj.R @@ -0,0 +1,101 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# References to objects that exist on the JVM backend +# are maintained using the jobj. + +# Maintain a reference count of Java object references +# This allows us to GC the java object when it is safe +.validJobjs <- new.env(parent = emptyenv()) + +# List of object ids to be removed +.toRemoveJobjs <- new.env(parent = emptyenv()) + +# Check if jobj was created with the current SparkContext +isValidJobj <- function(jobj) { + if (exists(".scStartTime", envir = .sparkREnv)) { + jobj$appId == get(".scStartTime", envir = .sparkREnv) + } else { + FALSE + } +} + +getJobj <- function(objId) { + newObj <- jobj(objId) + if (exists(objId, .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] + 1 + } else { + .validJobjs[[objId]] <- 1 + } + newObj +} + +# Handler for a java object that exists on the backend. +jobj <- function(objId) { + if (!is.character(objId)) { + stop("object id must be a character") + } + # NOTE: We need a new env for a jobj as we can only register + # finalizers for environments or external references pointers. + obj <- structure(new.env(parent = emptyenv()), class = "jobj") + obj$id <- objId + obj$appId <- get(".scStartTime", envir = .sparkREnv) + + # Register a finalizer to remove the Java object when this reference + # is garbage collected in R + reg.finalizer(obj, cleanup.jobj) + obj +} + +#' Print a JVM object reference. +#' +#' This function prints the type and id for an object stored +#' in the SparkR JVM backend. +#' +#' @param x The JVM object reference +#' @param ... further arguments passed to or from other methods +print.jobj <- function(x, ...) { + cls <- callJMethod(x, "getClass") + name <- callJMethod(cls, "getName") + cat("Java ref type", name, "id", x$id, "\n", sep = " ") +} + +cleanup.jobj <- function(jobj) { + if (isValidJobj(jobj)) { + objId <- jobj$id + # If we don't know anything about this jobj, ignore it + if (exists(objId, envir = .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] - 1 + + if (.validJobjs[[objId]] == 0) { + rm(list = objId, envir = .validJobjs) + # NOTE: We cannot call removeJObject here as the finalizer may be run + # in the middle of another RPC. Thus we queue up this object Id to be removed + # and then run all the removeJObject when the next RPC is called. + .toRemoveJobjs[[objId]] <- 1 + } + } + } +} + +clearJobjs <- function() { + valid <- ls(.validJobjs) + rm(list = valid, envir = .validJobjs) + + removeList <- ls(.toRemoveJobjs) + rm(list = removeList, envir = .toRemoveJobjs) +} diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R new file mode 100644 index 0000000000000..c2396c32a7548 --- /dev/null +++ b/R/pkg/R/pairRDD.R @@ -0,0 +1,789 @@ +# +# 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. +# + +# Operations supported on RDDs contains pairs (i.e key, value) + +############ Actions and Transformations ############ + +#' Look up elements of a key in an RDD +#' +#' @description +#' \code{lookup} returns a list of values in this RDD for key key. +#' +#' @param x The RDD to collect +#' @param key The key to look up for +#' @return a list of values in this RDD for key key +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +#' rdd <- parallelize(sc, pairs) +#' lookup(rdd, 1) # list(1, 3) +#'} +#' @rdname lookup +#' @aliases lookup,RDD-method +setMethod("lookup", + signature(x = "RDD", key = "ANY"), + function(x, key) { + partitionFunc <- function(part) { + filtered <- part[unlist(lapply(part, function(i) { identical(key, i[[1]]) }))] + lapply(filtered, function(i) { i[[2]] }) + } + valsRDD <- lapplyPartition(x, partitionFunc) + collect(valsRDD) + }) + +#' Count the number of elements for each key, and return the result to the +#' master as lists of (key, count) pairs. +#' +#' Same as countByKey in Spark. +#' +#' @param x The RDD to count keys. +#' @return list of (key, count) pairs, where count is number of each key in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +#' @rdname countByKey +#' @aliases countByKey,RDD-method +setMethod("countByKey", + signature(x = "RDD"), + function(x) { + keys <- lapply(x, function(item) { item[[1]] }) + countByValue(keys) + }) + +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} +#' @rdname keys +#' @aliases keys,RDD +setMethod("keys", + signature(x = "RDD"), + function(x) { + func <- function(k) { + k[[1]] + } + lapply(x, func) + }) + +#' Return an RDD with the values of each tuple. +#' +#' @param x The RDD from which the values of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} +#' @rdname values +#' @aliases values,RDD +setMethod("values", + signature(x = "RDD"), + function(x) { + func <- function(v) { + v[[2]] + } + lapply(x, func) + }) + +#' Applies a function to all values of the elements, without modifying the keys. +#' +#' The same as `mapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' makePairs <- lapply(rdd, function(x) { list(x, x) }) +#' collect(mapValues(makePairs, function(x) { x * 2) }) +#' Output: list(list(1,2), list(2,4), list(3,6), ...) +#'} +#' @rdname mapValues +#' @aliases mapValues,RDD,function-method +setMethod("mapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(x) { + list(x[[1]], FUN(x[[2]])) + } + lapply(X, func) + }) + +#' Pass each value in the key-value pair RDD through a flatMap function without +#' changing the keys; this also retains the original RDD's partitioning. +#' +#' The same as 'flatMapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +#' collect(flatMapValues(rdd, function(x) { x })) +#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#'} +#' @rdname flatMapValues +#' @aliases flatMapValues,RDD,function-method +setMethod("flatMapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + flatMapFunc <- function(x) { + lapply(FUN(x[[2]]), function(v) { list(x[[1]], v) }) + } + flatMap(X, flatMapFunc) + }) + +############ Shuffle Functions ############ + +#' Partition an RDD by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' For each element of this RDD, the partitioner is used to compute a hash +#' function and the RDD is partitioned using this hash value. +#' +#' @param x The RDD to partition. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @param ... Other optional arguments to partitionBy. +#' +#' @param partitionFunc The partition function to use. Uses a default hashCode +#' function if not provided +#' @return An RDD partitioned using the specified partitioner. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- partitionBy(rdd, 2L) +#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#'} +#' @rdname partitionBy +#' @aliases partitionBy,RDD,integer-method +setMethod("partitionBy", + signature(x = "RDD", numPartitions = "integer"), + function(x, numPartitions, partitionFunc = hashCode) { + + #if (missing(partitionFunc)) { + # partitionFunc <- hashCode + #} + + partitionFunc <- cleanClosure(partitionFunc) + serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) + + packageNamesArr <- serialize(.sparkREnv$.packages, + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), function(name) { + get(name, .broadcastNames) }) + jrdd <- getJRDD(x) + + # We create a PairwiseRRDD that extends RDD[(Array[Byte], + # Array[Byte])], where the key is the hashed split, the value is + # the content (key-val pairs). + pairwiseRRDD <- newJObject("org.apache.spark.api.r.PairwiseRRDD", + callJMethod(jrdd, "rdd"), + as.integer(numPartitions), + serializedHashFuncBytes, + getSerializedMode(x), + packageNamesArr, + as.character(.sparkREnv$libname), + broadcastArr, + callJMethod(jrdd, "classTag")) + + # Create a corresponding partitioner. + rPartitioner <- newJObject("org.apache.spark.HashPartitioner", + as.integer(numPartitions)) + + # Call partitionBy on the obtained PairwiseRDD. + javaPairRDD <- callJMethod(pairwiseRRDD, "asJavaPairRDD") + javaPairRDD <- callJMethod(javaPairRDD, "partitionBy", rPartitioner) + + # Call .values() on the result to get back the final result, the + # shuffled acutal content key-val pairs. + r <- callJMethod(javaPairRDD, "values") + + RDD(r, serializedMode = "byte") + }) + +#' Group values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and group values for each key in the RDD into a single sequence. +#' +#' @param x The RDD to group. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, list(V)) +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- groupByKey(rdd, 2L) +#' grouped <- collect(parts) +#' grouped[[1]] # Should be a list(1, list(2, 4)) +#'} +#' @rdname groupByKey +#' @aliases groupByKey,RDD,integer-method +setMethod("groupByKey", + signature(x = "RDD", numPartitions = "integer"), + function(x, numPartitions) { + shuffled <- partitionBy(x, numPartitions) + groupVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + appendList <- function(acc, i) { + addItemToAccumulator(acc, i) + acc + } + makeList <- function(i) { + acc <- initAccumulator() + addItemToAccumulator(acc, i) + acc + } + # Each item in the partition is list of (K, V) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, + appendList, makeList) + }) + # extract out data field + vals <- eapply(vals, + function(i) { + length(i$data) <- i$counter + i$data + }) + # Every key in the environment contains a list + # Convert that to list(K, Seq[V]) + convertEnvsToList(keys, vals) + } + lapplyPartition(shuffled, groupVals) + }) + +#' Merge values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, V') where V' is the merged +#' value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- reduceByKey(rdd, "+", 2L) +#' reduced <- collect(parts) +#' reduced[[1]] # Should be a list(1, 6) +#'} +#' @rdname reduceByKey +#' @aliases reduceByKey,RDD,integer-method +setMethod("reduceByKey", + signature(x = "RDD", combineFunc = "ANY", numPartitions = "integer"), + function(x, combineFunc, numPartitions) { + reduceVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + convertEnvsToList(keys, vals) + } + locallyReduced <- lapplyPartition(x, reduceVals) + shuffled <- partitionBy(locallyReduced, numPartitions) + lapplyPartition(shuffled, reduceVals) + }) + +#' Merge values by key locally +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function, but return the +#' results immediately to the driver as an R list. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @return A list of elements of type list(K, V') where V' is the merged value for each key +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' reduced <- reduceByKeyLocally(rdd, "+") +#' reduced # list(list(1, 6), list(1.1, 3)) +#'} +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +setMethod("reduceByKeyLocally", + signature(x = "RDD", combineFunc = "ANY"), + function(x, combineFunc) { + reducePart <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + list(list(keys, vals)) # return hash to avoid re-compute in merge + } + mergeParts <- function(accum, x) { + pred <- function(item) { + exists(item$hash, accum[[1]]) + } + lapply(ls(x[[1]]), + function(name) { + item <- list(x[[1]][[name]], x[[2]][[name]]) + item$hash <- name + updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, identity) + }) + accum + } + reduced <- mapPartitions(x, reducePart) + merged <- reduce(reduced, mergeParts) + convertEnvsToList(merged[[1]], merged[[2]]) + }) + +#' Combine values by key +#' +#' Generic function to combine the elements for each key using a custom set of +#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +#' for a "combined type" C. Note that V and C can be different -- for example, one +#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). + +#' Users provide three functions: +#' \itemize{ +#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +#' two lists). +#' } +#' +#' @param x The RDD to combine. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param createCombiner Create a combiner (C) given a value (V) +#' @param mergeValue Merge the given value (V) with an existing combiner (C) +#' @param mergeCombiners Merge two combiners and return a new combiner +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, C) where C is the combined type +#' +#' @seealso groupByKey, reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +#' combined <- collect(parts) +#' combined[[1]] # Should be a list(1, 6) +#'} +#' @rdname combineByKey +#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("combineByKey", + signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", + mergeCombiners = "ANY", numPartitions = "integer"), + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + combineLocally <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) + }) + convertEnvsToList(keys, combiners) + } + locallyCombined <- lapplyPartition(x, combineLocally) + shuffled <- partitionBy(locallyCombined, numPartitions) + mergeAfterShuffle <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) + }) + convertEnvsToList(keys, combiners) + } + lapplyPartition(shuffled, mergeAfterShuffle) + }) + +#' Aggregate a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using given combine functions +#' and a neutral "zero value". This function can return a different result type, +#' U, than the type of the values in this RDD, V. Thus, we need one operation +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument +#' instead of creating a new U. +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the values of each key. It may return +#' a different result type from the type of the values. +#' @param combOp A function to aggregate results of seqOp. +#' @return An RDD containing the aggregation result. +#' @seealso foldByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' # list(list(1, list(3, 2)), list(2, list(7, 2))) +#'} +#' @rdname aggregateByKey +#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("aggregateByKey", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", + combOp = "ANY", numPartitions = "integer"), + function(x, zeroValue, seqOp, combOp, numPartitions) { + createCombiner <- function(v) { + do.call(seqOp, list(zeroValue, v)) + } + + combineByKey(x, createCombiner, seqOp, combOp, numPartitions) + }) + +#' Fold a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using an associative function "func" +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or +#' 1 for multiplication.). +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param func An associative function for folding values of each key. +#' @return An RDD containing the aggregation result. +#' @seealso aggregateByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#'} +#' @rdname foldByKey +#' @aliases foldByKey,RDD,ANY,ANY,integer-method +setMethod("foldByKey", + signature(x = "RDD", zeroValue = "ANY", + func = "ANY", numPartitions = "integer"), + function(x, zeroValue, func, numPartitions) { + aggregateByKey(x, zeroValue, func, func, numPartitions) + }) + +############ Binary Functions ############# + +#' Join two RDDs +#' +#' @description +#' \code{join} This function joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with matching keys in +#' two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#'} +#' @rdname join-methods +#' @aliases join,RDD,RDD-method +setMethod("join", + signature(x = "RDD", y = "RDD"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), + doJoin) + }) + +#' Left outer join two RDDs +#' +#' @description +#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' if no elements in rdd2 have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' leftOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#'} +#' @rdname join-methods +#' @aliases leftOuterJoin,RDD,RDD-method +setMethod("leftOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Right outer join two RDDs +#' +#' @description +#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, w) in y, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +#' if no elements in x have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rightOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#'} +#' @rdname join-methods +#' @aliases rightOuterJoin,RDD,RDD-method +setMethod("rightOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Full outer join two RDDs +#' +#' @description +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD +#' will contain all pairs (k, (v, w)) for both (k, v) in x and +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' in x/y have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +#' # list(1, list(3, 1)), +#' # list(2, list(NULL, 4))) +#' # list(3, list(3, NULL)), +#'} +#' @rdname join-methods +#' @aliases fullOuterJoin,RDD,RDD-method +setMethod("fullOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' For each key k in several RDDs, return a resulting RDD that +#' whose values are a list of values for the key in all RDDs. +#' +#' @param ... Several RDDs. +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with values in a list +#' in all RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#'} +#' @rdname cogroup +#' @aliases cogroup,RDD-method +setMethod("cogroup", + "RDD", + function(..., numPartitions) { + rdds <- list(...) + rddsLen <- length(rdds) + for (i in 1:rddsLen) { + rdds[[i]] <- lapply(rdds[[i]], + function(x) { list(x[[1]], list(i, x[[2]])) }) + # TODO(hao): As issue [SparkR-142] mentions, the right value of i + # will not be captured into UDF if getJRDD is not invoked. + # It should be resolved together with that issue. + getJRDD(rdds[[i]]) # Capture the closure. + } + union.rdd <- Reduce(unionRDD, rdds) + group.func <- function(vlist) { + res <- list() + length(res) <- rddsLen + for (x in vlist) { + i <- x[[1]] + acc <- res[[i]] + # Create an accumulator. + if (is.null(acc)) { + acc <- initAccumulator() + } + addItemToAccumulator(acc, x[[2]]) + res[[i]] <- acc + } + lapply(res, function(acc) { + if (is.null(acc)) { + list() + } else { + acc$data + } + }) + } + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + group.func) + }) + +#' Sort a (k, v) pair RDD by k. +#' +#' @param x A (k, v) pair RDD to be sorted. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all (k, v) pair elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#'} +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +setMethod("sortByKey", + signature(x = "RDD"), + function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + rangeBounds <- list() + + if (numPartitions > 1) { + rddSize <- count(x) + # constant from Spark's RangePartitioner + maxSampleSize <- numPartitions * 20 + fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) + + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) + + # Note: the built-in R sort() function only works on atomic vectors + samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) + + if (length(samples) > 0) { + rangeBounds <- lapply(seq_len(numPartitions - 1), + function(i) { + j <- ceiling(length(samples) * i / numPartitions) + samples[j] + }) + } + } + + rangePartitionFunc <- function(key) { + partition <- 0 + + # TODO: Use binary search instead of linear search, similar with Spark + while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { + partition <- partition + 1 + } + + if (ascending) { + partition + } else { + numPartitions - partition - 1 + } + } + + partitionFunc <- function(part) { + sortKeyValueList(part, decreasing = !ascending) + } + + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) + lapplyPartition(newRDD, partitionFunc) + }) + diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R new file mode 100644 index 0000000000000..8a9c0c652ce24 --- /dev/null +++ b/R/pkg/R/serialize.R @@ -0,0 +1,195 @@ +# +# 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. +# + +# Utility functions to serialize R objects so they can be read in Java. + +# Type mapping from R to Java +# +# NULL -> Void +# integer -> Int +# character -> String +# logical -> Boolean +# double, numeric -> Double +# raw -> Array[Byte] +# Date -> Date +# POSIXct,POSIXlt -> Time +# +# list[T] -> Array[T], where T is one of above mentioned types +# environment -> Map[String, T], where T is a native type +# jobj -> Object, where jobj is an object created in the backend + +writeObject <- function(con, object, writeType = TRUE) { + # NOTE: In R vectors have same type as objects. So we don't support + # passing in vectors as arrays and instead require arrays to be passed + # as lists. + type <- class(object)[[1]] # class of POSIXlt is c("POSIXlt", "POSIXt") + if (writeType) { + writeType(con, type) + } + switch(type, + NULL = writeVoid(con), + integer = writeInt(con, object), + character = writeString(con, object), + logical = writeBoolean(con, object), + double = writeDouble(con, object), + numeric = writeDouble(con, object), + raw = writeRaw(con, object), + list = writeList(con, object), + jobj = writeJobj(con, object), + environment = writeEnv(con, object), + Date = writeDate(con, object), + POSIXlt = writeTime(con, object), + POSIXct = writeTime(con, object), + stop(paste("Unsupported type for serialization", type))) +} + +writeVoid <- function(con) { + # no value for NULL +} + +writeJobj <- function(con, value) { + if (!isValidJobj(value)) { + stop("invalid jobj ", value$id) + } + writeString(con, value$id) +} + +writeString <- function(con, value) { + writeInt(con, as.integer(nchar(value) + 1)) + writeBin(value, con, endian = "big") +} + +writeInt <- function(con, value) { + writeBin(as.integer(value), con, endian = "big") +} + +writeDouble <- function(con, value) { + writeBin(value, con, endian = "big") +} + +writeBoolean <- function(con, value) { + # TRUE becomes 1, FALSE becomes 0 + writeInt(con, as.integer(value)) +} + +writeRawSerialize <- function(outputCon, batch) { + outputSer <- serialize(batch, ascii = FALSE, connection = NULL) + writeRaw(outputCon, outputSer) +} + +writeRowSerialize <- function(outputCon, rows) { + invisible(lapply(rows, function(r) { + bytes <- serializeRow(r) + writeRaw(outputCon, bytes) + })) +} + +serializeRow <- function(row) { + rawObj <- rawConnection(raw(0), "wb") + on.exit(close(rawObj)) + writeRow(rawObj, row) + rawConnectionValue(rawObj) +} + +writeRow <- function(con, row) { + numCols <- length(row) + writeInt(con, numCols) + for (i in 1:numCols) { + writeObject(con, row[[i]]) + } +} + +writeRaw <- function(con, batch) { + writeInt(con, length(batch)) + writeBin(batch, con, endian = "big") +} + +writeType <- function(con, class) { + type <- switch(class, + NULL = "n", + integer = "i", + character = "c", + logical = "b", + double = "d", + numeric = "d", + raw = "r", + list = "l", + jobj = "j", + environment = "e", + Date = "D", + POSIXlt = 't', + POSIXct = 't', + stop(paste("Unsupported type for serialization", class))) + writeBin(charToRaw(type), con) +} + +# Used to pass arrays where all the elements are of the same type +writeList <- function(con, arr) { + # All elements should be of same type + elemType <- unique(sapply(arr, function(elem) { class(elem) })) + stopifnot(length(elemType) <= 1) + + # TODO: Empty lists are given type "character" right now. + # This may not work if the Java side expects array of any other type. + if (length(elemType) == 0) { + elemType <- class("somestring") + } + + writeType(con, elemType) + writeInt(con, length(arr)) + + if (length(arr) > 0) { + for (a in arr) { + writeObject(con, a, FALSE) + } + } +} + +# Used to pass in hash maps required on Java side. +writeEnv <- function(con, env) { + len <- length(env) + + writeInt(con, len) + if (len > 0) { + writeList(con, as.list(ls(env))) + vals <- lapply(ls(env), function(x) { env[[x]] }) + writeList(con, as.list(vals)) + } +} + +writeDate <- function(con, date) { + writeString(con, as.character(date)) +} + +writeTime <- function(con, time) { + writeDouble(con, as.double(time)) +} + +# Used to serialize in a list of objects where each +# object can be of a different type. Serialization format is +# for each object +writeArgs <- function(con, args) { + if (length(args) > 0) { + for (a in args) { + writeObject(con, a) + } + } +} + +writeStrings <- function(con, stringList) { + writeLines(unlist(stringList), con) +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R new file mode 100644 index 0000000000000..bc82df01f0fff --- /dev/null +++ b/R/pkg/R/sparkR.R @@ -0,0 +1,266 @@ +# +# 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. +# + +.sparkREnv <- new.env() + +sparkR.onLoad <- function(libname, pkgname) { + .sparkREnv$libname <- libname +} + +# Utility function that returns TRUE if we have an active connection to the +# backend and FALSE otherwise +connExists <- function(env) { + tryCatch({ + exists(".sparkRCon", envir = env) && isOpen(env[[".sparkRCon"]]) + }, error = function(err) { + return(FALSE) + }) +} + +#' Stop the Spark context. +#' +#' Also terminates the backend this R session is connected to +sparkR.stop <- function() { + env <- .sparkREnv + if (exists(".sparkRCon", envir = env)) { + # cat("Stopping SparkR\n") + if (exists(".sparkRjsc", envir = env)) { + sc <- get(".sparkRjsc", envir = env) + callJMethod(sc, "stop") + rm(".sparkRjsc", envir = env) + } + + if (exists(".backendLaunched", envir = env)) { + callJStatic("SparkRHandler", "stopBackend") + } + + # Also close the connection and remove it from our env + conn <- get(".sparkRCon", envir = env) + close(conn) + + rm(".sparkRCon", envir = env) + rm(".scStartTime", envir = env) + } + + if (exists(".monitorConn", envir = env)) { + conn <- get(".monitorConn", envir = env) + close(conn) + rm(".monitorConn", envir = env) + } + + # Clear all broadcast variables we have + # as the jobj will not be valid if we restart the JVM + clearBroadcastVariables() + + # Clear jobj maps + clearJobjs() +} + +#' Initialize a new Spark Context. +#' +#' This function initializes a new SparkContext. +#' +#' @param master The Spark master URL. +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkEnvir Named list of environment variables to set on worker nodes. +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors. +#' @param sparkJars Character string vector of jar files to pass to the worker nodes. +#' @param sparkRLibDir The path where R is installed on the worker nodes. +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark") +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g")) +#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g"), +#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), +#' c("jarfile1.jar","jarfile2.jar")) +#'} + +sparkR.init <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvir = list(), + sparkExecutorEnv = list(), + sparkJars = "", + sparkRLibDir = "") { + + if (exists(".sparkRjsc", envir = .sparkREnv)) { + cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n") + return(get(".sparkRjsc", envir = .sparkREnv)) + } + + sparkMem <- Sys.getenv("SPARK_MEM", "512m") + jars <- suppressWarnings(normalizePath(as.character(sparkJars))) + + # Classpath separator is ";" on Windows + # URI needs four /// as from http://stackoverflow.com/a/18522792 + if (.Platform$OS.type == "unix") { + collapseChar <- ":" + uriSep <- "//" + } else { + collapseChar <- ";" + uriSep <- "////" + } + + existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") + if (existingPort != "") { + backendPort <- existingPort + } else { + path <- tempfile(pattern = "backend_port") + launchBackend( + args = path, + sparkHome = sparkHome, + jars = jars, + sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "sparkr-shell")) + # wait atmost 100 seconds for JVM to launch + wait <- 0.1 + for (i in 1:25) { + Sys.sleep(wait) + if (file.exists(path)) { + break + } + wait <- wait * 1.25 + } + if (!file.exists(path)) { + stop("JVM is not ready after 10 seconds") + } + f <- file(path, open='rb') + backendPort <- readInt(f) + monitorPort <- readInt(f) + close(f) + file.remove(path) + if (length(backendPort) == 0 || backendPort == 0 || + length(monitorPort) == 0 || monitorPort == 0) { + stop("JVM failed to launch") + } + assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) + assign(".backendLaunched", 1, envir = .sparkREnv) + } + + .sparkREnv$backendPort <- backendPort + tryCatch({ + connectBackend("localhost", backendPort) + }, error = function(err) { + stop("Failed to connect JVM\n") + }) + + if (nchar(sparkHome) != 0) { + sparkHome <- normalizePath(sparkHome) + } + + if (nchar(sparkRLibDir) != 0) { + .sparkREnv$libname <- sparkRLibDir + } + + sparkEnvirMap <- new.env() + for (varname in names(sparkEnvir)) { + sparkEnvirMap[[varname]] <- sparkEnvir[[varname]] + } + + sparkExecutorEnvMap <- new.env() + if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) { + sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + } + for (varname in names(sparkExecutorEnv)) { + sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]] + } + + nonEmptyJars <- Filter(function(x) { x != "" }, jars) + localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) + + # Set the start time to identify jobjs + # Seconds resolution is good enough for this purpose, so use ints + assign(".scStartTime", as.integer(Sys.time()), envir = .sparkREnv) + + assign( + ".sparkRjsc", + callJStatic( + "org.apache.spark.api.r.RRDD", + "createSparkContext", + master, + appName, + as.character(sparkHome), + as.list(localJarPaths), + sparkEnvirMap, + sparkExecutorEnvMap), + envir = .sparkREnv + ) + + sc <- get(".sparkRjsc", envir = .sparkREnv) + + # Register a finalizer to sleep 1 seconds on R exit to make RStudio happy + reg.finalizer(.sparkREnv, function(x) { Sys.sleep(1) }, onexit = TRUE) + + sc +} + +#' Initialize a new SQLContext. +#' +#' This function creates a SparkContext from an existing JavaSparkContext and +#' then uses it to initialize a new SQLContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#'} + +sparkRSQL.init <- function(jsc) { + if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + return(get(".sparkRSQLsc", envir = .sparkREnv)) + } + + sqlCtx <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createSQLContext", + jsc) + assign(".sparkRSQLsc", sqlCtx, envir = .sparkREnv) + sqlCtx +} + +#' Initialize a new HiveContext. +#' +#' This function creates a HiveContext from an existing JavaSparkContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRHive.init(sc) +#'} + +sparkRHive.init <- function(jsc) { + if (exists(".sparkRHivesc", envir = .sparkREnv)) { + return(get(".sparkRHivesc", envir = .sparkREnv)) + } + + ssc <- callJMethod(jsc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.HiveContext", ssc) + }, error = function(err) { + stop("Spark SQL is not built with Hive support") + }) + + assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) + hiveCtx +} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R new file mode 100644 index 0000000000000..c337fb0751e72 --- /dev/null +++ b/R/pkg/R/utils.R @@ -0,0 +1,467 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Utilities and Helpers + +# Given a JList, returns an R list containing the same elements, the number +# of which is optionally upper bounded by `logicalUpperBound` (by default, +# return all elements). Takes care of deserializations and type conversions. +convertJListToRList <- function(jList, flatten, logicalUpperBound = NULL, + serializedMode = "byte") { + arrSize <- callJMethod(jList, "size") + + # Datasets with serializedMode == "string" (such as an RDD directly generated by textFile()): + # each partition is not dense-packed into one Array[Byte], and `arrSize` + # here corresponds to number of logical elements. Thus we can prune here. + if (serializedMode == "string" && !is.null(logicalUpperBound)) { + arrSize <- min(arrSize, logicalUpperBound) + } + + results <- if (arrSize > 0) { + lapply(0:(arrSize - 1), + function(index) { + obj <- callJMethod(jList, "get", as.integer(index)) + + # Assume it is either an R object or a Java obj ref. + if (inherits(obj, "jobj")) { + if (isInstanceOf(obj, "scala.Tuple2")) { + # JavaPairRDD[Array[Byte], Array[Byte]]. + + keyBytes = callJMethod(obj, "_1") + valBytes = callJMethod(obj, "_2") + res <- list(unserialize(keyBytes), + unserialize(valBytes)) + } else { + stop(paste("utils.R: convertJListToRList only supports", + "RDD[Array[Byte]] and", + "JavaPairRDD[Array[Byte], Array[Byte]] for now")) + } + } else { + if (inherits(obj, "raw")) { + if (serializedMode == "byte") { + # RDD[Array[Byte]]. `obj` is a whole partition. + res <- unserialize(obj) + # For serialized datasets, `obj` (and `rRaw`) here corresponds to + # one whole partition dense-packed together. We deserialize the + # whole partition first, then cap the number of elements to be returned. + } else if (serializedMode == "row") { + res <- readRowList(obj) + # For DataFrames that have been converted to RRDDs, we call readRowList + # which will read in each row of the RRDD as a list and deserialize + # each element. + flatten <<- FALSE + # Use global assignment to change the flatten flag. This means + # we don't have to worry about the default argument in other functions + # e.g. collect + } + # TODO: is it possible to distinguish element boundary so that we can + # unserialize only what we need? + if (!is.null(logicalUpperBound)) { + res <- head(res, n = logicalUpperBound) + } + } else { + # obj is of a primitive Java type, is simplified to R's + # corresponding type. + res <- list(obj) + } + } + res + }) + } else { + list() + } + + if (flatten) { + as.list(unlist(results, recursive = FALSE)) + } else { + as.list(results) + } +} + +# Returns TRUE if `name` refers to an RDD in the given environment `env` +isRDD <- function(name, env) { + obj <- get(name, envir = env) + inherits(obj, "RDD") +} + +#' Compute the hashCode of an object +#' +#' Java-style function to compute the hashCode for the given object. Returns +#' an integer value. +#' +#' @details +#' This only works for integer, numeric and character types right now. +#' +#' @param key the object to be hashed +#' @return the hash code as an integer +#' @export +#' @examples +#' hashCode(1L) # 1 +#' hashCode(1.0) # 1072693248 +#' hashCode("1") # 49 +hashCode <- function(key) { + if (class(key) == "integer") { + as.integer(key[[1]]) + } else if (class(key) == "numeric") { + # Convert the double to long and then calculate the hash code + rawVec <- writeBin(key[[1]], con = raw()) + intBits <- packBits(rawToBits(rawVec), "integer") + as.integer(bitwXor(intBits[2], intBits[1])) + } else if (class(key) == "character") { + .Call("stringHashCode", key) + } else { + warning(paste("Could not hash object, returning 0", sep = "")) + as.integer(0) + } +} + +# Create a new RDD with serializedMode == "byte". +# Return itself if already in "byte" format. +serializeToBytes <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "byte") { + ser.rdd <- lapply(rdd, function(x) { x }) + return(ser.rdd) + } else { + return(rdd) + } +} + +# Create a new RDD with serializedMode == "string". +# Return itself if already in "string" format. +serializeToString <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "string") { + ser.rdd <- lapply(rdd, function(x) { toString(x) }) + # force it to create jrdd using "string" + getJRDD(ser.rdd, serializedMode = "string") + return(ser.rdd) + } else { + return(rdd) + } +} + +# Fast append to list by using an accumulator. +# http://stackoverflow.com/questions/17046336/here-we-go-again-append-an-element-to-a-list-in-r +# +# The accumulator should has three fields size, counter and data. +# This function amortizes the allocation cost by doubling +# the size of the list every time it fills up. +addItemToAccumulator <- function(acc, item) { + if(acc$counter == acc$size) { + acc$size <- acc$size * 2 + length(acc$data) <- acc$size + } + acc$counter <- acc$counter + 1 + acc$data[[acc$counter]] <- item +} + +initAccumulator <- function() { + acc <- new.env() + acc$counter <- 0 + acc$data <- list(NULL) + acc$size <- 1 + acc +} + +# Utility function to sort a list of key value pairs +# Used in unit tests +sortKeyValueList <- function(kv_list, decreasing = FALSE) { + keys <- sapply(kv_list, function(x) x[[1]]) + kv_list[order(keys, decreasing = decreasing)] +} + +# Utility function to generate compact R lists from grouped rdd +# Used in Join-family functions +# param: +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +genCompactLists <- function(tagged_list, cnull) { + len <- length(tagged_list) + lists <- list(vector("list", len), vector("list", len)) + index <- list(1, 1) + + for (x in tagged_list) { + tag <- x[[1]] + idx <- index[[tag]] + lists[[tag]][[idx]] <- x[[2]] + index[[tag]] <- idx + 1 + } + + len <- lapply(index, function(x) x - 1) + for (i in (1:2)) { + if (cnull[[i]] && len[[i]] == 0) { + lists[[i]] <- list(NULL) + } else { + length(lists[[i]]) <- len[[i]] + } + } + + lists +} + +# Utility function to merge compact R lists +# Used in Join-family functions +# param: +# left/right Two compact lists ready for Cartesian product +mergeCompactLists <- function(left, right) { + result <- list() + length(result) <- length(left) * length(right) + index <- 1 + for (i in left) { + for (j in right) { + result[[index]] <- list(i, j) + index <- index + 1 + } + } + result +} + +# Utility function to wrapper above two operations +# Used in Join-family functions +# param (same as genCompactLists): +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +joinTaggedList <- function(tagged_list, cnull) { + lists <- genCompactLists(tagged_list, cnull) + mergeCompactLists(lists[[1]], lists[[2]]) +} + +# Utility function to reduce a key-value list with predicate +# Used in *ByKey functions +# param +# pair key-value pair +# keys/vals env of key/value with hashes +# updateOrCreatePred predicate function +# updateFn update or merge function for existing pair, similar with `mergeVal` @combineByKey +# createFn create function for new pair, similar with `createCombiner` @combinebykey +updateOrCreatePair <- function(pair, keys, vals, updateOrCreatePred, updateFn, createFn) { + # assume hashVal bind to `$hash`, key/val with index 1/2 + hashVal <- pair$hash + key <- pair[[1]] + val <- pair[[2]] + if (updateOrCreatePred(pair)) { + assign(hashVal, do.call(updateFn, list(get(hashVal, envir = vals), val)), envir = vals) + } else { + assign(hashVal, do.call(createFn, list(val)), envir = vals) + assign(hashVal, key, envir = keys) + } +} + +# Utility function to convert key&values envs into key-val list +convertEnvsToList <- function(keys, vals) { + lapply(ls(keys), + function(name) { + list(keys[[name]], vals[[name]]) + }) +} + +# Utility function to capture the varargs into environment object +varargsToEnv <- function(...) { + pairs <- as.list(substitute(list(...)))[-1L] + env <- new.env() + for (name in names(pairs)) { + env[[name]] <- pairs[[name]] + } + env +} + +getStorageLevel <- function(newLevel = c("DISK_ONLY", + "DISK_ONLY_2", + "MEMORY_AND_DISK", + "MEMORY_AND_DISK_2", + "MEMORY_AND_DISK_SER", + "MEMORY_AND_DISK_SER_2", + "MEMORY_ONLY", + "MEMORY_ONLY_2", + "MEMORY_ONLY_SER", + "MEMORY_ONLY_SER_2", + "OFF_HEAP")) { + match.arg(newLevel) + storageLevel <- switch(newLevel, + "DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"), + "DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"), + "MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"), + "MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"), + "MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"), + "MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"), + "MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"), + "MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"), + "MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"), + "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), + "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) +} + +# Utility function for functions where an argument needs to be integer but we want to allow +# the user to type (for example) `5` instead of `5L` to avoid a confusing error message. +numToInt <- function(num) { + if (as.integer(num) != num) { + warning(paste("Coercing", as.list(sys.call())[[2]], "to integer.")) + } + as.integer(num) +} + +# create a Seq in JVM +toSeq <- function(...) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", list(...)) +} + +# create a Seq in JVM from a list +listToSeq <- function(l) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", l) +} + +# Utility function to recursively traverse the Abstract Syntax Tree (AST) of a +# user defined function (UDF), and to examine variables in the UDF to decide +# if their values should be included in the new function environment. +# param +# node The current AST node in the traversal. +# oldEnv The original function environment. +# defVars An Accumulator of variables names defined in the function's calling environment, +# including function argument and local variable names. +# checkedFunc An environment of function objects examined during cleanClosure. It can +# be considered as a "name"-to-"list of functions" mapping. +# newEnv A new function environment to store necessary function dependencies, an output argument. +processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { + nodeLen <- length(node) + + if (nodeLen > 1 && typeof(node) == "language") { + # Recursive case: current AST node is an internal node, check for its children. + if (length(node[[1]]) > 1) { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else { # if node[[1]] is length of 1, check for some R special functions. + nodeChar <- as.character(node[[1]]) + if (nodeChar == "{" || nodeChar == "(") { # Skip start symbol. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "<-" || nodeChar == "=" || + nodeChar == "<<-") { # Assignment Ops. + defVar <- node[[2]] + if (length(defVar) == 1 && typeof(defVar) == "symbol") { + # Add the defined variable name into defVars. + addItemToAccumulator(defVars, as.character(defVar)) + } else { + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "function") { # Function definition. + # Add parameter names. + newArgs <- names(node[[2]]) + lapply(newArgs, function(arg) { addItemToAccumulator(defVars, arg) }) + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "$") { # Skip the field. + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } else if (nodeChar == "::" || nodeChar == ":::") { + processClosure(node[[3]], oldEnv, defVars, checkedFuncs, newEnv) + } else { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } + } + } else if (nodeLen == 1 && + (typeof(node) == "symbol" || typeof(node) == "language")) { + # Base case: current AST node is a leaf node and a symbol or a function call. + nodeChar <- as.character(node) + if (!nodeChar %in% defVars$data) { # Not a function parameter or local variable. + func.env <- oldEnv + topEnv <- parent.env(.GlobalEnv) + # Search in function environment, and function's enclosing environments + # up to global environment. There is no need to look into package environments + # above the global or namespace environment that is not SparkR below the global, + # as they are assumed to be loaded on workers. + while (!identical(func.env, topEnv)) { + # Namespaces other than "SparkR" will not be searched. + if (!isNamespace(func.env) || + (getNamespaceName(func.env) == "SparkR" && + !(nodeChar %in% getNamespaceExports("SparkR")))) { # Only include SparkR internals. + # Set parameter 'inherits' to FALSE since we do not need to search in + # attached package environments. + if (tryCatch(exists(nodeChar, envir = func.env, inherits = FALSE), + error = function(e) { FALSE })) { + obj <- get(nodeChar, envir = func.env, inherits = FALSE) + if (is.function(obj)) { # If the node is a function call. + funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, + ifnotfound = list(list(NULL)))[[1]] + found <- sapply(funcList, function(func) { + ifelse(identical(func, obj), TRUE, FALSE) + }) + if (sum(found) > 0) { # If function has been examined, ignore. + break + } + # Function has not been examined, record it and recursively clean its closure. + assign(nodeChar, + if (is.null(funcList[[1]])) { + list(obj) + } else { + append(funcList, obj) + }, + envir = checkedFuncs) + obj <- cleanClosure(obj, checkedFuncs) + } + assign(nodeChar, obj, envir = newEnv) + break + } + } + + # Continue to search in enclosure. + func.env <- parent.env(func.env) + } + } + } +} + +# Utility function to get user defined function (UDF) dependencies (closure). +# More specifically, this function captures the values of free variables defined +# outside a UDF, and stores them in the function's environment. +# param +# func A function whose closure needs to be captured. +# checkedFunc An environment of function objects examined during cleanClosure. It can be +# considered as a "name"-to-"list of functions" mapping. +# return value +# a new version of func that has an correct environment (closure). +cleanClosure <- function(func, checkedFuncs = new.env()) { + if (is.function(func)) { + newEnv <- new.env(parent = .GlobalEnv) + func.body <- body(func) + oldEnv <- environment(func) + # defVars is an Accumulator of variables names defined in the function's calling + # environment. First, function's arguments are added to defVars. + defVars <- initAccumulator() + argNames <- names(as.list(args(func))) + for (i in 1:(length(argNames) - 1)) { # Remove the ending NULL in pairlist. + addItemToAccumulator(defVars, argNames[i]) + } + # Recursively examine variables in the function body. + processClosure(func.body, oldEnv, defVars, checkedFuncs, newEnv) + environment(func) <- newEnv + } + func +} diff --git a/R/pkg/R/zzz.R b/R/pkg/R/zzz.R new file mode 100644 index 0000000000000..80d796d467943 --- /dev/null +++ b/R/pkg/R/zzz.R @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +.onLoad <- function(libname, pkgname) { + sparkR.onLoad(libname, pkgname) +} + diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R new file mode 100644 index 0000000000000..8fe711b622086 --- /dev/null +++ b/R/pkg/inst/profile/general.R @@ -0,0 +1,22 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) +} diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R new file mode 100644 index 0000000000000..7a7f2031152a0 --- /dev/null +++ b/R/pkg/inst/profile/shell.R @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) + + library(utils) + library(SparkR) + sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) + assign("sc", sc, envir=.GlobalEnv) + sqlCtx <- sparkRSQL.init(sc) + assign("sqlCtx", sqlCtx, envir=.GlobalEnv) + cat("\n Welcome to SparkR!") + cat("\n Spark context is available as sc, SQL context is available as sqlCtx\n") +} diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/inst/tests/test_binaryFile.R new file mode 100644 index 0000000000000..4bb5f58d83dc9 --- /dev/null +++ b/R/pkg/inst/tests/test_binaryFile.R @@ -0,0 +1,90 @@ +# +# 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. +# + +context("functions on binary files") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("saveAsObjectFile()/objectFile() following textFile() works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + saveAsObjectFile(rdd, fileName2) + rdd <- objectFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + + l <- list(1, 2, 3) + rdd <- parallelize(sc, l) + saveAsObjectFile(rdd, fileName) + rdd <- objectFile(sc, fileName) + expect_equal(collect(rdd), l) + + unlink(fileName, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() following RDD transformations works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsObjectFile(counts, fileName2) + counts <- objectFile(sc, fileName2) + + output <- collect(counts) + expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), + list("is", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works with multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + + rdd1 <- parallelize(sc, "Spark is pretty.") + saveAsObjectFile(rdd1, fileName1) + rdd2 <- parallelize(sc, "Spark is awesome.") + saveAsObjectFile(rdd2, fileName2) + + rdd <- objectFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1, recursive = TRUE) + unlink(fileName2, recursive = TRUE) +}) + diff --git a/R/pkg/inst/tests/test_binary_function.R b/R/pkg/inst/tests/test_binary_function.R new file mode 100644 index 0000000000000..c15553ba28517 --- /dev/null +++ b/R/pkg/inst/tests/test_binary_function.R @@ -0,0 +1,68 @@ +# +# 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. +# + +context("binary functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +# File content +mockFile <- c("Spark is pretty.", "Spark is awesome.") + +test_that("union on two RDDs", { + actual <- collect(unionRDD(rdd, rdd)) + expect_equal(actual, as.list(rep(nums, 2))) + + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, c(as.list(nums), mockFile)) + expect_true(getSerializedMode(union.rdd) == "byte") + + rdd<- map(text.rdd, function(x) {x}) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, as.list(c(mockFile, mockFile))) + expect_true(getSerializedMode(union.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cogroup on two RDDs", { + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + expect_equal(actual, + list(list(1, list(list(1), list(2, 3))), list(2, list(list(4), list())))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("a", 4))) + rdd2 <- parallelize(sc, list(list("b", 2), list("a", 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + + expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3)))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) diff --git a/R/pkg/inst/tests/test_broadcast.R b/R/pkg/inst/tests/test_broadcast.R new file mode 100644 index 0000000000000..fee91a427d6d5 --- /dev/null +++ b/R/pkg/inst/tests/test_broadcast.R @@ -0,0 +1,48 @@ +# +# 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. +# + +context("broadcast variables") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rrdd <- parallelize(sc, nums, 2L) + +test_that("using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + randomMatBr <- broadcast(sc, randomMat) + + useBroadcast <- function(x) { + sum(value(randomMatBr) * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) + +test_that("without using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + + useBroadcast <- function(x) { + sum(randomMat * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) diff --git a/R/pkg/inst/tests/test_context.R b/R/pkg/inst/tests/test_context.R new file mode 100644 index 0000000000000..e4aab37436a74 --- /dev/null +++ b/R/pkg/inst/tests/test_context.R @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("test functions in sparkR.R") + +test_that("repeatedly starting and stopping SparkR", { + for (i in 1:4) { + sc <- sparkR.init() + rdd <- parallelize(sc, 1:20, 2L) + expect_equal(count(rdd), 20) + sparkR.stop() + } +}) + +test_that("rdd GC across sparkR.stop", { + sparkR.stop() + sc <- sparkR.init() # sc should get id 0 + rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1 + rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2 + sparkR.stop() + + sc <- sparkR.init() # sc should get id 0 again + + # GC rdd1 before creating rdd3 and rdd2 after + rm(rdd1) + gc() + + rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now + rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now + + rm(rdd2) + gc() + + count(rdd3) + count(rdd4) +}) diff --git a/R/pkg/inst/tests/test_includePackage.R b/R/pkg/inst/tests/test_includePackage.R new file mode 100644 index 0000000000000..8152b448d0870 --- /dev/null +++ b/R/pkg/inst/tests/test_includePackage.R @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("include R packages") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rdd <- parallelize(sc, nums, 2L) + +test_that("include inside function", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + suppressPackageStartupMessages(library(plyr)) + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) + +test_that("use include package", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + includePackage(sc, plyr) + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) diff --git a/R/pkg/inst/tests/test_parallelize_collect.R b/R/pkg/inst/tests/test_parallelize_collect.R new file mode 100644 index 0000000000000..fff028657db37 --- /dev/null +++ b/R/pkg/inst/tests/test_parallelize_collect.R @@ -0,0 +1,109 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("parallelize() and collect()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +numPairs <- list(list(1, 1), list(1, 2), list(2, 2), list(2, 3)) +strPairs <- list(list(strList, strList), list(strList, strList)) + +# JavaSparkContext handle +jsc <- sparkR.init() + +# Tests + +test_that("parallelize() on simple vectors and lists returns an RDD", { + numVectorRDD <- parallelize(jsc, numVector, 1) + numVectorRDD2 <- parallelize(jsc, numVector, 10) + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + + rdds <- c(numVectorRDD, + numVectorRDD2, + numListRDD, + numListRDD2, + strVectorRDD, + strVectorRDD2, + strListRDD, + strListRDD2) + + for (rdd in rdds) { + expect_true(inherits(rdd, "RDD")) + expect_true(.hasSlot(rdd, "jrdd") + && inherits(rdd@jrdd, "jobj") + && isInstanceOf(rdd@jrdd, "org.apache.spark.api.java.JavaRDD")) + } +}) + +test_that("collect(), following a parallelize(), gives back the original collections", { + numVectorRDD <- parallelize(jsc, numVector, 10) + expect_equal(collect(numVectorRDD), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(collect(numListRDD), as.list(numList)) + expect_equal(collect(numListRDD2), as.list(numList)) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(collect(strVectorRDD), as.list(strVector)) + expect_equal(collect(strVectorRDD2), as.list(strVector)) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(collect(strListRDD), as.list(strList)) + expect_equal(collect(strListRDD2), as.list(strList)) +}) + +test_that("regression: collect() following a parallelize() does not drop elements", { + # 10 %/% 6 = 1, ceiling(10 / 6) = 2 + collLen <- 10 + numPart <- 6 + expected <- runif(collLen) + actual <- collect(parallelize(jsc, expected, numPart)) + expect_equal(actual, as.list(expected)) +}) + +test_that("parallelize() and collect() work for lists of pairs (pairwise data)", { + # use the pairwise logical to indicate pairwise data + numPairsRDDD1 <- parallelize(jsc, numPairs, 1) + numPairsRDDD2 <- parallelize(jsc, numPairs, 2) + numPairsRDDD3 <- parallelize(jsc, numPairs, 3) + expect_equal(collect(numPairsRDDD1), numPairs) + expect_equal(collect(numPairsRDDD2), numPairs) + expect_equal(collect(numPairsRDDD3), numPairs) + # can also leave out the parameter name, if the params are supplied in order + strPairsRDDD1 <- parallelize(jsc, strPairs, 1) + strPairsRDDD2 <- parallelize(jsc, strPairs, 2) + expect_equal(collect(strPairsRDDD1), strPairs) + expect_equal(collect(strPairsRDDD2), strPairs) +}) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R new file mode 100644 index 0000000000000..f75e0817b9406 --- /dev/null +++ b/R/pkg/inst/tests/test_rdd.R @@ -0,0 +1,644 @@ +# +# 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. +# + +context("basic RDD functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +test_that("get number of partitions in RDD", { + expect_equal(numPartitions(rdd), 2) + expect_equal(numPartitions(intRdd), 2) +}) + +test_that("first on RDD", { + expect_true(first(rdd) == 1) + newrdd <- lapply(rdd, function(x) x + 1) + expect_true(first(newrdd) == 2) +}) + +test_that("count and length on RDD", { + expect_equal(count(rdd), 10) + expect_equal(length(rdd), 10) +}) + +test_that("count by values and keys", { + mods <- lapply(rdd, function(x) { x %% 3 }) + actual <- countByValue(mods) + expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + actual <- countByKey(intRdd) + expected <- list(list(2L, 2L), list(1L, 2L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("lapply on RDD", { + multiples <- lapply(rdd, function(x) { 2 * x }) + actual <- collect(multiples) + expect_equal(actual, as.list(nums * 2)) +}) + +test_that("lapplyPartition on RDD", { + sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("mapPartitions on RDD", { + sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("flatMap() on RDDs", { + flat <- flatMap(intRdd, function(x) { list(x, x) }) + actual <- collect(flat) + expect_equal(actual, rep(intPairs, each=2)) +}) + +test_that("filterRDD on RDD", { + filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list(2, 4, 6, 8, 10)) + + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) + actual <- collect(filtered.rdd) + expect_equal(actual, list(list(1L, -1))) + + # Filter out all elements. + filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list()) +}) + +test_that("lookup on RDD", { + vals <- lookup(intRdd, 1L) + expect_equal(vals, list(-1, 200)) + + vals <- lookup(intRdd, 3L) + expect_equal(vals, list()) +}) + +test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { + rdd2 <- rdd + for (i in 1:12) + rdd2 <- lapplyPartitionsWithIndex( + rdd2, function(split, part) { + part <- as.list(unlist(part) * split + i) + }) + rdd2 <- lapply(rdd2, function(x) x + x) + actual <- collect(rdd2) + expected <- list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176) + expect_equal(actual, expected) +}) + +test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { + # RDD + rdd2 <- rdd + # PipelinedRDD + rdd2 <- lapplyPartitionsWithIndex( + rdd2, + function(split, part) { + part <- as.list(unlist(part) * split) + }) + + cache(rdd2) + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + persist(rdd2, "MEMORY_AND_DISK") + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + setCheckpointDir(sc, "checkpoints") + checkpoint(rdd2) + expect_true(rdd2@env$isCheckpointed) + + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + expect_false(rdd2@env$isCheckpointed) + + # make sure the data is collectable + collect(rdd2) + + unlink("checkpoints") +}) + +test_that("reduce on RDD", { + sum <- reduce(rdd, "+") + expect_equal(sum, 55) + + # Also test with an inline function + sumInline <- reduce(rdd, function(x, y) { x + y }) + expect_equal(sumInline, 55) +}) + +test_that("lapply with dependency", { + fa <- 5 + multiples <- lapply(rdd, function(x) { fa * x }) + actual <- collect(multiples) + + expect_equal(actual, as.list(nums * 5)) +}) + +test_that("lapplyPartitionsWithIndex on RDDs", { + func <- function(splitIndex, part) { list(splitIndex, Reduce("+", part)) } + actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) + expect_equal(actual, list(list(0, 15), list(1, 40))) + + pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L) + partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 } + mkTup <- function(splitIndex, part) { list(splitIndex, part) } + actual <- collect(lapplyPartitionsWithIndex( + partitionBy(pairsRDD, 2L, partitionByParity), + mkTup), + FALSE) + expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), + list(1, list(list(4, 8))))) +}) + +test_that("sampleRDD() on RDDs", { + expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) +}) + +test_that("takeSample() on RDDs", { + # ported from RDDSuite.scala, modified seeds + data <- parallelize(sc, 1:100, 2L) + for (seed in 4:5) { + s <- takeSample(data, FALSE, 20L, seed) + expect_equal(length(s), 20L) + expect_equal(length(unique(s)), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, FALSE, 200L, seed) + expect_equal(length(s), 100L) + expect_equal(length(unique(s)), 100L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 20L, seed) + expect_equal(length(s), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 100L, seed) + expect_equal(length(s), 100L) + # Chance of getting all distinct elements is astronomically low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 200L, seed) + expect_equal(length(s), 200L) + # Chance of getting all distinct elements is still quite low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } +}) + +test_that("mapValues() on pairwise RDDs", { + multiples <- mapValues(intRdd, function(x) { x * 2 }) + actual <- collect(multiples) + expected <- lapply(intPairs, function(x) { + list(x[[1]], x[[2]] * 2) + }) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("flatMapValues() on pairwise RDDs", { + l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) + actual <- collect(flatMapValues(l, function(x) { x })) + expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) + + # Generate x to x+1 for every value + actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) + expect_equal(actual, + list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), + list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) +}) + +test_that("reduceByKeyLocally() on PairwiseRDDs", { + pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, 6), list(1.1, 3)))) + + pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3), + list("bb", 5)), 4L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5)))) +}) + +test_that("distinct() on RDDs", { + nums.rep2 <- rep(1:10, 2) + rdd.rep2 <- parallelize(sc, nums.rep2, 2L) + uniques <- distinct(rdd.rep2) + actual <- sort(unlist(collect(uniques))) + expect_equal(actual, nums) +}) + +test_that("maximum() on RDDs", { + max <- maximum(rdd) + expect_equal(max, 10) +}) + +test_that("minimum() on RDDs", { + min <- minimum(rdd) + expect_equal(min, 1) +}) + +test_that("sumRDD() on RDDs", { + sum <- sumRDD(rdd) + expect_equal(sum, 55) +}) + +test_that("keyBy on RDDs", { + func <- function(x) { x*x } + keys <- keyBy(rdd, func) + actual <- collect(keys) + expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) +}) + +test_that("repartition/coalesce on RDDs", { + rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements + + # repartition + r1 <- repartition(rdd, 2) + expect_equal(numPartitions(r1), 2L) + count <- length(collectPartition(r1, 0L)) + expect_true(count >= 8 && count <= 12) + + r2 <- repartition(rdd, 6) + expect_equal(numPartitions(r2), 6L) + count <- length(collectPartition(r2, 0L)) + expect_true(count >=0 && count <= 4) + + # coalesce + r3 <- coalesce(rdd, 1) + expect_equal(numPartitions(r3), 1L) + count <- length(collectPartition(r3, 0L)) + expect_equal(count, 20) +}) + +test_that("sortBy() on RDDs", { + sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) + actual <- collect(sortedRdd) + expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) + actual <- collect(sortedRdd2) + expect_equal(actual, as.list(nums)) +}) + +test_that("takeOrdered() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l)))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l)))[1:3]) +}) + +test_that("top() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- top(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- top(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3]) +}) + +test_that("fold() on RDDs", { + actual <- fold(rdd, 0, "+") + expect_equal(actual, Reduce("+", nums, 0)) + + rdd <- parallelize(sc, list()) + actual <- fold(rdd, 0, "+") + expect_equal(actual, 0) +}) + +test_that("aggregateRDD() on RDDs", { + rdd <- parallelize(sc, list(1, 2, 3, 4)) + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(10, 4)) + + rdd <- parallelize(sc, list()) + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(0, 0)) +}) + +test_that("zipWithUniqueId() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 3), list("c", 1), + list("d", 4), list("e", 2)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("zipWithIndex() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("glom() on RDD", { + rdd <- parallelize(sc, as.list(1:4), 2L) + actual <- collect(glom(rdd)) + expect_equal(actual, list(list(1, 2), list(3, 4))) +}) + +test_that("keys() on RDDs", { + keys <- keys(intRdd) + actual <- collect(keys) + expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) +}) + +test_that("values() on RDDs", { + values <- values(intRdd) + actual <- collect(values) + expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) +}) + +test_that("pipeRDD() on RDDs", { + actual <- collect(pipeRDD(rdd, "more")) + expected <- as.list(as.character(1:10)) + expect_equal(actual, expected) + + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) + actual <- collect(pipeRDD(trailed.rdd, "sort")) + expected <- list("", "1", "2", "3") + expect_equal(actual, expected) + + rev.nums <- 9:0 + rev.rdd <- parallelize(sc, rev.nums, 2L) + actual <- collect(pipeRDD(rev.rdd, "sort")) + expected <- as.list(as.character(c(5:9, 0:4))) + expect_equal(actual, expected) +}) + +test_that("zipRDD() on RDDs", { + rdd1 <- parallelize(sc, 0:4, 2) + rdd2 <- parallelize(sc, 1000:1004, 2) + actual <- collect(zipRDD(rdd1, rdd2)) + expect_equal(actual, + list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName, 1) + actual <- collect(zipRDD(rdd, rdd)) + expected <- lapply(mockFile, function(x) { list(x ,x) }) + expect_equal(actual, expected) + + rdd1 <- parallelize(sc, 0:1, 1) + actual <- collect(zipRDD(rdd1, rdd)) + expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) }) + expect_equal(actual, expected) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(zipRDD(rdd, rdd1)) + expected <- lapply(mockFile, function(x) { list(x, x) }) + expect_equal(actual, expected) + + unlink(fileName) +}) + +test_that("join() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) +}) + +test_that("leftOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) + +test_that("rightOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("fullOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("sortByKey() on pairwise RDDs", { + numPairsRdd <- map(rdd, function(x) { list (x, x) }) + sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) + actual <- collect(sortedRdd) + numPairs <- lapply(nums, function(x) { list (x, x) }) + expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE)) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + numPairsRdd2 <- map(rdd2, function(x) { list (x, x) }) + sortedRdd2 <- sortByKey(numPairsRdd2) + actual <- collect(sortedRdd2) + expect_equal(actual, numPairs) + + # sort by string keys + l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5)) + rdd3 <- parallelize(sc, l, 2L) + sortedRdd3 <- sortByKey(rdd3) + actual <- collect(sortedRdd3) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # test on the boundary cases + + # boundary case 1: the RDD to be sorted has only 1 partition + rdd4 <- parallelize(sc, l, 1L) + sortedRdd4 <- sortByKey(rdd4) + actual <- collect(sortedRdd4) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 2: the sorted RDD has only 1 partition + rdd5 <- parallelize(sc, l, 2L) + sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L) + actual <- collect(sortedRdd5) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 3: the RDD to be sorted has only 1 element + l2 <- list(list("a", 1)) + rdd6 <- parallelize(sc, l2, 2L) + sortedRdd6 <- sortByKey(rdd6) + actual <- collect(sortedRdd6) + expect_equal(actual, l2) + + # boundary case 4: the RDD to be sorted has 0 element + l3 <- list() + rdd7 <- parallelize(sc, l3, 2L) + sortedRdd7 <- sortByKey(rdd7) + actual <- collect(sortedRdd7) + expect_equal(actual, l3) +}) + +test_that("collectAsMap() on a pairwise RDD", { + rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = 2, `3` = 4)) + + rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(a = 1, b = 2)) + + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) + + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = "a", `2` = "b")) +}) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R new file mode 100644 index 0000000000000..d1da8232aea81 --- /dev/null +++ b/R/pkg/inst/tests/test_shuffle.R @@ -0,0 +1,209 @@ +# +# 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. +# + +context("partitionBy, groupByKey, reduceByKey etc.") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +doublePairs <- list(list(1.5, -1), list(2.5, 100), list(2.5, 1), list(1.5, 200)) +doubleRdd <- parallelize(sc, doublePairs, 2L) + +numPairs <- list(list(1L, 100), list(2L, 200), list(4L, -1), list(3L, 1), + list(3L, 0)) +numPairsRdd <- parallelize(sc, numPairs, length(numPairs)) + +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge and ", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ") +strListRDD <- parallelize(sc, strList, 4) + +test_that("groupByKey for integers", { + grouped <- groupByKey(intRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("groupByKey for doubles", { + grouped <- groupByKey(doubleRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for ints", { + reduced <- reduceByKey(intRdd, "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for doubles", { + reduced <- reduceByKey(doubleRdd, "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for ints", { + reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for doubles", { + reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("aggregateByKey", { + # test aggregateByKey for int keys + rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list(1, list(3, 2)), list(2, list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test aggregateByKey for string keys + rdd <- parallelize(sc, list(list("a", 1), list("a", 2), list("b", 3), list("b", 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list("a", list(3, 2)), list("b", list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("foldByKey", { + # test foldByKey for int keys + folded <- foldByKey(intRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for double keys + folded <- foldByKey(doubleRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for string keys + stringKeyPairs <- list(list("a", -1), list("b", 100), list("b", 1), list("a", 200)) + + stringKeyRDD <- parallelize(sc, stringKeyPairs) + folded <- foldByKey(stringKeyRDD, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list("b", 101), list("a", 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for empty pair RDD + rdd <- parallelize(sc, list()) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list() + expect_equal(actual, expected) + + # test foldByKey for RDD with only 1 pair + rdd <- parallelize(sc, list(list(1, 1))) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list(list(1, 1)) + expect_equal(actual, expected) +}) + +test_that("partitionBy() partitions data correctly", { + # Partition by magnitude + partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 } + + resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) + + expected_first <- list(list(1, 100), list(2, 200)) # key < 3 + expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3 + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("partitionBy works with dependencies", { + kOne <- 1 + partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 } + + # Partition by parity + resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity) + + # keys even; 100 %% 2 == 0 + expected_first <- list(list(2, 200), list(4, -1)) + # keys odd; 3 %% 2 == 1 + expected_second <- list(list(1, 100), list(3, 1), list(3, 0)) + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("test partitionBy with string keys", { + words <- flatMap(strListRDD, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + resultRDD <- partitionBy(wordCount, 2L) + expected_first <- list(list("Dexter", 1), list("Dexter", 1)) + expected_second <- list(list("and", 1), list("and", 1)) + + actual_first <- Filter(function(item) { item[[1]] == "Dexter" }, + collectPartition(resultRDD, 0L)) + actual_second <- Filter(function(item) { item[[1]] == "and" }, + collectPartition(resultRDD, 1L)) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R new file mode 100644 index 0000000000000..cf5cf6d1692af --- /dev/null +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -0,0 +1,695 @@ +# +# 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. +# + +library(testthat) + +context("SparkSQL functions") + +# Tests for SparkSQL functions in SparkR + +sc <- sparkR.init() + +sqlCtx <- sparkRSQL.init(sc) + +mockLines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}") +jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") +parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet") +writeLines(mockLines, jsonPath) + +test_that("infer types", { + expect_equal(infer_type(1L), "integer") + expect_equal(infer_type(1.0), "double") + expect_equal(infer_type("abc"), "string") + expect_equal(infer_type(TRUE), "boolean") + expect_equal(infer_type(as.Date("2015-03-11")), "date") + expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp") + expect_equal(infer_type(c(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(a = 1L, b = "2")), + list(type = "struct", + fields = list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)))) + e <- new.env() + assign("a", 1L, envir = e) + expect_equal(infer_type(e), + list(type = "map", keyType = "string", valueType = "integer", + valueContainsNull = TRUE)) +}) + +test_that("create DataFrame from RDD", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + fields <- list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)) + schema <- list(type = "struct", fields = fields) + df <- createDataFrame(sqlCtx, rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("toDF", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- toDF(rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + fields <- list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)) + schema <- list(type = "struct", fields = fields) + df <- toDF(rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("create DataFrame from list or data.frame", { + l <- list(list(1, 2), list(3, 4)) + df <- createDataFrame(sqlCtx, l, c("a", "b")) + expect_equal(columns(df), c("a", "b")) + + l <- list(list(a=1, b=2), list(a=3, b=4)) + df <- createDataFrame(sqlCtx, l) + expect_equal(columns(df), c("a", "b")) + + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- createDataFrame(sqlCtx, ldf) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) +}) + +test_that("create DataFrame with different data types", { + l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"), + f = as.POSIXct("2015-03-15 12:13:14.056")) + df <- createDataFrame(sqlCtx, list(l)) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"), + c("d", "string"), c("e", "date"), c("f", "timestamp"))) + expect_equal(count(df), 1) + expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) +}) + +# TODO: enable this test after fix serialization for nested object +#test_that("create DataFrame with nested array and struct", { +# e <- new.env() +# assign("n", 3L, envir = e) +# l <- list(1:10, list("a", "b"), e, list(a="aa", b=3L)) +# df <- createDataFrame(sqlCtx, list(l), c("a", "b", "c", "d")) +# expect_equal(dtypes(df), list(c("a", "array"), c("b", "array"), +# c("c", "map"), c("d", "struct"))) +# expect_equal(count(df), 1) +# ldf <- collect(df) +# expect_equal(ldf[1,], l[[1]]) +#}) + +test_that("jsonFile() on a local file returns a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("jsonRDD() on a RDD with json string", { + rdd <- parallelize(sc, mockLines) + expect_true(count(rdd) == 3) + df <- jsonRDD(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + + rdd2 <- flatMap(rdd, function(x) c(x, x)) + df <- jsonRDD(sqlCtx, rdd2) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 6) +}) + +test_that("test cache, uncache and clearCache", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + cacheTable(sqlCtx, "table1") + uncacheTable(sqlCtx, "table1") + clearCache(sqlCtx) + dropTempTable(sqlCtx, "table1") +}) + +test_that("test tableNames and tables", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + expect_true(length(tableNames(sqlCtx)) == 1) + df <- tables(sqlCtx) + expect_true(count(df) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + newdf <- sql(sqlCtx, "SELECT * FROM table1 where name = 'Michael'") + expect_true(inherits(newdf, "DataFrame")) + expect_true(count(newdf) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("insertInto() on a registered table", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", "overwrite") + dfParquet <- loadDF(sqlCtx, parquetPath, "parquet") + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + saveDF(df2, parquetPath2, "parquet", "overwrite") + dfParquet2 <- loadDF(sqlCtx, parquetPath2, "parquet") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1") + expect_true(count(sql(sqlCtx, "select * from table1")) == 5) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Michael") + dropTempTable(sqlCtx, "table1") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1", overwrite = TRUE) + expect_true(count(sql(sqlCtx, "select * from table1")) == 2) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Bob") + dropTempTable(sqlCtx, "table1") +}) + +test_that("table() returns a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + tabledf <- table(sqlCtx, "table1") + expect_true(inherits(tabledf, "DataFrame")) + expect_true(count(tabledf) == 3) + dropTempTable(sqlCtx, "table1") +}) + +test_that("toRDD() returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toRDD(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(count(testRDD) == 3) +}) + +test_that("union on two RDDs created from DataFrames returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + RDD1 <- toRDD(df) + RDD2 <- toRDD(df) + unioned <- unionRDD(RDD1, RDD2) + expect_true(inherits(unioned, "RDD")) + expect_true(SparkR:::getSerializedMode(unioned) == "byte") + expect_true(collect(unioned)[[2]]$name == "Andy") +}) + +test_that("union on mixed serialization types correctly returns a byte RRDD", { + # Byte RDD + nums <- 1:10 + rdd <- parallelize(sc, nums, 2L) + + # String RDD + textLines <- c("Michael", + "Andy, 30", + "Justin, 19") + textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp") + writeLines(textLines, textPath) + textRDD <- textFile(sc, textPath) + + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + + unionByte <- unionRDD(rdd, dfRDD) + expect_true(inherits(unionByte, "RDD")) + expect_true(SparkR:::getSerializedMode(unionByte) == "byte") + expect_true(collect(unionByte)[[1]] == 1) + expect_true(collect(unionByte)[[12]]$name == "Andy") + + unionString <- unionRDD(textRDD, dfRDD) + expect_true(inherits(unionString, "RDD")) + expect_true(SparkR:::getSerializedMode(unionString) == "byte") + expect_true(collect(unionString)[[1]] == "Michael") + expect_true(collect(unionString)[[5]]$name == "Andy") +}) + +test_that("objectFile() works with row serialization", { + objectPath <- tempfile(pattern="spark-test", fileext=".tmp") + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) + objectIn <- objectFile(sc, objectPath) + + expect_true(inherits(objectIn, "RDD")) + expect_equal(SparkR:::getSerializedMode(objectIn), "byte") + expect_equal(collect(objectIn)[[2]]$age, 30) +}) + +test_that("lapply() on a DataFrame returns an RDD with the correct columns", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- lapply(df, function(row) { + row$newCol <- row$age + 5 + row + }) + expect_true(inherits(testRDD, "RDD")) + collected <- collect(testRDD) + expect_true(collected[[1]]$name == "Michael") + expect_true(collected[[2]]$newCol == "35") +}) + +test_that("collect() returns a data.frame", { + df <- jsonFile(sqlCtx, jsonPath) + rdf <- collect(df) + expect_true(is.data.frame(rdf)) + expect_true(names(rdf)[1] == "age") + expect_true(nrow(rdf) == 3) + expect_true(ncol(rdf) == 2) +}) + +test_that("limit() returns DataFrame with the correct number of rows", { + df <- jsonFile(sqlCtx, jsonPath) + dfLimited <- limit(df, 2) + expect_true(inherits(dfLimited, "DataFrame")) + expect_true(count(dfLimited) == 2) +}) + +test_that("collect() and take() on a DataFrame return the same number of rows and columns", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(nrow(collect(df)) == nrow(take(df, 10))) + expect_true(ncol(collect(df)) == ncol(take(df, 10))) +}) + +test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + first <- lapply(df, function(row) { + row$age <- row$age + 5 + row + }) + second <- lapply(first, function(row) { + row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE + row + }) + expect_true(inherits(second, "RDD")) + expect_true(count(second) == 3) + expect_true(collect(second)[[2]]$age == 35) + expect_true(collect(second)[[2]]$testCol) + expect_false(collect(second)[[3]]$testCol) +}) + +test_that("cache(), persist(), and unpersist() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(df@env$isCached) + cache(df) + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + persist(df, "MEMORY_AND_DISK") + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + # make sure the data is collectable + expect_true(is.data.frame(collect(df))) +}) + +test_that("schema(), dtypes(), columns(), names() return the correct values/format", { + df <- jsonFile(sqlCtx, jsonPath) + testSchema <- schema(df) + expect_true(length(testSchema$fields()) == 2) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "LongType") + expect_true(testSchema$fields()[[2]]$dataType.simpleString() == "string") + expect_true(testSchema$fields()[[1]]$name() == "age") + + testTypes <- dtypes(df) + expect_true(length(testTypes[[1]]) == 2) + expect_true(testTypes[[1]][1] == "age") + + testCols <- columns(df) + expect_true(length(testCols) == 2) + expect_true(testCols[2] == "name") + + testNames <- names(df) + expect_true(length(testNames) == 2) + expect_true(testNames[2] == "name") +}) + +test_that("head() and first() return the correct data", { + df <- jsonFile(sqlCtx, jsonPath) + testHead <- head(df) + expect_true(nrow(testHead) == 3) + expect_true(ncol(testHead) == 2) + + testHead2 <- head(df, 2) + expect_true(nrow(testHead2) == 2) + expect_true(ncol(testHead2) == 2) + + testFirst <- first(df) + expect_true(nrow(testFirst) == 1) +}) + +test_that("distinct() on DataFrames", { + lines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}", + "{\"name\":\"Justin\", \"age\":19}") + jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPathWithDup) + + df <- jsonFile(sqlCtx, jsonPathWithDup) + uniques <- distinct(df) + expect_true(inherits(uniques, "DataFrame")) + expect_true(count(uniques) == 3) +}) + +test_that("sampleDF on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sampled <- sampleDF(df, FALSE, 1.0) + expect_equal(nrow(collect(sampled)), count(df)) + expect_true(inherits(sampled, "DataFrame")) + sampled2 <- sampleDF(df, FALSE, 0.1) + expect_true(count(sampled2) < 3) +}) + +test_that("select operators", { + df <- select(jsonFile(sqlCtx, jsonPath), "name", "age") + expect_true(inherits(df$name, "Column")) + expect_true(inherits(df[[2]], "Column")) + expect_true(inherits(df[["age"]], "Column")) + + expect_true(inherits(df[,1], "DataFrame")) + expect_equal(columns(df[,1]), c("name")) + expect_equal(columns(df[,"age"]), c("age")) + df2 <- df[,c("age", "name")] + expect_true(inherits(df2, "DataFrame")) + expect_equal(columns(df2), c("age", "name")) + + df$age2 <- df$age + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age)), 2) + df$age2 <- df$age * 2 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 2)), 2) +}) + +test_that("select with column", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- select(df, "name") + expect_true(columns(df1) == c("name")) + expect_true(count(df1) == 3) + + df2 <- select(df, df$age) + expect_true(columns(df2) == c("age")) + expect_true(count(df2) == 3) +}) + +test_that("selectExpr() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + selected <- selectExpr(df, "age * 2") + expect_true(names(selected) == "(age * 2)") + expect_equal(collect(selected), collect(select(df, df$age * 2L))) + + selected2 <- selectExpr(df, "name as newName", "abs(age) as age") + expect_equal(names(selected2), c("newName", "age")) + expect_true(count(selected2) == 3) +}) + +test_that("column calculation", { + df <- jsonFile(sqlCtx, jsonPath) + d <- collect(select(df, alias(df$age + 1, "age2"))) + expect_true(names(d) == c("age2")) + df2 <- select(df, lower(df$name), abs(df$age)) + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("load() from json file", { + df <- loadDF(sqlCtx, jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("save() as parquet file", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", mode="overwrite") + df2 <- loadDF(sqlCtx, parquetPath, "parquet") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("test HiveContext", { + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + df <- createExternalTable(hiveCtx, "json", jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + df2 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) + + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + saveAsTable(df, "json", "json", "append", path = jsonPath2) + df3 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df3, "DataFrame")) + expect_true(count(df3) == 6) +}) + +test_that("column operators", { + c <- SparkR:::col("a") + c2 <- (- c + 1 - 2) * 3 / 4.0 + c3 <- (c + c2 - c2) * c2 %% c2 + c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3) +}) + +test_that("column functions", { + c <- SparkR:::col("a") + c2 <- min(c) + max(c) + sum(c) + avg(c) + count(c) + abs(c) + sqrt(c) + c3 <- lower(c) + upper(c) + first(c) + last(c) + c4 <- approxCountDistinct(c) + countDistinct(c) + cast(c, "string") +}) + +test_that("string operators", { + df <- jsonFile(sqlCtx, jsonPath) + expect_equal(count(where(df, like(df$name, "A%"))), 1) + expect_equal(count(where(df, startsWith(df$name, "A"))), 1) + expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") + expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") +}) + +test_that("group by", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- agg(df, name = "max", age = "sum") + expect_true(1 == count(df1)) + df1 <- agg(df, age2 = max(df$age)) + expect_true(1 == count(df1)) + expect_equal(columns(df1), c("age2")) + + gd <- groupBy(df, "name") + expect_true(inherits(gd, "GroupedData")) + df2 <- count(gd) + expect_true(inherits(df2, "DataFrame")) + expect_true(3 == count(df2)) + + df3 <- agg(gd, age = "sum") + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + + df3 <- agg(gd, age = sum(df$age)) + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + expect_equal(columns(df3), c("name", "age")) + + df4 <- sum(gd, "age") + expect_true(inherits(df4, "DataFrame")) + expect_true(3 == count(df4)) + expect_true(3 == count(mean(gd, "age"))) + expect_true(3 == count(max(gd, "age"))) +}) + +test_that("sortDF() and orderBy() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sorted <- sortDF(df, df$age) + expect_true(collect(sorted)[1,2] == "Michael") + + sorted2 <- sortDF(df, "name") + expect_true(collect(sorted2)[2,"age"] == 19) + + sorted3 <- orderBy(df, asc(df$age)) + expect_true(is.na(first(sorted3)$age)) + expect_true(collect(sorted3)[2, "age"] == 19) + + sorted4 <- orderBy(df, desc(df$name)) + expect_true(first(sorted4)$name == "Michael") + expect_true(collect(sorted4)[3,"name"] == "Andy") +}) + +test_that("filter() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + filtered <- filter(df, "age > 20") + expect_true(count(filtered) == 1) + expect_true(collect(filtered)$name == "Andy") + filtered2 <- where(df, df$name != "Michael") + expect_true(count(filtered2) == 2) + expect_true(collect(filtered2)$age[2] == 19) +}) + +test_that("join() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", + "{\"name\":\"Andy\", \"test\": \"no\"}", + "{\"name\":\"Justin\", \"test\": \"yes\"}", + "{\"name\":\"Bob\", \"test\": \"yes\"}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(mockLines2, jsonPath2) + df2 <- jsonFile(sqlCtx, jsonPath2) + + joined <- join(df, df2) + expect_equal(names(joined), c("age", "name", "name", "test")) + expect_true(count(joined) == 12) + + joined2 <- join(df, df2, df$name == df2$name) + expect_equal(names(joined2), c("age", "name", "name", "test")) + expect_true(count(joined2) == 3) + + joined3 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined3), c("age", "name", "name", "test")) + expect_true(count(joined3) == 4) + expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) + + joined4 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined4), c("newAge", "name", "test")) + expect_true(count(joined4) == 4) + expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) +}) + +test_that("toJSON() returns an RDD of the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toJSON(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(SparkR:::getSerializedMode(testRDD) == "string") + expect_equal(collect(testRDD)[[1]], mockLines[1]) +}) + +test_that("showDF()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_output(showDF(df), "age name \nnull Michael\n30 Andy \n19 Justin ") +}) + +test_that("isLocal()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(isLocal(df)) +}) + +test_that("unionAll(), subtract(), and intersect() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + + unioned <- sortDF(unionAll(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(unioned) == 6) + expect_true(first(unioned)$name == "Michael") + + subtracted <- sortDF(subtract(df, df2), desc(df$age)) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(subtracted) == 2) + expect_true(first(subtracted)$name == "Justin") + + intersected <- sortDF(intersect(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(intersected) == 1) + expect_true(first(intersected)$name == "Andy") +}) + +test_that("withColumn() and withColumnRenamed()", { + df <- jsonFile(sqlCtx, jsonPath) + newDF <- withColumn(df, "newAge", df$age + 2) + expect_true(length(columns(newDF)) == 3) + expect_true(columns(newDF)[3] == "newAge") + expect_true(first(filter(newDF, df$name != "Michael"))$newAge == 32) + + newDF2 <- withColumnRenamed(df, "age", "newerAge") + expect_true(length(columns(newDF2)) == 2) + expect_true(columns(newDF2)[1] == "newerAge") +}) + +test_that("saveDF() on DataFrame and works with parquetFile", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath) + expect_true(inherits(parquetDF, "DataFrame")) + expect_equal(count(df), count(parquetDF)) +}) + +test_that("parquetFile works with multiple input paths", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + saveDF(df, parquetPath2, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath, parquetPath2) + expect_true(inherits(parquetDF, "DataFrame")) + expect_true(count(parquetDF) == count(df)*2) +}) + +unlink(parquetPath) +unlink(jsonPath) diff --git a/R/pkg/inst/tests/test_take.R b/R/pkg/inst/tests/test_take.R new file mode 100644 index 0000000000000..7f4c7c315d787 --- /dev/null +++ b/R/pkg/inst/tests/test_take.R @@ -0,0 +1,67 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("tests RDD function take()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +# JavaSparkContext handle +jsc <- sparkR.init() + +test_that("take() gives back the original elements in correct count and order", { + numVectorRDD <- parallelize(jsc, numVector, 10) + # case: number of elements to take is less than the size of the first partition + expect_equal(take(numVectorRDD, 1), as.list(head(numVector, n = 1))) + # case: number of elements to take is the same as the size of the first partition + expect_equal(take(numVectorRDD, 11), as.list(head(numVector, n = 11))) + # case: number of elements to take is greater than all elements + expect_equal(take(numVectorRDD, length(numVector)), as.list(numVector)) + expect_equal(take(numVectorRDD, length(numVector) + 1), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(take(numListRDD, 3), take(numListRDD2, 3)) + expect_equal(take(numListRDD, 5), take(numListRDD2, 5)) + expect_equal(take(numListRDD, 1), as.list(head(numList, n = 1))) + expect_equal(take(numListRDD2, 999), numList) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(take(strVectorRDD, 4), as.list(strVector)) + expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2))) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3))) + expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1))) + + expect_true(length(take(strListRDD, 0)) == 0) + expect_true(length(take(strVectorRDD, 0)) == 0) + expect_true(length(take(numListRDD, 0)) == 0) + expect_true(length(take(numVectorRDD, 0)) == 0) +}) + diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R new file mode 100644 index 0000000000000..7bb3e8003131d --- /dev/null +++ b/R/pkg/inst/tests/test_textFile.R @@ -0,0 +1,162 @@ +# +# 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. +# + +context("the textFile() function") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("textFile() on a local file returns an RDD", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_true(inherits(rdd, "RDD")) + expect_true(count(rdd) > 0) + expect_true(count(rdd) == 2) + + unlink(fileName) +}) + +test_that("textFile() followed by a collect() returns the same content", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName) +}) + +test_that("textFile() word count works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + output <- collect(counts) + expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), + list("Spark", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName) +}) + +test_that("several transformations on RDD created by textFile()", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) # RDD + for (i in 1:10) { + # PipelinedRDD initially created from RDD + rdd <- lapply(rdd, function(x) paste(x, x)) + } + collect(rdd) + + unlink(fileName) +}) + +test_that("textFile() followed by a saveAsTextFile() returns the same content", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + saveAsTextFile(rdd, fileName2) + rdd <- textFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("saveAsTextFile() on a parallelized list works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + l <- list(1, 2, 3) + rdd <- parallelize(sc, l) + saveAsTextFile(rdd, fileName) + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) + + unlink(fileName) +}) + +test_that("textFile() and saveAsTextFile() word count works as expected", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsTextFile(counts, fileName2) + rdd <- textFile(sc, fileName2) + + output <- collect(rdd) + expected <- list(list("awesome.", 1), list("Spark", 2), + list("pretty.", 1), list("is", 2)) + expectedStr <- lapply(expected, function(x) { toString(x) }) + expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("textFile() on multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines("Spark is pretty.", fileName1) + writeLines("Spark is awesome.", fileName2) + + rdd <- textFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("Pipelined operations on RDDs created using textFile", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + lengths <- lapply(rdd, function(x) { length(x) }) + expect_equal(collect(lengths), list(1, 1)) + + lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) + expect_equal(collect(lengthsPipelined), list(11, 11)) + + lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) + expect_equal(collect(lengths30), list(31, 31)) + + lengths20 <- lapply(lengths, function(x) { x + 20 }) + expect_equal(collect(lengths20), list(21, 21)) + + unlink(fileName) +}) + diff --git a/R/pkg/inst/tests/test_utils.R b/R/pkg/inst/tests/test_utils.R new file mode 100644 index 0000000000000..9c5bb427932b4 --- /dev/null +++ b/R/pkg/inst/tests/test_utils.R @@ -0,0 +1,137 @@ +# +# 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. +# + +context("functions in utils.R") + +# JavaSparkContext handle +sc <- sparkR.init() + +test_that("convertJListToRList() gives back (deserializes) the original JLists + of strings and integers", { + # It's hard to manually create a Java List using rJava, since it does not + # support generics well. Instead, we rely on collect() returning a + # JList. + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 1L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, nums) + + strs <- as.list("hello", "spark") + rdd <- parallelize(sc, strs, 2L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, strs) +}) + +test_that("serializeToBytes on RDD", { + # File content + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + expect_true(getSerializedMode(text.rdd) == "string") + ser.rdd <- serializeToBytes(text.rdd) + expect_equal(collect(ser.rdd), as.list(mockFile)) + expect_true(getSerializedMode(ser.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cleanClosure on R functions", { + y <- c(1, 2, 3) + g <- function(x) { x + 1 } + f <- function(x) { g(x) + y } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # y, g + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + # Test for nested enclosures and package variables. + env2 <- new.env() + funcEnv <- new.env(parent = env2) + f <- function(x) { log(g(x) + y) } + environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # "min" should not be included + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + base <- c(1, 2, 3) + l <- list(field = matrix(1)) + field <- matrix(2) + defUse <- 3 + g <- function(x) { x + y } + f <- function(x) { + defUse <- base::as.integer(x) + 1 # Test for access operators `::`. + lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. + l$field[1,1] <- 3 # Test for access operators `$`. + res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol. + f(res) # Test for recursive calls. + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". + expect_true("g" %in% ls(env)) + expect_true("l" %in% ls(env)) + expect_true("f" %in% ls(env)) + expect_equal(get("l", envir = env, inherits = FALSE), l) + # "y" should be in the environemnt of g. + newG <- get("g", envir = env, inherits = FALSE) + env <- environment(newG) + expect_equal(length(ls(env)), 1) + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + + # Test for function (and variable) definitions. + f <- function(x) { + g <- function(y) { y * 2 } + g(x) + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. + + # Test for overriding variables in base namespace (Issue: SparkR-196). + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 2L) + t = 4 # Override base::t in .GlobalEnv. + f <- function(x) { x > t } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(ls(env), "t") + expect_equal(get("t", envir = env, inherits = FALSE), t) + actual <- collect(lapply(rdd, f)) + expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) + expect_equal(actual, expected) + + # Test for broadcast variables. + a <- matrix(nrow=10, ncol=10, data=rnorm(100)) + aBroadcast <- broadcast(sc, a) + normMultiply <- function(x) { norm(aBroadcast$value) * x } + newnormMultiply <- SparkR:::cleanClosure(normMultiply) + env <- environment(newnormMultiply) + expect_equal(ls(env), "aBroadcast") + expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) +}) diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R new file mode 100644 index 0000000000000..3584b418a71a9 --- /dev/null +++ b/R/pkg/inst/worker/daemon.R @@ -0,0 +1,52 @@ +# +# 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. +# + +# Worker daemon + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") + +# preload SparkR package, speedup worker +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) + +while (TRUE) { + ready <- socketSelect(list(inputCon)) + if (ready) { + port <- SparkR:::readInt(inputCon) + # There is a small chance that it could be interrupted by signal, retry one time + if (length(port) == 0) { + port <- SparkR:::readInt(inputCon) + if (length(port) == 0) { + cat("quitting daemon\n") + quit(save = "no") + } + } + p <- parallel:::mcfork() + if (inherits(p, "masterProcess")) { + close(inputCon) + Sys.setenv(SPARKR_WORKER_PORT = port) + source(script) + # Set SIGUSR1 so that child can exit + tools::pskill(Sys.getpid(), tools::SIGUSR1) + parallel:::mcexit(0L) + } + } +} diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R new file mode 100644 index 0000000000000..c6542928e8ddd --- /dev/null +++ b/R/pkg/inst/worker/worker.R @@ -0,0 +1,128 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Worker class + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +# Set libPaths to include SparkR package as loadNamespace needs this +# TODO: Figure out if we can avoid this by not loading any objects that require +# SparkR namespace +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") +outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") + +# read the index of the current partition inside the RDD +partition <- SparkR:::readInt(inputCon) + +deserializer <- SparkR:::readString(inputCon) +serializer <- SparkR:::readString(inputCon) + +# Include packages as required +packageNames <- unserialize(SparkR:::readRaw(inputCon)) +for (pkg in packageNames) { + suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) +} + +# read function dependencies +funcLen <- SparkR:::readInt(inputCon) +computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) +env <- environment(computeFunc) +parent.env(env) <- .GlobalEnv # Attach under global environment. + +# Read and set broadcast variables +numBroadcastVars <- SparkR:::readInt(inputCon) +if (numBroadcastVars > 0) { + for (bcast in seq(1:numBroadcastVars)) { + bcastId <- SparkR:::readInt(inputCon) + value <- unserialize(SparkR:::readRaw(inputCon)) + setBroadcastValue(bcastId, value) + } +} + +# If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int +# as number of partitions to create. +numPartitions <- SparkR:::readInt(inputCon) + +isEmpty <- SparkR:::readInt(inputCon) + +if (isEmpty != 0) { + + if (numPartitions == -1) { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- as.list(readLines(inputCon)) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + output <- computeFunc(partition, data) + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + SparkR:::writeStrings(outputCon, output) + } + } else { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- readLines(inputCon) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + + res <- new.env() + + # Step 1: hash the data to an environment + hashTupleToEnvir <- function(tuple) { + # NOTE: execFunction is the hash function here + hashVal <- computeFunc(tuple[[1]]) + bucket <- as.character(hashVal %% numPartitions) + acc <- res[[bucket]] + # Create a new accumulator + if (is.null(acc)) { + acc <- SparkR:::initAccumulator() + } + SparkR:::addItemToAccumulator(acc, tuple) + res[[bucket]] <- acc + } + invisible(lapply(data, hashTupleToEnvir)) + + # Step 2: write out all of the environment as key-value pairs. + for (name in ls(res)) { + SparkR:::writeInt(outputCon, 2L) + SparkR:::writeInt(outputCon, as.integer(name)) + # Truncate the accumulator list to the number of elements we have + length(res[[name]]$data) <- res[[name]]$counter + SparkR:::writeRawSerialize(outputCon, res[[name]]$data) + } + } +} + +# End of output +if (serializer %in% c("byte", "row")) { + SparkR:::writeInt(outputCon, 0L) +} + +close(outputCon) +close(inputCon) diff --git a/R/pkg/src/Makefile b/R/pkg/src/Makefile new file mode 100644 index 0000000000000..a55a56fe80e10 --- /dev/null +++ b/R/pkg/src/Makefile @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.so string_hash_code.c + +clean: + rm -f *.o + rm -f *.so + +.PHONY: all clean diff --git a/R/pkg/src/Makefile.win b/R/pkg/src/Makefile.win new file mode 100644 index 0000000000000..aa486d8228371 --- /dev/null +++ b/R/pkg/src/Makefile.win @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.dll string_hash_code.c + +clean: + rm -f *.o + rm -f *.dll + +.PHONY: all clean diff --git a/R/pkg/src/string_hash_code.c b/R/pkg/src/string_hash_code.c new file mode 100644 index 0000000000000..e3274b9a0c547 --- /dev/null +++ b/R/pkg/src/string_hash_code.c @@ -0,0 +1,49 @@ +/* + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +/* + * A C function for R extension which implements the Java String hash algorithm. + * Refer to http://en.wikipedia.org/wiki/Java_hashCode%28%29#The_java.lang.String_hash_function + * + */ + +#include +#include + +/* for compatibility with R before 3.1 */ +#ifndef IS_SCALAR +#define IS_SCALAR(x, type) (TYPEOF(x) == (type) && XLENGTH(x) == 1) +#endif + +SEXP stringHashCode(SEXP string) { + const char* str; + R_xlen_t len, i; + int hashCode = 0; + + if (!IS_SCALAR(string, STRSXP)) { + error("invalid input"); + } + + str = CHAR(asChar(string)); + len = XLENGTH(asChar(string)); + + for (i = 0; i < len; i++) { + hashCode = (hashCode << 5) - hashCode + *str++; + } + + return ScalarInteger(hashCode); +} diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R new file mode 100644 index 0000000000000..4f8a1ed2d83ef --- /dev/null +++ b/R/pkg/tests/run-all.R @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) +library(SparkR) + +test_package("SparkR") diff --git a/R/run-tests.sh b/R/run-tests.sh new file mode 100755 index 0000000000000..e82ad0ba2cd06 --- /dev/null +++ b/R/run-tests.sh @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FWDIR="$(cd `dirname $0`; pwd)" + +FAILED=0 +LOGFILE=$FWDIR/unit-tests.out +rm -f $LOGFILE + +SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +FAILED=$((PIPESTATUS[0]||$FAILED)) + +if [[ $FAILED != 0 ]]; then + cat $LOGFILE + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 +else + echo -en "\033[32m" # Green + echo "Tests passed." + echo -en "\033[0m" # No color +fi diff --git a/bin/sparkR b/bin/sparkR new file mode 100755 index 0000000000000..8c918e2b09aef --- /dev/null +++ b/bin/sparkR @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Figure out where Spark is installed +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +source "$SPARK_HOME"/bin/load-spark-env.sh + +function usage() { + if [ -n "$1" ]; then + echo $1 + fi + echo "Usage: ./bin/sparkR [options]" 1>&2 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit $2 +} +export -f usage + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage +fi + +exec "$SPARK_HOME"/bin/spark-submit sparkr-shell-main "$@" diff --git a/bin/sparkR.cmd b/bin/sparkR.cmd new file mode 100644 index 0000000000000..d7b60183ca8e0 --- /dev/null +++ b/bin/sparkR.cmd @@ -0,0 +1,23 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This is the entry point for running SparkR. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0sparkR2.cmd %* diff --git a/bin/sparkR2.cmd b/bin/sparkR2.cmd new file mode 100644 index 0000000000000..e47f22c7300bb --- /dev/null +++ b/bin/sparkR2.cmd @@ -0,0 +1,26 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Figure out where the Spark framework is installed +set SPARK_HOME=%~dp0.. + +call %SPARK_HOME%\bin\load-spark-env.cmd + + +call %SPARK_HOME%\bin\spark-submit2.cmd sparkr-shell-main %* diff --git a/core/pom.xml b/core/pom.xml index 6cd1965ec37c2..e80829b7a7f3d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -442,4 +442,55 @@ + + + Windows + + + Windows + + + + \ + .bat + + + + unix + + + unix + + + + / + .sh + + + + sparkr + + + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + sparkr-pkg + compile + + exec + + + + + ..${path.separator}R${path.separator}install-dev${script.extension} + + + + + + + diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala new file mode 100644 index 0000000000000..3a2c94bd9d875 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io.{DataOutputStream, File, FileOutputStream, IOException} +import java.net.{InetSocketAddress, ServerSocket} +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelInitializer, EventLoopGroup} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} + +import org.apache.spark.Logging + +/** + * Netty-based backend server that is used to communicate between R and Java. + */ +private[spark] class RBackend { + + private[this] var channelFuture: ChannelFuture = null + private[this] var bootstrap: ServerBootstrap = null + private[this] var bossGroup: EventLoopGroup = null + + def init(): Int = { + bossGroup = new NioEventLoopGroup(2) + val workerGroup = bossGroup + val handler = new RBackendHandler(this) + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(classOf[NioServerSocketChannel]) + + bootstrap.childHandler(new ChannelInitializer[SocketChannel]() { + def initChannel(ch: SocketChannel): Unit = { + ch.pipeline() + .addLast("encoder", new ByteArrayEncoder()) + .addLast("frameDecoder", + // maxFrameLength = 2G + // lengthFieldOffset = 0 + // lengthFieldLength = 4 + // lengthAdjustment = 0 + // initialBytesToStrip = 4, i.e. strip out the length field itself + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)) + .addLast("decoder", new ByteArrayDecoder()) + .addLast("handler", handler) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(0)) + channelFuture.syncUninterruptibly() + channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort() + } + + def run(): Unit = { + channelFuture.channel.closeFuture().syncUninterruptibly() + } + + def close(): Unit = { + if (channelFuture != null) { + // close is a local operation and should finish within milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS) + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } + +} + +private[spark] object RBackend extends Logging { + def main(args: Array[String]): Unit = { + if (args.length < 1) { + System.err.println("Usage: RBackend ") + System.exit(-1) + } + val sparkRBackend = new RBackend() + try { + // bind to random port + val boundPort = sparkRBackend.init() + val serverSocket = new ServerSocket(0, 1) + val listenPort = serverSocket.getLocalPort() + + // tell the R process via temporary file + val path = args(0) + val f = new File(path + ".tmp") + val dos = new DataOutputStream(new FileOutputStream(f)) + dos.writeInt(boundPort) + dos.writeInt(listenPort) + dos.close() + f.renameTo(new File(path)) + + // wait for the end of stdin, then exit + new Thread("wait for socket to close") { + setDaemon(true) + override def run(): Unit = { + // any un-catched exception will also shutdown JVM + val buf = new Array[Byte](1024) + // shutdown JVM if R does not connect back in 10 seconds + serverSocket.setSoTimeout(10000) + try { + val inSocket = serverSocket.accept() + serverSocket.close() + // wait for the end of socket, closed if R process die + inSocket.getInputStream().read(buf) + } finally { + sparkRBackend.close() + System.exit(0) + } + } + }.start() + + sparkRBackend.run() + } catch { + case e: IOException => + logError("Server shutting down: failed with exception ", e) + sparkRBackend.close() + System.exit(1) + } + System.exit(0) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala new file mode 100644 index 0000000000000..0075d963711f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.collection.mutable.HashMap + +import io.netty.channel.ChannelHandler.Sharable +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.api.r.SerDe._ + +/** + * Handler for RBackend + * TODO: This is marked as sharable to get a handle to RBackend. Is it safe to re-use + * this across connections ? + */ +@Sharable +private[r] class RBackendHandler(server: RBackend) + extends SimpleChannelInboundHandler[Array[Byte]] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, msg: Array[Byte]): Unit = { + val bis = new ByteArrayInputStream(msg) + val dis = new DataInputStream(bis) + + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + // First bit is isStatic + val isStatic = readBoolean(dis) + val objId = readString(dis) + val methodName = readString(dis) + val numArgs = readInt(dis) + + if (objId == "SparkRHandler") { + methodName match { + case "stopBackend" => + writeInt(dos, 0) + writeType(dos, "void") + server.close() + case "rm" => + try { + val t = readObjectType(dis) + assert(t == 'c') + val objToRemove = readString(dis) + JVMObjectTracker.remove(objToRemove) + writeInt(dos, 0) + writeObject(dos, null) + } catch { + case e: Exception => + logError(s"Removing $objId failed", e) + writeInt(dos, -1) + } + case _ => dos.writeInt(-1) + } + } else { + handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos) + } + + val reply = bos.toByteArray + ctx.write(reply) + } + + override def channelReadComplete(ctx: ChannelHandlerContext): Unit = { + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + // Close the connection when an exception is raised. + cause.printStackTrace() + ctx.close() + } + + def handleMethodCall( + isStatic: Boolean, + objId: String, + methodName: String, + numArgs: Int, + dis: DataInputStream, + dos: DataOutputStream): Unit = { + var obj: Object = null + try { + val cls = if (isStatic) { + Class.forName(objId) + } else { + JVMObjectTracker.get(objId) match { + case None => throw new IllegalArgumentException("Object not found " + objId) + case Some(o) => + obj = o + o.getClass + } + } + + val args = readArgs(numArgs, dis) + + val methods = cls.getMethods + val selectedMethods = methods.filter(m => m.getName == methodName) + if (selectedMethods.length > 0) { + val methods = selectedMethods.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + } + if (methods.isEmpty) { + logWarning(s"cannot find matching method ${cls}.$methodName. " + + s"Candidates are:") + selectedMethods.foreach { method => + logWarning(s"$methodName(${method.getParameterTypes.mkString(",")})") + } + throw new Exception(s"No matched method found for $cls.$methodName") + } + val ret = methods.head.invoke(obj, args:_*) + + // Write status bit + writeInt(dos, 0) + writeObject(dos, ret.asInstanceOf[AnyRef]) + } else if (methodName == "") { + // methodName should be "" for constructor + val ctor = cls.getConstructors.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + }.head + + val obj = ctor.newInstance(args:_*) + + writeInt(dos, 0) + writeObject(dos, obj.asInstanceOf[AnyRef]) + } else { + throw new IllegalArgumentException("invalid method " + methodName + " for object " + objId) + } + } catch { + case e: Exception => + logError(s"$methodName on $objId failed", e) + writeInt(dos, -1) + } + } + + // Read a number of arguments from the data input stream + def readArgs(numArgs: Int, dis: DataInputStream): Array[java.lang.Object] = { + (0 until numArgs).map { arg => + readObject(dis) + }.toArray + } + + // Checks if the arguments passed in args matches the parameter types. + // NOTE: Currently we do exact match. We may add type conversions later. + def matchMethod( + numArgs: Int, + args: Array[java.lang.Object], + parameterTypes: Array[Class[_]]): Boolean = { + if (parameterTypes.length != numArgs) { + return false + } + + for (i <- 0 to numArgs - 1) { + val parameterType = parameterTypes(i) + var parameterWrapperType = parameterType + + // Convert native parameters to Object types as args is Array[Object] here + if (parameterType.isPrimitive) { + parameterWrapperType = parameterType match { + case java.lang.Integer.TYPE => classOf[java.lang.Integer] + case java.lang.Double.TYPE => classOf[java.lang.Double] + case java.lang.Boolean.TYPE => classOf[java.lang.Boolean] + case _ => parameterType + } + } + if (!parameterWrapperType.isInstance(args(i))) { + return false + } + } + true + } +} + +/** + * Helper singleton that tracks JVM objects returned to R. + * This is useful for referencing these objects in RPC calls. + */ +private[r] object JVMObjectTracker { + + // TODO: This map should be thread-safe if we want to support multiple + // connections at the same time + private[this] val objMap = new HashMap[String, Object] + + // TODO: We support only one connection now, so an integer is fine. + // Investigate using use atomic integer in the future. + private[this] var objCounter: Int = 0 + + def getObject(id: String): Object = { + objMap(id) + } + + def get(id: String): Option[Object] = { + objMap.get(id) + } + + def put(obj: Object): String = { + val objId = objCounter.toString + objCounter = objCounter + 1 + objMap.put(objId, obj) + objId + } + + def remove(id: String): Option[Object] = { + objMap.remove(id) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala new file mode 100644 index 0000000000000..5fa4d483b8342 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -0,0 +1,450 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io._ +import java.net.ServerSocket +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ +import scala.io.Source +import scala.reflect.ClassTag +import scala.util.Try + +import org.apache.spark._ +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( + parent: RDD[T], + numPartitions: Int, + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Broadcast[Object]]) + extends RDD[U](parent) with Logging { + override def getPartitions: Array[Partition] = parent.partitions + + override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + + // The parent may be also an RRDD, so we should launch it first. + val parentIterator = firstParent[T].iterator(partition, context) + + // we expect two connections + val serverSocket = new ServerSocket(0, 2) + val listenPort = serverSocket.getLocalPort() + + // The stdout/stderr is shared by multiple tasks, because we use one daemon + // to launch child process as worker. + val errThread = RRDD.createRWorker(rLibDir, listenPort) + + // We use two sockets to separate input and output, then it's easy to manage + // the lifecycle of them to avoid deadlock. + // TODO: optimize it to use one socket + + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val inSocket = serverSocket.accept() + startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index) + + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + val inputStream = new BufferedInputStream(outSocket.getInputStream) + val dataStream = openDataStream(inputStream) + serverSocket.close() + + try { + + return new Iterator[U] { + def next(): U = { + val obj = _nextObj + if (hasNext) { + _nextObj = read() + } + obj + } + + var _nextObj = read() + + def hasNext(): Boolean = { + val hasMore = (_nextObj != null) + if (!hasMore) { + dataStream.close() + } + hasMore + } + } + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines()) + } + } + + /** + * Start a thread to write RDD data to the R process. + */ + private def startStdinThread[T]( + output: OutputStream, + iter: Iterator[T], + partition: Int): Unit = { + + val env = SparkEnv.get + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val stream = new BufferedOutputStream(output, bufferSize) + + new Thread("writer for R") { + override def run(): Unit = { + try { + SparkEnv.set(env) + val dataOut = new DataOutputStream(stream) + dataOut.writeInt(partition) + + SerDe.writeString(dataOut, deserializer) + SerDe.writeString(dataOut, serializer) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames) + + dataOut.writeInt(func.length) + dataOut.write(func) + + dataOut.writeInt(broadcastVars.length) + broadcastVars.foreach { broadcast => + // TODO(shivaram): Read a Long in R to avoid this cast + dataOut.writeInt(broadcast.id.toInt) + // TODO: Pass a byte array from R to avoid this cast ? + val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] + dataOut.writeInt(broadcastByteArr.length) + dataOut.write(broadcastByteArr) + } + + dataOut.writeInt(numPartitions) + + if (!iter.hasNext) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + + val printOut = new PrintStream(stream) + + def writeElem(elem: Any): Unit = { + if (deserializer == SerializationFormats.BYTE) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr) + } else if (deserializer == SerializationFormats.ROW) { + dataOut.write(elem.asInstanceOf[Array[Byte]]) + } else if (deserializer == SerializationFormats.STRING) { + printOut.println(elem) + } + } + + for (elem <- iter) { + elem match { + case (key, value) => + writeElem(key) + writeElem(value) + case _ => + writeElem(elem) + } + } + stream.flush() + } catch { + // TODO: We should propogate this error to the task thread + case e: Exception => + logError("R Writer thread got an exception", e) + } finally { + Try(output.close()) + } + } + }.start() + } + + protected def openDataStream(input: InputStream): Closeable + + protected def read(): U +} + +/** + * Form an RDD[(Int, Array[Byte])] from key-value pairs returned from R. + * This is used by SparkR's shuffle operations. + */ +private class PairwiseRRDD[T: ClassTag]( + parent: RDD[T], + numPartitions: Int, + hashFunc: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, (Int, Array[Byte])]( + parent, numPartitions, hashFunc, deserializer, + SerializationFormats.BYTE, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: DataInputStream = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new DataInputStream(input) + dataStream + } + + override protected def read(): (Int, Array[Byte]) = { + try { + val length = dataStream.readInt() + + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null // End of input + } + } catch { + case eof: EOFException => { + throw new SparkException("R worker exited unexpectedly (crashed)", eof) + } + } + } + + lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) +} + +/** + * An RDD that stores serialized R objects as Array[Byte]. + */ +private class RRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, Array[Byte]]( + parent, -1, func, deserializer, serializer, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: DataInputStream = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new DataInputStream(input) + dataStream + } + + override protected def read(): Array[Byte] = { + try { + val length = dataStream.readInt() + + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj, 0, length) + obj + case _ => null + } + } catch { + case eof: EOFException => { + throw new SparkException("R worker exited unexpectedly (crashed)", eof) + } + } + } + + lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) +} + +/** + * An RDD that stores R objects as Array[String]. + */ +private class StringRRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, String]( + parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: BufferedReader = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new BufferedReader(new InputStreamReader(input)) + dataStream + } + + override protected def read(): String = { + try { + dataStream.readLine() + } catch { + case e: IOException => { + throw new SparkException("R worker exited unexpectedly (crashed)", e) + } + } + } + + lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) +} + +private[r] class BufferedStreamThread( + in: InputStream, + name: String, + errBufferSize: Int) extends Thread(name) with Logging { + val lines = new Array[String](errBufferSize) + var lineIdx = 0 + override def run() { + for (line <- Source.fromInputStream(in).getLines) { + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } + logInfo(line) + } + } + + def getLines(): String = synchronized { + (0 until errBufferSize).filter { x => + lines((x + lineIdx) % errBufferSize) != null + }.map { x => + lines((x + lineIdx) % errBufferSize) + }.mkString("\n") + } +} + +private[r] object RRDD { + // Because forking processes from Java is expensive, we prefer to launch + // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. + // This daemon currently only works on UNIX-based systems now, so we should + // also fall back to launching workers (worker.R) directly. + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonChannel: DataOutputStream = _ + + def createSparkContext( + master: String, + appName: String, + sparkHome: String, + jars: Array[String], + sparkEnvirMap: JMap[Object, Object], + sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = { + + val sparkConf = new SparkConf().setAppName(appName) + .setSparkHome(sparkHome) + .setJars(jars) + + // Override `master` if we have a user-specified value + if (master != "") { + sparkConf.setMaster(master) + } else { + // If conf has no master set it to "local" to maintain + // backwards compatibility + sparkConf.setIfMissing("spark.master", "local") + } + + for ((name, value) <- sparkEnvirMap) { + sparkConf.set(name.asInstanceOf[String], value.asInstanceOf[String]) + } + for ((name, value) <- sparkExecutorEnvMap) { + sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String]) + } + + new JavaSparkContext(sparkConf) + } + + /** + * Start a thread to print the process's stderr to ours + */ + private def startStdoutThread(proc: Process): BufferedStreamThread = { + val BUFFER_SIZE = 100 + val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) + thread.setDaemon(true) + thread.start() + thread + } + + private def createRProcess(rLibDir: String, port: Int, script: String): BufferedStreamThread = { + val rCommand = "Rscript" + val rOptions = "--vanilla" + val rExecScript = rLibDir + "/SparkR/worker/" + script + val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript)) + // Unset the R_TESTS environment variable for workers. + // This is set by R CMD check as startup.Rs + // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) + // and confuses worker script which tries to load a non-existent file + pb.environment().put("R_TESTS", "") + pb.environment().put("SPARKR_RLIBDIR", rLibDir) + pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.redirectErrorStream(true) // redirect stderr into stdout + val proc = pb.start() + val errThread = startStdoutThread(proc) + errThread + } + + /** + * ProcessBuilder used to launch worker R processes. + */ + def createRWorker(rLibDir: String, port: Int): BufferedStreamThread = { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!Utils.isWindows && useDaemon) { + synchronized { + if (daemonChannel == null) { + // we expect one connections + val serverSocket = new ServerSocket(0, 1) + val daemonPort = serverSocket.getLocalPort + errThread = createRProcess(rLibDir, daemonPort, "daemon.R") + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val sock = serverSocket.accept() + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + serverSocket.close() + } + try { + daemonChannel.writeInt(port) + daemonChannel.flush() + } catch { + case e: IOException => + // daemon process died + daemonChannel.close() + daemonChannel = null + errThread = null + // fail the current task, retry by scheduler + throw e + } + errThread + } + } else { + createRProcess(rLibDir, port, "worker.R") + } + } + + /** + * Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is + * called from R. + */ + def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { + JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala new file mode 100644 index 0000000000000..ccb2a371f4e48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -0,0 +1,340 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io.{DataInputStream, DataOutputStream} +import java.sql.{Date, Time} + +import scala.collection.JavaConversions._ + +/** + * Utility functions to serialize, deserialize objects to / from R + */ +private[spark] object SerDe { + + // Type mapping from R to Java + // + // NULL -> void + // integer -> Int + // character -> String + // logical -> Boolean + // double, numeric -> Double + // raw -> Array[Byte] + // Date -> Date + // POSIXlt/POSIXct -> Time + // + // list[T] -> Array[T], where T is one of above mentioned types + // environment -> Map[String, T], where T is a native type + // jobj -> Object, where jobj is an object created in the backend + + def readObjectType(dis: DataInputStream): Char = { + dis.readByte().toChar + } + + def readObject(dis: DataInputStream): Object = { + val dataType = readObjectType(dis) + readTypedObject(dis, dataType) + } + + def readTypedObject( + dis: DataInputStream, + dataType: Char): Object = { + dataType match { + case 'n' => null + case 'i' => new java.lang.Integer(readInt(dis)) + case 'd' => new java.lang.Double(readDouble(dis)) + case 'b' => new java.lang.Boolean(readBoolean(dis)) + case 'c' => readString(dis) + case 'e' => readMap(dis) + case 'r' => readBytes(dis) + case 'l' => readList(dis) + case 'D' => readDate(dis) + case 't' => readTime(dis) + case 'j' => JVMObjectTracker.getObject(readString(dis)) + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") + } + } + + def readBytes(in: DataInputStream): Array[Byte] = { + val len = readInt(in) + val out = new Array[Byte](len) + val bytesRead = in.readFully(out) + out + } + + def readInt(in: DataInputStream): Int = { + in.readInt() + } + + def readDouble(in: DataInputStream): Double = { + in.readDouble() + } + + def readString(in: DataInputStream): String = { + val len = in.readInt() + val asciiBytes = new Array[Byte](len) + in.readFully(asciiBytes) + assert(asciiBytes(len - 1) == 0) + val str = new String(asciiBytes.dropRight(1).map(_.toChar)) + str + } + + def readBoolean(in: DataInputStream): Boolean = { + val intVal = in.readInt() + if (intVal == 0) false else true + } + + def readDate(in: DataInputStream): Date = { + Date.valueOf(readString(in)) + } + + def readTime(in: DataInputStream): Time = { + val t = in.readDouble() + new Time((t * 1000L).toLong) + } + + def readBytesArr(in: DataInputStream): Array[Array[Byte]] = { + val len = readInt(in) + (0 until len).map(_ => readBytes(in)).toArray + } + + def readIntArr(in: DataInputStream): Array[Int] = { + val len = readInt(in) + (0 until len).map(_ => readInt(in)).toArray + } + + def readDoubleArr(in: DataInputStream): Array[Double] = { + val len = readInt(in) + (0 until len).map(_ => readDouble(in)).toArray + } + + def readBooleanArr(in: DataInputStream): Array[Boolean] = { + val len = readInt(in) + (0 until len).map(_ => readBoolean(in)).toArray + } + + def readStringArr(in: DataInputStream): Array[String] = { + val len = readInt(in) + (0 until len).map(_ => readString(in)).toArray + } + + def readList(dis: DataInputStream): Array[_] = { + val arrType = readObjectType(dis) + arrType match { + case 'i' => readIntArr(dis) + case 'c' => readStringArr(dis) + case 'd' => readDoubleArr(dis) + case 'b' => readBooleanArr(dis) + case 'j' => readStringArr(dis).map(x => JVMObjectTracker.getObject(x)) + case 'r' => readBytesArr(dis) + case _ => throw new IllegalArgumentException(s"Invalid array type $arrType") + } + } + + def readMap(in: DataInputStream): java.util.Map[Object, Object] = { + val len = readInt(in) + if (len > 0) { + val keysType = readObjectType(in) + val keysLen = readInt(in) + val keys = (0 until keysLen).map(_ => readTypedObject(in, keysType)) + + val valuesType = readObjectType(in) + val valuesLen = readInt(in) + val values = (0 until valuesLen).map(_ => readTypedObject(in, valuesType)) + mapAsJavaMap(keys.zip(values).toMap) + } else { + new java.util.HashMap[Object, Object]() + } + } + + // Methods to write out data from Java to R + // + // Type mapping from Java to R + // + // void -> NULL + // Int -> integer + // String -> character + // Boolean -> logical + // Double -> double + // Long -> double + // Array[Byte] -> raw + // Date -> Date + // Time -> POSIXct + // + // Array[T] -> list() + // Object -> jobj + + def writeType(dos: DataOutputStream, typeStr: String): Unit = { + typeStr match { + case "void" => dos.writeByte('n') + case "character" => dos.writeByte('c') + case "double" => dos.writeByte('d') + case "integer" => dos.writeByte('i') + case "logical" => dos.writeByte('b') + case "date" => dos.writeByte('D') + case "time" => dos.writeByte('t') + case "raw" => dos.writeByte('r') + case "list" => dos.writeByte('l') + case "jobj" => dos.writeByte('j') + case _ => throw new IllegalArgumentException(s"Invalid type $typeStr") + } + } + + def writeObject(dos: DataOutputStream, value: Object): Unit = { + if (value == null) { + writeType(dos, "void") + } else { + value.getClass.getName match { + case "java.lang.String" => + writeType(dos, "character") + writeString(dos, value.asInstanceOf[String]) + case "long" | "java.lang.Long" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Long].toDouble) + case "double" | "java.lang.Double" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Double]) + case "int" | "java.lang.Integer" => + writeType(dos, "integer") + writeInt(dos, value.asInstanceOf[Int]) + case "boolean" | "java.lang.Boolean" => + writeType(dos, "logical") + writeBoolean(dos, value.asInstanceOf[Boolean]) + case "java.sql.Date" => + writeType(dos, "date") + writeDate(dos, value.asInstanceOf[Date]) + case "java.sql.Time" => + writeType(dos, "time") + writeTime(dos, value.asInstanceOf[Time]) + case "[B" => + writeType(dos, "raw") + writeBytes(dos, value.asInstanceOf[Array[Byte]]) + // TODO: Types not handled right now include + // byte, char, short, float + + // Handle arrays + case "[Ljava.lang.String;" => + writeType(dos, "list") + writeStringArr(dos, value.asInstanceOf[Array[String]]) + case "[I" => + writeType(dos, "list") + writeIntArr(dos, value.asInstanceOf[Array[Int]]) + case "[J" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Long]].map(_.toDouble)) + case "[D" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Double]]) + case "[Z" => + writeType(dos, "list") + writeBooleanArr(dos, value.asInstanceOf[Array[Boolean]]) + case "[[B" => + writeType(dos, "list") + writeBytesArr(dos, value.asInstanceOf[Array[Array[Byte]]]) + case otherName => + // Handle array of objects + if (otherName.startsWith("[L")) { + val objArr = value.asInstanceOf[Array[Object]] + writeType(dos, "list") + writeType(dos, "jobj") + dos.writeInt(objArr.length) + objArr.foreach(o => writeJObj(dos, o)) + } else { + writeType(dos, "jobj") + writeJObj(dos, value) + } + } + } + } + + def writeInt(out: DataOutputStream, value: Int): Unit = { + out.writeInt(value) + } + + def writeDouble(out: DataOutputStream, value: Double): Unit = { + out.writeDouble(value) + } + + def writeBoolean(out: DataOutputStream, value: Boolean): Unit = { + val intValue = if (value) 1 else 0 + out.writeInt(intValue) + } + + def writeDate(out: DataOutputStream, value: Date): Unit = { + writeString(out, value.toString) + } + + def writeTime(out: DataOutputStream, value: Time): Unit = { + out.writeDouble(value.getTime.toDouble / 1000.0) + } + + + // NOTE: Only works for ASCII right now + def writeString(out: DataOutputStream, value: String): Unit = { + val len = value.length + out.writeInt(len + 1) // For the \0 + out.writeBytes(value) + out.writeByte(0) + } + + def writeBytes(out: DataOutputStream, value: Array[Byte]): Unit = { + out.writeInt(value.length) + out.write(value) + } + + def writeJObj(out: DataOutputStream, value: Object): Unit = { + val objId = JVMObjectTracker.put(value) + writeString(out, objId) + } + + def writeIntArr(out: DataOutputStream, value: Array[Int]): Unit = { + writeType(out, "integer") + out.writeInt(value.length) + value.foreach(v => out.writeInt(v)) + } + + def writeDoubleArr(out: DataOutputStream, value: Array[Double]): Unit = { + writeType(out, "double") + out.writeInt(value.length) + value.foreach(v => out.writeDouble(v)) + } + + def writeBooleanArr(out: DataOutputStream, value: Array[Boolean]): Unit = { + writeType(out, "logical") + out.writeInt(value.length) + value.foreach(v => writeBoolean(out, v)) + } + + def writeStringArr(out: DataOutputStream, value: Array[String]): Unit = { + writeType(out, "character") + out.writeInt(value.length) + value.foreach(v => writeString(out, v)) + } + + def writeBytesArr(out: DataOutputStream, value: Array[Array[Byte]]): Unit = { + writeType(out, "raw") + out.writeInt(value.length) + value.foreach(v => writeBytes(out, v)) + } +} + +private[r] object SerializationFormats { + val BYTE = "byte" + val STRING = "string" + val ROW = "row" +} diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala new file mode 100644 index 0000000000000..e99779f299785 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io._ +import java.util.concurrent.{Semaphore, TimeUnit} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.api.r.RBackend +import org.apache.spark.util.RedirectThread + +/** + * Main class used to launch SparkR applications using spark-submit. It executes R as a + * subprocess and then has it connect back to the JVM to access system properties etc. + */ +object RRunner { + def main(args: Array[String]): Unit = { + val rFile = PythonRunner.formatPath(args(0)) + + val otherArgs = args.slice(1, args.length) + + // Time to wait for SparkR backend to initialize in seconds + val backendTimeout = sys.env.getOrElse("SPARKR_BACKEND_TIMEOUT", "120").toInt + val rCommand = "Rscript" + + // Check if the file path exists. + // If not, change directory to current working directory for YARN cluster mode + val rF = new File(rFile) + val rFileNormalized = if (!rF.exists()) { + new Path(rFile).getName + } else { + rFile + } + + // Launch a SparkR backend server for the R process to connect to; this will let it see our + // Java system properties etc. + val sparkRBackend = new RBackend() + @volatile var sparkRBackendPort = 0 + val initialized = new Semaphore(0) + val sparkRBackendThread = new Thread("SparkR backend") { + override def run() { + sparkRBackendPort = sparkRBackend.init() + initialized.release() + sparkRBackend.run() + } + } + + sparkRBackendThread.start() + // Wait for RBackend initialization to finish + if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { + // Launch R + val returnCode = try { + val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) + val env = builder.environment() + env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) + val sparkHome = System.getenv("SPARK_HOME") + env.put("R_PROFILE_USER", + Seq(sparkHome, "R", "lib", "SparkR", "profile", "general.R").mkString(File.separator)) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect R output").start() + + process.waitFor() + } finally { + sparkRBackend.close() + } + System.exit(returnCode) + } else { + System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + System.exit(-1) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 660307d19eab4..60bc243ebf40a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -77,6 +77,7 @@ object SparkSubmit { // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val SPARKR_SHELL = "sparkr-shell" private val CLASS_NOT_FOUND_EXIT_STATUS = 101 @@ -284,6 +285,13 @@ object SparkSubmit { } } + // Require all R files to be local + if (args.isR && !isYarnCluster) { + if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { + printErrorAndExit(s"Only local R files are supported: $args.primaryResource") + } + } + // The following modes are not supported or applicable (clusterManager, deployMode) match { case (MESOS, CLUSTER) => @@ -291,6 +299,9 @@ object SparkSubmit { case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") + case (STANDALONE, CLUSTER) if args.isR => + printErrorAndExit("Cluster deploy mode is currently not supported for R " + + "applications on standalone clusters.") case (_, CLUSTER) if isShell(args.primaryResource) => printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") case (_, CLUSTER) if isSqlShell(args.mainClass) => @@ -317,11 +328,32 @@ object SparkSubmit { } } - // In yarn-cluster mode for a python app, add primary resource and pyFiles to files - // that can be distributed with the job - if (args.isPython && isYarnCluster) { - args.files = mergeFileLists(args.files, args.primaryResource) - args.files = mergeFileLists(args.files, args.pyFiles) + // If we're running a R app, set the main class to our specific R runner + if (args.isR && deployMode == CLIENT) { + if (args.primaryResource == SPARKR_SHELL) { + args.mainClass = "org.apache.spark.api.r.RBackend" + } else { + // If a R file is provided, add it to the child arguments and list of files to deploy. + // Usage: RRunner
    [app arguments] + args.mainClass = "org.apache.spark.deploy.RRunner" + args.childArgs = ArrayBuffer(args.primaryResource) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } + } + + if (isYarnCluster) { + // In yarn-cluster mode for a python app, add primary resource and pyFiles to files + // that can be distributed with the job + if (args.isPython) { + args.files = mergeFileLists(args.files, args.primaryResource) + args.files = mergeFileLists(args.files, args.pyFiles) + } + + // In yarn-cluster mode for a R app, add primary resource to files + // that can be distributed with the job + if (args.isR) { + args.files = mergeFileLists(args.files, args.primaryResource) + } } // Special flag to avoid deprecation warnings at the client @@ -405,8 +437,8 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" - // For python files, the primary resource is already distributed as a regular file - if (!isYarnCluster && !args.isPython) { + // For python and R files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !args.isPython && !args.isR) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) @@ -447,6 +479,10 @@ object SparkSubmit { childArgs += ("--py-files", pyFilesNames) } childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else if (args.isR) { + val mainFile = new Path(args.primaryResource).getName + childArgs += ("--primary-r-file", mainFile) + childArgs += ("--class", "org.apache.spark.deploy.RRunner") } else { if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) @@ -591,15 +627,15 @@ object SparkSubmit { /** * Return whether the given primary resource represents a user jar. */ - private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + private[deploy] def isUserJar(res: String): Boolean = { + !isShell(res) && !isPython(res) && !isInternal(res) && !isR(res) } /** * Return whether the given primary resource represents a shell. */ - private[deploy] def isShell(primaryResource: String): Boolean = { - primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + private[deploy] def isShell(res: String): Boolean = { + (res == SPARK_SHELL || res == PYSPARK_SHELL || res == SPARKR_SHELL) } /** @@ -619,12 +655,19 @@ object SparkSubmit { /** * Return whether the given primary resource requires running python. */ - private[deploy] def isPython(primaryResource: String): Boolean = { - primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL + private[deploy] def isPython(res: String): Boolean = { + res != null && res.endsWith(".py") || res == PYSPARK_SHELL + } + + /** + * Return whether the given primary resource requires running R. + */ + private[deploy] def isR(res: String): Boolean = { + res != null && res.endsWith(".R") || res == SPARKR_SHELL } - private[deploy] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL + private[deploy] def isInternal(res: String): Boolean = { + res == SPARK_INTERNAL } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 6eb73c43470a5..03ecf3fd99ec5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -59,6 +59,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + var isR: Boolean = false var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() var proxyUser: String = null @@ -158,7 +159,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given - if (mainClass == null && !isPython && primaryResource != null) { + if (mainClass == null && !isPython && !isR && primaryResource != null) { val uri = new URI(primaryResource) val uriScheme = uri.getScheme() @@ -211,9 +212,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S printUsageAndExit(-1) } if (primaryResource == null) { - SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python or R file)") } - if (mainClass == null && !isPython) { + if (mainClass == null && SparkSubmit.isUserJar(primaryResource)) { SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") } if (pyFiles != null && !isPython) { @@ -414,6 +415,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S opt } isPython = SparkSubmit.isPython(opt) + isR = SparkSubmit.isR(opt) false } diff --git a/dev/run-tests b/dev/run-tests index 561d7fc9e7b1f..1b6cf78b5da01 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -236,3 +236,18 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS ./python/run-tests + +echo "" +echo "=========================================================================" +echo "Running SparkR tests" +echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_SPARKR_UNIT_TESTS + +if [ $(command -v R) ]; then + ./R/install-dev.sh + ./R/run-tests.sh +else + echo "Ignoring SparkR tests as R was not found in PATH" +fi + diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh index 8ab6db6925d6e..154e01255b2ef 100644 --- a/dev/run-tests-codes.sh +++ b/dev/run-tests-codes.sh @@ -25,3 +25,4 @@ readonly BLOCK_BUILD=14 readonly BLOCK_MIMA=15 readonly BLOCK_SPARK_UNIT_TESTS=16 readonly BLOCK_PYSPARK_UNIT_TESTS=17 +readonly BLOCK_SPARKR_UNIT_TESTS=18 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f10aa6b59e1af..f6372835a6dbf 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -210,6 +210,8 @@ done failing_test="Spark unit tests" elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then failing_test="PySpark unit tests" + elif [ "$test_result" -eq "$BLOCK_SPARKR_UNIT_TESTS" ]; then + failing_test="SparkR unit tests" else failing_test="some tests" fi diff --git a/docs/README.md b/docs/README.md index 3773ea25c8b67..5852f972a051d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -58,13 +58,19 @@ phase, use the following sytax: We use Sphinx to generate Python API docs, so you will need to install it by running `sudo pip install sphinx`. -## API Docs (Scaladoc and Sphinx) +## knitr, devtools + +SparkR documentation is written using `roxygen2` and we use `knitr`, `devtools` to generate +documentation. To install these packages you can run `install.packages(c("knitr", "devtools"))` from a +R console. + +## API Docs (Scaladoc, Sphinx, roxygen2) You can build just the Spark scaladoc by running `build/sbt unidoc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as -public in `__init__.py`. +public in `__init__.py`. The SparkR docs can be built by running SPARK_PROJECT_ROOT/R/create-docs.sh. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a @@ -72,5 +78,5 @@ jekyll plugin to run `build/sbt unidoc` before building the site so if you haven may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). -NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 +NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 2e88b3093652d..b92c75f90b11c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -84,6 +84,7 @@
  • Scala
  • Java
  • Python
  • +
  • R
  • diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 3c626a0b7f54b..0ea3f8eab461b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -78,5 +78,18 @@ puts "cp -r python/docs/_build/html/. docs/api/python" cp_r("python/docs/_build/html/.", "docs/api/python") - cd("..") + # Build SparkR API docs + puts "Moving to R directory and building roxygen docs." + cd("R") + puts `./create-docs.sh` + + puts "Moving back into home dir." + cd("../") + + puts "Making directory api/R" + mkdir_p "docs/api/R" + + puts "cp -r R/pkg/html/. docs/api/R" + cp_r("R/pkg/html/.", "docs/api/R") + end diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R new file mode 100644 index 0000000000000..6e6b5cb93789c --- /dev/null +++ b/examples/src/main/r/kmeans.R @@ -0,0 +1,93 @@ +# +# 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. +# + +library(SparkR) + +# Logistic regression in Spark. +# Note: unlike the example in Scala, a point here is represented as a vector of +# doubles. + +parseVectors <- function(lines) { + lines <- strsplit(as.character(lines) , " ", fixed = TRUE) + list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) +} + +dist.fun <- function(P, C) { + apply( + C, + 1, + function(x) { + colSums((t(P) - x)^2) + } + ) +} + +closestPoint <- function(P, C) { + max.col(-dist.fun(P, C)) +} +# Main program + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: kmeans ") + q("no") +} + +sc <- sparkR.init(appName = "RKMeans") +K <- as.integer(args[[2]]) +convergeDist <- as.double(args[[3]]) + +lines <- textFile(sc, args[[1]]) +points <- cache(lapplyPartition(lines, parseVectors)) +# kPoints <- take(points, K) +kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) +tempDist <- 1.0 + +while (tempDist > convergeDist) { + closest <- lapplyPartition( + lapply(points, + function(p) { + cp <- closestPoint(p, kPoints); + mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) + }), + function(x) {do.call(c, x) + }) + + pointStats <- reduceByKey(closest, + function(p1, p2) { + t(colSums(rbind(p1, p2))) + }, + 2L) + + newPoints <- do.call( + rbind, + collect(lapply(pointStats, + function(tup) { + point.sum <- tup[[2]][, -1] + point.count <- tup[[2]][, 1] + point.sum/point.count + }))) + + D <- dist.fun(kPoints, newPoints) + tempDist <- sum(D[cbind(1:3, max.col(-D))]) + kPoints <- newPoints + cat("Finished iteration (delta = ", tempDist, ")\n") +} + +cat("Final centers:\n") +writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R new file mode 100644 index 0000000000000..c864a4232d010 --- /dev/null +++ b/examples/src/main/r/linear_solver_mnist.R @@ -0,0 +1,107 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 + +library(SparkR) +library(Matrix) + +args <- commandArgs(trailing = TRUE) + +# number of random features; default to 1100 +D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) +# number of partitions for training dataset +trainParts <- 12 +# dimension of digits +d <- 784 +# number of test examples +NTrain <- 60000 +# number of training examples +NTest <- 10000 +# scale of features +gamma <- 4e-4 + +sc <- sparkR.init(appName = "SparkR-LinearSolver") + +# You can also use HDFS path to speed things up: +# hdfs:///train-mnist-dense-with-labels.data +file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) + +W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) +b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) +broadcastW <- broadcast(sc, W) +broadcastB <- broadcast(sc, b) + +includePackage(sc, Matrix) +numericLines <- lapplyPartitionsWithIndex(file, + function(split, part) { + matList <- sapply(part, function(line) { + as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) + }, simplify=FALSE) + mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), + sparse=T, byrow=T) + mat + }) + +featureLabels <- cache(lapplyPartition( + numericLines, + function(part) { + label <- part[,1] + mat <- part[,-1] + ones <- rep(1, nrow(mat)) + features <- cos( + mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) + onesMat <- Matrix(ones) + featuresPlus <- cBind(features, onesMat) + labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) + for (i in 1:nrow(mat)) { + labels[i, label[i]] <- 1 + } + list(label=labels, features=featuresPlus) + })) + +FTF <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$features + }), flatten=F)) + +FTY <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$label + }), flatten=F)) + +# solve for the coefficient matrix +C <- solve(FTF, FTY) + +test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", + header=F), sparse=T)) +testData <- test[,-1] +testLabels <- matrix(ncol=1, test[,1]) + +err <- 0 + +# contstruct the feature maps for all examples from this digit +featuresTest <- cos(testData %*% t(value(broadcastW)) + + (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) +featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) + +# extract the one vs. all assignment +results <- featuresTest %*% C +labelsGot <- apply(results, 1, which.max) +err <- sum(testLabels != labelsGot) / nrow(testLabels) + +cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R new file mode 100644 index 0000000000000..2a86aa98160d3 --- /dev/null +++ b/examples/src/main/r/logistic_regression.R @@ -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. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: logistic_regression ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "LogisticRegressionR") +iterations <- as.integer(args[[2]]) +D <- as.integer(args[[3]]) + +readPartition <- function(part){ + part = strsplit(part, " ", fixed = T) + list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) +} + +# Read data points and convert each partition to a matrix +points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) + +# Initialize w to a random value +w <- runif(n=D, min = -1, max = 1) +cat("Initial w: ", w, "\n") + +# Compute logistic regression gradient for a matrix of data points +gradient <- function(partition) { + partition = partition[[1]] + Y <- partition[, 1] # point labels (first column of input file) + X <- partition[, -1] # point coordinates + + # For each point (x, y), compute gradient function + dot <- X %*% w + logit <- 1 / (1 + exp(-Y * dot)) + grad <- t(X) %*% ((logit - 1) * Y) + list(grad) +} + +for (i in 1:iterations) { + cat("On iteration ", i, "\n") + w <- w - reduce(lapplyPartition(points, gradient), "+") +} + +cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R new file mode 100644 index 0000000000000..aa7a833e147a0 --- /dev/null +++ b/examples/src/main/r/pi.R @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +sc <- sparkR.init(appName = "PiR") + +slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) + +n <- 100000 * slices + +piFunc <- function(elem) { + rands <- runif(n = 2, min = -1, max = 1) + val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) + val +} + + +piFuncVec <- function(elems) { + message(length(elems)) + rands1 <- runif(n = length(elems), min = -1, max = 1) + rands2 <- runif(n = length(elems), min = -1, max = 1) + val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) + sum(val) +} + +rdd <- parallelize(sc, 1:n, slices) +count <- reduce(lapplyPartition(rdd, piFuncVec), sum) +cat("Pi is roughly", 4.0 * count / n, "\n") +cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R new file mode 100644 index 0000000000000..b734cb0ecf55b --- /dev/null +++ b/examples/src/main/r/wordcount.R @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 1) { + print("Usage: wordcount ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "RwordCount") +lines <- textFile(sc, args[[1]]) + +words <- flatMap(lines, + function(line) { + strsplit(line, " ")[[1]] + }) +wordCount <- lapply(words, function(word) { list(word, 1L) }) + +counts <- reduceByKey(wordCount, "+", 2L) +output <- collect(counts) + +for (wordcount in output) { + cat(wordcount[[1]], ": ", wordcount[[2]], "\n") +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index 9b04732afee14..f4ebc25bdd32b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -274,14 +274,14 @@ static String quoteForBatchScript(String arg) { } /** - * Quotes a string so that it can be used in a command string and be parsed back into a single - * argument by python's "shlex.split()" function. - * + * Quotes a string so that it can be used in a command string. * Basically, just add simple escapes. E.g.: * original single argument : ab "cd" ef * after: "ab \"cd\" ef" + * + * This can be parsed back into a single argument by python's "shlex.split()" function. */ - static String quoteForPython(String s) { + static String quoteForCommandString(String s) { StringBuilder quoted = new StringBuilder().append('"'); for (int i = 0; i < s.length(); i++) { int cp = s.codePointAt(i); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 91dcf70f105db..a73c9c87e3126 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -17,14 +17,9 @@ package org.apache.spark.launcher; +import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -53,6 +48,20 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { */ static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell"; + /** + * Name of the app resource used to identify the SparkR shell. The command line parser expects + * the resource name to be the very first argument to spark-submit in this case. + * + * NOTE: this cannot be "sparkr-shell" since that identifies the SparkR shell to SparkSubmit + * (see sparkR.R), and can cause this code to enter into an infinite loop. + */ + static final String SPARKR_SHELL = "sparkr-shell-main"; + + /** + * This is the actual resource name that identifies the SparkR shell to SparkSubmit. + */ + static final String SPARKR_SHELL_RESOURCE = "sparkr-shell"; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling @@ -87,6 +96,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { this.allowsMixedArguments = true; appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); + } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL_RESOURCE; + submitArgs = args.subList(1, args.size()); } else { this.allowsMixedArguments = false; } @@ -98,6 +111,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { public List buildCommand(Map env) throws IOException { if (PYSPARK_SHELL_RESOURCE.equals(appResource)) { return buildPySparkShellCommand(env); + } else if (SPARKR_SHELL_RESOURCE.equals(appResource)) { + return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); } @@ -213,36 +228,62 @@ private List buildPySparkShellCommand(Map env) throws IO return buildCommand(env); } - // When launching the pyspark shell, the spark-submit arguments should be stored in the - // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable - // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. + constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS"); + + // The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script, + // followed by PYSPARK_DRIVER_PYTHON_OPTS. + List pyargs = new ArrayList(); + pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return pyargs; + } + + private List buildSparkRCommand(Map env) throws IOException { + if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { + appResource = appArgs.get(0); + appArgs.remove(0); + return buildCommand(env); + } + // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS + // env variable. + constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS"); + + // Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up. + String sparkHome = System.getenv("SPARK_HOME"); + env.put("R_PROFILE_USER", + join(File.separator, sparkHome, "R", "lib", "SparkR", "profile", "shell.R")); + + List args = new ArrayList(); + args.add(firstNonEmpty(System.getenv("SPARKR_DRIVER_R"), "R")); + return args; + } + + private void constructEnvVarArgs( + Map env, + String submitArgsEnvVariable) throws IOException { Properties props = loadPropertiesFile(); mergeEnvPathList(env, getLibPathEnvName(), firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); - // Store spark-submit arguments in an environment variable, since there's no way to pass - // them to shell.py on the comand line. StringBuilder submitArgs = new StringBuilder(); for (String arg : buildSparkSubmitArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(quoteForPython(arg)); + submitArgs.append(quoteForCommandString(arg)); } - env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); - - List pyargs = new ArrayList(); - pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); - String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); - if (!isEmpty(pyOpts)) { - pyargs.addAll(parseOptionString(pyOpts)); - } - - return pyargs; + env.put(submitArgsEnvVariable, submitArgs.toString()); } + private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); // Default master is "local[*]", so assume client mode in that case. diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index dba0203867372..1ae42eed8a3af 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -79,9 +79,9 @@ public void testWindowsBatchQuoting() { @Test public void testPythonArgQuoting() { - assertEquals("\"abc\"", quoteForPython("abc")); - assertEquals("\"a b c\"", quoteForPython("a b c")); - assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c")); + assertEquals("\"abc\"", quoteForCommandString("abc")); + assertEquals("\"a b c\"", quoteForCommandString("a b c")); + assertEquals("\"a \\\"b\\\" c\"", quoteForCommandString("a \"b\" c")); } private void testOpt(String opts, List expected) { diff --git a/pom.xml b/pom.xml index 42bd926a2fcb8..70e297c4f082a 100644 --- a/pom.xml +++ b/pom.xml @@ -1749,5 +1749,8 @@ parquet-provided + + sparkr + 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 a5e6b638d2150..53ad67372e024 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 @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.NumericType @Experimental class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { - private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { + private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala new file mode 100644 index 0000000000000..d1ea7cc3e9162 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -0,0 +1,127 @@ +/* + * 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.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.r.SerDe +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} + +private[r] object SQLUtils { + def createSQLContext(jsc: JavaSparkContext): SQLContext = { + new SQLContext(jsc) + } + + def getJavaSparkContext(sqlCtx: SQLContext): JavaSparkContext = { + new JavaSparkContext(sqlCtx.sparkContext) + } + + def toSeq[T](arr: Array[T]): Seq[T] = { + arr.toSeq + } + + def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { + val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + val num = schema.fields.size + val rowRDD = rdd.map(bytesToRow) + sqlContext.createDataFrame(rowRDD, schema) + } + + // A helper to include grouping columns in Agg() + def aggWithGrouping(gd: GroupedData, exprs: Column*): DataFrame = { + val aggExprs = exprs.map { col => + col.expr match { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.simpleString)() + } + } + gd.toDF(aggExprs) + } + + def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = { + df.map(r => rowToRBytes(r)) + } + + private[this] def bytesToRow(bytes: Array[Byte]): Row = { + val bis = new ByteArrayInputStream(bytes) + val dis = new DataInputStream(bis) + val num = SerDe.readInt(dis) + Row.fromSeq((0 until num).map { i => + SerDe.readObject(dis) + }.toSeq) + } + + private[this] def rowToRBytes(row: Row): Array[Byte] = { + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, row.length) + (0 until row.length).map { idx => + val obj: Object = row(idx).asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def dfToCols(df: DataFrame): Array[Array[Byte]] = { + // localDF is Array[Row] + val localDF = df.collect() + val numCols = df.columns.length + // dfCols is Array[Array[Any]] + val dfCols = convertRowsToColumns(localDF, numCols) + + dfCols.map { col => + colToRBytes(col) + } + } + + def convertRowsToColumns(localDF: Array[Row], numCols: Int): Array[Array[Any]] = { + (0 until numCols).map { colIdx => + localDF.map { row => + row(colIdx) + } + }.toArray + } + + def colToRBytes(col: Array[Any]): Array[Byte] = { + val numRows = col.length + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, numRows) + + col.map { item => + val obj: Object = item.asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def saveMode(mode: String): SaveMode = { + mode match { + case "append" => SaveMode.Append + case "overwrite" => SaveMode.Overwrite + case "error" => SaveMode.ErrorIfExists + case "ignore" => SaveMode.Ignore + } + } +} 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 24a1e02795218..32bc4e5663062 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 @@ -469,6 +469,9 @@ private[spark] class ApplicationMaster( System.setProperty("spark.submit.pyFiles", PythonRunner.formatPaths(args.pyFiles).mkString(",")) } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + // TODO(davies): add R dependencies here + } val mainMethod = userClassLoader.loadClass(args.userClass) .getMethod("main", classOf[Array[String]]) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index e1a992af3aae7..ae6dc1094d724 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -25,6 +25,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var primaryPyFile: String = null + var primaryRFile: String = null var pyFiles: String = null var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 @@ -54,6 +55,10 @@ class ApplicationMasterArguments(val args: Array[String]) { primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--py-files") :: value :: tail => pyFiles = value args = tail @@ -79,6 +84,11 @@ class ApplicationMasterArguments(val args: Array[String]) { } } + if (primaryPyFile != null && primaryRFile != null) { + System.err.println("Cannot have primary-py-file and primary-r-file at the same time") + System.exit(-1) + } + userArgs = userArgsBuffer.readOnly } @@ -92,6 +102,7 @@ class ApplicationMasterArguments(val args: Array[String]) { | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class | --primary-py-file A main Python file + | --primary-r-file A main R file | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to | place on the PYTHONPATH for Python apps. | --args ARGS Arguments to be passed to your application's main class. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7219852c0a752..c1effd3c8a718 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -491,6 +491,12 @@ private[spark] class Client( } else { Nil } + val primaryRFile = + if (args.primaryRFile != null) { + Seq("--primary-r-file", args.primaryRFile) + } else { + Nil + } val amClass = if (isClusterMode) { Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName @@ -500,12 +506,15 @@ private[spark] class Client( if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs + } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) } val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++ - Seq( + Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ primaryRFile ++ + userArgs ++ Seq( "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3bc7eb1abf341..da6798cb1b279 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -32,6 +32,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var userClass: String = null var pyFiles: String = null var primaryPyFile: String = null + var primaryRFile: String = null var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var executorMemory = 1024 // MB var executorCores = 1 @@ -150,6 +151,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--args" | "--arg") :: value :: tail => if (args(0) == "--args") { println("--args is deprecated. Use --arg instead.") @@ -228,6 +233,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) throw new IllegalArgumentException(getUsageMessage(args)) } } + + if (primaryPyFile != null && primaryRFile != null) { + throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" + + " at the same time") + } } private def getUsageMessage(unknownParam: List[String] = null): String = { @@ -240,6 +250,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | mode) | --class CLASS_NAME Name of your application's main class (required) | --primary-py-file A main Python file + | --primary-r-file A main R file | --arg ARG Argument to be passed to your application's main class. | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) From b9c51c04932efeeda790752276078314db440634 Mon Sep 17 00:00:00 2001 From: Peter Parente Date: Thu, 9 Apr 2015 06:37:20 -0400 Subject: [PATCH 683/817] [SPARK-6343] Doc driver-worker network reqs Attempt at making the driver-worker networking requirement more explicit and up-front in the documentation (see https://issues.apache.org/jira/browse/SPARK-6343). Update cluster overview diagram to show connections from workers to driver. Add a bullet below about how driver listens / accepts connections from workers. Author: Peter Parente Closes #5382 from parente/SPARK-6343 and squashes the following commits: 0b2fb9d [Peter Parente] [SPARK-6343] Doc driver-worker network reqs --- docs/cluster-overview.md | 6 +++++- docs/img/cluster-overview.png | Bin 28011 -> 33565 bytes docs/img/cluster-overview.pptx | Bin 51771 -> 28133 bytes 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 6a75d5c457f02..7079de546e2f5 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -33,7 +33,11 @@ There are several useful things to note about this architecture: 2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor processes, and these communicate with each other, it is relatively easy to run it even on a cluster manager that also supports other applications (e.g. Mesos/YARN). -3. Because the driver schedules tasks on the cluster, it should be run close to the worker +3. The driver program must listen for and accept incoming connections from its executors throughout + its lifetime (e.g., see [spark.driver.port and spark.fileserver.port in the network config + section](configuration.html#networking)). As such, the driver program must be network + addressable from the worker nodes. +4. Because the driver schedules tasks on the cluster, it should be run close to the worker nodes, preferably on the same local area network. If you'd like to send requests to the cluster remotely, it's better to open an RPC to the driver and have it submit operations from nearby than to run a driver far away from the worker nodes. diff --git a/docs/img/cluster-overview.png b/docs/img/cluster-overview.png index 368274068e7548517e3f5572916e38b3a63b355f..317554c5f2a5b1022ac594f4bcae774e2ca62521 100644 GIT binary patch literal 33565 zcmcG0gNq2WQ(jlFKbcld-r<4dtr*wP|9^Z4` z-yiUKU0yS@=ZV?-+56t>UTfWisVGTfpc12kKp+fR8A&w|2wnpOf+L3@0WAu35$(VW z*i1}O3o>N>rp&I^cEA}Rkf19+@YdSY={3yN+RDa}-&Kg}&k_8< z`^RP$D%hVxoGgW?UMZ@;By1f_VO-3d%&b(xs4y5z(BaKnel^J#e@_QK2~n9lIoa{E zu(-ImFuQOt+d7!Bu<`Nnv9PkUu(LA(M=&|M**Lv+WwLRk{?A4JxsIf%qltrsos)&F z4ear{uZ?Y;orI{U9&hyDfB*TNP8M(fcPAUizuf{l$ny9N3mY>l%YUy8oGSR(%CGET zVG7*&@%qASf`5+uKW%^SBgpc2^MC!A|Mc`vE6`P8R6&;i4x2ElLN~282qX%Ul@!x( zh1=6doWqgE$!fp_y6@r=u5172l}?4Op0_Y@0{_pa*?6CowUu@36ItPJDL*sZvIdv- z4=P$+zO|*LeYoq*3zP)IiHSm?;J+`ok)4g+?P|=p|1|t*Hv$K$d<%dZV> zU_r$T}WT6152sJif5AS(I3jx`pD;?cG(woW>l#+*OdwuqK{f=Tub=RbpG#}Drs zhsUTj1k+GD^yMThhk`708g0lDifudYW?-t(vGgFCK!#UB-0ZLZjG7(=9F)OIx6#J= zZba_M61ks0BHBdA6N+CghwbY%sj^AbIt-K=HV<_Dpg)h{h!o73)@>jOwZn&pXK8C z@_QO&)$Qi;nELKw#pi5$l2$~-NBm=^+iU^T-=|9P_>`$7)nL9@tO@iT@$QrwaGvS~ z{QvpduT$FSbaPmv_KmFj^;wCg$z-we)>+xhDxJcrS2g|vDJ-MJkTP_`ERFJ4aqssR zHU=lW8E4A1IZD-wF)ih-ED70+!5D*K&g+@weI$~kK*iPL;gI5t@!5L zb-g#r>EYgM1^+^w586;DpE8lkX42k#-xG%8^s_&%YJyC0uGSQ7sm)_{d{D5x)qC@7 znFRq~e1l->C-L9j!K7f?wQq1)3NPol;osn!n|fcaAv?y) zlhZ0?rXoEdSJ)c);P~y{?Om~G!L3A*Zv>P<*^B>;-k))I<%O3D$IT@Zcu$xg*vnu# zkT~_(-o#%PN<-SeF_@ywsNU*oFC3y(=CnP5E8^`rkR>F5LBbtL79V%DH_ri#CB;&+ zb40Vtp%M}Mn^j9Iks`&6UhsZ*5L)K%A79blEjBPUX49+F?2P9lxA{C|or<17Kb~Ek zZcP=);xMQeMOr)ex(%mu(fi!rPSsne+ZrBK8MNgoec-o$5bZA2s*r1DX=Y!A!zNwwNe&-9;Yuu28NWztE+B;^%7 zTy9$@x1Ou^lS`x%gXzF{7bKA;pp@d^fn+9lTevbE0kkf^KYtC%;27Cb#vtQ*x~mhsCUC-(8w_zOjgYjbo&S#Qtx&1USIM& z8kVD>NA*uM5sjtJ3H6u4A|709v}Iy&{}glA8*(`_2%-Gh1>J+ z%Om?Q!5B>8J%rogoJTeHD)(A?F(>M^blqS2=(=eQXug@r48@V%)n&h`+dn9ivuazdk+{sx6Y zxOHRUlciq0R)r4!hQ?@LI80kS7~P`D^_VJq9pQ7Ka`ti_r*39Zz0wOeeHn}L&lKS@ z4rVsl*W=gxuX+oU(C>z7_j0c21Yq^DXu3 zmag9w6my43r`||0@rvJkL#EXh~f_aOiBMIZYViIY>JWzc~*1m*rXse_@ zKYu1(PMLZp9)fAjMJ(BiG!FVU2A@r(+x02+S%@755JXHU7!$jkR6uL!-;g*>?FnXrd5ct$h7?$1ZVA;uQZclZ%Lt+#-t$(T>(=e>b41~0Zs{f|EbDT-{MY$I@h9rF(7g3=XqBD@ zy7;OK_85Fr|3O;t%FZg9f=j+U5uU=t62>au0C2>ggkoVQ^u77WTOQ0?AEP&*akPEtY7QNtg z?~5Xt<*WKW7 zTzc%+y`&I~YlMm^AbO^KUL&?@AHU4OSagt)(4Di)L#U(SG=|xmqv6IT*F2A_#opa5Az05S_w1J`b%yWL%Ly^bIkYEBtmYH|z-%VN@= zBmlWc`XLjcNBe&^LBoG;gu`|_wyGS0f(WZlhu$EiR7Lf)H^~kB7ZTV+`=a_lSE+ru z`_tfuC|9P~X}w3gQ25G>Y(rGU!VF)cYTf1j8XeLI;?U+W%?P1%Tu{71h>1Bk+hOc|(D-k>L`4 z{JU|E3KlqkkHtsuFC5w_2As_YNyYs?jrhQFZwT#g`a4hmykI__E!J7^`QMEVz{XW8 zB{_olFTiUD1{Dff0}Y9brVDSvS4TURd)YqE7+>>`o-s*@e^ZSkR`4d1 zzSwVFmrqn&UubS6g^**oU%2n=A-DH%xw%8e~h2B0^xIBfflTRNN zhG{rWX9Ta$wsE%B@?}G^>P-&eu4@eBOBZW7mf9{l`_Uo;${G*G^L!e{!lPQY4+8q& z7J4c?o>;6L*TLzSsb~T?{$p&)`@ho$p3Lv!aG7HbZkx^aaHV6bVenQZl-p)SaS18% zr(d=Nq_CKyPGz@`nA1~%@(TT{yk ziih&Tfekse>nZw@l~5FoD$SSc*CO`@#i9CUsALR@JEI=q_*}Jd1hL3BG1>WVzLKbx zP7Hs1NW9W$*`<@y%0p4qy;Sz)I_+!zpiy@E5md%P$B3O^sFKHG{dcj08Q}AaZWZRN zO!ro^rrNa;!pWH!hSNCX8K$ho5KlJ?Fxx^@Ix4N{d=By4M4gsXHb zsJLua)Gexe$NZugagVj7k)tzn21B?Rjd6ko10ly`)+=7VC~?*;o2N2Z)GAl;QWkTU z*&ox{6?|5 zcHhKKvKY@UO&(Gf%FrzCIW`_YO{uczwq^Xdn>w!n^Kc(XWsvo4*gi-?$#u98pu77U zPL8nsik!<-6Pl+x29%*Z(Q#U(uf*QXSL5Rp!iUTJJ^-L;fV#)Xb0Uk*{;(GdCL5yO zw`Y9UK>9=`;1*o_R>3ls9(U(^X^ra4ctAewh)QpL2|9bwt~?%-43M5;i0KxgXhxS4 zkyR|@o-iRRdVZPl7Gv9J3dMS+Bx1JTxJD&7 zQI<|RIPnEJra`;Agus*Rygp?^m2dF#bUHsnwXUj?-rC$9P@nyrM;l1tA}drN9c7>> z>LNo{o**fV8_amdRH)|V$b}`o@y{@KFY&&h$38bvaS?Q#GIP{aGkhzRT+wJou53Ao7>YK7p{wJ`;=t=+Y&LP% zaHue8+rv?pOmr@%rQfquIJdJsylYc)93R zN4)rbeseopXru+Zyz5W-(C+#Hb!PX)3Oy+YeT7!teS(EX%8RK68^jTx`yH;0is>|Q z`aKFkZ>7eh?3wsuq3u3nH(sClh5f05L&tTXtN8eTxrdh zQF4`wXN9{FAWGXYT5r(bli+fm5=FhbyO0w&eTI+E$vbPa%udkAuawM7$#lDJJL4S$ zIljWf7{31D-o0w#ZPYD)t(;zSY1qEG{DpQ( z53BmznQvl~#o6GSDR|`o! zZ?bY$YLwdDs688Wt{Yze2q3>;p-=2-@xsFToWWFsoZ}S@Dwp&ETB{24TGVc02#; z$8Uk}XV>~6n)(zU%inFq%eA=fbDPR%`{zlbOP4hqT$Fey4HIA_sjnsX{>=q4!#eBv z&`>gjn!FqcFvu=&mV7Q#xvL2KrF3{I^SepQU&nuP*=OIGu6d^4;hZ5iYo2$TZagL- zHFmPuFPD$xv~^;k{;ZhDBt)gfAi#X;XW9VzW%MfRa+^NK$l-i0Vp7R;qH)oP;%o7c zJi&KbhHAvF&Ie=q6J$T`gR{b2 zPf`&r&gFH40(#Z&uX^3e&EvLP4ux+lDA_#cSO;3K;!d$@;eID-VxBZOytJs&lMPg9 znplc${|gSmgz(ah_a=B9-x_{=RHahinu#j0O_>o~d~)SF|4?^iMx(SNG&0zA%kNxI z#jpR%f5ORBE+w5xy7v7?8;urJO?tz4B|7QfxUx9vLxhI?5}W82+Y^4nP8xhw5`%%a z$l;w`FVV1d;>r^Qv?iM(Uedii8=dVEDSMxE!tlM@O-VkB6Dwt!g7$rRo_Xn^-!oMz zsT^=kJ|pe%mvY5g>*A6VvjxXJ^reMn-y-_S<$mpsNlw!d#w;GjuWH^(j>%Q|ceEpn zRFc7`SL+45fGLBp-|mswM&q|t2`@lyB+60+r& zFt;QB0}n+-DPv&ErOPx=+3Ek8Tt>V`|!#>&s87ma!C|>x-67!G#_&yf;|H0y&NPaCYhYJP$C3RRI^#rrSMKq&yFf--WH}7SerqSC!geZ;U znffy(i7;Z=yo_nlXcSjvSa}EgM9!IX_i=`f_IL4sA36M_QTJyy3#VC{ski;g^)_3S za#flAVp9uK%+klkUX-z(ejlFnH1E86kltbSq^VE^qt3G2yO3-Qq=paR4!zzA&ymL# z$8(yKAN(7a-a~)Q*U^3u_DVlkYLTun_B#GlqplGTkL4#8&dF7(=ifjdHBm`jJkQo@ zexRVyprVvNjs31@DH_q_SCyj2ubqmUsUjt}r0jrUUzVx7saP6Kc{QEFl55L#^6NKO zl0J(2wml^SANaU#Oo;GG12V&(Q||dEtLl^BEG*?H#8v@8o+w~Jb&ZtBsbO%hLv@GG zbOZMm)iDtsARsh*wZp&CC<*^~gT1*zzn~zgH(r(XW<9ix4W%$H8w;!oL{+|;$z`Ef@gNrQ?a5;c_I#kbQ2#3hZ zCeWD&-!hD?8U?)VYn?CJ_UX7pwpy<5g;+o~AwS@U6P6dr*Yzx`TNf>d3CnS5ZEa3{gx8mNIKZpz2-yF`+ng*b5+C;Ik zvP1}GAto6=E4LQnPk;=E1Jc5#PQ_T}$C$PbS$*cS-iu+<0x)y-k!JwAUXl}K>!r_T zaLH!eV}yZL|9q3l;b(t&QrGf1uW?UkJ2%Q@GY7K{^sO9}(P}hXgeTe_bC|5&ZBA6} zcg=D@r-iu3-W*$qbZA41^X?1z8dC~1Ak@7o*)|!nf3Avx;2zy<-YAQ@1!WB~k`R#VB zd;qYTaHgNuq{V3)8;7>lybUL|5R8KT${GcShI3^7Yeb*Uv%{TO!7WNC zjRN8NvgCJ)#2} z`G4WO1gMzfA{5*q2Llv?Oqhaf0?h^n=C^BJfRHcOKH1Tcbuk3Dy3??~9g+c<-5h|J zth1Q$lBN^7&;Xv0+s=n1dlYyvUywm?vC@ay1<``lJu79=bkRh$A_dtJwZb1>CqKWX z@qUe=$o^2fuTN`{72!KiNw)cw-+6~M&R@Mb6t+6>I0ZLasJ(E&0I@d&FJS)z+|_gm zP^YcPaoy$o9@t#(G~`m4$qJP|qyxWDB@BmlQ``EyGOm6#43s&?s97p$Hk!3zhE|{p zOdaIMsq;lYUKkZ#EEGf1IVn? zI?X&ulyQ}7Bv#}>&r^K}!_4QZfR|0S$0=1Sd=;mFfP(ai^C{%BFg)aq+tI3+iGbZI zXmp@SG`D~wU6_Cs<|W>?g8d&HYmc@bvyaq?(_ReL$rV|tWj+E6%+lu zn0pA68h%mLQ!J3rbnv0n^*#^Yj3;kKDS5>MAzD2sVAOQlp<0~3G3zoCU{Tz#!G1=* zpdrW9*NtYQEQh0+e5XnCnCgplZ-)o$y`j6vB$+*4Ey619zVzQsR7+ZallgsIpO_tnx+{f*wyl)&U*Yb~gUt;R!) zygq#paG5yYU$p3pCIi$JEQ8(Q@7j`xqY1Q6Y?~animc{qkKG+M+5cfRR(XIMUSaSw z$w47%;A%nSe&9i|uB0LA&_IhkuTHS~aYAwZ--Q2zQ7{3{2_E@ni6#`H0)}Ro4JI?y zyPdvWW;SRw*L(o9E9EAKjXXd+Q}csIqywS^F0&qQ8M_13-&DsHm^7$@R)ufHI!7U` zuE+kLl+#5iLyNSlV{PwmuQ1?~0M<3l*$hc08qx28T4k?~|IO85e%}wj&uYR!;!yNV z$D1G(K4dcsDd-WtAD~WP84FkFHb(Q1QY)fys*Ldis$XO{KGT5GPfGFr*_Z3o9%7x0 zk5mu9?r1hF7T?gw6dSE(EyoG|4Ut2!XDS*LBJUmA?)b>&SRm$ngmd0l6iW9*wJP_0rMI6^|vT0sc*(gE2e^^u&Fe#13TlT`3-^*>m%xxH`U{1dYkGs8Jr#ZjWMH|A7NrgRAQKm{Y z%P!~id;l7y1bD&)v7{(50Rkba-1kvDtk8tdJ#|XEgEU+V!8GlNIHg zje^%dHB+H0L_$WS?@Q&IlRzQt2@_gcQ7AnUY~rs0q&4yiy{7p4>s^)^nMX+!Nf?mY za^#X2#$!b8Wr#T5a!zdg)yiS6uvOOF2_5=9DKMNgw3FlXcs&K=0aK||DgcBUiNO?> z;uW8V4+O+Bqe9owruv*=-RMGCK(~(2!Ab_7cFZ>6;zBH_QNghP`x+o^d-H{ zPY5p0Th8ooB<=esfhWI40{RRZVpcjyiPYSIV~}=yj(0E$6|=^Q^K23hS#!Jl&*Z;a zzn~{`CIaW=`ilw#btcWV5SN^UOLqAIGyO+-P^n5TJg~nnnc5{q0}D#DlTxMXhLGQD zJ2(X|nJG2kK67Do&@-ET$B9geYXG-En91@=gR}pnm_=I(K#AL?H!I#Z7;Svb6OTs~ zG2e!~Cli(uAc#6G`MCH~K;h%j36A&%_r+`>fA|&>ap__EP4u@!?_|BMsV%ZF2K{DE zKr@H}^h;$xb#b9rDC7Ww2=oZDr^g#DtSwxnQT_S?t905FcmKhQ>X{-mg9I(t`-`AH zQhkX;Tl!0$U?L%9MdFzn;ZGk`du0+3EFUN5Ov+#yY?ToHNB|Iu6X^RYG3O?LWmzb4 ze;I%es0IS>_gL@gL~(*Z8Gx`rfQ?rjD{t-Zs|$lNTbbu)zn1VZ$b|t35-z3QE9$%H z-NyREOTK3JUFNT$#AzpQW%Gwzv7&CxSPZcG=gMYn=@e@N6FE#6s7KFN2Zmcpvpf4PUt#zKNdvy?8;d z+M1fk<4BB3vWeEP+3YvFHc3FQ?a!OYYNOzoyhwTzrh=FpjJlhm-|jgNVgWEpqM)0D zzYl6X@iK}*bgGzEjj>4AI4PbU55sU3E69qgwvqJU0quR~GJzxsnVUD854 z=m2S^?k65Dq&y6k0aH!=P)s%$BJ}PwW)jNsXE|&fpD?!u9lcjiP(8h`PEor;V^Q9sv zw|v4Ivi~xWLHat;=|x36%5(z+IbRbR>^8la+B~+w8UpmHBi2IC!Wgl<`xKA$`WmKXhE(;() zVrhE9bUxcrhVYG1Vv71=&ExmCnh}2%O)yA!8pn}>8~00oe@83F&w7`5`@pVxHG8=JVvdi$@h$$>aPpuQy^1fa z>9t3;G#ZOfvNYc8!Y?g1_~tPndm1HnD7{>0e&C&^H=H8;5%IFBMB&sMSIMIC{@x;I z(EL{?&;nrk`lYM?iX2CFu-e2Z#l~xPRv!x z$2S&p^Ltl%f3b1veU96FhmQ!d{oYgwj^z-18^VNd3S^b<`uD7%HVL9}Y`JRO)-+iS2D~otU43UOJ`a5Bdp;A9le!n7s1{&o^ zHY`nhN6TW6|CudsVv>NR=_|O#WNc4lW@2nRqC|ZBd!lq&^GMa**HWI;Z65n&Lya~K zxJGJqLYcaE15Dz-h2#n#13c#<&8~Oc3Ax3ihR3|?ec`?Mq0ISas!OO+C89Gj?Y8V?)T;x;`6k1$U{OqR!G0aa=Nz@(ckY-c zp5VAwuvc~a~%KrHU9J0wWfpkO5Lwmm3XDL`r`s_Foxv;pF-MaKN(}IC**># zRH`b&j**C6V{V-yh3x5m>1fh5&(x<_@+i@BcAsB({i4O+|4Jyz8UIZF?Lqh=>3aK@ zOk~9^0NX}-2iQsA1eqbdee<}v*fGOj-UA8i;K0z6ml#%?BICK@LB@`)jviaDEHB7~ zbG;L`CL0j?whz7(C$gCuyWj2wojkcCRe}%~18bM{HF2Jd=H9Qk63teu(Cvy(wyWph?4jD5ooIIJ7wY$#uM;^Q%+ObPxP3HU?o`?7 zoE>VRlI?h0*Ss=FAFJcDEcfk78{{Dhkmos=sBwHr=TbB!^IrFAM&EroM+^Z#__!?0 z5iEv)g~noUt_JMj&owDpt5a{@^>YKr{WR*{D!SiZ*02f2mIEI-a^advbsL#;m7q^r zeQSCbmfJj<1mTl00mICqFFhokMH<{g3&3Pg6k|d^=#RbvgEsfuFL1R;Y8-Z}qTN55 zINlDyz`>F&I>4HdhP{SGA2Bfk*|_ueL{Ku5Zm*Q++}qKt;es0b^az~>!#r# zeDJX549yP7t!IK+zvaCtdyr2DlWxP$OI6snF&fk~hpR<;vNIL%Q3#gLbt?||PU=YS zmd)9e9$;bg+wDWViDtsc4-CX_QWWz7?TzEp5k{9Vv3>{;&lj>SEjEP$kvqPWUU9^? z1a0o=$G++Mcd`rIKAbni&J>mvZ4;(?ZY#DYg6TDHxPDt}CsbNx$`j0v<_>&4)yCS~ zDMXvlHpLda8g(L`Xcw+nvX9llxXvq-D;+9A#(lqmwpGyE);GE6a;WdtFMK|{!KL92 zUZ|22s0eu8RYJ((ZC;bpGy&h|^!&o_(==dp=>C>E{QJujk?JtBE`P*FkWRzHpA3~j z9PqCf6I~di;S^q;zY@z5oq>%-i|!&SDg!`}7lYy#?aqtr;{Z5o+=ZZXlsj-&nPG=; z3LKk11);_|%ssX8iI{eeXg6U!Z0>3Od!0IPEm#q+UU4CeLK17_8m0v`KNckwONYT6 zMDEUT`~rC8H6l$kuZoZdp8`occ!vE=e{7{7csm0I zeGc`-NA*EEMxJ5pav|`EJlYy31k;n^w4Z=xW* zOQVkTwO3DHz9`jhV7Xl+e9&Qr=)?nnPxFMP`+q)^c02NBm##D)V+cWjg?*X*W>2~s_5=m*~l3=F6T9=sL z>0@R3ho8BM2x;BVIa2RM;m?R|KUcaOE-ToG{>({&FByM>ink_>bZ`3BPeO3PFAdZG zc$-Ze4e3W91#5i?B3M3wzM$(C#X5~9YOLHZr+a8S+@I@Alr#4OLhgkBhJk2a4hs1F zBOHg;Lsof6cJ)WHMY5Mj&TK!o8LaU@bzE-e16fKL(Zp$C)mznR^1r^FrjYJ?9G6z$ zDoY9ZT>d0-A>%l&eYr2n#gDONhqCx2(`X-!le+=FLg{N>(?{(UStqNF`l@d1xn!55 zQ=M=bXVPPi-)WbOU&F|gjIKTIPq^Atyj2fZd~zIu9Mnc@99@eH3n@;BVCq9y-N#6Z~PtSmzC3clNa0i$_3ZzC8e#waN*gun2H&B1GXD$F1 zx`8^xnh|2K4q!55*1np9kqK)oC_pgO%3m42HG!>UMN)_?+sQSi#Bg37{!+ml6Bff} z?4Ymv<(SHx>s&Y6z@*ywT)d5$e5d;pYfyJ0GOhBe$*vQ|1GsKAWa#~Sk_kOUQFgJs zBbJM$mQi_e$#C2-#YWh0t#?-065F=2ofK#WUWf?X(E+_JT2-d}PQEIDN}*KOshIn2 zx>ReE%F$beoK~JNq?xNyTs4tfb})18!>_|(4~xlV69%v-=e6YmNUG9cELdOqGen%0 zRG%&BE+4bB8Ztjom?Il5^NYw^F-gJ&X~U(^w07Xj9CY8M+cN> z0z^{L0(Z``USSoJ&D-4Dfh@b_=pYk3nh?$Br6zJD1kq;4cq3_XyT0!k=N}LXso;JK zd7fmB7&dLx5)uYn{ODwY;3*h8ZVtWnFHqE;TZ~6IB?K^;d>EdEANS5c663AuG%;nt z7cjTiwbMHGEu+8N6G_xSMa%9;_3W|b6^*5Oi!j|!$5fP`V(K?B8La@AfB0nxDWx(C#Etw5Ca-r}S) zQ;d0B>UY@_uLmNbyiu)s781Gy)1pHNIR(h0+9K2ihR7}fN7B%_SW1bxC+NZ|5H2_$ z`pcNKyBc_RK{!U2mJ2L6r1R3js)!OAI)L%&tRhcxUkl zbUzAO@oG-1xmW~N4<2C6)EwCd6`^uC07<;ESZ5&wujqk*PNlOGUupIT$3IL-z<`SAoyGeaN3sIRumJ$GqF6J$#A8Wk@PpN4a6(_WdPo2(LCFPCxF5b2)?g z%FxK>M+%+iiDDmMrek)Lp8ma{biT#=_j_~Wb~3YG;53BZ5$L6EPBtq6ESw#Df#$6W z3CJ}G;LB&32nKpOCk^s(J<33tMH4C`dWPj&r<(u53as*4`(TfMoQMZSQ!2f3dA(MvqbI@vL(rt4!i(m0j%*pB-=E)KDT82#9`NE|0uF%`D zd7MjrH@xS#_v$cX7|~dfSdLuZ6C#D6SilT@ynO5DqVF2E$aS8+M|Y6@63wldnu+4P^l1jv4~E9F-()}vKD_>>#`-_RJcUg%UH4W zpzb{U1Q5Lzkk7ceC!BV7{N(1QIbr@LLbo_a5=f3u?@_4cid#?>K96~K)fbzoch`^$ zJB;-g$|m&chtS;Ng9VY`E|a@6k*#rQ&dmLOgPx-Le6G(f8cApa#s~^{KE{hd50K+S zsOrR8Kc)zk%QJOC1Y_sd+ay-dp?iy^W-iNB=*@styz{s&)e<&X5FhS#zDo`jrW%@1 zOuGd*$U`2ER!1{in58#rcodN~9UeL9Cf$=J2^b;diSIBO;#;9uaVR~MfhqJX`kC(| zF<5t&$%Z1@$m9Wl^y(KNzrZy#)QV#u3E{;_5UE}O*Q*}wbKX?x7&4gwTcptoKv5vV z;5+Amzf=u&6f_?E-r^FoE3vB56D9r`;paB2_})f=;gPSS5s$unx+V}ljU-wyREA-M zhZswDX>E*@6$10?l*0sa#y&}%XPRrl4M*RLQBxkFAu?4W=4eO`!BFAF@pgg}r8 z$NA7!s8q-_lF`H@vB_r~V9UZroXjdl?q52a?e1)Xv(YF6_|lALfdhQJ9vY07jS<=* zFtr5{RFTbBsf3m&-A*v$C|{b4nGy#YV2S$y)Rnh10&*q7pxkA zg64}26;=MZ%|a^PguZDC+a_%n63z~?tP-7q2>5*v3y^xVc1}?Dg(E;8D^wm8Bl{E8 zhbkgdZB*6;4zDMgh;=4C+8+m8`eePnJYiD+y^>iVtPa$at zll#iNmZetw=-DcY3W0x+25ky@&H`*`ACBM=`aDoy(W|Cr$KH6+*T2jaPXjDaxDa1x zsqdEwCQJV?P}eOCrUa7*C^pkN@*xj*Zs?EKzxU?VNr&(3QngO}uqquJWoj((R^O1)q{;r=ltzN8D^N?_Wj<3rMB-UgAoy#Hu>QWP#KxcR_aI9ui70AH?F9TGer4+Q4z9<1ju3y z;<*dIX6~n1Pt3xg3}%%3#`{LrooEEibNhHdCSiF^GM$vZDIb97MF5VcQ5--EF~PU# zkEP^W#!U|KGsN>G;c99INP=o7(Un@~3`3(NOFfVuu&qu2*wOs>$5-Ytk_Q(tNYVqF zQwuRr-|+%aI!`U3*^i}WHM+1oLowkH}2Ir`B3R(|M95L(|Gv@+~a2>d}{=c9HS zRy$i|s1C3}SA6e_?-zG>8_%(y4Lf3q!j!-(XG%sdjx6ZW4l7eytFWrQu)Ouf)xdB! zkkxqr-S?e5pYM}ZTSD^#ywY{Sc%!GlcKf~T)Zk1Dl0?wyYRrC*)!C$pe0ORSdHwpL zGV&+j3l&|Z&XG-omUQ;%=-6SxyApJ)R6JmT;=;)|_`il@Ma|_SKfG71xW2qKsTdVGl*eaz>*B?_nAN$hd`oDh8t3@9 zSRv7ps4P5Kqf#KBxz{_B{$`#>t6hR6;>C#5%Oax&IXDclCBuVTNO;^6l{nB?B5oy+ zcKRPNqZ0vIJ*~7&DJG9-SPG;+yHAVv#qwCcNhHv>xzr>0DSaf}k4^lSpLkRfgyzy0 zj<;LZB_)7hRHn`x*SECO@>E+v(Ss zK{7@;`kmg|{6qn2-=l_Ny9k)1{$2ML?du)3_7$m%kGu7#ep{JS!_S)pz6~yxjtNjS zgwjTx-Hi@@I|d77akYg7Gq9?rH2<~1W`_HEf6G-RHJN-fP(l{h3y%T!Jih?3Z-hpp z+4|0NyKP2gjWq9~xz)vET*WOU`f&z5_M8_3uC*1g;rF>Vt?3;)XHD`y#Xuy6KhmDa z8@9gwp5~#LTFpaoQ~1R2#m|(ojlcC-KpY2Qb;>K^(B8#=Y4V;&G*Va7gi?X93laM^ z9X}ilwY6;x8l<~;VT*5&@yF8w0}R}4)d@Vr!cM8K6tc37@s1;7lBB_6?DqToo2wm- znUhmI1vQz!kCzU5bSTeEcm3QNNf|6DfhTy_5~=g&yGAilnWV+<42CXKp33%C!WYiZ zn6*?mHJIN^6#Qend%TtRBQWUjpb^6VpBcIhHIo=L=>UQpm(5u6v7T!i0B(CkEyN3Rz6B9icR!s$l+izs5fO z=1(4qA2UzMH+-Sq!erI{?AK%UOuYR;b^znc$|FMC??=}jt0~v2 zE74>E=0mA$Zw!LPXFmZ7yH2|&Uy9rDk07PwN4yAuXAbin6ez6HtBV`S?2z2Lu+9M= z{~&~MA5Mi_N2Q7~5l=l0tayUzc5_aaRJ#1iLSJ}n`im~T-skG?QE*EOsuzCwA00c^ zH1cG;!$}0Re4P$|Bv~eMs``UU(3138J)^JzgR{}^c`fkpFhDk20M$(7(gi~%h_Zn% zAnSWTH6NGla^_1Vo)nbOb-L&-N&2k$L;iwQC98D!rjX z?oQDnbq*VHr3P)ynl3(fzhBZs^hLW>Q8qNoDE+ZQLa4xc1`^H}_Y#F^*I9Xz;pT{{ z*&`JtF2!2qc;dYU**^J*gIW`xv%O!}>3(TgvH(1=TC;kp=|x)lH|l_g{xdo!=LLlz z`@odVa@hL=AsJIk?n3v9=CS-fbB=f>gv6BmHdg+&=ewAgEpD9lZYD+Wm78kww^zyT4w4 z?{LCt{k=UR-FeSC5hWA>N^fqJ(*!qu#95K}lF-69$XZOW!9_xjkBF+!ICcWqe`(!jdiS+6dK#3xs<#MarqmU$A3Fxbj zP{>gHk$SAcr=V3LX4qr2>)sw(dhbHXSJ<$U-O^cx^Z6^&Z!Z`z+ zXQ9$d3*7nUs`*DB<~>#+IH5s4w~7@RQ@K!vlXqX;nJi}8B{F@smc0fvJ`4Kg)LUw0 zKb9~sYE=|3F!2+)yua$$y%c}_pe~K|p<~uMk6Ur)xMQacIYuq; zXpPRY*XaeH7nIwIkeK6Raa^NYA31|w4Q89?Rr&&Vg*1y{TM?DWOrg)L?V7yJN}F;A zP%pxTH~#3o^RK^we=Vt3DcDb8Mnrts0vV=-czHR6uvj_14P^7Y1Y@t_0KqC zC`4#t;UWE+E${a)-D5FgIb!1M%QK%;VD}H-UY(}J`Z>8h14@AlfdA0z5=3XfFp?aO4pf2t^S1y9SSk}-pJ2}d zC9?;NkU=1gORKK{YF|&+b`e)<00Ddwn6>6WB&-KaK?k&`g6?ObqUS&QVn*-o0ii1G z$mQUBT2&tfR^&Q>k-+hM0LgX@2pdgGLJdGkp$d@XJnD*p?*WuA|0pBY^VlvA zNwRrepK0|Zk;FX#65Asgu-@{|6e*PZs}b1 zyeTC>t%_v>Qp^YO4n-0PyeU_b>C$eUri1gfvQZw?LSEh!}728D5-zlx22 zDk~0y19WB+fW+p6ZGGT(&SN(nQ2EglhC)or8(*dKDIA4-C(z^;faBU6;CKW-YCf49 z<%lftMW7m+o91aJ8hjKU=gaftd_o?Z%-kmbhgTimcfSDbNu$9^yB(jGMZ31goWN@= z5Urp>=hM4#vA|$7`1niUiJ!7TX3{v!egPIeT*etQb+2G_qE{UKh*QW52x0zj(f!-k zM|%M3v1wEWAuJ#mGf+bk008YTI&nTs=x3>{aTmy@W!~!xMi|gnj9{9^2hiJ|`>$9e zOkD37+zEV*#ELr9%Gyf<5@+3)dRHvJ$%M0p;b2b@9n6ZoLZZh@90`7X4;rku8N}Q` zzcC=#@``?*Rz70y&u_ETB)yJwQF{^+PC$z<(C_2yrwG5mTee`SGrAyMp!Y4WT=UCw zF($s+H&Y?yLf1!Y)E5*iTy@`Dy;8^p-4$4XigxcWw1nEQu#o5voFjctwiUG_w4Oqn zQ6z3uiq0$|TYdRE2@r~C91Bjl(knTTx!=`?LZaZ8&nk8g{L);%rt=8`8 zMx$ymQ9ubbfR<7C+{R55-sqf$yXWR!9jzf)`V4WP7n%%@J58hpmr;iUz zlO!EVzA?SN*3rejd!UhYl!{Aw&|0K%P71c7^6NMMmZ#w5av+pLz+$-bwCt`LMP$5r zSU?7cCUtfQUL2`wP*(o(L7-t>FM*CDVRob5XSQ+0vur4&Dw3&~>J6U!Lr#S1ud!UZ zcZvO;5RI5__%>6 z|F5;T?y9o;zK7v(fI|p}76fUP?(RmqyBkDGLAtx8JEXe>lm_XN7Lk+|=@jAF-1q(Y z{@%bd#u@y4f=9+V^XHsW4iBCnLV;{ICw>W(;tB4iqC^X+wNl~2tfeha;nxGCs77 z-lY%*G?h=NeMYK%`q{9?4cipT5}uo?A#VSQ2Mk4?xI>K&V$+tTqloDk&9N%xM!wyz z9ekQ}+6i1{V*;1?v+}-cxYutiH^?WQr+xDSg!pv~k*T>e_&X7Aw#=+KV;tjwyaEhAf1DPR;If=6MP1 z@te>P4CQ93J^OR(xby;F6xFmR*mN51{rc>ea07=E0vF!u?1ERoO$iR1?uw(y=4&7q zEo(%E@t|+jlh~_K5)5jGmkiez`smWCi!>J-=1wG_dB&J`PNI8XTi zqB=P~M6k1YRs?QK(vzc!*5eJ_NF3i*O&*P5-+pLZYH4zdFlL&tHO}_+mSxx^WbV?1_f*|w6j>U;8tdDf(?=@RS}g3PghlFQ~W8z46ez_oZydPl<$(m^CZlePdB z*aw2L&jZ0Jvtv~`STs-1gk7QqnYU!)oBstoc6QQT?2LDn=ac%7xCGdmp{` z1Tyq)Yjh~Oxqk}oK_I+W23?QUW)wb7>3wx1J$n=u)XNW}sSb+M0+pVt=AjofsCVKB zlDgaz1EL}{)$TQ}E2N7R0atP}ammP;H{DI8dd6K?`F#l#!r4<*fKW&4eR(H zr^`y`7$q;iZOjtICqm&PdAW`bIgLxgt=~^1^4qdPk6(;s7Aei56V4pp4y(%7)1Wcg zr4*{J6a{6rDy)oG@x8uT=$7`Wam?Zz=JV`yifhNSoNFj$g+v#j|FltM5sjKc(3!yi zG5>QyOW$?tk~ zW+A#s&v3%-QrbLD^^zP@q;L$PrY`i7(zCuJYTfXBTZw7WLH0CQ*!VX6q@OW!+a&K9 z>+vOOC;6W8%lH-jzgg+IBBTOTD|xk!kJFfoNml*p!n*@$S-lf_kWZ(#HsAQrZ&i5P z_`{cQJ5;wQa$6odXH&c>iT+SPpQ1i&5%Fk+a{TIkVD}Btb*@cD(vbT{a!z4YO?T*( z_1a@$4bvX)uZ$E-87E>>R^qo&AuIURXf@)`(l)E*L@F93Clx3wgM_*tR~vMSrB4;r zx#We-?dC^%Sw2l|=WGmc0UAOcZrL)^rIN_ zE~#5bc$VSz2#Zpe<`UcNk?taq?yyN$`2-cCnUdr+1sapyHaH!sh&UnkA7KX*PBKnL zcN-AvB(N9@-;L+xdIo(YU3qelSi%OY6y|L(`kNz`$@}rg+fN*SXkH^@K_>Bg-(r1D z6oZt|*1VcsAv;~Z@G3fOtKO84J_>f~9_P(teE1;1sK| zjO!catd3`Smq)ZsGQ~%jHz^ve1o`MT2t~qRbAm%hVO=7aY-OuXu?zZFY#{|VfRSU) zo0T~-;@kxwbNq`m0><@pn5aa^;S=bqguNkVF*d#6NW|{mTrRw>48Z9( zu`Qmal0Ky96symnwcJ$PHWhiZL0Wo!tuPz4%_Xg(rqxJ<93D9RuVWw7^r>LEL!Y1+ z@WZ}H+<&AY6nXGTt1ka?<-KDeh5RCp4%}T`H9fgcieWzk1Ows9C-c-uJ z0J7*Ll|NBBygaWRb=>BMcN`flGvW`CGvq`=RzzgH*S9yA==tb70@UaKp|=mPsy^ka zF?&cidcI4AK65s~zi=l4-{%X8TT+mbz&c@p3sefyCg_24 zIU;_`4tVs!BrN8ZJktS^&=0N_u864RZr?vl%C5B@#krve5s7bpjPrl{ua(GzWS2eG ziCu8GMO-rC=*WwFMk{M7A5aC;N{t2JUn(M;Sb-*7qiUHf*7o!)Z=1VuN7DR=)3 zj&}$G)prRz%V!#_BV|R^uzjWsG=iyG<^nzO!BSMq?Xk>(MUvC?H4080WT=>-DTxFo zyH}uK5>T3@ynpm&10)_|c6`)E*6P+237x;658E%)idp+K$#C9*%zP50>z%_5ie1Af zJLGf8jn?!~y55#L>1Ri^GLPd!k$VD=8-2&AT(&x(P3oVRs(pnn(`wOtkwvTjXIZBV zHR*UD|Hh05l>*gF8nEh;2e{V;ZR7{(nBibVrGfB{*ef$YDW#U|m)7uROp2i`gimT# znGQ+9Hnpk^hN3KkbSPIqM*HNaLe=>o6x{=86Y~Iq@y1`S(fNE&^Feho?{RzOthO)V z^29D%MlSK$Bg(NH6TTE2YH1aaalNd{e+js(AGM=_GnIHaYxP`$MNF37VZsCdQ}s?v zrIrR?A>8h-#&8A~LmY=@xrSh-qINhtm)(rX3OnNmtow41^-cpcCE7Cmt?su!(GS)l z{jzfZApR7v7K#V&HOi({F2|jxFijw8q60=kZF2<|9)Ydc?^|X+tSe}N65y9$lwF0Yq|44MaK_lit=e zayl%f0V0UdcCk_Eb60kAiE;xS41k5n%6nLF33ISclWZU=hRxW$}s`ZPKpm*}ui zG%J31K7d2_n2Eh?rfUcJWHSkQ$F!^Tl2e_h@E9(K`3>E z7#&LR_-asGY>?oAAzN&Xrr&-?ZAgCyp$b_U0KRjz1((5ScsfB)@=7p2>#b0sFn~G9 zk_ATGXqvC?;2?*S zHYH&+?hq_lD*zy9<*dZ#0J^PhEvxHm++8O9=Ii=Gd<;gYiOfKfPA(Lit9ZYMNb~`8 zGC+C|T}}fse^q#5dyZ5%JOI!jELcVgrUBO4%`vsA5cAE$>Z@dl@826~cB{z;o57)% z00U1e?LkHN)c_3c&G0>GQx+`InY^mK@69ZKbOX}5{Lrk#uV}>bhNh==e9 z3cxZEa#_Bcl)YeB=YRxM=byA#D zl~0nq4wowgu&s~=q+M`fq75RYG7dqt%F#q`dv)wz2UyvT61@yru;qh*T zGQz&|IFX34XW=rlpWiw5xO168TE$-=+mx{?MSEQxt&v(mg5iNewO!;6Hp2O{;_ZNk zu;G2}gc;zNv^ch&DgTyjA=)$!<;Wc-wS401KT8WMhpt{`|}<-L2e>J&|^H>tPt zVx51&@Z!YBIaOfu-nVB(9q);K|8~$P zu`|9MIhH$`3LE}D7xA^?B;moPYg|n9KCf3}w!_DZN7U%KD2A9roiwu8Ezv7NaPo<;V*i;!XE4B`7HZVEdiovR zJH&(yAgUaIN)(-{5P*j6D>P|9xz0^mcQA(8P$dt{#H=aus*G$fCO$q72Y#O|Q5~d6 z3a?aNg<=qm??qBdgieC;%ePkCX=@x)q6;@(E;IVN@Ozx2By!AqYDz9%U^1(+_BeKO zQY#%fcqk?9XU~Wo!gX~)Y}hehQ%mW;V|E)(THK$ z`g70^!BRY?SpI#{#EJ0d$g!EM*j*7glIYXUYhAD^0;`xj1@Mr6xQtfM*!J~wi*Wos zR5&y&i<=^v7f0|5t~Jg0{A_Dda!sJZoKy_Sk`U_Nhw_@EQB6mbXrz!L6qj)13yN9j z3euIqn&-Ro&+KvfKqH$c*`=(s%ZhDF@J7YWXz@Y-qC<$-jZgv{Nipme=Yn^ej#{Vi zuvxuO)ZejTe9`K)^s3U{4nHPP~Ob zhrfW+@eY{vULt#)zT47ndJ9`ergUj`gw|!i5%7M3Dwt-Qdu<1n`XQS-{3rVU8uAzN z7v9)s@1&jy?+R?BdrA7uC+5{I=fyga;}<-7QSz2jL(KWoAWyOK2lO7jkM!7q{tyHw z={ly}%CK;P6mij?Mt$whlA@(>Nu`MolfiQLr*romWM$K{BkRcd#<`si=O3+BBe zNSWb)nTN1~Y_BH}a##w`?L*A!fKKY$LQ(<{vu~UM$>?b)MZVtTQ!|DD1InF(oS33h zq*pba3|9_-`NWuHSYSk{u_wGkgh17`mx}BVcfG*nkb2hh9O^XTEpH~;RXhC2sEkx( zC+%gh4%0FY`w!H{pbK~jvU-9=6wI@*Lkt(yms+4f>Vx&)r3XPG~f-}$-TDWdgBFlKAF;`4g@pn z?p$mL5nftukX)dGLat5p)X&fUa!U*8 zj~5bqh|Lm(wvz+TKzUHE=I&IHu2+apq3run!_$+;EUdR$t8r<%!7Axrk485DpUl|? z+X=N{m9Ia6hfb}K^|`<%?;wmox+EfTPQ!K#f9iashwg=C92;@EEq6?~1+vzC`T=L8JDSddOu!3MMf-Yb&ICIDs}wsTh=y zP6`RL{gFM{z9N5Ep%yRY_hDzj$VlDrMx z0^Be%A$5^_3O#UzUg^AUV3YAgQm%k1SdJQi$X$PNx*{=v;E}CSyF>Q`1u+;0(MZAf zx8J=F2c)W~=MN<~^3JPx9C}U%`3(CBRcT2$0^}&w|CIkO0svE?VgbeIA~_%(F`dqf zr}NN)A_9uqA?lSOB`MOK$r?{LL%<=eFUwIvGOvbF{!hN$Ni|T*W*_t};Y<*dD}DRi zf(3R(h?s^1PbO0;1@+>s5g_L664nA1zi0`iqa6pIlR%;t;4g+yoDJeCPkmj0uJUrq z1G9|iWRkYed9rKb7T0IG7{3kqhWmzc`9knjinTOTDG}i*JZBI{Hm*?oxjp6_K)11i%hLIOBtR##ZefoXy0x43(=K}NM zBi#@_6oQ4wC@}p&3WMM4g7Gg%cu47pL~5Nh6hBaHa%(u-f@m4X>5-h`1G<3=Gci=* zNbl*xA4RwWZr)`;eZ&dekd3m)D%efPRB;}7>##KkvXnbd6v!1(c>xpk&z^-ka}TRk z;Qau5LOKruXk}mpz2UBz!)8YuLIL;2aR+L>^TU!V@&6v{^pf!ElSM8(Rv=5P zWuo2WM-Z_5ZLH>nR||MwIT;xcY;^(gUlfuEiy6_oMTX4=^JHN%!7 z4-rQF2A=i!beC!(8e6*L(i31q`rGF|(C`otT;u>_W&5#Kcw6+UUwC;Na1|QP6%nx< z%SZ+=niR;c(cKmX{Q-q=^%c~tKU|4;s`$oE{yirfQHR$d;*SABcuKB2w)3GN$)Markd^Eyy={SzqL?whsE+fFL|`1B@j8aVhVP8hdw4zFd9d0|fi z^3g7E){*ANs{YX;1zgHY)U8=2XZLxkPJYttJW#9j!&8Vh)z7VZOfwY?}~Ir5vCYuVr+s^M5z5O+#OCsn-zbzrJ@mcEm-j6aaL+$H#N(XBT9!S!PXE>`# zLkW`^MQ>V`0v<7x+Wy#UJe*YBoAiPKPk0tCX%q2uBR@v{K-r3x5o&o-e!?{dCOok0H!%e@G$wg2`T}0*` zS4x00CjX*PDkDqqFs)*u$mtM0)#gyU~{?I6B#pf z<}u6znsr1M$o{|CqfDoJcpdFDpr93GaC^d0fdziq)2hYVjL`PoD{jxuifA8RBjxJ0>HL?BtH7UG6KB@rU9qZ<27p~ zbvP5)XdYN&4_a!eD8exZvV;dmpx<$#e@T+dj8nOnv1q_k<@{i*Z8upg(DYg@b*kj$ z(B|4_Xl`llHdB#qq>k-ZO8q=G{^~}}p+uS~-8SVE8cNgCsH{%!kjS4Cz}MW{L&$n8 zEtX0qQZ}{5_jq3};YEOP`QD{8NA&FM8{jQphVIq~PpdM@SlaQ<50@QzZ(&ahb z7peF?j?2-l3ltLQ@TO~(9?`i@vG|bkyyN@Z;6W~=)?sRiu~|6RG^`5D>F(#3J-X(- z+x>1;BZSOoaTw|@=aOzm8_iPG@F`#0X}y}!(vy0RQZ=hZ_u5aBda+ZD%Z_uVSVX@x z8SYhB8vOYFl+U|{0aq*`^RWFot0Tv|^HN~KBlnMsh|l<&9CMHUz!mY$hQm8$=KD8_ zdvm2P62n|m8L=f_za3S)cE2&u?eGngTxp|oE!U3ELkvh5pbgvf<@hxzVHs*vI)rBVgV!<;xPAWhz1sU1K(Ar5Dy5>HiZsZ&Xb?w(qt7=KVIa zTou;eFG9x5>3@g*oE(3?bKwbf>tJ|P^$$Sn{-006IJl6~! zP3G^iy?MRnDqT*_dUb=3+_OJBeN9NaLO#dB-|*Wh*JNh#!Xm}R)FeUfjXa9fR6VSh z`u653*jMqZc*KBJcDuWjr>jkI$dq4saw68)zP#f-Vg0>quy23N&SNw!v%RRe<>dSy zi!Xf+<=9^-7!M^fgk`B;l5E}=$G~AtfGZ|j5h`47e)w0DO}zOqxk>+Wa0cg4*5s9l z$B&wijJbSzv65L|j}7|&S}h2l(PiXwS0bS;_#@oa;6c#cfa5^>rt++Kt9vyAbkaIV@0fr_6hbFva$K)T-_+Hc&%Xao;m#uXm&jRj^xt3d=^R|L z9;F7GnL`_eYVB6Ig0E9BfByhh%8y^LG-Ji5>bs`+^XLtA_sUqT6E$pZ9~SS+AUAp` zKAX7GGO}Ng^L+OxUOu&Euu)fJFwR(cYPn2%@6X^4+H_GV$%m25=glPi1{4nT$w8_o zKgWySg~esl7z%V!wnP5T!Z*B+ZKsqx-(kd0(^o-+-yO_o4X4mMETw$VuL*RxxyG>` zmw!0{QT8SZg^rZwx6|%_97Lr3&=Z?GBen(iKe{De|0SY1qMNg!+okAx^)`O9LH?pr zt?Pg?5uH|Wvw8C1=Jt23dB#Ru=N;wgOteIS#Yn0GjTWrmmcCNT9@{oy_tLIxWg$hGv{ML9= z8q`fb_TG)-VB8M7*rhB8yGWc={GRkW`DH=a z%g!9fmm#RmdTIGQuD4gMw-KT8FTL!nsd?Nv~B5HQe|wkKS0l zmwt1n%AGgAztG;|U&#{v%V1qK`o0&~gizJj1o~VQHYvZB?wdv9W3srltBpD-3FJL! zhii3(9FLvGm|*rR6=t8!l1iLMQ%@}h`}hL$#W@Jj={e3(_RaVqX*l8DT2FW!lBi{3 z2TN583W16P(1HlsKbnAQOW(+dJm6nZ0Y6So!@s(exD<%E?oT1DhV>)sw&W4kV(&b( zX)hU`Mmgg=fV{K^j;93fy}l=4RE8!KslOD3#~nzxaCufB_~(m^(F z%x?KMMepEGlnj}lkiymMz~Sz~3!iShJB#ro)f8_wwG#ExJ5R?U{f2Bt!)DEn>%v^o z(e$!H^=gh}uD}qe?5Zxz*T^(wq9Fi<_#Xr5&?=~%)JgU1LDuokGU3xvK&)%b)~n8&(o zYH(lpx^+v4M&XNxP}$Yh)y~jM*uKBzXqsxtsA#r|-#)kCF5j7oPz2V*dr=4!^rWJZ zB_VnGwEB&K>0eQ8ZSB7vJ;7+6t(m|KVq`L3!gcNf`n0zfyd5+qM9{XRl3{+Z)D^9L zttaFUeGJ}w?eSl6;dEI+I(3BVXk@+4G$)O|sIlo~Dk|w^JUT?5sr~&h?dGei)R|0} zx`A|6F7Bxiw2Tns#gyIM-D(BWj{;gqIBnwX0pOM3*Z-Sj1U(S}4a__mS^q$KF-}1- z$b>Y)@7H}Apr4T<6*V=;tGMFAD}XS@_BiC%FSsGVZ#1T39fiKjyK_DKFy5aI4V=AR zUx&`?If652)?*me(lf;aYH%q49~lY3AUazN1V@|g4c3|DqoI+kuE^qHe7HqX$i++l zz+vG(F-;$-A#m--gu!8ZAp5L0AHX6O_HIdH&{m2n1qYA=)%bIgAW-t0qQdMA`QMKv zz>j@zBDq}Q5MYg(0vLgNTu9jD!jKEjho9u7O)(}b6AfLf{?Zd&}|bGt9av@klD;@LC)%F zQ)@mD>xq^HR5Oe6r#i|H(?XU5+H$6lv1`XY^9vd;2}1tyMKg2rChNUU-76n_z=Y$Hm5%K&xN!8fdCKUsnMP-ZlW)8ANPM=#@#kgu=;CS&A5(ygWH`vhgi`Zp6xn z&4Ys~&uxL9`cO71MI;Cze)KemLas1#vp zg99muAygDo$D(t4p4nu};?t4Dx02~Yn-3S83k=V;+L>2j&vyb2`RzbC4z#-dDg{ zXwzjj0X*|wZe<)h|1)`4pf5^)eZJT6!j;8Egi%8)dsX7^?SLDVo&lxAA7DAP2@0G1 zM1(G8`6hJPpaN;*jQk!s0J(bcz#T?$Ptc}P8-Rqm#}Kc9 z--Eu)U=M1G6{%z2K-W5oO9DAy|5XGAQmx(7H46A(D&%gEVkYpIrCtI7l8U#!QtI)tH~1)&jA7Wf0m_;WTk;;^7RV%)y`_s#;@cW6;a=FzO4` zU@m(7;+ji`DoA&YjVS^HmGZGtJqwFMDFf4Il%X)dNPNv_We~<>g)458%%racs0Srg z8f9t_#D4LA))G;$96a@maHT;Mpiu!GtpWh?xJT^|u+G$9uc4?l`Ez-g3PjDXI8r9M zAHiG(*!e@d0v8@y*_jYO?X&-9*!|PNyYtUditFSA0XxlHl(6}lb%BYMsg6$C_?GW# z7a~(FMSZO|7G&LYP8{QMb0XJxg0b(H_^Y6jk)eqDN@Dv1XRrM^rv7i=%m?bmd;C1E zXde~|O-Pz<_?mE;r*JMge*|Prh>3d|c->vmD16Zo_gVJ?&(V8( z!FbBXNER;S@Vm-+eO&z!Sz}^qBVhsh(aElp2pSJId`a+xu72SRuCG(F1lqqslkUJW zZMu$hXigvk@~pb(IxTpEwJPGi)&Gkz~{N+w3? z&%YN*GyI?YOdYK)mj{#uxi=nKIJG;AO2ew9-rVv$R?C7;@wu`D} zxWSxJ_SHo}K>@NEYN(Hn9Z_FmK*XAE5YgDez}!v4T6UCU)aHYkIAe8nbsQJkq5GeM zW>*D*P!;uh%)P1plPiKJZZ=|EZ8BbTTD+Z`Ed__1HHS>B1)?fVo?m2^&;4e3`(Bem z@HL$MkEkD(8yX%&_wZHS09tJ}LCJpZYSf{{UZ@z+B}Z&`L+~eEud2#+-F`mzuJzME z;$=32R8r(~v380cif1u_=M*K%xz+QboUuh)gy&hUPOXdvbz-r({0!9|28Fgih%9Sy z%>cNP@oI7u?&{la(22uj8AXtSbNDviZc|QB>$Tu3lOsmHcXVL2LEIU;Vh zKivwx4ki`wkk)JG(F=8Z&CtPBF}P)jTKHW?omROzYpQZNiV$T%x5c|1>U-2WO8kAo z;S2tvm>4qpe*s=GrGEs{<6+5u?{NAC0(bM`m>s&inw!1y*(z1Bl)rU~ZXrJ={5~Ga zw%{}TWUeB)ku8oUF4X(BK2@tl5E0XEZW}qJb~0YNs~7$E_;1O&=8_}E{P;Wys6YY_9WTtD5NkpiE_&=T@|#x@*Ws8o+x{GU2UI8 z9Eo}yZ0bF8a9-Ce$dk*$sR8TyL-a)%5FZaezP#iqN`r(7XY23f3y%g%IGuO&KK~hy z3r`-wpHAYZ%Rm3SKD(}3QowmuMj!J^J2gd)@#pBB)1%>C5$5EP;Y_@RPx(J0=r~UZ zXyNU%_5-Ny}0BtmnZsqbobiy`LZS?Fk)9RdQEw3p@JnV{&}P zoTt@Og-VY$YJWH+TCaY0$ef7$H?mKQ5%m_^wJP5T{uY&_vcqg#CVHU5HNtk}DpIB~ zJfikxk3PlbFrh}nF2v16)x*V;URmRMOX%UkrJYsR?S@f>EH0EQe#X7_itcr5_)#yQ zAn1c+u~Wz=XONSKvYcFKnD{B>#(FH_Nr+k22n~4tdwC04q7R6t(;hbar+3f4@j50zv42r9`Mesw9#M6X zL7oA7FGXk@3bdozydd^oSjJ3aFT!KwZgTq=Rl*d&-d~L}ne7JmSDvt?yt%05u)8rJ zurM~jV*aCN6|>9W*kt#jnS4}O>Mu5F1RQhQ)ozX9^!cOr9Q)hD3|tH&N%wbVwnoY9 zQs>t%B|Z&X1HholtA}y>mvb11CcI)?pZ)WQWeFhTN=gayv|(VzOHIf|$Fx@k)@@g$ z+Y|lVs?sqmVdcVHuyv~aLhX9fW7&%1BQJ-4>|QyZsJVIrDnk}Wec$4;IZgv^t6MkK zX1F3+J><3DRFVAdWfTfR#G-=vd>ww|8EQMzu(swZ=HCBf>2XYK_NzkK0IcS?V( z0v~_Ka6C&~T7Erq;j_C~{J!zkt&{Aijbg_V-)2Mp4~yaq;ueQ}hmG3Z=D#cUOyiLu z70O;i#-D>QRRf(q$ay$PJlZe~HU739Vmae=$L`)^KXj@lK&MlaXz|uj^RUeDAM2VBvE7TsH$IM5#kp6@_u^>Gw=DtU1mi!=Dh@f%k zyJtZe$l<2Uq<;i%8?pEYMnwzkG13{8=%d8?lwEt|E@g3WvBQd)5wP)Kr%=lEs%2 zXx|jKgejc!QKBF(od~N~QHY9JVz>A4Cp@HD51+J*hmhxUSTglL*YdD%pn^T=|EB}RSJ-Cz`GW#xBI! zYZc?11}C_)t_lC|Hc+35ifwjfj}*WE-)$^}g4=k_d&>0R5de~<03z=IfmFr+cVmx0 zIRD*gOaEaX`sYKMhG2co=d@jK`|z{X9q3ehyb(#G@5tB8p$nnsh~6UhnjnRc$~Xi8 zAs8B2H^}}SO!30{B!K2f5X$@WQgFr{`@fL{gQ)`&#QMN)!~eU`2Y?D+V?rY4^54;Z z`0GyqE`#Cm5{Q=mef}UzeIS(nlvY=vC8i|*2ZNxA&w?gw3^4KS_gWV{f?O@rZhn># zOXlu)ZcqEa^8xH8fxh@i2fZA+dcyj@|N3v>`KNiVg%CB;O6jm+CvW&^FNi;!sg?oK zAEyU^w72btATnjeH?I&m!bYPBK7GeZyvUv5zXQ@FBlyOe z0GdKF0N!Nn?JK`HtaMbxt&#eDOpwdC4Dw$~4@?W>6A*aCA^fCg8EzDzS`bFhCN8M_ zkpIw_s6+a(bwT?Crgbz4cDOn#ek6USTSRaB=vAy~)FCmZIJ&;~yh@IiEV0vzQcT>X zf8R_HA#ccxUa3liIAMyo%`MZRAE1@IML+c3p@+`{*-*Rrx6>qIV}|1ED%^sF{5(3JUV9nRt0N^?WWJM~DH|1ZpU}p8xO0(n7mV?@H5)5i4ag zsJj$)ZL3GVnf&j43HS+K?BpfMYr+5f52p-(z{Xtpo+$itU*Oi_mp~eykoh84=Kub) r^?)t3GFu1zy9$GsJ0Y{{9_cnK$N&6iV>9qjKzJ#oBv~zP68QfC2l@U& literal 28011 zcmb?>V|1lW)9#LKO>En?ZQI7gww;M>CllM4U}AH^i8--z_B_vrcdfI|zmp$Xz3=Yo z?yBytx(ZQB3X%x0IIsW!06|(xOa%Y{#RRrrph1EEo%u4=007tyYf(`pX;D!kC1*zq zYddoQKnkHY%|i`s4Qq5;&bt^QQQ^1;4jftP7zDipwg~l`kQH+{I7$O?3~hZ;q*PO3 zj0!j{QA~8CiFkwYHn7pjfN59pMo~Y^-*>z5>F+r0bL+M3aWw7cahU4@LUd&c7lvJr z3djU)pdy)#XeDCE3-2_O_-rm<)eEJAyPE70p z0AZvP(wlS&6oaFQ27@Tk!+?S>kq+H@bScbO#_A}jM1$Nc3Ymwf*p%|Om(mb~c-PJ>80GLxVW<`&4(V(6dk2o|9KjaIDu&4_OT8 zZ;tL7!dHyj;$VxnNO5K7?B9!k<=AN~h;cXLkwrm%j1VbQ;u$bSWcnH6^b5)v`fIQc zBJ>U`A{;IdpB(_+a5r89o#%GxAV$$5dT+*vm;zuXYXsHFuGTvK0Z~E7JzsgY60H?Wf9suO95Ir?!JyFs8r*l^lrK zLHESYfy3(0-jQiXbO_(rVZAng;-CvN9Nec0Bq>}}hNQwp$3KXH75!2CO_^lTMh&G3 zR6W8**sbVAiBnl+k#^CN4O3oxVV1)R$rYIgXG)k+v`!+upn48!*8bSwPW}$r8~z<% zbl}*ihUxMPrUeKEL=&EBWM%)NX}syQv4g1u6Jfnp4bL+CSv11ni19)_Z3Fo#{;Gx@ zr#pZ*7JqctDT|9Xhmrw)bR_C)SUZg#>m}PIqyzXPjN;(gKK>t&C(mv`enMS8=bqxO z<4*oACA17US}-qhY(IGoQ3~n<3J=T*G-)VI5zAccv2-j+8OjO@Z4~{m%W%Z72N~*k z;%|v8MH%v1a`kVBNeqeMQu*UFX0nZp+(C52jwDx<9?4fp+sahvs<0ubiYTfa$~aV( z)Ctu2=oPT7uyhdL2XVr1!Z^ZkVe2qpqA8-~e*B2;h{lMP#Hge0RUx6uqf5q{>V0~j%^z@CIC0Vw1?uzO)~1j=kiw9tA^ecb@#ye4#vG;|w`|Cs+V4y~ ze82S{WHNEWzK^w(gR{VC%6wiQ-IDz6vUyZo*L6j+?I4(9SYViB*w+^w&>Yb6xepi$ zObILtY!Eu?PYwDdoKcWiKrb9Zr;i!2)QoHhoNz>h)) zLwRE+F%RfE+qG=Jj_f7NqNmrV!j?{KrrYasP{5)rSqf{%H1Z5rN$om|n=m{}v!q<4>LuvhsGZDmc(74&STkF!+ zn!4~a=I6GfZsUc?kMn5#ji?RsjndS@RDAUj_3Y|ijoWI+m8Zq#<Q!pQcBFUsz3bSDG(%2=_YoKkm05;2va=#PE*o zo$Rfw7Om;*gR{xxi>F|wR*+us&#fA)*A}X_bbO)lzTtU)<@$4-OG+&19Y3GYipR>W z;gjRtU4dYuJIAp+?U?R9>;|6$EikumGUxTixv~A=f$h4KqsFW49qn1Qv-taDb7BA2 z;H9Re$~62V-HDfz`@ouR&#lkvOju5rlfcf~nt;Xj30xjlCqt(p-`6Pr%J;^JgI3yU z&3Hq%UhnSh(!PtVvzGWXhZSC^L-?G|UBCTF`M&HU#|8)eUjD0x{@i0;4=6OqQjZRg z{OyVM$3y>z_b4b^LK6Z^eid)EKUzm$e_dhrwaMt0P)59kf9H$xy)nvQeY;7h5$uge z=PTu_^(_0W+3ncB{XQ2scgBd>`^Agem-|oEwc>%GtzmYbzysEP9jz(ch{RUpmRv8r zSNXHU=j+;)&&bdyVZhiM)1QXpv#yEdXEmD1I^DRo$*sj(&2#L4~K^hS@;iiUI&2xz=i0u3GYP zJf@EJ48~@TCgu#D_D(>M3jpwW@&KFm=B~y>p7wSQE#-y0k;2sW+Wl{XNapU zKZ%yS5|OB*vpEqv12Y2)i2y7S5fPuWnFWuEn8bgP1Ap<8Sh>17@h~!acz7^)urfG0 zTQV|pb8|B?u`sf*&;v)%yLdUc8hg?^xRCxk$$!ryX6|C@Z0+P~?dU-CcV1%?M>kh~ z5|Y0Q{mgtKAcKs5dl;D+m>B*DM)}H2eT4L7r<_<2v zH3ZmM+4%k$|9|!T?~4CHsr|o{EbL7GN%$2{QD*WF4`ZH8^S3SUQixi=3&eh{?W1GyEK*B5kVsynTJofE3*q7~;;);_h?F zyXBjC&ik6T%{|lMT2)aY#AO7|1o{Oe79dfu3;kBTEA9jXYy(IGQTm~gp=X)k|LvPX z>0cnW-vURanXOc%RF<(55u5A(`j<)_W#{&}qocz}upY)*eG`{lPFuS}v(CUsq(bOV zKShbM)}9D(4j~t!Yv2pQ<8iIwUf6?r+PGY2&3!m)#~^=A!nNxzs}r(VyGD)XP?)TAZrfuKt= zvMZvhe^S1$8`%RdTF^rnK!2ehv2fz6qJHcm^4=DYA1Jdn)w$`L=5oBgqt0o$CN2KgrlB>geO5zScU(Sy z%n7SNSoK)LMCS)HvC}gWd9dYn19OsY8(FOJln<(}g8;;TPjw09=_o%SCpUMuWXqd8 zJr4wT@Qxb1Xz+_)XfRifJ#>RFcj7e%WDylf9be3ajC<<~9cFUu>M$Bav|j<>jTH3k z&VXY6k^XxH)l7^T3N;6_SE&#h_*+m_Rf(mUL0qu3QW*`pi*HnL4m5J{6^^n(3D)Z% z)i7mLWN#@~NT#W>iL?X8!OZGnaa^u`DNm$$BL;2m4w-#neSN*{%J&2Ue(#5KY`itN zBYKhBpl?#^=s#ZmK5cw(ka0OFaZF=TE$LE>AIy2PewONL$lG#;#~=Irg`y9Ww2G89 zfCZS8P-&;Q&}sg}dW@GKX(J38E-^*IjC3sNFW6<4JSlZ7BE8a~1CvZ)OBO9%^(~Cp{1xP~Hx7htu&O~cB;97KJRvuZ_aVtoeo(e<%^lUpO-F&i zoUI}xtrMh>l89%%ttL+?4~nuqqFKn*D-c|%(9ROo4wgSr@{w|EXd0-bvtLmPtPvy9 zU&qdpG())DYj%w&)Lci;NZrNmTA&ddIp)%;UF8C_Q%lQWijg3GE=;Ihl0@S#OvBuq z<9n!XIW@jo_WERsgm!ssKYF-1oA@81^hV=Ah-W5#yRBZ3-aIOwJ5M=)=P+IVL**-1a6fjN9mW=L(VM8 zh6+g>itwBocCNk5od5% zBq^7`xR`8;?C_{AFp4mi6hg#l^LQd+?-tgq6J|>}3#mo+Tra_;(|1DUOczzyL3+b=PKm@lM z$`~{81f5cIxHZ4!CnGm0>1pH!^eft!LxVu>fZlm%5k8hZ;?&qw9Dk!t4b)^QN+H1s zdJw#?OiUBxSrAa7u1w%$uH|4fi_sK^tzU>me&h%p8)*0b=An|JpSl6jRl?w#K|X7t zlH{aKjMgj(am>|A690`pg0|(PK}6T;osBM*McT-;5T#QF@!i8%IJlQ4@zT|XS>0U4 znrFACZoY(1pnkqgh-88K8vA4y>fOW_CY=ZS?!rz;9oMEXo$Ew>u;iqR!NZ$UsjdLRGp}0)vU};enns9)wNfgJ1sO8e;bQ6K<<&k=KOof>>zzUSuce~_W#e(3 zi8B>3tx)=i8NsMCh?dYqsg}$YIi0MS*2KxLXB5NHg*XfS=hB&hg4l_e6=l|c`$t;} z;h^*bZSw!W?ruggiYyq~D3oI-=<~Iv>>=WKN;}TorD{#aJi+vjQ!5)AgXng;!o1Z` zDQI+xzxGQ9BS;YGlY7C}*%0A0=RpJGm&o9BV?M%L>(TE&raF-UP4#}7TFv`AU&pnL z(MY&8GEjk%wc-coaXV%6tE$ZVu5cD*XDpqNZ{|^44B=ANTAxGF5b!|KCrknUDv|}x z$q@Sw?>sD^tD&yHmq*=3fy*+1%8LN96DbBF;7tDltDyK;00%LC$Nkq|%zt5)P%<(Q z6G>n14*qLbNdKY*Ne4eYkAHy&88{F{q58Hn{c{Y*W6(Q(|4V%$Qb5J{DE-l;c;L*> z_KLJ*#3s@a+|AefO-IoDSt7+=48=|a2Ly*97`U@$X3#eeFE73zybD<*4F=>fX77Lv z(ZV=(+vX>vOx*O1%h*p91CN(0%hskBzskMznoiSvk+{R}+DDouoFxufhm+PB?o-s* zw}b!i6(>&gezg>Swr7*(Zl2x&UzfS6HzSUHd7GdB_drm(sD~&1+`Yy#<8C<;h#z!$ z3*VB0eD$zB8eq`3&=~Ofw0$Rj(uU`4DsX zd^cM^LDCR{Sj)K{`#sslCw?GMoFQS+*+W&vNU(|6^Kk5jjbrz;kP{_Jb*3>!+wPJ( z6dzDJuj#YSwqIDaoj9C-DWTVuao4Q|jAd4ilj+OMuDDg_gLW;PjY1un05>=v1wcZ$ zC|s&rX?thyy^wsvoi#+WXpnJ6MS6w!HT^6_LoXEv&x_|f@>2abZR#iYndp(998m$18(}9G~ zAu@BN59VMC?GS~|S!8tGGTd%!;E??gQLd4O1zW|bGI>B^=5SwZ)%G!UcabEnxDxnI z<&C@^Fb_LgFDGuE(mrb0+9BS*uh@U3{DhdBdA4~KphV~Q+p4YRF>QlLEhtFkv4*vw zOG)R&CZ_!8I0-5rwF(`AE_+h1ffbCpphlE+(y}IQtX7j$nzaEY37uLl0(haM&_J1B zcmIHc3iQc6A$7`-w@kOKbl~8g2gECl_n4B>;Z1j|Y7`cZ?q)ap=Ulm`Z{r{EG8$Ft zKndr^&Az^~M&L3!$xGfTdi0dNb2+c{Jicx6!fEj8jTgltd*}XcTBKRW*2d(C{*Iy2vLMGn{e7=aNyVPly#kN z(08`>{wnWt&vibkSfs5kbT=m3ZUk_*B5>^fI4AC0?m?uxiJFPkW^|O>2ngy%cenvG z16>ox*QA^ui}ccL+I|X-_YJz2`^GwFE7ZL@HE~^}6ppiGunHUK-an1e^BJwiVzFu! z1w-e5p4Ic*Se%&}9v+na>h>e2p1r-@K?i0VtBW^Z*p^}+Zs-C*CAnp{=Mn#Hi zVSb4}UsqRC4@9oIIn<^Q3my`IEmF=5<`X2vE(EEQL5BaD^#!PyNCMlrfZWegK=)G( zypZ`+$#AbplBgNPvq8CosHcQ$k6tOT>G=T)u^UD)s|TKlZa<{Q+THxH-VLw`DZ-0( zPh{p*O}#R*8}mNE4m6Js-1np4RCX*OqiT`z&vz!CE65(l{gOD%|9PwZZ1L~6^Ik7Y^_+# zp{0ovWMB>?Z4rvgreW8Y}r21925(*o~0Kji03ETSVlL_ z|Erk2PymL|3wGxTCWa4PRqo}X<4OEE1#hf>w4V0Z=R$5!SXOsz3OMS! zHeHNvL_LfdrBsn6D)&qTt)1%+M9K+8SB)25aS8`%<{LXlP}RXr`q=LzNt<0Xk&jYx zo!*za&~V-JelmHiP4mM&n0DGuJFqfENF`?e(WHBcO?g4Wj!`;@Xit>@~!mZ6hdz`4F}U!lm%nVKzep8HllU!300foca*~7kIsW@*Ih4LaA*Z+Ni?FgezoTP*u@Sx+=n#$x zNlc?W;r{p0LKz>i614HqTYITY!LRL27P{MnreyJt!};aa%RN!Ci*W^S)qulUSWJ=KB*IHCyi)@ ze}>q5V`ZccXavI*EKZ8d4oNW^oDyIs{jTjPn)B0;^68jB1dY^SCpvyh%C zBW@kF7lW+$H*svSqjT$W!FS$`mh^V_yJ2!BO?xc_RWMb;+(Sf+J}35$TKc>Uzlk?6 zukK#eyOJ2c(Jk}lwf-^2fe`XNy*Z$rEd=9ilBt2*xD-%6cWClF)GG>XeeO73^Najq zZ8YYHgFpw%{il@Y#kYOb@i31 z%Gb|HT+9$VB+P!d?F&G{M$;hB_cR&^o?3AE(8Iw~KkqgnpD-g=78<2pwfr#Jxi%e+ z^IP7>hi61_c>X%!@orrmV-;)T z*X9pwa;e2gVtnHGu5CcZUD=-)n8xpmo7kCi{Rs{hZ~?Rp+8TE#c1xTxKXb!+T>1#< z_E@Kgb0w~7N&wk>HLc1uT~^=M@w4qby~{w3BlSp!v^5sEC=m zo{^lNRgVn^)ZIRTT`G7OIFJhH@~8k2v1pTcH>y^x61`K09zA_+dwV;BekaqH*y&u} ziHV8NGsDks^_NSYyHNkYG@=;j*#XbUIbGq>?8rX_(8u~}!-@09BqI?93;mJCzb=#O zd3%0wGJhwM0AgV(Mn**$85vpGWQl`%1t83nBt{htg8|~fAPOQSEv=_|8a~f!B6fCl zB4pRo#qrI}ZWz6BOP#_WtbXQc^@a8vHZ|1xV z)cop*Jf`Gk%ucp;xutoQ$^oJC0z!7@8Yc3;`dtlNA&(E{0gPt>u|3ylT%=kU?1d9! z0F&YK(r@>|hx!{FFy{8(gUP@LhlW<_43X%3{rsAmn%W$+n?VPePo(|o_3QF^ zyxf6P*3d{Jo@%Hv6bC4p`C*XA{f&OI5rAa0@;Lv7j+1~xKnO#_z@3St%k;eZ5dIVj zExC1CRj3YVT3NwWDUPSEJn8!Mkx0OcUws~@obJ6Sf#_3>P>*X`i|DjgIQRKsWorwH$=QW2}vgm;=X>CaZz zBwjbi@+XC+I@i;;<39@&gTa2p^5Do=e?638qU+0FyjsCO_Ww}GXEpmgIxwSiRq7J~ z{g5sz2|bwDKsI6kRL4vfJ1r%p<@L@0F)?wGY<-Eee2pwC(Y2dlWV3frR^biH0I0+z zOCcpN8bTztvtk$!7Zs(ftjrzo;oIi$EA)hG{sXyUA=LNf?j%3pqwpN6mBfDOIyySK zJ_VvAgm|E}q*e!H!X6pa=i&VMc1Cd518yJbOBh=P9Yq>9_;7Y&m&X+j4vtE0`lNd0 zFDm#npW#RpPKV7vxai8jTHpoWe9%dnNN23mZi2Kq+TUmWx;OlwBd^Cu47w)|%{4GN zIcbDUQ!3U#w^^1E@ zb06#;PL#M%EDVZl1lk5Hh(JCPDbkmSK<;x$)6JwA@BV$I5?1aWe(!t8sGJ> z@$nIk3&YO~;WFzjRN5^>mnzn$S*y&=Oy23&WRQZbJX zUp@J%yi(YvXub{i^n2NIohGA9L$IBKrkB%$47!-DvthUBq-uFdMShlXh6*0>N27b_lRG_(FCF+(+NIq8C zkCuTQ#^5~2pqZJO^;XDfMgs>SPRMC@OL|HaMXGZlvmePCT;b+4G{wXG0!>r&pacOT z$5RqL9ZmWSDKu2wvDeRQt~x?Abydilwu8J4-s|w?xj#;gVs$tJ(V&P=R*ibgK)4wAM{$ongINB4!`5;)p9ElSsqIPD+jTe?Cw7iH)801 zRJtz#{ZzlZu6kczulkIC7>UZX!PIuIy{x8tby#yc8jg?4$I7+`&Z;Si6~+*V@M3A7 zv$3u5Ev=ee#25A$VJuTZKt`{f5`-jPR)N`@2?}vz&vrmiG(PFI@3Q2>iO^vo*sb(IVn${7rU-AX8>aXvO5At2gfIQ*gUH6LkE@7 zI$k2x2qDti+Ep{Bm%Ez)?twj?ix9XgSPC%>g=y`?%klQ+CeNH)9I~2SS3fBr(7yTQX4oMu7lsDcp%1SCHGkkRw3xWKa-K{39A>=mw|2d3 zw0U%+o<{vY^#0$VsFG<7rD_EU8QD#L4X!(svPB-EfTaQK;YaY`uG$Y`%uGnlw9z zy#=w{p4#S9G?1=j-J^_%5LCyuH)ac@3RH5pvl<+|sx9kBCl`FaS&Lz=ERF6NRmkTQ zNAlGZq8l985YA2!!3drC$a-H|S{#?WH{aqVeTZ+*xi-1qSs_M&NG`Lcgn%0M{c>{3 z5|)qR#_{Ivplp3LV5;3zW~HIS>|kd#7WaE|-XY{@ie*yPvrN!OM$teft}*yLG%|)$ z)ey_=jNfx}re+RI4SX}UKGKqO)bC&$8;IrQ!VuJ-Bs-`m<1|j0NXR4c+ zr#bbz-$myZD_)XuDK^SGVJnK(t5ma8#5Hp@JXKA3*V&?hQrri#>(CSa#t?>VC?*&W zoTfJyL0(ldKKgFay@eq@dizuPnchQIQYakkQyY^(u+}6~XS8pr*UJ9HF|{q(e@z0L{Jw-AW^&}pD@GeRf$!c`k{eX z%jpAyZXmd+-v8x<($LUQyF%J>V@GR)M*Q{N2qoXz;I1L|AM8KUZ-bl3zUXQ!U4~II zBpcAiuoy*A1aVMaC#cfc8!bB#fYjo|*ieB^`p(nVFP6p?xdgdU0L=IBEn;}P#`_zo z7>J8!1i^xG;_1ha=)nj|ex99#ZUm>Jc3~B=YLLR2AO*c&%hULN!NgQo_8yVmHXZ!X zQ_B^-Z}agqkqiwBkBA5lha_AR=qabps9F<5B0gH~H}iCcm8BIK#Rq>KxEZIV{DS2Y zjLc))MjD-s4E->anv3DK$#`yToQQuspQ>dpGC851il95v15?M8=SW-h_7bWzrL{pqTNEF`Kj{qNeb~z9$wlx3rdjxWA#;^jG%zv*o)nNRcsQAvoRkD&FaiW0 zKPOc6YipJwDYoS^h1U~KvyxBLABw#-vIV8vrs+9gD}#KnU5f(r=ijVv@of(iD&Jjqe+ATm4BzDz# z!QzY703T%rZ9xgNuBO%ePDM=}77#1{X_OFr*P&JdpekLSpT+t|r!r*_sA;)b2tzZm1yqfZI$`JN` znl^M$dUn_(Qaq7VVp7)#Q?^E!x5AfZRdtT%T26cBM$2b>wL6KIq~vB|`FP>Px*Ki> zA@`}^&5B>;a4kz1^Hr(U&w!Z5Q>44^JwZFlBe>wVR$^mFykinoWm3XmU-61>@h8W3 zhlY-!GsZ|@!~}0HOr?Gy>w<*$8mPS{zQ`F z>t*h8Z0@0%x+Mv<7SG8Sq;l`~ercV^VLx1m)&}WTG=pCfggM66b;=o|8`u*2I!C8N zLikKAaBPr_i_qw@)z9T!4w}}=;+gOvi<5DJ@Un-u<0fX6U!5M2L}kaO{c<9D>x4CK zQ4+*$NP_dt!GnNUa5{?}&H*n&FrYUE?)iw4G|QA-7b*yoh1jarT>Z0p#`*qN_xy6s z4Zl$BOR|AYmG^a?;XP&@%hc9V-o-_SmC8YAV)WceNkJ)T4dwCBDRl;wb2PVcxk$}j z)kx5usQ*vq9a-U;Z4isfP*K?MrQI5j+;kqDgi<~1JWH-i38R)p26J()X*TmX2oHgU z@%RobUkWi~`f3i!c}lC`eJ%W=iri{iG=oXHfZPY8Ug&@#q5Ihun}-^27XLwLDNfKI zKYlQsdwzbrXE2+-z0-aYl<1n*=-ZOczEOl0w$&7wXye}-(Z!&S7vqMit3gvQm3ykI zdBPA97g-o#XfS7w9>7>d)4>01oUQ(fTv~R9kF`sYH8iWMrQC2W3+opcVz^i*z-K_M zV#ah#2KlzB^%)c90E6LQl5;+5?(@Dal!Rl>9*0kmwjnId9S*BJJCv%6*f}Tm#EL25 zhoZrVW5_nh4D{@Y0d0$u*_@2@^cNeBef%Dmcy6zaY*b$K6n}DW3Fix>5%5uyRVA@M zlt`1jyK|q6qSQ{`h{#Pp7`LOAvnx@oIX_&t5l~7YGHp&^}pfG=WvFKUSw(uw#;i#*&1tHBlir{D! zj(l5KoQi2OLA-d}K1A%2&d3W&qQ$~7b;W_BWT4!D28VgB)fQCmv8!;owQES0s)Mz@3He z1$v~75iQ}+?|()KSuG~5z@hX?_W!!Pqt<7bbtgbqbJdPxvM_|q##SLFW*F3l7+^89 zxTvPDpW8@?&_Xhk$DhsZa(Fb8Ut3e7ryrG!3ABfcA;8q$ePBovPI!M99G208>CY5u z1lsV*Y>c3cj4$nq$cPK?;qdi@O3t(@Dk>%p4hFzThWFiZAYhM61dj>y{%`W{{b^?t zk_fPmgoc)^-{sC;08e4>0M5tHZwxTX8KMQ03tB25c&b~$h3e9km)>yX#6Kv9(%&qBYy#01B zKHbz@u_OXKd`aOqUXRP*uC-EOH0?Q%k)-;R>DATtfB-I(oJY`{`0YL$%3 zYUrI7EyJ)1H|PT59Xt19B{Dpmksm+S)=*Y(?be&e2a8UtqL=4d9H@8HhWuMPcTtjA zeID!(Bf#7rfxdb@u^PNM99he>bPs(2mqipX7_%4zFy|&~KAy=Jj7aSQR)jg>M|O4j z;M09=Y$gRv5KiZCgXsk0!Y_;YzCAU#u{lgq{eHOEaPm|*J6=I4IX*kHwX$*z&D!IU z!3CY26Y>Bm*zC`@BS9u868lqQKQb|Zf~rBUZ%5+O?ob2=2Zs}D2AKepapj0Oj-AMk z{g1IRD^M4^{olJP?8u(aEcx8Vdjla*wk7+Lu|l$>fT0va|7VC!+*L7923K+MBwQRF z9gf3oUubAWRZMdj+tZJ$&ySgj>h;QyBJe2;$9kBce0)02z9-JD!C!pxJFiC#l8QS5 zhQM1<8`uAGf42K3R*z>S*dN{8>Pc6}jT{J5SgBe*X#KM#bAWjd=&Q7}wtmlDils*2 zAfBSaZTg#}U55o}4@mSD4|}G~_+dlKUcXsJ3b+DE*`}qTfn|USfpDW#$SGKD9mIG9 zmbLXjFhj)##u6tcCK}n=!pZn}dIn}7H5FvO4wW+)KnE3z zW9$qs5T*g1ohU&R6$8wKpn_yD8`c|vbXN>MlRG7$=a|nQE(PX6Z`tn3h(!jz0}zOt zJ2Y6T0B%iWcUQmw?UVsT{UdTg=FS)xG;-nJeVr?rhX36cpsZFzR1|n5T7O_HsareJ zX32vPXAgwI;bB^Gd#75y=|fZ`GlM9$+cuKh2(Kc~M*~B!vi&h$!00Sa5Rj=M;@*Ni zU>FR>vO_lNyIQ)KpG=fYV>M)ku@b=u6utKxzp=!a5J;FLapP;qK$a;unfT6Dk53FH zJ*W*QXIt796;Y*vx~p11=uutOwbfgjle6>}YreWpSrMNo3t-t^G-SLf_&PD@^YinL zNW}iOG4XO5y>3ZUI6Du;iJA&UGwADFmeHOmrum*`Q0g$WR5SqZHZMJ$vIKGhT^ z45(aAMY z#2_Fb2nBq!qM_7@Lc9*RHy76g6VIEqSl6t6#ggz;BK{EkXFr7XfcCx?7IZ?)eLo5#U92=B!9SenJ=9Y`^^;c*XD@ZjE`MQ0cYdz@91 z-A9gym7Pw_zc^~;rfFjqWd1i05tm1>w)BJiJR9PqEP_x&F)fpl>(b<0%a=W5vN?T{ zdL5RcgwA^#RUMy{w8=|V3p2cUl&ZR&qPM#GHXXe`W#KmBIDdDEf>Fke^RuwmCubkzV^thK{btJ2eDSh+Itw&g!jtBe0H^q?=yA=Rr;blWWkR-tBC8 zDqoNgk*kUjv!O#ddsNq3a^qQ|m!7^3ScWK@P%k={_JTM*``y|39mVF_R`6oM+A(2q zL_dO}T!^OQ(Ii;v45m@BHZJ_`eyOAo#|mKd)(WhaQ$ig(c##aBhpJKhxG##f*Rd|sGDZbaTWShm5okvb{Zn35OT<%8vc=(>J2(27GC@>qvn~bmTlpNT= zz{`{S@O+|HF5KYk>agD;@Xci62@Xz~4Pan+ThB2+c1 zh=?T5uwy<7*;TuqPJF3(>`BuqgR@7Wzp2}!v3VZ4ey{L*Hm_t((sU^6W)P>z+RmO~S-L1BqWTHl zbGO&^as2YktB+ubo#aM|h#^YFd#Iwu2mOtZi9U8(>w2Xc(RYrhIx1au@JDRHHim}uv{V68<>R9WdT->Lo1YkBv9G$mumQ` z`hoi{6IKdhl_bJP-bPkPiJpi{68!n|Cj<&9JcJMCf*fwM{E-Vfx1ow-bdX6)!@!2T zfH{#U!bCO8Wyx7t=#9?BZC*6V#Vxg^*xoqoBX`OIjj!llo8#rj))oOYx408lUu$#3 zmp#wbCi~#%IJRlVGnYB%iyvlj;7@^U+*3e zrQ&>#&10Xit2<8vAxzfR3Oz-nyI8gTF~L|wG*7fHAzIGwVzk3;3-MfZyBi)7jhNS~ z`r5lz=JUQl3SYN4m6F^Cz_z6hyi^18OWXBnSPE*PfHtHhL*%2MJfMsD_1C|?r&f`k z3xuK^&lT{7WY>pM7olibAo^)zKX!zE3Xq+X*sv_VI8nWhC#NKkKryo?+ftv=|JU;Otrdc3xvqE%F}v&e2hr2DEMbwbT;|GPz$9lkTNE)=!o>Bchf!A^DxGrZpM#6^6C4{aft*gxE{(4~#H@ zM0+%;R0=I-Ke(Wh6fikIsna&*(Bu)$D5>yRR)rVn4uNVaL~z&=!i*HjvVdnIh&8f< zuXf0drf4ov7-7srX1ef{W01^#--w8g<(G{XRt#Ci)Fg|b=A60HXN z-yJJw^ab;i4DQ37(MPbF69n~re&G7kGm+0`u_vALW8c}>6(Hp$xpD;z&+E(52TKu{ z;&ISFO--Nc@_b9C9Km~aETO|Mi^Zd;J2qvpmHiEse--ReKEvyDABi#7@I9|=VT104 zpYWcfysF$Dmj8eWHa^VXodw;EQ8xOH-N4Ccy)3yL&%dNFp>U;+gw!z($E}Du5ysrO z3$MSQtmk?;?9Z%Y6XpYFb_@Al$Trz90PZNqedsFyp_YA(V~pn6Eox9sG2c2uZuYX8 zm#&$i9eZ2MXoLd*;%s>Q-Gt42V=2LB@?ra0fP2fW1y_Z~6@YCI@BF1STmk}LYEE#h z8^etLnePLb;oSK_%+IeM*-%+Y-40C4UjoZLfhi!lI1wWvP{s`8iZ=Ks^ti~8Af(K- z$~>us6JvDHo>V2vFJW&lXLUPcC}oR;mFzIu($hzOtYYiF51CjYEJ*rbQ9BRv*;^)j zT5Fy(f{^VT{Jz6tPP*Isz<-)-_w;;qbE*Oc)r>X^U$P>81`fJ>ihwmdb{~%4FE+V| zoBmqkox+DDRTLx54l!-s<0NAHrZm$}esYnsS>BM&xYdV?DHQ7&YTSF^dFruJYZ(w4 znU6QrU)Vgmr>$Uje9`|Q*8k(Xm#}_Y_ z+Pkpbo5V>#MMDF#W(*p(=wlE0&3Q*jyxml$x#x5@u!#xu8>!IyIeEDwIJx}FV1wPu z;JIYN6$PcCz)_2%=w>hS5oH@5$_upRgI3^spwNmzquXTusA2m2RJ>C@pQ3CR4nbPx zpKA$EfC$ClrNzx~VL<~bv)dN#4+pmRuy4!2Dru0QWl6NcMHU1O|C$G<`8^!jqs zrTM)*G1JXDdw4`(!)WawNxH=3HOMKq?ozhfhw7uA8}d5s2!&W|v`FBP{`P!?nGsSf z@%cRxe<`3L$n^fCH<#rmK4=Iym9dzeeB_hM$M!y?+ZdY5k}`If=sG2_!5q^>JnTxR zz7jxr&VB9T;id)_;iIX@neY7)x1+);V7~QDYJ0!%joA$O7-Lt-rtoNrW+Pn`9YwVZ z>Wxbd1I8EaoVQ{p%}#BhadVnYuHk<6@M%t@Ane(lm%Xutd6D3-$O!%h<0l+egXBK# zzUOH^@PJH5R_*e^XbIhBQWVn17oHpPcLVxrQQ7BYi%`ierrOA z;BnbKlHQZ9kEA2)=Zf_?S#}e!y>-2a+j9>JX_XU2uaFpV+5S&sUmX=k_ifpD-Bo>Wox11L zJ?HGbdq;kz1qn-_X&2dg5R6Ktb-C%9tWd;HR#6nrum3S$Yn;?3+^Sxb8Rumf+Il!4 z`e6)_xvShU661Xi7rd`eER{TmTCLA1 zFO3v{oMlzD3a6eXWlF z^-@NQo)Jmtz55^H6VJWR!zcw0GJ>nTjp~#yoV}a4q86Rxfd;c{9xId+f=C&8EJ_>% zUQzt}GxWhN3biI?@CwF}&yS(RI%V&N6YX$t`1((VnGufn;!oT8zq_0$$axnO7JP8!^*ZY!vKz6ohGI)Zn3yV1_s^u@(D%DEG$@qKqLRdO7SD7v1&{fKlRQg zA4hw6%yEL(zLC*GaX{H_DCKQ>d*;&RAYxAR-dm*!_IeB{cKOn7OUPdPj3q-e7ModV z37Pu!gMaX+v7zZ#cAni-)R+iVZ+|~#@7&f149>ucF#PKlOWPT)Ma@w4@v`ar&FNLN zUM%8N>+1XWazi%InyZ(eA$!h(4R(FxV%~hZ>r0v^>k9=)Z~?D)21-1m$1G`d@%bdZ zQ6cFrc6KN{`X4@g;I@Po!;k9%42l~4dP42odWP!6%h0&swxFxs{L(ZY4^N)n2fbak zzw))+h6}v{0Wz}_F>*^C62wgJ@3V4yi9rD-c_wW!D9UyY_E04sAEm#2bSV`}`C;FU zf4j&Qs45K2^0!*kT!MMpnWz*yzd)UO7mD8ukB8K$>%Akgj}WvN5v`1Sn*tHiiR~iG zPR&n++$JC!NM_s#)-T7)EqzH(%P%}=KVX!{3V3&#^ZQKRZQ$DY=R5^zK7{PEP7FDe zxj3MuF1zbmMOJnXvyz)gML`<#i(jW80j&ml81OXE&kS-d+< zr?UNq9kCV!*}@C2okbK2E+jf^x80@xvo6?)Qp(75vpzkdFMCWmSV$r7TX^;JmASrc9$)9ftAUuMlC%5L)57e!n=r&wm@j|VGjPE>YtZ6oKSgX`l@PqR%M-QOL?tUp6K$z6Kq> z|5_Jj75+d*Nh#vdTA!z_7y05Wvb^N6JM4CbLI|jjZ{_j!U%QBxeQf3T68)*oC0vN# zAAeZ{q62yZxoTWEK02OP$40YEI5xOa$97IZy~YxSc_i9ZDth^NyQP$eTzc}#gX2xj zKmncFb;Mlz38m|g5*ib$zHFVu&)JotqJD!(D$R3F3eShH#Z|hsh zodjoaKLsp=((COc?oG9^a_u=<6n~@$hprZftCNwD&3Q~oQq;*~ai~Bn2(YnKoncW^pr|S z3U8OAyrS`1s>3|x3YxDWF3e1~p*zD$=C3$XivJ*}OKKm?$kNpXUru`>*_!lTNdX8< zttG`U#QS8?PDxG0T(_Z+m*8t#Frk-b#(lC$r=-r4urm7ab-3$J8aPz9GK3=VoNo3f zc16DvvRh>zzKnuGtk>sBrz&)b{Ls))`DBX2`Q}PF!eD%<(0#kb7hl+l!tCUHLXJn8 z>}bi_7PeN4FK4Pq(qk427z$TW}>i_ z(S=s0yNmtyXBeX0G8{>fSRf&xXEK=p7U}2t8fy^`N~z zG3u$MO_VL$o-rPW)Pm3Ddvn__mv3JhOXL_M!;P=M3}ZZr9JswcrEO(L7!aN+t}G9t zzL2UAP7!kT==xkyzRwzW#tGk)05V-#>|tfwvqWla8tIYj<-s7;GPm7yJhnMm0GC4M zED-k_DphjfQ7CyM4M!%R^;1pooL8@8L2X%m`>}2`=zi_xs5d_04>(tfQ{qXl>KI-{ z0~LB5Y0$Fz+AzeRR#QiZ*I|M#$F(+VNe+81xCQ4Vi3t5Jshd_=2+G9n>;NS-$2R$g zZbb#(FhTT&&Uzp*Oq&QOPBY;Vq474vzbGu=9(wti12HkC>H*Z>h`PHYA!73MHX<}F zO-n`w(QjeRz*ewB|L>%|FqRCg0#g=%%0|;f?{dL&CxT##A5A{JudKXU|AuaDZEc|Z zvM;megEAZdV^ff#wIBdW?6Dy75Vwchi{q{*aYjaz0UFn3ouamhcoqxILzL&1in{T7 zo&_SEO!0HI^J*VJO+|c_UnTN?dUUeuRQOJm-pUMLw~AhED)AcXXdSN234cfl2V0~E z3hiwdJr3h;8zcwT*<-?Ip8#(B4GkmtX`Ef^Mub8*JWjaqKJ_!bTez8ibk)-`urT(` zd*UGjT!Ifa5S3077sE`JbWVF^1`vn}_N)c4G8g&XrrENb3tm-TQd=9ZR^gp^P+EGr zWtW0t+>T<2s@;kx`TwSXr;%}NRw0XuM1n;QL`0}8FL+SjmTe%aO-%?c%8>Oci(tvm z{ht(Yw6h5;CkIuP@H1mu37~;<=N#(cO|lT^1?{BOqzHCdl_d>aO~0SzrFLz0&apHj z%cpSRq%Vi6PI!1UTxqw$A?;09IrET3Fus0i*yaC#8RF`J zD9j3^XSejl-xI))x?#oZEU`gnc^jm6JQe{44QwhYaQ_^&-=@U=>Do@DwUwlbiWZ-p zw@4I2xG6lbs(G#5GeId3n_?u7Pc~ifB=;S8FkDh@ZGpS^>VZffoH>LLzbw*J1&WZ! zmPnQP?#<1?x11@hZ_7YMCjUK3_y>Z5<3lQsCrNr>TRxkgb-Oz)*ow;iX@2b>;WW3N zpaYXaBEgoo_-_?Mu+;E;I!<+c0xzb)MgUH>wocT-M2!dO^%6s83UWR~ka@Gd>oIL1 zcM{oZ`SMRh3`T@lpL0FROfdrO$#KXN8P*m;m+%~}va&Mlol^V}QHg_Ulk{UY9*n(I z7gi1Tiz&AGrd(X)V5>Q5ApnWwYSk>;`IdBk%>}`blj4_Mqt1UKeu77qCUA@hVj?e09;tt{gl>_1llB1O(sQdWd8Y(*H^#4iatU!=L@7S$j{^e;$cW zPyHon9$CFpamxR$+^!(BWWRY2Np_lb>l$jQ<}XXQs2rVw?XD5R)icM4>OyECg0ntn zNVh-11{L{}p=&nAaFO7z?7>nwVW{}cTak!hF7z5`!%0$*TTF&>oBh&+QxKa0KV&k~F!~F_qnfh2*3H06xP!r%wW#W)UAawa%kxY&$=2^YZhKjq)dQu&D zBtR*t=K2ez_8jHzob%maAR+edSicTA;sWSS%mN`!BioBbpFtd8{l&fNRkA`fLV%ld zQ21fh<9v2``NvDyDfS^ABX7&i?^E&@3V*&>>p@biS5R5sFXC46GUf@ZR8Dx#pOgmA zvvmy}^EhuciYsz*hD~LVI1dY`elEKUY+^Eg-5B-d(2jMxI=XxOsHQghQ?yrKv$U8A zkNaci?EwiaWBxy>@7#+$qz@Ghs|Z}_m-K}tSi2UBOdHz9pT9eDP;;QtWK6sM;%Rbp z(sZV%_u*IT0n9Fp5C7`bD^5yoyD{<*pgW!Ge!R?r0wsGdIcX!j+kEgcPUw9qCz$&6 z_j)#)pW*q^h94uQcgnv~9|E8q8_AoGXMI>rsUI9(W?B*)wsk$2AbNgoW~?<0>Oa4R ziU}dF`Iv8dE{2i|fjvrn-my>V=_}Oe!hGPJLuJ#jogz!=1tkn5q{Qb7m;JkCOPR$c zT;!l*zNA4)-Hx)1ZcD6_$3;S_rdcL2URs$8f6KU$lA&cyh7DT8VQtbD0Avi<)I_02 zt?p^*>(91n%P{5gCpxHz8{EsM2kh&TIG}yyPQ+J1k#?Au_{G~RH&$9!3+P7w1_ zv=G0=EAj#{$Y30ae~5W2vJj1h=%sC?PdZ%-e1XO`1ccPd@&`pzY!NP!g7eIhE{hH7H9W74)*?NC-Ww?pIw zp#vQ~eGenzo+Ih5`%qgp<+s^=xK{{AuxLF0wRJB8OQdYo9@VO(W_lDXa)VDE^s$SL z)-fjxOZ84vTu5vLp6>2pheW4HH@Aj<31pSI zd$0zIJDzJ=p)jAs72zt_ctY+iIjU>D8y`b0efi9NYbg0>lQ`n)EIdB~8wv(mb)!Pi z)WuevK_mP!vPB#xB5^{u_18bn?-Wa=uNDRQ>@f(2Lw>>5rL-7Emsy|X#4~hX!&iC8H_Lc5(u>^%^Pm!KoGp1M{La1ybG-)?GfP(p~5Y5m`K zbcmcItl}Gf_F_ID4Xef-8nGJAk^^C(F*8xkFiJ~n>vu_7iv`_gq@f>sy3u(i>`@Iw zfod^_*+RwPvR?D0QnN3cfV&g9Udt1G?ITtrA?6Gf#KOpX1V9mPGm@S| zxFhn3s3kViF|t3(wd&Jh@(^@$Wyap*&?XuSXkIyoJyKNl3ie}93-8p$nd_xF`fCUr zD}25zfKc&0S`?ZS*Xn{7Z1;wQYG(oP7wll58DA0I4{!zAS{^`g7rBrRPvK+eJ8sDu zrT#{1se4GP+Pp2{i?49tgayzkM`Z#ap?$U8@!bO#hRv@`VQz43P0GnGFAVX3jI_9z zO3>DD>P`?CX~r=A<#MECpoS?(MM=XsLRDe^NlrG<@|WMfQ#id9A_$Hcf`W`3o`8?_ zi(5dT%VQIbye29&6&bNYH6MA5#eiCs83CQPq})ipQCJlmOLW*Kvr$({lJzV{pg;zH zqK3-x#X{ENFXC2{#jmX@&C)W}p|{o8Y+sN|DW2s%_yG_Wtkwu6=2k^v-#)bbnF7ci z34$q9Yl2PK;b%8{r$4zZDJbty;^3s_=Fa05{ElZN@4~)|GFJPXmWB(HvoNcP6Y*5`?d89Un1}kE?Tq)4l zUj){Pc2{#)=wg3A;@D$4$#mm;RcRp0s#uFrepa@igKt;|QV{?Tu(rTuaBQrl6&|(= zewM<*F(R48p&cg$W03NK?`fV9KOKjw-!bPzCP=j!6ZxNWHU20{^g(b72VykV3pA$| zfEb5-hq>0vD;WPJXuAHJlXgZnKunwXAyOdk&!@heso^7Kww!=nhaw`rD%0!JWAn_0 zkO5;*ij!X^bbpi_mGJoo2P*Tfj$J zToDr!6T5>|fKX0Hho~kU);yT+=cA1Y`}2ncEX3%(6oY|NrD~!B-^r z`tFRHYzIF@Snblvci4VyUqmiJ15`Tf3gecdCMNWaMaATkMXxh7%9$+YZMNbINfh90 zU*uVeTRY2~kt;-~exEfa(tZwGNzx!WYM@&gX)Jetf3H_(G%d*Boe{1HPG*Fh+}QSp zjO(&gdv|^vo1U|H@{CkEdO>En=C)PXgoY`GwNiGm@ut1w{)lPCA5M8%7<~IWa{2h3 z(4t?MlkoeH&-;9bgQ*=yU$?iH4I38Ll=L<=TkF%`WEs&Do|u^lc@dPH$8P9T5TPHT z2xH~hpfB$1#GS#T>z;L)QOn?)SUTCf>e$r?J=;Bx&Ph>M6tGTRv$!B_hVnW>_s$B@ z#T5em$LQ;mx*mL(c`R6I^-znCVmjW6C>j4uQk1Lsp?raqg%@5O~(`_+>F^ z`@wU~jN%f#{FNV2uc#(E3Q{NQ^B422HB}!#)H<6#c-FTvR~u5o?R6@se*RRDv+u6K zMU*aJKzd-2Y@$7a3@uL6gU7K+SS$dq_N(-tBB=U-Yo{3vX?o@oS8$MOb&I1 zab8<@ct{+#S%ILFLJ zzc5=C`SQB6l=W&?DqDLeiJ~xALuY93@=^$GMv9)ZmYSi7>GwI;Q2;saC&5$&h69z` z&6vpJLa=PIT0c!)MP2D(g%fELvFY6B-q*@IKc)^)d|hi>^VW*O2q_y#RZ5xdY0qyD zo4>pdUu>y6bGD>G4KcxtNyWzoOk^5Rw|qURyH zSs4ntqPb|poE%!>(WZ*)!QZ4|UnUprKmGPhPzN?N$nfm!?3|^o?i+G*J!-amYY=?o zfDAv;7IeSq8t9GPO<41fY3zR*Vg*#kA5-^?2(j{mDLVV!TnE#(O!V#P$gPXnQZ-y7>++5^*^;$BhPfLr1v&+j#*7d5Nj9=0p2Y+J^y}gQrapOm8b1J25!?8b zsr)m}17Z8q^OHdyp*?FxmkW4>kaF>6e-ZAxP*-XJ*O3E_0R z=2{;R-#^8Q&IY&N!!|n}Tc%}sd)g4%qFMP#Ddxe;8T7Aq)Yl6snH{jq!^b;l-_mq{{F^Tzu@Knj7 z724Xp>9NN7N>H*{Ih%dlv4IV0X7*_QbekRNT|a;>@%(Zo4vVH~0D#JPjA~O%(~-b# zN)1>kO|ha?^D6M*|wjKAaEbty1^`tD1DkJ~s>X&7|GpYOL%Q({C#ES3$ew-a1uY~y!{g*T ziNi+=>UNmao+aJ{6^xVXpA5vF4qm%bh3(;d*MY)c&W+Z;#gEHOF)kchcj4fj@QtC& z<+U5uzh{ag2H#rp-aCm0zwBjZyz(C{>_}@K==?8@if7SZ4NPDnA9}(l0UQ1ElJFygJ{~%DrwG%Ai==CF>@Dd`aMEdA!{S^ zq+x{FC>{Tsdg-xfbjtFVy?MenA}{~;ev@;ER#hmq=!<%O7;TcE1NoDol*Z^M7xXpr zR1)v~BCYH(Wup11;*P&hhf5(Ucy`1nCqzOD{GrUPr$kc!@5IX+6p{m@=i(PHamyj7tzg#sA)VLbv~u8l*wY z%k{T|0`q)p#MRNl@q=cd@A1;vmv>jN!49Xj80)*skW;N;-sDZKoQLYput(%gOef4> z#t7VXffxLdAVe8R@kQzQbsISBiS^Ikk7Mj9o4qrd^!&MJabR2NhD@Z5^ zk!qDW6nC9D+#f6nf`>quLUCyzXg{bxaA3)9Z4iCGHsaYor%`Eu=w+zz^Q3^YLPIM- zO0kvTE$ZjfB#VIqNhaP1;Kx94>Tqv3{=F!46zO2rc;N*(kz8!VHtJgMj{n>37X`ON zb4+2CUwjhE?fg{Ul!cQ-)zuyPU{embe=KmA;SE{t@%+_D8m8F%Gj|<)ZZi-o`8@vP zyY3~{4Q+k@-%*_xjy@Uf1P@u>x)nCgqYo{fn~}ja(e_e<%e$XNUGMlQcsJEZ*PN3E zxi5u8@FY=BiIGR4Ab4&f4Mr1Ty)~+W`2EFpnhuuMlYLv%w-iNgdu!o=72|1!Xjzqm zI3xQk=H>d-Yl}AzND01Arh29sqG?`dbhqx*7k%?wTS6icO}riheTcxVTf`#apO#vi zQp*jOH8=Z@QShvlu!;VP4Si~<)}&d{L&0EF z{Gt8D>B^yd{eDG2E%Bn6j05k{!%Cbk-uvJ^!yinuEAPsMvX%JkSV9MPBLaynTmh?l zAzlXtc{CQ*k@NUoEU+}Qq)i>O8Gez}5BQXj(smzbm>@9i#hO_cqs%?Aoo%+`2h(Jt zeqp=6%`wc}d4~8>?lt2rBnGm(omc~Di$Qu1Jj-?PTu3!D^#;bQin5hGe6)Vkz+P9i z)AZo!Mda%_;|vL7Y5vYESg`S3(r>z<`N+9?%DKkFIu)q3O zOT~zGrUprCamiM=h+trhC~&7`?GAAn7XudGON)<>OzgG}3x5}rlX>L_C$}-r@l`2N zlG#7&_*;yS<+joK4=n!K<*2K6sc?Z%)%<_Z*Ohw(*=2s2mtyfoj zrOo`zF}snAp|P=XJ~;P2=dQWyS6Oz-uistV07hwl_{ZyJdMrCpl0S*Kz1Q86f`Iqn zhcOkpJ5LNvOl90yYv~sjzBZ_%=Nqi|a!gY{oG`y`5)`uib$z}V|LImc-|XNnR`6Bc z$Acesj)_R}zwKo$xtVn$s|r@C7`eZHswcpEffxq(VYdPm)ZDBlgqSrr`l!r5?C?m8 zUDOJe7Hr|GnChf!72s3pk7}y+G)Y4M;}!!w7uzbe5f(L*SWPVo_4hR zajpe@HKyS5np>G?TT#2B>__fWJ*zky;Qnyvsz9J}M`B<|da?-Ygb7`f2EFc`{Dx#4 zw4}s-c*-WjP8%iq9Lp5cfFS1~9Ul2S_wTa6!T;}@{#V9-CIwhK6T9JUmBwzCumVCVA~p!W{U?;5c4 ze_R5XbDR<$6dLLWe?CH~tE;P5qx%IPVwfVSudlD5@JylFVJ)O^#)HP3{K8oN=SeZR z0Zih70lD-c8(&qROY4Q&yaWDI6Dn6a^!7a3yKjMCJS=*WZaYinyQ!AzM-w?$@eSLJ3jG_2nN>xT+% zTPSv8=O(tO$ggNnH_*it0wai IY7zdw01+G|`~Uy| diff --git a/docs/img/cluster-overview.pptx b/docs/img/cluster-overview.pptx index af3c462cd904d459a433ee6a8b97daab63e547b8..1b90d7ec5a7ae022db7f11402cd93f11c863b60c 100644 GIT binary patch literal 28133 zcmeF3bx>eSlCPn0cZbH^-D#k4cN%whcXxMaym5DThsNFA-Q6F3XEyfUdE0MxWB!}B zL~x=`5r>GZI{8&*)~}9?I1n&00000u062JvP|q#3>GsE+%qIW~H`P@?@=)@PG$eL5S)W(SXMRRg4f5^Y zJ_R8PSRqBv%KC>Gf~yZH!keSbbItH=|urphO8faM&IAIqhbo>GcJc zSY2Mh7frJJ*PTQ9=X()RT9-*#auCS* zgy_eV$x|;(YNU0B5mS}InLYMbVturz%9cl9nZJzB6QF2A-^UnG0s;Uae2kC2m7a{P zm9-tMo|UZujkAUMe>F-LQNyM^ywHO8uaG%y)SNzY5@{#RV)zrN%{WZOj5F8TxpRl> zPP(BrgK{f&aLO$`ebTdVMmF@&6U@{K*kr0`{eWxh8ONlB0+N-6p*~;_20lHW!bJt& zALI45N+a_qGVr+b0LO}pgpoOFSf`K);x8Btt!mR;di4*6!^2bSJp{&0OJ)6(b<&jx zUnr@|hHlae1hpYtbjSHXTJn3aO48a^sHq0UG<(ZtxW-3YH5#i6d!mxIj~Ut>H=j7^ zH{Lek{yGkv=n(D+ALFq9@eBXwanP~0{@pO7kF`N|(;)|Lu9kosS;^xFh)vUMoye?y zwS58_75y2K*XBm(i?;D_+1)d5pQ_dy8e=348EF9WY0#N-x^QE1JroLDDy3ADE6W!L zqq)9G7?&;|Lq*^FTUWeXVV*u?*0lPbL=!X*NBUKG&P@*;R;%l^t%ztw@e!4cxCp8R zf+wDPpR441_FU@t7tcGlpBd(GR(~}s*~U2~^J8iwzyUstLI$w5wx_kWHLx?VwAZmW zv9kP~X`_i6R=yvmo!?WYtkJ0Qwi9Ku4p$Q|S4-RjHQUO>%896@KO8GxJmSPV{r=2B3^=sF!}1N$nf;3ty5X>!5ohR5d<A?ee> z?mhKQvxe0W&N}8$Z7zqlZQgSQRO6n92|DyAaqn4uk|)TaJ{WgT()|)V!fQjBvNd0D zvNTc(G9_t2@ZBZrvpD@mb3`7*1SQ~sruLDTwkd64K9{_vw-qm1csAU(udIMA%m+}#=-$NrnB-ev007*-Px9Y5_m5Nkd+x}qpxf)jrD zGC26@X_oIK_=EW#ZjA9MkqE6ncOPxXGNX`wx;`_D*aIB;L&e9d3$Z+}gERQ~c5MAr^-gdJgGsNdK-mk47Y&pXL*#)rt=oT$G?Q|!z5Qg5y zEd{D38z!!@0nISV*7#br%8Vj(YrOGgvDpG72yFBoh|FsbQkSaHsHDfB(6Amz`qDo% zI&jb>L{KWk&wCsx1R{^9+*Zt*8$0UkzI~pBA^AFf%s~oge2uvENtS;Hu`B+kq!a?F zAdvKkOQjb9LYjjDd6~i7tG`Mxiyf7h_S7azf#1~Ul|xekwqp^~vg(e7LABQ@QQr~I zlF>fazm7NXuwro42PXvn&dmRCsz1Q_JCBNpYJu&hgBEmZ&%S~shXz;lpbAa~9UFRi{l`o)1QnLQ) zlTYKZ2?QxOLl1M0FOD9|uT0l}0F{@EGijnyDJ<5J;Wtg6t|Ox{k>u~BRCeWFmc{~n z?(w~PznzuZvWdHi=)xDcRvHN-A_;C(I?Kc9g^ZiPU`IT(W+WuRuUPz6OLP3}uQn$j z#hsOX*i8B-n;lIIoc`Z7NB^hIQu65V#ZTzGBhVLsvd8r?n4FVa#zVO6}Yo z!k#P#^!aH6i~J>1_(TvT+UTBaP%Iy&AM_6m*LE^3_A{)e!bXzMa*a!V{+cvGmPmL-m;m&S4!$) zdt6*WOuZ(Pz0QB99PGUIIXw>P73EbdA^oep^T0Jp1t0ceeb|focc0R;vat97C!d48 zv6ZdeZzp+}GDFFcR|8B0Gxrx4kq>i1vfvp|wpKj=X zXZfkBh1D_({9C&gzHw(ebU9I|YZY2SWn2^Ggd<=2D}ZXYMb^x)$dZHVO~*M|Nc|3( z-I9H}MK3PVl`|K&Q!D4^WcAjS+>vkcG_hP*DPvH}Ovb3YHnG%!ds+=~n#P#?u~Z~f z;|CM1Io<9qF0#B%u1nGB1%PhSiAO8WSP+%Hz20~sozgFA`&~Ctks@?l(uukN+4G}9HNzsnu$P|CzMJO#f~(nU|f^|3M7bf z^5?4S*QDwXl#ty3OLdC)a2I{%=hy@a&zI{+B8`GzuhgER`MLt!1+QmhBC-Zeh$IC` zk6HPZN>c=5R1DpLWyt`FiZe+lu)oEALRNb&6FNU zwgm1{gN0OmlrDKiwINhE$Z$b{%djx_2w7S2EIL<0WzR?iy*}S~>tAW996+ze_BBp)VR{Et^%Bf@6jfw0IvZcm4>g2(bpkn;T6yt-spiZ65D zG_th3yX2*$-eBwrka_8mLAPVyTOoL}q98T!9+quV#m;f#3bR`LQSHVCKu~$q^F}_9 z;AVxMy_I9wd<|CX8K7oCR~XzyHfV*D_q@f$6PUA&A1hnuF|%_IP^+h3gO@j}@dd{n z;9lk63AVOnd0GzA*=s^lnm&PST^K&(hLj0(Bud5bRd%_hqIx_&*-$-28}3gI3LOmi zfd+k+*V;1sdj?sRmG=sjJo9WM-w4ObRV3fjGwtnNBdO!nG|?=y$e(MdtjTDN9~bRzQ!VLXA~U z0mP=V?8<4oX7*-UlIR#J;#8CuM&e4sv4mYsgBpUBz!`c}%X1A6D{H+tE6_*7o(oIQ zpcl{HuhRq*fp2sRI}8EmzA?P9LsD7DK6+Z%Lb=xF;UqMw4AYbn)DnR7B_E{=ZLZSvYf zIQ>>lNx|QUk|+PAtTfnSr+HlI<;k>F)(g!~orc`&g5zo9a?1zG^mc%+xh*>pYoGpp zx+7mDY7myN5KxSx^am)xB{xmnnB43cRAG%m-SgBDmEvjp0f)F^3+=!k-F3gZ5zl^g zH*$(AHtY)4WLKtm1yp0cKe;|m|3esy@Ibi<@v+?Yg8=|g{a0(y-(uQ-T#0@g)Ar4_ znfB1ZgI%;q<6Bk`bB1jDlc{`UsYSVPcsti%b;N|^d7jUZpNIqTIFKz74sxH5Sr1_g z&TX`{4&1FG{JD@iCydqZ^Aj#?Ts+FXAAd-cftty5hkuqD=;4H==tW(>jWeR}afZ+U zpU$wW;KWWCKk+BYV491d=q)pQ%)KcvjX~p&Us*NkWQa>QgE&*)@1(i*_ZqC1oHlc> zdg922fXmpxzte?QDDT6=AGShb#9B&{10B?%_nG)I?8(0|7{&D^N>aMQ#vAQFBu2;a{aMU(`yCN=B>m3Y- z72zZttTa;1H)lr9i?L7HnLm$?b*S2z5sfQ5h=f^ab>CsPB%2cfKBjRF_3w_k%f!RNvGpen=Wy zSV%@k$QD5QQSD7{AEmpUUORu|`*xa;b&w%rM zogKxyh{n=VleiqX)ba)P%szKr(aUpE3k?nSrsZfHKm@squDROpaUnH5X#uoQ#PONO z?klVDHr6W~$HSmG3lu5!%a49AoRM zrAq(xn=x%>Miip4g#pb6_ciZ%vI4&JY;I++koZ{IJciXS70=)Gb6{ZMg__sE5nvr-m3Iuq)y2ELsJ+ z6`M{ROK>IxJcg76FMk|@6kAGIQm6WYHv8!XzzD6Aj!=3Fi(Pknnr!Awr3EtU5PuUm zhgzGU6;NX!2rsW)4TE)@z6zD!X^LQ9ibgplw+tV@CcSXAF$9vXX(J{;lS@Zx zB3Df=Ud3w;gJU=(nb@;#`8V+m5yspa#z5D>J>0oTAS?0%wz1RRuK`R(VSo_G&?wYJP59H3V~0_8WvGq=5DC@)9oB$u6` z@ypA|kHoq#N&gs!BAE3B&ps@Kn>d1d2vz}YHZmBhXGwNGpqzJRyXG&m@IY}5zdENz zn5kblS9goI-s$R6V`j`wR$*auPt7nGA^WtHtly*z(Io1H~v5;MAK@)o`q-aL+ zWwqEmBc6~}$b64~PRdJg8)N~Wc&pnUJVZQB6N66YQ>AT}(N;gSHqbZ^gcfZ*SMiLW2W=E2caGqU3$*=G@M5IzB>&(Qcq5@u z%^j0c)W*=#E-IO$hcdjO+pP@oZBxOJ`xqOus9++9S9lXmm%2FP0dWlS=aa%gbSXRI zPx;HT#ScmZO=M0kj2XpBjka8;t>A_r*Tk7Llxofdn-UP>p3ep(ZA;*yrB&OU=4PZj)9 zKVU{>gMNU#jM-Oh2g%NdH4i=m;8xIyOOJ2Cexlcy+SM(fWM+w@DWn=eg=543<2S^- zFBC}t2eO{FQNDaGx!|C?F@)&x!jalCpV>b;cK+Id-SvaZc1IX?NV&?{;lAug>wZ)- z-yK|mulgC?~jX^E*CDl@1=YAKA|vOMih3JGmkCs_Tx{4 zf9R+{7^m%dkh*`rIeAMOf7W^f{A;>6PAru6_)$?$`6w&?C0&%zaj|l+|9iUlx0~NC zC~Zp1wo7#I9_Px&H&XhY3$RGB5VYM)PL zP4A7z&s2x|6cg+e81icP!>#W}dV)h5MHs3uH!H49NV`R_C=iFBx`dp2nA)3LqrZ;E zaW<3BfdcwYeC6maqPGm(ZSKz z@r|bD`zrD!hf((oUZ{;Jo@=_HT!ZlyGE>oaQ1pEd^w>p_4O5DZ4Ak*^1+08S@5NPR zuDp4I;j*bc!i-B`h!nqdF2=MhjB&)h`r0(g{!*eLHDaSHQTC{yE72?}o60&;l2IW& zAyZP)XyvmKrQ9CVBHQP{n;)kBS%Fy3L`nk`ep{*-=e0IZ&b=&BUvR5PP674LWrmAk z&zv~;#ElZ=24yM@9C!!Y4VLnk@}V23PDvIA<_1DQC-JUKL05m?`&nE6WQXp_IA>OO z%~;2|0R;;`kfE@!REZ&^46oF!>Qa&(=Cm-Kp6l()O%0Y{riqQHQ}0}!%>G(FI%3Xd z2X**48d1|WRkTH7;R?$5CJbi5tY$g71b4Ll+)Yw;WY^#5l2sqFH+JMl^JDZdx_gRs zS|1&(=&Uo%6bt`U;?b#ZKoAI&G}(89h_0sU)#n0-FQ6#30KMnzX)JjfbDC46>A}u0h?$v$(yxkH7M8ynl{>t^M6f)?x6!Pr=kQXM1j`KB=6-IY3VrNWix->!eH6N8GmXFZ) zFJbS0_|l)k-amQNZ^Pb*@n5?VA7Srm0eC}dEiA!QZ5TK#O*Wi0(LBon2|k+O`dT;5ClmK$ z6Zqc;BzWF~Ww_YIjdmo8D!+pcVdlP^(I%Rz5b)MTsJ1Sb^Za!{d`3RErhmX?_rC!b zhTqkqW|g{r!KLkb;F}QOY}uTHe)| zw!U@A!}VOn?D3_V2Bi3VmHHt0S*9R5eR6U5Z~^&SWAh;~x{9!YP(ZACDC@}9aF`u; z6+uf5R-Gd_d~W&#IZfI$62G}>W$Y4TUb26adU7m>Gm|tC;|)f%I3K7i69MrqV#Dp; zch`h{E`xarjU0IS$Kz8QBW{{tA z*_w)9&%DzLr)#JR4GVGG4wevh&zldZ?&mh83Nne759$+&BODqe5`MzvFI3LBNs!hy z*OcQg=K%vR&$YPbS4WJ$0N?ZaO8Gm8Hy}`P9asZGWFu-ed5wH~14np95g?Y`7ZT5F zs-3pMt|Ms+8t2H;%iE5E*eL>5t1HNY+ z>5{b`JL!%(zID9j?8$u67Pi%x6fRt&84GP>wVkTZ^jXVmfELoFpU9Z#OG85IXWv|Q zIzOB475P^gkTL_~br*1x2YKU>MKM5`uX(#~BSK0*5LIvlB2n&}HvnuT(luu4um0 z7hcdsIu!l{{Q;?h_q7Dz_J}y=lteK2sk6z9)tcb262-oWSTV&p_YDxE_8_wHqt=~k zpZiyn6;bY7rcHJo<^muq$pj|{7yPHtT_=4FTBc9HD5_yR{S=hOf-@@AWI=T* z)Rftqieol3gN%5}&_V6E+oAC*W-q(sZ@1=3hBx_Rs3^NjJ|aWGfd`7Rkh9q#@)Q2K zklLm9o+7T)-X)rV_t~CrV)6O5pT2jU_3GJ-4i8DBxa%V&5=v2Aq)v?RX4A&+C@d)w zXAtEX_9ivRN%ZE_O5G0&hYYO{a2mm}LPcOWN;!JNt?Zvh-zYPxB$Micmw|qcx3M@| z7@15EN=_ZAuTiGd@|T*qN=6ot0Y0a$;FjAX4`o@;Gn_GV zNfB%fE02_y?5@P}PyCAfc-iFD$srm`cLg*+a?qcZd2`1W(;h9hKGn=`NOwUrGtPLe z?^009Xd!7XQ5+&t5$Mp)k|^)|$)UR@8Q1 zHXHXo_Tb?lyEwP#``{!msm(`++|?NjwRYS&bf+w0mr7QR%mUSZbfBCIl~|phFCVwP z!X>=f*iPZM&mjwasVAKoNG0Gg4t?;?_qKcVlx;$NYr;eHHFYI&?+|`V$iCMPkD6VU z8r%5EB;b{IX^ak&lPbl~eC9azN_Lbh%5{=!;d;aIebb%2y(-r6H>?9I0Aa!_r>= zP#OB4gXjP+;|GXLq5e6Dn0^PM(Z8fQ?p=WpgTPZe#Ax{wa55GMZl=Q@ zk%;jx4w(<|iVnx?=W{JwiMidS&C*Og?l%9F?aLh!^qPK)}QzpeIT0f~ss3?+W zrJ{nzh^{xN8>h#SJ)1toyyzGrC==14DLtgGr{w9=K+GrHG-cN|nEVQV2Z(q@!8$Z* z)SZ~wnca>hE0r1%DE4Zc@3C$YTS%Mlg6bvMEI>8LR2&AP;xQoxKc;OQMr6h{Tx%e* zH1mev)zbzH7bQ|T8PVloadcYYvPMb%1w4z}yzB-na+||#c9&C=$vUMNVg>J{dX}JV zcx=%o;T~&FtQV=?T!^tuMo@Q3?5yh74qzPHmHU|1MVWK zYek_?gqS1cer=)^Q0v-NKBxIWNqS;X{pMh}n5%d3L;A1a$PP7$i;unJndI?k_19b_XL<3LQY^V4F@ z&*()~7A!VRtp!_E)v8d-?eb=b)*5ATHFdze#Jd!XXu`R`7gXPT5hN*|zf9m%(kAVM z!FBx0So*^k|9!F4o>|9#`jH@!{WlUM=HFqdO>xhtA}U%C+->IhhpiWhNc)VHYKJBUZ~|~{UA;6+LSg>{wq1~E(!8Bd zcZqov-+l622|{$o;hTPnZn)r+e($ew&9 zGBGODjxyv(Xs{`beBWAY6nO{{tEK1h}g{2s3=Mqz2?qMBC;U9V5iuPj5c zVTma}Jw#lz0V3H|j>OEY$-CogpHLHvUCmgNn#jFR%8sI?xJeR(mM2hzr;j~B1*KJ3 z*<_f9+5^ zix)=+K|QD-OOrbA*Srm!9`-z+$*Dzy=@F$3eToBYi|pRKIAUGH8~M=!Kk;T* zO}PH>qJLkYP=VgZseJ&Y_G4S{pOtf2ehUx6Ug9CN~MlZ=Ni%R3q{LMGhyBa_m^vGU8AR()(VUkLt>Ga7X+QbgQY{_O*q@* z$IpDBF;xnC@yf9gHa*bo=_`dXn~#JR#p#0Hc%vUNj!qdSf^M}hgeeV0AJK^g1$ALS z-de2sG#W2znm->r5}$ZTi>rr}N8MsPMym3n6rG70{3u&p=NfVz19u7sn52mzK)*7s ze09X9GDo$*;; zQ&7T?np7??1y~N@@W6sN6seomcl~WDj1*=1;7i>!gtJ!)L)XmqG0q&CC2fN(rNkr3 z5f7Hg+f2|k2lxu~(6xeeVzG;|2=gL5v=ewO8TwNUP+Qo_a3d__pOMG=?bS~_Rd*wX zUq0oa2?(z&Bv`OrL+yUi2CHb&+e>6$cJ~c_gtedg;Zz?t>)Cut2FXG1$gpj2hq4G+ z)mmHldLDfbX-SN|Pgg2DMV(=^#7N|k`ZiU%qfryfm;&PT?m}8=^lTx_9Np~=S|9Xo zx%pnR3dU+poj!qU!j&pCZOkKL=#=Dlj28-Of+!7U;>z9@dN$1_y7%R>q?^GAktz1e zve;TVi^+KhYveI$F>2jBg?o=7PwvL1*(kdRpp0Z5`6`ls_@@ltUG&M;yL2b3uWHv% zD^ff!|8l|fk9YRp*L@+h!`^uPbieOZ6!z7g`va?5nc?!R_pERp#|BtUyM zaxc1x8MAQl^x5xzs1f%qoZhUv5iR4_a$V?(4j3u79B&UFD?!d!ADfmjxVtiR1ykmS zsh%>tY(KJy(%WVf`$CpE2XdZW+|K5yaLBGqOPzCn0^Lny>Pfga6b;rxNt#JF?m=m# ztV@{bnLHEGImDt9=gW^9OcPP1z_|KC01@Htx}hPJJL|ZVy#ZDsKgll z*qy!+7HwzTm~Gf-o3etNaj|UZba7Q*P+%=i%tF7?R+GgOB;K&)n>C~2xHe~pGrH5OhF*aDyi43CoJO`O%2~-s!ZRNV7aORLM_QI7;W{nEoVj7G=a%e+~zo>=r)>MF+ z-PZ}Kn(u6ni?{P2#R{xldhMhxj?98lpXP#nRNLg#-auC`0Lr@Ik$RjBl}=kJWMWUA zw*Dk_;Y@c4*e^f`%bsu%<2Si}=$-rT%@r>xEh<{Ox0L-1%=*BiVlEU4Px{)WhPG z5Fv>sp%v}%zX3v4)0VbSN$>~yaIcnFIzTZfY6~rMmo_*1F->y~b?Rb7W{KkLJ)o0J z7j*iJIh8IWpqI!({EiqRclx>v~_wvIhPgK!eu!`u=00+ike({ezx zHdGE_LJH{iTiGoY21BLRrZ`=K{;5Du<9fhE5T6JP}N90W=XXj--m`) z@THCEJ7q@lnh^E$pd#dqB-Tq!-Wms3N7v6UNlroa_2hGgVxDECD(4sni?qQr)HKwx zM>KXU=AbkfR*slx2T7WjXqm%x3)gwteD;ZbaM^J@(FNKY$^!2K6rLhyQ5eDi6RBnw z7EJ6aYV9mEsIcm`ve9}t)G)`<8vTrirdtE0JZ+H0@7?jqF^?? z3DqOgy;j!nO`(p{BeKAic}}r((W+*(Ohc{`s&?t4e0}J)ep`(M&t|(Km`7Y?qZ=-3 z4|ZRV%2D*X&nDuEdQ;RgX9i-I=>xiDtnfR_8p4RFw>B-Y{gGj$M{zg}i)cb_ad{=Z z!K`C4%BWBsm{4GSkkPrxPP?P^13ry+vpz%o0V-_-pDW-i{i^t8)E?VH^7ckBdtgm> zkjrj6N9Mi)e^_~cvzv42`#QYpHnQio(7}8zD`sHNMLtqgx!+h5T3|ZS84RZ=B6Q9L z%VFQ;C~K?IH~nxwp{wiK*ajc~n|+YfMmZjt%Y|=c2{`L9%n)p3O*`g|0EM~gD0?D4 z@cw~wKCs(UZ|0-eraI1e~!kiLio_`R=;lGj3v;U4TbxIa?*>v!4 zo*e{n=PZTbGexHgZl_7@Y)0o`0>Rzb{_2QDcYANo7Kk!8=Eo8RTG}-yJc~y6KGrsM zG`0%#Mb)uUf2b)g8zmUpu{Sfv8`8lE^T?%*HEmhCvGOlwcLUIixta=6ZSxTDteJBO z(@d!Y{R`O)KopJgw6Qa&#Z+Wq#G8mha%sKQcXIb#0LhK}ZPIC_2Wip5s0>*@^7%gq zlV)0|nU^dBlfnifDUuJ+Z+M=R%lfJDh}l|tI0>wnDUPx8q#kGbCdDYgCdIS2hKnx^ zHJWu)w=EXBO`k&$=>stle;|fRb5)ccN!~;rx1>lwMfVa1nB>-vBfOEwEZXnJKyPZ( z6(7njh$rh7yoZi;Pb@sDj5#!&am{7>=1{~u&GQSQWD1Ok*~g9t%Kex0w;!8~;va~a zgZZ`PK{J(qoIHkM>6{Xj8}YH_aT>VarirYA=m{Km18WTTLUz%Xk;4e?TZwyWll>Jx zpp7tOn}mFTt4TrjMBtg^lCG63X}@6-yOU-r&q5ZP4AvsNyb{K zo7QMPZ{|pzv|Xv;Dp z{-&dq4Q5+l-dXbYBZxl#RtDGDm(0XPf2&A@3ZN$7sZlvyQ7yJbA& z)_Y~T6&!uFIuz+Ni4&%#ogSo?HhBm^2?UwW&=sk7H(U|16Q*R>qt8oWz;J8TEn7ZP zV@Gm7LNSID(zf1Y{So4dh8o?!Z$*_1Yx>4GC>Sb`4dy%dQ1NhbPa;uPn>EXt1p)9a zL8NbWvvcG3=7Ufcam%HMq7NsK7x|UD;3a28d8KLL7OwQ8(7(q^yBSVi^LG=%35kV!^0tz$`)CI)5$GGlcm5dfJ)ImehK~cP`wF zj1gPw--_boi+=&_q1%5M4_T{Q2Q^))iP7&auB0qPd)w`QviwU((TxZLNpi}|8Nno) z#iyD3#a@!5$WH!=^;^IZdEj}|rSz5_@25OTI*-WwzEw)Q>8&2h8qo%TMb()S)1R*t zM$RjTDyRyKz^_vCk!0_4f<~>Z@J<}Bc#A>}WeOIhExN@5?eu+td24zcS+jKxFpHLA zyp56PLEI3g1vm6nGrz5$W2aM5@an+eaB{a0?m3sx5qv4wUyc~e|Vn{1J) zNUMkH7dKG0kX7NFGv?UHH+(8xwx4@Ih^99{@^QXSsXW(r+0>d@W1s%gQsB4!Us?*8 zKeQC*T6i&$Q?M0pfUSvCiq`PIv=rb$U4Z|gr2v3T`lY2{3wnTu4C##PjK7o5Gl zFXU)su$m`^?QJhF@}XWr-Q3Wk=!PHJWVz<=AkBHC%IHIDyL@q9;EeoFEyelv@GmXJ zOQ-bTv=mm$Vi~l2Q|hT*Y&w>O?Ipf*4Dsh&{!A~lOnc^%f74Rz{il{Fghl?frQRlls4U@BeQx)gN#Fzb^^il{Uv0KS&Vrk-q#>68u)H`Abgqp{DplPQ}^n zO|P@1uUjxkY{FHw+yZH5*lYP~6VtobakamUGyM=GclPY4&7-t;v9EZi&`=C5 z0e$l=BnSb?Bmrg1#(#ah!i`>NNhU^C6N07NEdR3aFcOoyn_ZAk`i#Izp1V1au|JzWc{E z<8YBa_#RFLwcH)`b@@fTW#hI9+MuChTc#=UV?kC!Qsp^JpyeQ9h&Hm@8!AqUNZu$+ ziI_eZVd*zhkQ|6It2z^PPBbv#2JI13bpx~GM})OV1j$Ag#+tnm$y_&HZs$>I2F}ri zEkLR#DDcm28YXI*qd6L|Un_|9#z)g-#V*9EpY+xxiuG(XR^NJb-v|kojHqR+nBBODKHZQ6$7a==B2fCXE&5&H17;@U=}CgzHVE! z`7WI;&c7o$pACy9T0u4o>RHRD7C4RgS88%P=98p%^@w})GW!2WP5zts_`_xXeeq$n zkj1h6fe$L!|Ew)qTG<=e{l~%JKi&M69;->RRv+{jSidKy>X0gfhz8D{5}U4foLW*8 zzXw#`23FAHS~d{<@;c@XG+12ZmMW9?z?MmS|2*I%N?Y!jzzZ{qJ_^$Y`CCnM1|$Y0I8w4g=UMU2N?q8@fZ*C%P#dQy5!Y z_hZTCB@6pqCGD>*&^^MHP8Ge64`T##?28))eGSUGDwl@|##`&&;cGLp4t=j7NVHtz z%|Aj3o$u6zWh@mzs@*`cgm$!wsqcPRCWTL@&Z`*JutEAp>c))3Z2)s-(Dpd4ZwlCa z2koLq)*!byVXmOu35PZtqjL0Ia%ZlGy2ZDU%&+))#xwJ{CdkvhWILHa=V)>7+W=SW z0jMZI2uiP*+LuY9IoNT~#5HO62+FM?IpIQ!@%AXe%!CoBE>-E3$d=w`Q(=wWwO-VF zjVJMy)&?3^XduXISbr1g<0JW6AY5mnvGw9)81iCR$d|C26RRS?_4d&{3!3* zV~x7+l6*4Et(V4;?W(&Z8g%YvjrRk?0+(ZD?FTYlv?P@Ul)nwk5(dIn#j_vm9Eqcz zJxRi|V@K!E+rqI4?ilk%4*Yzz&Lptq!W$p>>3Jaq@4|ye+y67D;u?a17IbeH!LBw< z(!rY=Y;PwR!@Gc^E7pgdTUV-EznG9p@oj$RU>=dt7N2^b3U-KdMwYT}uK*G|1ISsr!sV{e&@MoaM zT}FPDq{KVORqi)|ZMdxd@<~*MjTrSjx+fIksl08Az&0dz(O)2R6ETQ=-&&Vy5Ti_F z2F%`o@zGe2#$tvF+yxC&MP3mM^$W}c%is^TUcOo0e&&rY^_e7umnXYD*~4F?D%GpM z@gguA!p2+1m9f-1cQz+_yw07j{WUxLCnpYntKmNV-xG)bd*bkaPaOX5iNpUrarpoJ zi9>wOEb-Wn6n4-V008CBiT&Tt8vZu+Jk?OM%V9(HT-V}zbfAPmY|igz+$Bn1D@(~Z zOSw7YqC{UdZ=}{HO)h&qr-TTu;$P7Ob4n9%u?C((;Fx;6vB?E`xf#|#IAde@i&Q<%(3VB{JOoo3xUOMKs;3xT*LJ7?WrEyo8j>GCb5q1sK)Y`=@To4kgF{#@Yb8TZ*Jy8u%P?f7mCb%yh=tXk`(iNV-lrI(T zzrK}$e@zfm&J2B_cvn3*Sy$46B278skc;_Qj@2Y7BtKFXVBgK$yKfwPv`Q8E_blz=ot>UX=SeVJEaUiq*fzhptX;YhPY)Ib)1Tl=J-KfDq%;Aaem z*F93P4CBGiEy2PJ60bL+T?Poy35I^2??4UF9BV8N7a#EyWtSqScDnmZOYTVbGxW)W66ZGb8~JxmsP%SYeA z69r{NymfgX_s(2rg&Zco@5r@3AS2U*{Y%7~Itx`8Nzey2p z=EFAIMwcpTk&D@!lL#3Zc337~99dN2H1zE7l>hk-i~B4xc;A+$MrdMj_me1h9$5uDt5uTf}uh;dpm>v4c&zr*SpUk@0rDV|1pH zlhNXhW(n=P_;xe30$kF&W+(wR`{nwy9BZ>Fl3;41CDI`S`?xOmKrquceK{Xj`ziD4 z(rT5N?|dGB+&>H)6jukb&hFITe^L&VG)~*`DISRBZ=N_MtGtoZdgIQ_lsVN^wkUO5 zAY3z9RyKrSo#0@zwv&P#Bv*Ttaavh?nz!qAOeIhT^^m+fY~ z%I!Q=J*=*Bw>zC!HeG^F*0R=@ky`E0EFS~`TLW4j0&b)Fj*bRjFguPWNOWGARw2kj zE09fel(FJV*zmP$eOFiCnjSMS%p=g_r^;v_@~3sZs2`x*OLP905$3&I@vkme%0*W-XrtkC^g0a`xp$|8+*TKby zd?s()t%U_MlkFgECI!>eqv-Oprdtu6^s~O`8yM`{A!A?gIIYdw6-3FCa>xu~L=dS- zMve(iokV@z_p5$)cgeglXgm$b6pihzG-vQP0QwZOZ&77=V8h=?EGiyL8+GkO`DsKl=^H;qZ65)te<605w z6@Lx-)^etNjaTd8=`&ub(d>CIKHs|#kMCzbL>CC|u~dKBU8m9L`3yC`F5tq*1!Cvt z`^ekQhz~7Va2NbBjf^OD$mZk^)H0P-FkiptX^VFy`bH)AlCR`yLxNrTR!KW=gWS_- z$@?}&-DhhPhh#{0%{bHfxq@YB&wkS-Uy!?i%h5J9AC|EugFIWP|Md^7GJVGbuxB^f z(Xp_@gQAwN0CljP*CUp%yzEy{nKK4|yupPoS4%T4HkQq!HV8(fV}-(A(Ou>=Qa{sG zMJ&7O*{H>1QXIJlbzTzfG(T z4UO9nz;#5(8)ZKI9aGvu#*#JSH1rnhTa{tI67=;+ew9X5J>}d$v&l%flNCLa`^ub+ z935i@qRxa|W!^>uZ&yDH^ffhz8e~1Pdo8kgt6SLQ=61bnQiT?mWf%sW3!V9PJrzg? z2qa6r0eY2(Fx5;1{02_oT^}laPs!k@U`|Cc@rypHnMNzOj9&*8^(xAw$WX^Xwr=Ff` zpPs$9*+68{t#y#RbA*(9K32rYI-u7|K#2Fadv2i93{XT41~l@p^uM{-54hp=1?U-V zk>`fwUSDUz%N)+&FLfzn_5GeqYXF&O24w;xoHkZqAx5>NV4d`#iIx~tFCsqDPNn#k5Lp3r-5A{e?{r3i=-6he^>B26jMi=tEs zaRVyoD#)(%Mfyq;kg5_PfC`96mtKq_MT#_otcalCjiAfSFg(w_;ji#K-;Z}rhMALd zzV{t3+`Uw~8J5t;Qd2aF`c*b20eyiYp6|q}6RIDbV~c8~j1PG5RnY_PR|g zK1uo{Bcd$Ax_lGUq4unsag2yJVa^)N)Ac(#5!#K*j1y&xZ5ZctT~u%=cU|-xSBva# zB!-Za4qhW47G{ZmDo=J(Azd@LuF}XCUrH5U9WWu77(KF#Fy?-6dvA5p!G5X3!MWPg z&hM2!&Z&F!>g8%jFpjog5EG|yIjf1+2#<4R2uDgf&8DW1L%XNum#u9@Fkey$%}9ZA zN6nzTeC)t*L*T%j7S^$`vZk7%gG8=1lc9jihO&kOB6B>k34CI*;!BYZd2Nz^$NQ;X zc#aL+qgNLxIPP&uw|_nHJ0wbCcH(;r;ivsw1U#_=?A82V%Oe!ad#?^noxvw$)HsRm zwZwSVURRRG!r!_dv!2WS#`>s8BL-ckI((iue0)}z;pKbTsJi&DWC1qs zfV-RlDx-Bed<_dl&P^&G5?quzugg%pb5i#D+>_?H4mtww z$~RvNO{0!r)0MC+lPa+t7%`scm)*2e3vD+VdM)}zCN73Nux#Z*7I^oL4dIp`qrk3> zkhi3?Z_arl5ZpgU0qgAIj!1cOQgH>+Ir)!T>OzYAY(HmQztB>5_?6?-bXXM6c@q2@`r zknGv^phP}{e@aMscU2Ru0%Hj?9YKY@Gvf}gZ{NL@g&I!F@$FvS8_s zj&1pqSk(lB!f(E~m8Y5{srE~s66ek8{pS(k<+IK+0t%ncy1ArsX%Sr+k1##W?n%za z7YsO}3zR1K_C|TSwAe3+5hIhirI~a2b)RwUQGP=Lg~mJU&bHZ`VrVrsoZD zdIJ^gYl!_Qt8-QO3YD~$f`@Bq9=1>M3udv2Rzt!*GxLA6+EyFoMBn;w^)H*Xd-XNw zAi?{nH8=E|-f-b$5AO=uN5MBZN;_To^?=E4 z$@UW8fpo2*b4A_NmY;>)Uin7t>0QC0%l4bf5&Et;m`1S`6_gkMfw#wh7L`xE6n`aU zAf0>f>QJfC!qlt+DatHCjqRepmz)?H=NMIt!S$PtSz+>;3>A_t1=VrWRo%GU#ED7T7RF<$<%km^+Q|s%fcvbW50hB#X5U;d+GXDvP24Rg#jM?StG2Tpaet=rIY6gnJ)j(#nA$U5YMYNG?455nNVTy!x z{R1w#qV?Bah~^@d{hh*)CcC`uMb4=+ zg^K=S#$KyE_LaFKyMtANjFzigS)Qt>f8@ASI@N++c-#5#S*UepgqoJIvFzR#Z4aou z1fnK5#M$z1W|SkwWjY%*Ukd*7TB>fevW=X3>+y6riAqB_gYCq{g9X}s2uoJPJ_Il3 zI#WJ`P@Fr{7qMQvAS)$JA*8brH-#u~$Burl@~gM1$9EBRdT3 zxGDQTZ`e04Om^D7K3GZ!uA?R1&pp6?r_Pb~M0;P-W`4NX@JWf<@{&}BK7!Awula-z z^u%U^F}d}a%NYygi$bJ*Ix88HXDgAfCh;BduD>^oT@Y_|V(n)vb~(hHCxb|MZ<&A8 zBgE&aiJbLFOyOUNwwi6_BZCtc{L7R_s9X~Mw7Gg#T_@IvDdx=$QfO9!v0Za)*r8|^ z)AdE6tQBm~9!XnK>9mXvr`85uw@X$o3xV1_N7$d)EVwVLIPUX!lVmO-+;p4D+Hk!V#hg)0I$*i`= zY`#dpOJ1J1oHJ5=TPw)P+x0-!+rwIgOnL*Q;bw(*+>PXVMHQ{k$DPe-mv?ol)Le`9)4mUzTg~HTn|ynv0JAp3brjF-|JPFC0tVPhXaqe(u$i^vv@- zxYEY+mI{}76LkHe?ekgb$Xc>rGt%*c@jY`dH0g#{PL_7|(dRvQi_H_P08x-!Y_n^e7mvbvQ1PWHG4Ex1$R%Q?2Ol6jLXXN}Y0RgMTsu z<=3Fs=DmMoUlm#ieq;f#N`$iAvh4u|&~k9nKd@Yya-RQN{xkO z2gd6|3E;GLAYq&`0r)}KJa*XZb|?%SMGk~nQqr7%$@w>=8(I!d^#+#fQ$jd@l!G$9 zq2=I^Y+(6eO5fR!^6in?PzE@88pzm7S?>G600e$+&!L9mz#*bQTp{HOwhafEVs36| zZ%s6zWbiH)kj%xoN#4lM1f*y}3E1ajpingl7&1WUhL;_R+gWJ+UKrdqp<4!dC>iW62MX<#h9LulZg^y& zxSfUaP)^x@{~dpw^4o-N8E>IvuxAx0R8kg(3=q2E287~v7TO~ZgWD!_%VG#6gWY*R zq2mW&$N-@m#wsXoXQ2v8Ft}|(x7=D#GT7(@6smO)h71t8;m?5Lb`~0shQVzUx@G!+ zlD~oc5=nVbDOS8@J5=U8>wM7~XcNJKu=GEonfh8akV3KwnLO zJm3wiQgE*h5Qo!;!EGvqLSe^r=NZFwgH1AE#sGWg1n+tRwKbUnMiICN3jH9!>uW$n zrVR)I_y~3$d+<6Hkd|lr^N_%+R!}+M?~_1)h9d$2^X>A45iMmVO9?xn{GeG7h$QD9 G@BRyj$kC7h literal 51771 zcmeFZWpE^2b0yqjW@ct)W@hFQGcz+YGc#+%Fk%`pGxLmCBW7AXkG*Sq{YKbd#P`el z){nYXx4JvxW@Mgw@?=&k%7B2P0>A-~004jp@II##Fb4zxbix7vC;&)c9T9sw7gIYI zeN|5fQ)gXz4_h0;B2Zw;0s!#W=l^;9KWu@?^bO?!M#Qko+-p4Uc2|o&5^hum-fx~+ zNCbqfFro{j$+VRAm#u^@m?gxf=W>~dZlCs_C58*_Z?bBP;76r%NYz*fIwg{gfkXf3i%aMGbm3UM50 z_T8BPeycWlH>(z)v2c@4^>#4#+Ig@;R7LVKlTT> z7QC5Nxmg)XCbp*#NS`5?sq@Zbb7u-(k>c`4q`)356Eif1p2@w76h?Ops!HmDAD|&^ z!zldm(8Fx5BB%NFfwmB1#(`@Jx`xchW$e2|KZ%l#6hr279%7L9BL;A?3&;~_(+}a5 zt1R+RyNW>pOMQ)tUNl907VkzuVHsG%&w+2{D16#qC{#s!p;<6odTjzOE5>HVTS+)6 z36m{nGvTf#Qt1qnJz_JlVmpJp*QeleEq~VMX4`;g=Q!+Kn=Haf>p*d!G_>vH(kS`k z>D*tMhuv5$2s{RDo2Sp$q5tK?pPwKA#s7c=#pglfa9>EU{}t#kUr3yeI zp9t{(aM=I5rPn0&%MUWbhFwZ_iTrxavLg(co!3R2w}VWDV2s=VUniC3ZSLkX3{RCV z&H^1-z0SX!xx6GWJ>976t z?~n?K@3$IcL=?RYeG6XoCgllNjxyC3CQMLqXn0cI3>UE zPT%CAWQ5Tbrmapmnr)Cvz_$51aSa1%eusXvD>Wz$tLyz_;b$@YO_;ZRv`CTHI>RY$ zN4g=?LMV+WiD4RXKB#?^wJPE;kfc&Y&9|!ARIt-Hs(LA*wiSG?u7UM)27YiCHaFOl z{?~0b^iWfi%OD%CbH5Qbm34xX_tEzXQ%?fbjf$vnK5b^hWLhAx)&cK>JfUw{JiMJc|%@c;Hvv#6vH%!mka<&M$|a?*)fCT|Sx z%|!0a$Ykdi01t0LYrDWx?&FNr3=8%kAL z%kt3R-R^0Elr&qaC@wP0E!a9;SEqhi0QwwWpNLM9U&Le#m*~5X8x5`<*w9e$E^Yda zW;DI_P#rbD2)su!iOl@~HZ9*&Je%3rPG!5%VnwU>_H(VQ$^>Yp?8!=KA(ZZit1`+D zI>cuTQ3ASea48LA19!u$+cHy6rhB@YM7!B650h?ILm#xC!kgiuxTHP`h`!eF4X}}f z0wRn)g~ns?D=yPlh>L^|(5;J90{8DVv$HwDB}7COCQx<2?^o~%N+{128GQL-4$Gh} zMR$;X59Oo&Uk>u0L+!HBPg?V9lWO^*tf>DQYX7&Fou}>CWiz4#VOmp;sT{5jHN+OuU9s^i@} z=`4y`sTijTH!E9S=+0&}itP!hK`HtDSK&)53=`ayY?zAbnzW{r#j?c8e$B1&xFaSK zl%eg_R*IM#8Xc(U;V-+twBLlEmAQM)!sMx-hhDwjGheKF8JM4xOywdc^hH@m?FbGv z7Z0n{{l#s4qI%m*L}4}#rvh{}#<5EVGc9cHHB(hh*u~6n-Xz;6G)4zU-^v-$?^Hf) z&Zl5$Zc<+L5W%i_VG*n3Ex?`m{?w#R`;U((D?E#tAjMU^7kcG%z8kDj{eIL#{6YV~ z%2Uo8gWtMsPj$xygBgvKHBPAu%2-7Bbx@qPA z@4W)BYB~+iaQay~E0Zm5nzmo8i>Z2Z)tA#9Om3e{J|ssL?V~HX1~*!MMagCjR%@n^ z?zU;-TFt!QihWNwAOM2Km~)YGS0E^w?+*U-HQGP$c$3zD2meQ& z_;FkjPWY8a_3;A$X#eVo{|fs52KVPWI`$jvn7#z{pM=MJEft+H+#(0;8v!jer*^(( zWb5cfIFxHr?B~6g35!E+CFCtDrZOqvt4RV$1Pn)=xJeBDKcrA>Cp@r@)?_IdV0DeP zG9(;Zvrc+_yLfM1zt>ox&ymio%J!O;ehfZ8?sU_ZnIcXp)*G8SM{ z6RzJ<Z2f-?6?x43p1EwjY)6;CWYC?k zh%%g`xJPKo@R=kGV@M9(uP!6!?$)}wdn3y&vR|K)9ogN9`lJ*sKQEgXzGpt)4m;EL zHW|Fn#y^o-0I*u%pmi(rb!^~uo=iia8%=9KE!4D{3%|*?WpPV$vC4VdhDTWP^n@x2 zLt!;WlR;$8!fT+i9hi3vxD;i^bBS1nlfPn=4mm1jLMFL0?Y3_Z>PpdM>J%f#m_1L2zozSTCX)}Hx|d{IcGq|;N0 zsb5oUec{}=M?eXxJHb@oiwd(un_mG@W4^%hGrxAu1k6A!vi7>`1y$p0C<|SaDeOFC zwo8Sfa4+>#N4chQ9?L+8O;>=3F1%b#oWDU{LAEsu&PTlsDvcqRd<~-YXVj1rvndF2 z`BtB(##EoAl#)p#kqt~@kFeAdYL%%NQ>mnm;=NOSi@J7ehPj#!V{B8-W6OGFhw2To zfvZBarK@wIL|akE%oC_=b$%0avWyK*a(}f+o-t8`=h%R2Tx8XvD1~VYVsU8Ta8mN8 zqNbdlrXyKMV!2A(BUB&Lf|lFE2JqoE0_~O;)|Ird8jXr9O>u@N8}NXvskW<5JsBL+ zt3ZRvL=|VrI!~)%)kFPT5r8nTEEIg{*&2V)%=*%^XP*_#q8}Iyc;7rt?2@XS0KcFd z@zY%O3_CT-0JH%#;b5DMFAftO^T2(>ST_DQPZ}A3Qa0Wkc>;2gk;4tG zY=re&GtJjTM}{zwL)UMtDar6D3UhHUS_y0P5B*l-JR9jJrVF~MQOavoObSMM7kmoBYz>c2*KkmkrA#=&I1MP;}`@-aio}|+*lO-mxrfaEyEc?j@ zFE=E-!7xW<1gvS!!R1d_Te3pQqp`e8&Ph7b?Z^9jWgZhh1g{VAlgK1VzR_1|mY zn@%3dbf+Z-^gZ}xTxQvrwAAtkOC9rij(PgliZmiVInGn($f2#!pa*F9s%4-N<4-E% zV}zy8P=v|76&A|eZ>`XY+^$lK)7EJFRUsleGEj5ZUW3dQCQR$s+ntiCPPQME5J+W*GUo-51pxqT|6q z-S_rZ1oRvKr+p>21o!5b;4!_=dK|UO^3X0&)!R4V1a$o-&-`w@`2((ld&Ew;jPJrKp2Pts0p;zz}hom_3EZWBcni3l7kIAj=)O&ioItSv5 z8K6QCgV!HJScJ=`>ibZ^^7?+K&oK1+xO=9A>CnfaVSyak<@}iQ7)2&lsb*P1rlwz)j2Q}_D=<0Qv zop1O99#u{^OP|6j0)g+MYlfRAJ!%{$rPD}mz8kEX>};u8p~>KEsUJ}#7AiZ; zw2EyOwjRdzyzb97GrHr&>r-&1oA3!MsxY389)ky&A4fxCyIIbk`{zNc>E;h-eLUW- zC*0=g`#s}t1O9An+i#@@vsew~MCLclcQaF^c3oZ0iQKUwdFjuMUPy;0pcEI5B;Lxu zR$?$bX&FpWihI`R7>ZWQEKz=-7t!BKZ`Eh~E?-o~O{-?m35W8RiPmpp#u~zN+Ef~l zg6yJpu>K|pD9VjE=ApOzg!@O7Vreevy61~Z{FO)2{(DLxYv^h3>hd45$$yz*|36vP zS5682CD4gp27Mwv*v)$(CYI@mUhRtAfuxyifNzkR6d&(o8-w*9oHN?HWxZQt4<#da z?y@a50t;<}p83@6x$)Ju5^@%|ae^ygZ~v^fPmewnCOJgj?-xT4-PamSaNeXEGAn@6 z3X|;N)RjazBvz>-&f<9}SypRNvYZZhh^CTP7vWyR{&|llbL;PR%x!**IB86r@2^UU zpIge}RHj;=;q^spb9OxGt+ciwme^0N?Oy*O$!DEvpC!S`V#Io1A_t&;W8ANSUH*#=G;J~hYx3uAw|xP zNw!=4&vMF%LPtAJMXL#LZ$lKmf*XBmTd%#h#YGpy-{BRBW|mnhEK>nSX;RZk!CW%K z0pEuE3EusG)K<0rG}qPhR{fLCQgNXjS!cAUabz4?Ip-*ubkdP4k16wp0C*ovqqB7G zVa=(uqq$=b!;@kDQ%kuxTLiX6Ga=llHaV?Mw8mX#gO_A$G|o<>g|CtpZnBZlXe)VW z|KNdfMo+pil5!!!e7G~>QQB}fB5poofBpZH!*TwPbh!WTfyebf-VSvCEdXNrx5)dS zgh2n^A^w|&VEiBN5&yj{iQ}Jk_yniZ6)cFV}T&2VI9Qy{-v_A`ae#N=GcybJ5|ZNW9$t+}h;yA=tNF^hhP#F$Pr zTU%YDMk=lB)LqP9aKQ-3}>?&K!w+;C3*c!1+tGCYQYgTG#Cc{y3a@cdh z4#Ew9&ra7RH|+0x5Z`~7fMA-wFOoIb6KNBQ)6xiyl~ASgfKffGOhoyz8M9kpRb4jz$=x+642@XoqET9C;@zUy7y8D-N+ z-hNOG7F|D@hB#zs0obcTlJtHYYG`9X&eF3(Pop2Akg?LUzx3*?TaqUKXKzTTkBEy5 z0(6hEg;d*oyEm~5Lzyxv|6O9#cZ^@lL8!3nsAM|AGXWaPy#T-V+NbAZJ$-8PTQ+kX z`ek#T%U;wT?Of4tKd(9DTi3{`dO`(i8Kj%_acVG^R0an$qYmb%$YqR9l{9kb>>(_79OfFZ* z!f0x8hf9ND$(^xXq%&t>k!V_Rj~UetJ*?d|q=9KB(;4Tj0uZWI}D(F9%2rr*1xU42+7vcT!fWm+cPIhtH&OL0U%BUA!#G zllfU|vFP@wbF-@LU^Xqc~T8HdWt0E@t$5{9n(2+pFlU_yh++9TxgScdFX$ z25BgJXk~=mu|>wXJ~r>nrA*`x4GpG2W**D6cr(48H5|HJ-UNi02zf$(6u{V^88@Ey} zc%!qsY@PLTfM$S7_*7|G9IzI$*X6ZKY&ctqx!(DLjsINu>pRHGc=^%;4(b2uT>E!5 z_W!L3{0%JsP7OHM*0nnjNA}YvbSKOh5hS_JM;3>t)f$K2m5o{D5{$OdilIu{u<<+x z1v-TH4&Eg0wC5VnAyr%2LN%3PxmnlK3fhMqP-X%U3I8ADk; z*se_;-Dm8S!*mEHBRkz_jC3<s70|Cck3jJ=>tcb+jpw7;T7f>Q zAc_~(7J9ub)V9alyrfZZcDv?&oFE_iIih0Mkb*}9+*52bGFwKCMDh@V&N6xQSlW{A zEik9{tuk*a_uTuzk3&w3%*l4ww5AY-abB?6gs9_?Q>IxXS8Z~Q7PS-yGu2RTYRVDF z;oF1is!%d}n%N}L2Cv+#kX*E>m#jN$_AQGc1Y$zG%Hi0z*=UY?lukQ=pPq|J2Tytk zb`A{ILa7zM-MhYNdoVSFDUc!s>gF#i)3M1w`nDum5!#EJdPN&d?g|*dqeILvNMA?W z{=SUzvUtSX33r$2D9l|?vzVD=H#6a<4|KW$r9n<$X-1*4U;xwHxRy@4mA#bgM%K?; zx%swzm}3F7+W=I7#&00PB@%ojc2w@XA%2>%JkAHeHAb}TIYaK~72ssSiR_or<u`xe#wILwNy12$3pG*dGgue>#L;>)lsGgVrU5>th%iGyFHhM8cisvp z5(6wQhsSeTHT}<0LI2wMJC1v#a%AN~rAGPJOLQ*%rV)PjB;i&{8RKAL7(-?dvAJzoBNC}TNP&> z)*3*UBi+2;#XyIN%S)wbe@ zZz6mwhZ)|~rJG|bOpMJs+~7{;>Rf6}tl^m@jd;_t6n22lZH!9&^iNoT5AJ*A<@)@i zr;`fK5T6yhk|!H?K*@RHF5fj)u|V*AFnw%-?1t;_#2ia;O&XCqY`rqt1-u%F?@nO; zesVyd@Bi|YPjR)ZnwepQ&#YYWTEE)2=|a|6_QOlj`vwY;2KwMIGYKc zcNs5neZ89f298sQ>z@EA|2^Ej(k`%Xx4U(SSw>VV-T9fEDqVN!^H}{i1br295mmEb zPTrqlMT`L`;9dluKR7&r-AZ=GZZHyz-(yKJcACb6hQUDHpyVJU7DRZugR0j`qm32E zCUAa8*X>4;LAl5ng!V9!FxH7@vE^X}GRw*J_uqwLK}|x6a2VUM5M+O+O%Lq8sHf~(~1V;3zMQL7!T@|ZeC5C$Qj>wMvY}y9bY?*oVz)n zNa(3MGp&AE=r7&wIsqdx0r5-EF|d2Z_trv9x24g9lgKVbd7aB;J?w#FBA=&S6p@g` z)SaAD1x3o&oVe1_NMpLYQtjALIQ&;$RJv>qyzPwpt4DnefCH}TGy_OQn78diD7EUy z2_o1M>NkYhhqT-Xv2nZdunn6YqaoQ9Q?r2Yn{I4tfil!jXF;{|psM5ZfU093bZVon zXtL}e5@62RQZ|Plj)3179=f`7-5#7Dz6kIfe)JQb!kma87Xe*txMeMy1B0=ZtlXHq zf3m}0ZSu~ct_2ZU@%>;Bwu<-+cw1K?xckf_X%!+Mf(ou7mpy)T6B4t3e_t|O8_ECd zzO01@g8wNvh}60ZiJRZGP&m*)mt`DuwLb+SH0vQs$NQ2f(tnADAg$h;04Y$|Dg5=* z*VqWD8+Iic;lFF)m-!JvVkH&)@v^4pG#78J_YCsTTnu_c{8l*b z>;eiOE4X2eOyOoM-JwTkBate|HA55J_tUN9BC$?8q7HUFQ9@*1-1M_L$c?ns5V4!v z(#nn_(~PG*HWpqhTvoc0_7Jth`m`<%;Jl$F(5` zvEqp;4UMfY_oo^6P7RdK{Hf1X`z?P`DCsl6-!2+$)`fJOV_LG`6L6%RI zVeF7d%9a`Tv{wu~`7yvMwZKr~DTNM;Xn%MvvByU_qgJi-R+(+{Qa0_|y%$Jo6@>Yf z{=Q@IV!G7~#)Qq$n8=5aqg|s_b@d%Xb^u-brq#EC?r(@B^CX_!l|A{`^nCLHx8{@C zO>_Z^0|?qG6LekBx+~_|HjqDpbT?O{7&kciJ}J;rl4d}WwUe2?^Buy}f~Cqnt>o6L zHg#?}+U+t{CXDR6<5_^~QA|Xb%cAE`=Xa&uP=eXLEZ%ik?xw%5KWkuA|3+8bMhla z)no<)8wFE3_Syhr||7sAgta}n*7lUV=Ouc8i5En z=o47T5lP`Q=zxf^m|;CZG)3_60dPVaJ3r*>!;(llt0Fw|6$$jaB0T>7bUlC=j1@4= zN`~ln>PaaI$Ql$Fn$owRbovND&W;4zlsSC3q&KVi7W?SH$PxGRXO}XJs|4wJt&lJf(q_s=*Y1+i4$#Wzirtv%e)>#cJg`6CWOTvp1hO&~ zyhyQmjb136+N$#Oo`+DcW*y+R2K-Mc3Y5T>>^p{2`pa54btlc%;9>_)R`Xu8tl}38 zjLH{i8@C9vr!B;r5;{6FY#W?wfVLV6UoOt{dwpJC9-)kuQl>@PpuZp3!n{iuXN{&( z6Ftwn;CiA!Qyqb`@MOdpznYCg`jfL%k--x2sZrd~wB#o$r&+2ub>i{L8luUkL>fHau@8r}X@q-)go z?7rld4*`QqL5p5At+ew|;q!r{QbWNkP}dhgjHuW*szegYl|PCFbuE1I`6d$5(%Fk( zIcxqV(H*ZIk6YIVy4`DRu(pw^DQ(Upy_j4P&0q|cubl4NJ4bfE2umGxjD~fX-5*4` z?9qEaoXCuZ#;v=KaAE<)YD{@NEz86YylRUj{ii`N^8;;B6X4b!squPL*Ajhp865br zWqQGuXLlbrnErZd&4e_1HrVs0@S@0JY(ML)s22msC<7X{XGG?`f!1!q1@KH_ zf1&I7nge-aSgRMIbJRO1u#rHn3Q^g!5o&>u{UIY7Zr=K!AJX2GI&*L8PL46B)VrojclvfywpF!R0zrNB+j6&GvM^wFU^-2iD7%f+jI z2!kUGDwiwVESfkTaTylj#>4wiQO~6UU$NHnQLOl#P%hOgQ}<&H-hkX3gu!Z&sSd8G zTlaD9hG~^%cQL8d9g5+R{2Jn^C7RPry6!#?Nd__`))){DF*{RlX*1t(xiY&NE05Jb z`^4wzmB_5^BbY^^tQ@;-R2nJDj>!oQ@B}-zpp-|DTEBspdHVy_IEFA%&jQLP3hB=S z?JsY_h`8pC(qa6Akj^B}qv37g&80$Obj>rSgpy~ku@ zx!@tDv{JI`Ada<$>;~Fp`hs@#$d)=%3{e(WCphyge)@8}JGe5ia7xHcIi}^!gX`rT z%I3MenueILBn0=7u9t!xcS|wrGf530y3E?ix2Vn@^CPE(08O$-=}&Jlt87`i&@p+j z;~{u4qhqoKuUoPyDm}|ieq7O8(?YZ+RHfB=LJq$jY5P{j84&fSFfIJp>}2QN+pbRC zz{l`?7g;7O*%%iG^JDk=Dm!-zWonwP+Q+&96c21DS21jEdg4jnXx3a(LAKoORZ#kv zCH1DR9U~Xeosd<{Lr%MSlMR-xygG|Jnn@&&%)p|Y`r6|CpiQcTR*D?)XQg@SSXbzA zob6Jb{>SshS-StsZ1qDJt~kXH#9Fk?<^g$D9I3r4vN0QOc@j=1zl_d3E4LH++moMD z6hfF~Qq8i#=PaP~Z;bS$4aQpprUH5c`v@e4A~0iUb$4R@ICa@{eEkB*RGcjbr2?ns zi*#X~(*$DutRW+l%|Jh4l;C^WwKbU!3ZXg;Zs@*bxdeRNK~~uJ{B<&*Tl=H;+{iKC zjB5mt{u_+u@9N8>=Eh%)=3gaNvB&7p)#%Z*4d6A>iY9_SNf%mL zS1K{@3v`%s`9Y+DNsV%})e{;u zMO#`6jo622u@w~|`TN_{pzoj^G)Bz>88_Way_-0blzk^j?_T@ zZPXj`*47Gh%h1b#*ex$ar0v%I8*#sT8MQG*p`zemxrRY~W-z)4*}fYX^gFT2D<(j( zZM3unF{M;<2t>QhZ9}#kTRD?_ahfE<217WM;EWfoU`!v)__Q5sIo36LF*dGpDA8b> z5pYu5@Gnjyx2m4hMKFm_{CxTXwdRw`M`UXW5@2ADdb9O_?|T+kz12=fX8v={dnsC3 z(`)S)0gp1rE=9*qXS%*^GxHtRQX9>DxL#3}P;I{I_3TY35$2lGy(U5$(2xt>JPc5$N_ZL6wv~`S)Tk`F zCU)>C3q%0GLS8|C$iGE7Adz&yjdp`w-CoZtckTMK7S;sQ z@)Bbcv9trcAovHzlZjTnK>4vQ%lh9uc0F(D7CmXocUsTBa$uUXkx>t)6<*YaE`zmK4Io*}7-hdq~Zj&WYoK)<$zYwXEWl{jJKHXj$TSiygD9Zi-tJy4GrG(=?S|H# zy4oMVe*F*y#TEpQxeJl=;cBvA-JAn9s4v>4t@n}2=5(NVE(pe>Q{xexetY4;tGLy` zV;1lsf&(u?$7=`}Qo17y>N%rG925A5^hl3RM|I+h9!-7GqyMrT^>+tHjk?lT$3CLp zSI54f#Y_~fj++UQo8j6#BcWW-+7^$D5|WbiqU3rD2kP<8x}UuNSX@fRbR`UDtiDM! zw_n%mpFam%U27}27Zs|qtSk&&*d7t=qNp{VIi8p7?F8PEmYb*<(P7hFAZpv$&ocsB z&5`>$D#RKLlKxew7&)A`W#i+H7UCFf9``VO8vH#*nLPTHusMeDa7nWQoR_oyY z^*7h6m=?3JMv#s5`Z2tyava-J9rZGG_;OzGvUC+-_z%CE6oGaGdc<6IcX&5gA()i|91KnwmzW^p3rGxBKd$Yuo({{B9cihVNYkrY>)C_Q`j&eqM5LzStQRW7 zvA~_#Lkkeg6AJIf$8mx~BZDblZ$DIh&;tuXk~^cqY6yx0aH3tJk1pBLJCu&HDMby4 zJHX@0h4NST1@Q}eP9aP=37Jn+70Xcly?O!yG#l_f>z?~wVrMQ03lq*$tFSI~^Iluq z8NW7vPB*YTot9UA)V=|*KSHk?62-TZ7JdgSD4;e+;N2_03l}?@JkIA9Hg(ssj69Qz zph8|A0~ZNF0(u0xY;l{WC3T`J_zMuBeE=TZ*a?xP3a>$=zkis`{vcEitUM3Gp1s80 zVwBT(eMj0qAlHzmAAvG4!xxp&b7kuQ3`z)hVm2rjhR@ zk2=noITYFpyWEfx`qpUE96AiyNewbm^dU$?pVdip#Vh2ci;WZG`8u?Fpy?R|NTrvf zv!+;?oZ`zlXsBDZXOA-;So*@>4R?W$lNU8F+8w6M)`)xX>&iZ?Qc6EqW7GUP#>xFK zO{;mQ_ib>)71*j}f5CaU+g18Kzs)e{^M85=YPURA;xz9H|4?&aXmk5mlLntkFIyOM zz!I4VRr(`QebUl9N?|NO5MN}GIUN0)@)b0$%`gQ~>Vox2XHKEC{%ML-i-kB8xwMD4 z2)-kc)nZ73TBrY96&wC#&Mkbe7Dw1Seq_(&v;?fHd3?Vs6mjw^hRDD53|+j#mq1W3 zii*55TqwgBH(^o4$;zJF!RCK{m5Kgy0qG)XKhXP&HCZ74*O}7ay&C8LK?f?*pJ+@l zL0WTCBFqLoI|ssyj9V%dO)bImjs|N=Q(=<&}O(H7H*;F z6k};LfA|_$j9Nh%-OgVp;GjLYUgR~)~CL}WIG6_A^*NG`uLGGQ|oSUtJMl}kpUtYd|$YGR@QRd z&eg+o)S8k)Xm6_q^92P-zx6?@uvRN4?WZQ{(8V(vwIc1!DvEU@PaQ5~ro#JSc2$$f z(y3_F$FF#v>M$IILvu#a$g&U>aVQ9JKLGEj)fMC|@1DR)x?@CSm_M1_jxYPdvNb^fHuWH9 zc1Cc3`aS2GM^T8c344NN008G=5F23d`dSYTSoQd8f@d|{7AWeBJ~zeKg}q-M6nWY5 zcKlo!sU%s3cdV8@xHWz;&|6ekL2~*OSgDcl4sgvq5r^o{L-V{h?<$N-#8(U@)l$1V(DKK$%2EWXn+l0B?= zq92T5DYpX+cyZvquSU4{=Z*}JgNE4pu)*gzl-nMLNzO$OX#S{MeW2+1a9mZ)N3!*m zyklCwn-`GF4G}^(Y8F9IW-iL;VPNHa5J!L#ev3DJd|T)^Sst^1A+N9uSLP{yBeAQ+ znAO<>J>)_B6zN{<6NPxID2PWi?aV~X{l~~Pe#=QB@I_+Uzuq?d52f#apkV*~LO_kW zj@e6i*o|;5Zb0F&CZt#+>BTx7@UJi6a!R77qLNBRg;e}p zMQ-oh%*6Y1dhDUY(p)I55ftBbRd7T!9!W{#$8u9kf8=*cWXs6Fgpw22g}1bQ{d5pc zl4+@)x0|bp1IqMc`f5}Mi=>Hg)IGFThXfBq{;@_o3i~QG9=42Q*!d&4^Q=!6YmLSb>db4)TW`9zH zv3NllHtjnkIXCFl`1l8~90FTO5Tsk1dUr!uI@lsuqkmbZ)=clzlU47-ofYr^m_2wG zi>wg4!%$ouu<-{ZRAaxy9TXuuwtZj+?I$>1r`D4otx5K-Z-wSR+uWwg4vL#r`YK5# zY@C(;&|X(<#Iiub3kV7fs;{KHYQW68+Lpx_&HBpZVJT>d*(D-H zq@@a#YvNQTr~qvOLwbUqV^YfFB;DKq)4bh_nM5O)J1~=OnL@_Hg?^GV|B1*k6qy4X z45d>O`>S5pZmSKa14`%eEhRm_oebh-Rp&9!I zNfUY`7Wt@@QmQ1U*7Lm~2&H4dCEWyJe}{qOJuVTfS~_Za`Tib8FUj``co0mu_p5md z)_d7TNp^nl!{&HqpZGD{lllHui|J~xBx7oUnC`{$sY1&((RuVwx3ce#&-;_Tk4OCG z(+)`vy@Hi|i(~P5*Y}pH-J9Rf4v2CpKb#DiACuRvX7eW@eEqlMlTEV9lP$k(3hgt? zG8a{3`{<9m0T1q1S88d9LeZri21IaM4BHb9xmT4`Y=XjjUtsPSfYX^GmO2Vxi{}Or z$*gE~*Z>9P=Ca!kEK;Q%;aKo-1|xvaelF5UJ&9U`|4fb?+V*vd`TU0|gqE;j&>9#3 zz>fQ0N8jJQjptfhcE#eTJ#~xjf_GUh5R3*i)6^f;)TYT5*Zh5$!%Vsc$Bb@C>NFSD zE%8kX^UmPI!~UH=$A+ZuV`HyJMxHZo9XXw^Kd9tl&``uCn|h?STjYXGiQ@Oq3+?@K zz1eoz+^11ZpCl>(Ki0DX-X%fZN^ zvmZ!W#lN{dgj-V9YxY|j=>>i6`66Rs$?g42DCa_R>}R=G zNK^nbju=z#je6tnML7-1Udmy=*e<@42?7Fkq!PDtvi8UR7ZG5N- zQT}n46`l`Ga`2_Q=yzaj*RW!}#N0g(R0zkbqSd~s42c}TL{vhj`CwliW8vlZw{c5( zOA%fEb_S*jI8YN-1-4)?y4{(WU6v1SbLHfe@-~3>5o|<`a_wV2JK$Rsq7H{DR-{m? z`2CwPa!?`c7kIRhrB6Yq$v^@%G#Iu6&^fZQmQo{aM{!UPlFlS1+CoPB{ z@A)+MdB$yLNSIB3G9+#N4lxL(0!kW94OwnINo1eJ1+7%ufkNw&U7KarE19#C#U3s}lgKR^x@`t)Bk~tD8C;tAH z`2>z4lc{7oddz-Zk7cLDiEO8q4$kRp*pdp8RULLi>!Vg8^aX$=7tT8U$9NEV4e6@6 zSXI>0Rz30v1%43D9Q7|Z$b{;mi$x0%9!2&X>F950-`fgXE&#F~p)I3ob1!V#JFbh| zyi3}@sw{%x=$zOU)&f_FHN-Z)&w1f}s!@s1my+FQDMp2H1gf>A(Yrn_puvMlKz(Z{ zbttdMs8+KXWr1qxATyC>+f}M2KxObrF>qzDV6HQ%Nx}4~sbbs{diM;1;-WWZzuxM* z^br}(u?rV;BrJ-Iqr$49u)36LQ7TVr1P>6QBX=bmMAkND1Jr91TWOY(TGxOB)K`4* zqN|-^mjHdIM*LBK4bXgl4~;y358B*+4QXISJV0prWJ%ULelgI>QvjdiBDI+vjApV& z+Fx2Gq&7oIAYdIbb*KAacMD>)Sxp)WAHz{Bdon~Av9azGQ8w}fGYE|$_BUl)S`ugL zE^Rz2==tfzEjyCJ?GEaPNW6j$N zxgV`Lg=u!c-HmUHAFO+pTai+2&d3%A;cs0GXR4bRq}|>_9dT-YkLP@ElN|phywUZ!OSF^e z-b2b{kCRv%H|mQlG5JHQtSR?L#r2>y293=H;J$y>$2{yNX|`tN+`|VfIaO%|o9iHO z3Gau9H6yk+y5mT(6eIj_Za~e7O<|B0>OJ+j^2d^Ry!wc73-ft z$==@TmblP*SM5F+H_d8xel1QPqoUKsEuUiROXTZO11|AD7%km#i*zU$l`p6*jQc@0 zTa}&CA`n=6bpsU_@Xwcp0^yb_aUYHdVwJ$LJH}c89 z-xjO+k1G7KF8V$#v3}IQsqi&+r4}RGMuG3&ADka#xJ4pcQ#o}+nlv+ zM+Rk#;w~Yvr4mKM7RO`Il0S=XGy;2TrSy~s^ZBhHK5)WfaFl5X2cDO>&^){T{+9Co z+bD6^MtQh82dxNbkG2^2t}fkDji&h&wK=K@+u+xqT4^Jlz!o)Z0UK80p(ku~*$Zge z_#-W0=&Zp3U+ic!^Iu$OGR$RYr$nZwr-3YC^FuH0dQ;Wj^ljU;DH#RT5ZRe&8WZeknbeVOdj5$8iOf)ndQ?IiW;oIX zcahE_>{vLDh)64OQ$=2>Qp6+%3~tY)UU!XlE8yCL=eQNRgF5&%X4po{42sgQFuFif zHDE>>s6r5K9L!0JY|=AwH#6}2~ ztuNncMM!=%0reu%_Mw2*r6h&}S3crba85L1dJK&-3&7kTd;*VM8-j&1}*KtMo1L5PhaRgV;H&wU9Fq40ZgFu2`C*Bk5e^SDEkow`y11jvz!caCT4eygt7(+MfQ( zYu#89hDCVf74sg$R4^!Uux9-ows(oSJkU@aH%z1-Z1Kh<*}#(XWjL( zFY8T&l%zYJ6etO}z8X|-+{6E6b!caYcm5Q(!yA)uEyLxS%ly4N0Y2}oZGYyqkWTD} z?HADVJ#_BF^Us&Vz`Eh#2_W8#wq2Mq2`IRT$l>6T~*D-QSp;Z2hMBUTTv&AaP#HP3x zmA|o`*oFI=wvn%J{(@$jiRt#9J9ITf?S6elX zth8kZ5Kiu6p#~<8d}LV|M(;n*Wy963Ugq2H4QUNMl(_gc&;Nj5`OqjqrquFXZF-&a zxM@Zg^zpUT&s$MbzXC$)YszYmS=qiwnV2q`(~fm>weiyntEmxpC23zeW_S4G!Glda zufElOpobrOqUiLZkaNn1C+K9xtG7-4PDR)na{SQ6+*<|*kV*}ePvY$xsQgJ-gVpS9 zx0=x{Uz<~yl&rv{IZsEo-{ug`q9r-U)2UqL zi~ACEabrcpZp~g4^)eNfld|AHnma9gT{P}+&)IIwN4Gvb2Sx{_mv-*>aBgo|w!GW4(9XxMc@LB+t} z*vC{4aWAI9{zyl)Pc5${yj(M53$o195!pNjxGd$>Z`BHS%PX>#%Qx-PdaB~X&XsZ9 zD-f7&-#6=0^K#XxPr=|;fm6hW$Gi~Zup~#1q#|a{dBWlC1(B$T%(gEvRto!1hTWZh zD?Brmb|LpT|7VRolu`xasS792em1`vjYKTwV;8vmZ5V29zh6C=q+@JXr zA|f3laUkyTwylryTIQ3PO}{+L^tJ`$d~c!ddc33V`n!n9t@~&4)zWl7Sf)?E^~x~# z{IU^Kzldk{2?=fC9(}2(_iQTLb8PXN!8*TAW&OMRJz1Xk=rd;1AGs;*$#eL7S}q%w zEuHZF-L=ZkKfhuK@<|xHxM8~df@M=j-}O-KCep?2gP7VT?<{&snIA?Dyujx=7A3qv zD6TaOS&qk_*^_Q?fS`HzPQj(frAD7NkCK)XQr5o?CiO@^lzG~ICx(^pn0_~IM6ga? zBBsUpar)GMG*AMRDm!gK1EuYMVW9l+wf;{I6G0@+{n)k3QwB4xI>xN-bPxI2(!u9` zX(yIB_c(Kf`}bsU-wO-739E>D5tSzML7byqRqlgPU~8gf{%7{tksk@6PYT6;gm!&Y z;)HGNz0ea|?Dz}eiOhb~`{f+^u;-;9y8$J4L*E(vW*^kW?rLr+#siugm_;-4x$1y3fC#P&DP> z=d|Oye$Mi_!%L2`qCccvB0WNppd99AGImzl~OyM;#D zs3yG^UTwaR7|lAqEBWPS2FIiE-z~phINB(#02fc68}_)3;>uArHu;jN4;=q0!>3;( znc`aR?i*V2{Pz>>&Q+2M_o^_;zpfRg-OICL8n=88DE?6NISuRdvWs!v3jKc8(o-m< zCc_;4(%J02EAY1XlN*mkpU#`4!KFCCu)W7sV%`Q~5?xZHW)(%2o)t^M`wt>xW-dNg zKESnD`8L=!IqB>7>6;=yDh|b*OZ=Yt*z_zHN0OB`TRL+QSG{*1uSYh#6N4b1(p@9p zjWgv6yDRiHTl9HiojTO~urU9r-%oIC+sWV7`@Q!2X_3AC>Z0?<%D8`k=d%rnA5ULC zpIe{+f3CmJqUcJ~rvb6!_tV?3PF@JVW@c1=jPO?N^bvVpmp8iHN0u>0=Q7VT77s71 zBD?u@7p^?t+ugk8_K>3=F~zUPW2m6MsKr}66hOM-d_EFZJW^fZlm$T$p7f7iSp}y1 z17c)Z29o5oJ zs`c%m`?-4#)PMNd^&(Bfw%Dd6%i4VM_4j*1QAK>;P7ff?i`tbalqzM1zqD@%-OUVn zLRno_Jo5QEq*gXzb<`GNzkBxV#u!6Er>-Hq>SlHIP3wil39FL5yZm98a(j)_V+Jo{ z4V4@oBLiCZ0^Am7tu!v3TUW>NASTQtGN+dW?6M-sOru`$_IzjoQ9tJZ($yk1ht zKfbj5ST9P*2RFOkG9AjIp*DDa_tP_f&0+YcVJg2u*Ci?b!k3myMLu)RN>>$KhmL%^ z%>UDW#{0Y^0IEY`5Bn_i!FaaY}kiqr_SD3tLL%Q ztixz~sv>4GpjokaUjLX@dE@EeL+e;^-Z~?>h)k;xjPgyMFv=8O^;QosWMUrtDk9H4 z+yB!eV=cF70s8EhgPk{_VLUf=5B-u0Gr`|J!)M}}h&knK)@%0IqRTn!3X{5-55q<= zx1?{_+6qVEf%T960)Y`3cPOx*d(+e&(Xo1aj{JC|qdDOQJsH@A^?Ib&9m;bXar%}q z>>Qb|>1umf4qwhq7QX)U;g^ZkM9t7kK*D4nixX|rW-I*u>iPU5R-sXwAu|ymK7^#0VF3kC_3$&YuLB;t}Y6B`EM#uv&UVH=p8tMPNB%sDbx`F$=l?tQ|M-s|Jmc0K062Ee zlMnO=cJ~Ki1i07U&;3?V2mtK<4yK<854i=xJYf73*dGKE2*M|KV6Q*n3p?L>fJU|!;!pb-PCEe>^;6G^qF~D+6JOey~;O>WaWC2R{Cfv{SkBmQW z|0VlhF#qpBf4@+W=RYnP@V+@1ToT|A1e*yBS%7@NqEx_gf+7chcoR zcsbP9)&PVR0brNN%^+jjKk;|z-to4v2VrqA-2)zCYV(iuI{`N=EI}B2ch@W5K(n24 zc5r@o4>km4xwEfj*QB?n=}sGi^l$3@`S$J@{X zgu!w+gM8dBT>)W95KasVwA!f)q?z-Tr=Q7AU10f~UAIE6{DTJ0`2atQow`7Nxdc6f z|KJfUlXE`A+Xn2XJMVI7h6LIC<6SPNn{Z>(f8Zl;`TfxsLE5;AfVGA{Lb58E&mzk&l(o~ffL9j z!F7`BH29~(b(u?#>kQXf;4l}I>l~Lp*99>3ESJt-6kPciIu8S0zye6=pXJ;JLcyAM z=&%FMKb8zt`0Jg+_k&SJohkoALIV^_xs%A+~2^@7+?UlfZrb!`GZu!|7eNe zzwlK4jfXZ6GX!u0??ezU5Tx~=l>S?b-VAsw`FGBaBs>!=s20F|<-b!;j-FgOIefC< zWYNi`f2-r~68}v~oavls&KAxF&PL8pz+ui-&NfaQXBTG;XVYJ0{^z#;vy{Ku{tte4 z(ssDrX@!5*_ph?{1pEZg3rYzd6TBb@5tJ3Y{1?_?K?%WAf)@ql!5pc-+Rg9p{D0#G z4mg6c`lnR?b6qv>W+j1fC#YEe;p_Ppp;97>lByi|LC=6cKY%^ zsn~6_`@(Jm;P7t6-Dh{7+I?vU{^!WsEf40M1@A_GQUB-2`3vLUIPm-zEKn+cm80@U zuiNRXJ8y@9bo+sHdxm%-z_|ke3-Rh6|jYGWe@3ate0MJ_nb=H@^)0{#9pfVVomev1G zQvh|MSUv!}*K`jJ3j4D>aN_L+fZ&^_fBWNqj=kqW-89K&uY>xECk+6YPuOhwD>j>1 z2$ug1JRjJPO$0)W5b&D-U~ayn^}%0X;o^`4K-cUK%Jey;{(_A;6hH;OlfDZ865s&W zKX7(*_0F}UUpfAI?s~cFpBNm#OU{47Kr4s=u*U(z9qZx$=-Mgd|EPeSm)Wg=@E)L* z1K{Kk26hQ^a0+vKmpE`xQdUl0 z;e?{PhNjk;v)YCijEqe#nqIQDv9+@Yo7dgL^QMxD$L!qv z!XoA8`o<=8i}q`Khp!zz|9SmKzCa;x?Be3$_7jc;B6c4vC~#$u^U1CeB#H|iL#U(*8bq^f5+JU|3{qt#n|6`5rDm%V5b!3 z6b7IGi=_a(+;x`^Gy(p9{SeC1hem&e?Pdcv*Tj)}WQkNi)+t(6H)=mphr-jz+DnGD zsf|s|jIIcia@Kf}%0pNgZ>&DWxcC*4j}}bfqvIQS5<*jxA%b-kDfS&#;bB=$sSh^B z&GpS1;y#YwMQky2^QW_z!YG=8JWC0};uB^AU&hYy{C}S;gC+}6duU2@6U1P5z5s(i zkdlCh>|;qGyjmwUC|>ulTC6^Z49Y7|Y@DV=vM8JB6>4);8mNOl!UD0+H&wXUec=S(B5J6l_D_YgS()Dw)Aq+B0LWV>qlnF4e1LJ6h9xB)E(gj1Sk+ zB)G7FB-J`PBsqI4E_=(c#a~rVr%hYFqi-(_)0qV)=JV3-E4>k*s}a4i*&3@EB>sivjeH*3AaORon)X8@Vr3(_qDjx1*q2O2z^sPRkdF7*Ze^P)0+uYuvo)oNQac$DpuLe{X5VHD$j8d(5jffC6A7Gf?YOKDmdve6SaSb0n zys`;P0dPM>mibp4RA%Vf&1-<(e^v7=yzCc`=n+S$}tviZuPv=<99<I`{OL{91Q;zzSXGI3c5Y zczOBGIICCYL~7u3Ho&dOtRMW8dBz&mkvi5A-m96fS+&=!i5ezVoOKD4Z~f?0R=VYV z-{IYpT2I{RfzpI57Jt+VgSoM(Mi=E|Y~sis+DnZ5)B0*-Y``{q{n{W26izN{>>Z4D zJu3`g`{G}e7Tk8F#HOBeYM4h#GHXybJ z_3k6owzJNHHD{vQ4C(?CrnuCn#wPpPkF`$;V-I{?l)&VR>FQPZ*lP~Jv7$6?*{KH% zqZ&l!`lT-ntx~x5g>7uHZlh~*$=KA?w{z(cj+p&7-$Tnx1oj^Pk@)DSHMQ=C=W-Ex zmip!4kpsb(g3{mFKP^fd9QsJ}QtXTha>#!6zFSCJIH9&o3g;_Ps#ti|A}Hr-?2Vjs zpHmi=uI;c`Co%NXt?Vp@SPzvA^vl*JTSgfgQy?n`$= #ev!3s=3C2jTKm1BO7Q_ z(do=K#*`WQ-|`EQwlu06S^XaGaQ#i1hj;fAR>SRMaAj2y`51gm!HLkuyI9K1CS7wM;&P%Tdur%f^OZI{i4i=q>Ag|5Y9TLt~@l!Op}6eAmEjbZGmP8v6CIB^)>7G%kE~EqVVVQYgWNA3sYvRsi1D(6AnDH9~!UlXG_wxDmk4;AR zKm=Kb5T$MUXK6KL&h}{e23jZy!hwjN+!G5ER#A;UN>f#FH>bsre#h183REG^```Jv zdSI1^yWV-%x<&3mkaX_a3)M^giNoE9n#=QTb z>_Lkc9@F4(&7Vve`cZk!8H;&$M(GS{I(0;pc0bZ`+>Gj<#JEb7mTGnCM-gz-{-3h+ zh5AFB^<)GDm6}?@B9|^?o1ZotepCz(-HmpOa;NHvS`ReqyOPPJTN+`h!CZk;0Ef|YD#{}vVYywxWqKH zLr4;_M6#$LS<+0dGfVV$FKyR!2qoc!W3#yjoV~Nnx2SQS3rUWfUV~MP6GVqPrjl*m z&(X`0(%;0l{WIPLb#`g@mmTQh)1umnJUp@%urV+?8y?v-xeUMlL)K^G#|iU~lZ2e$ z7aLWyIU?AB*@We0^`Z8$ zBxn^xt@w%MjR}+iCG|lwbOt4Ws2YV*MI znpa;t`~^ysc<;EWMe7<37BDf0%++yNmvyf)dA;~^vFqoI!?0}Ak899puOl^{85l&x zWmsm-%sE9$!F4V{KMqCs_eh24vH{g#YU-{DlV)%@6ntV%nnXnUE{PA-P+YH#2Bk-5 zkh7@iw16p*CoInBK2)tfj*`_mDMdlWhD6D?;>gx&V@=(zrzqk%WmAhdk!DNxBARhu z7h!k=gTUBnXC+pb)DBc@=txVMWlkz3yKtJ*HL#)Y!=<&aWvnap#}-^IyROuA>eho| z@$Usvvf_m|m&cndEM}h~61e%QLw?&_DdojabbonKe9&|_zv=U@e2pKGXOCBfin%X9 zXvf-hVf%1i7mN6gr?Lc=z^Or@=$^y71alMWq#a+!3D3QP2g=jVwtiOH9Da3?QP5OR zZ_4i%+x3@6z$mcfLV7TZp?;R*YIrreNAzGmSF-^5v=xnqsK!APnnvO{)KP+Xp=6c- z9I_8-TII6^-KWpVFrnyIR4+DeC8~;;B{444YZ5{;bG_V4oqnA*y~XjC4eXsZd4R@o znS$Eaj@hunuqI9^_(~ehC9?q!`iUypEVF@7yq{bXGtCX%(2VIvGdb;W@9N&?tyv7B zR|L1N#1p0*20a>p@nov`ND2B)3yh{{x$?Q5kef=)h^<9TQ7>TVws-gRg^FB@{wRF$ zM)D^Eo&7R~ZysGw{rQDQOJaG;UoAz~kW2Y}eGgXS+XB_ec1v-uoHgZk#3&YITll1oloS5v^1asiMQ; zOQwgwCMO-)fH%R(5iv(S-`LprW_*8s0Or2Xn6ar>?U=3*^;v(xxVkw`LEb%a6z+?Y>vJU`^wk z*21UmhdZ&-NkRgSza;Ef$LRu{ag&8!s8W|oir#ykCG`I^-Z_31Q`S49<(O~vPtdZc zvNR2nfIA+?%McHtK?C|8>5C7k*0q=!^>4Olk2(zEJWz4bvV%Y37x6PfotA>IvBi9G zq;27ZVL3yLrdCPlDVcynMfBwv`XdZ=j&%vfe*SjsL9%=Ylc!Ce|BE_`4*_k5kPTC5 z%BXlJ`k`;=BJT7@l)EMW{N!QUJ^IZSY7DW(**c~Z_U;|rnwD&Y9$r}q@C3^XV|~!S z`b5sMF6!y$$LJxu!M5Z5CBsi3zBMFSv7%aIawmFJ^#sQbS*0nXA;iazM zOv@tIvhswtY-;3{e!e!XM5S&|#j+Hm+gV(+2A+0bNPIRMNYpF1cpVw;w9lcg^kBpJ zB3B;a<0Dmi!*rd^v!~(bu}0Ry%VGyVI~WrcWY1`%793O>(;FNukM?>M-RHDBUn?H! zUnA^$zP}Qr>pbVZxP<0I4Zu&ZSfjp=2* z=0-n1n%Ivn6e~1KXgIX{{~WkhT?GH^!>6L1_4MUPufmz3plkif%TVvv&C=E0us8Xs z&3e=R(6&KZDHTmSQo+1$DYVe2mAHXGqqvZUEjb3G$I$f;d{vyyGvXd$O4L)RdR7CZ zk@IVT&D2|Lpze`pajGXxcopzG>kYAXy174?t9^ zR}v3=*6n}&&C|aqFU`Kc+4lvwHXrir>LLGfyw=NGg8jQfKcY>)>{f4Q^c$CNlrhXZ&Ibd2p zk>*rC`9WpeZyk0ZdKf~!90)-)(mVA}(t2i?VT3i3<$dfa!)f|d%#>7Wtoqu6XpTW$ z;g|FJXY1ajPQLS5P@P{GyJ$mL`7EpVw9`VZ&kB;Aa;Aqn z7Y6*0!v4N0naQEDBUv&y-`!Ozm7SOOM!5^I0q8tf@>kSPgJmT)(3g#)siM9ko>RgZ zB3PU^h+cVvOlmf#P@!xvuO9F1E^nYz=RF##RWP7~-C~J?bsK)-)Z;$eeYW&|ZWu2^Cw1P^(sn7WkU{ zOl1R-?5bGnU2mM0SZ5NdR@8oNy9>fX^@F9pVFTY|zS@J9xCV-RjSc+1k)J#zsV~9$ z0;^MFh!m1a+xiYHV0FlX4U{En!N3L^*thUx%Go$Ha1Vo%)-(jGc3$W$U&v~le7wG3HTeOK#5^(B+bYvdw?&52Vkb|KN0*<_)h55Htr$`Q z4bnR#Q&ZGJ=0n#6qkCXnPAnHG81q$gWbtM5%)$!vcP-XIP;YEmd$57H3d*I4nfGy6 z8H5xmvw`?VsDQ=6;H5>(RQ`?Fncm096xiVrDb??!q`8&lci39C2B}jMRuX~ zHAj(r-!cnGeTn*d6yt(&T1^-B#L)Aa%{5`4#`uD>mS=A~VDcF3C0kGp8v8Eu&Fa5>})8z^4(-$Hf0*kYxK-yt@q24gs zQIkZZ#A`$oF~5xs>|#jOQv|cv0DoKbZ#0})CL3s<5@JBbHe+Q@XPNjmIR^SIR~0wC zhew-<;%{|b66qwqK|Hs%jypS4m0BvxJDlrk_lEZ1WHi%M=Y4~(Z{3s4XVJ$ph0Wb= zpZTQgbLMl=5fe_U-(JAmvwlN75G$#XylsBs=R%}Q(+2b_8^Gi<_rqu>U@RFmnpQd+ z$khl&mC+#E64#;RQcX56J5$IE=t6zLgMN7I7&s139k{cFhY@iT6X3N^Rlu$%vX+yREbh(~C5Y7vWKXId2@>!)Szp$P9j3wc7y&LZt!;A8g z2eNH8oLmgFM%J!gCHXw_{&>L8PBZSTO$N+M@+^{6s^>9OK`b8{Uh>k+GCS6d^0rLL zuPa@COGb6!Wb}0yVc_V0f{-Ah@Jx+RsEF@6(h8iRqR|x@j@3tN-wtl1aN?)9H(|nv zRF~nvYLyKp67_^JIzphl8mJTq zWF!ABq)u%d7cfOgAZo-{yhMi4oL?a@Gi-n%%-PQ3rl?j<9vlTH3qD_MI#Rx0UkLoI zPXE1#wAIBWyc!87I6?5OhmGkg4w{WEN*YWi?6q$H7~D?0<#4CARMDbH>89B60=XAy zDM)jlJ3ouVcaIw9m1xj15wE)rRN85_oawc(nKR+o)6i*Z)NlnqP*!?6%-XbNHMIdX z{XBZli2i=o;G_t+)=<$W_>HzSjbZHDV)06oQ%PdDxMrggT7~fdBb>a}1@_^DxMtfD z$EEDc@5X#Iso7o6Lo!F-WIUpRXZR*vuos?N7J3Z5^DlIz}SHK7atW;5q-`%sGKA%q>=NrnS9Fn2?%!HH6 zQ8iHqXKxD4881>&Ok^q^)U~bhy)&62P}G>Qm87h)AuV;u+^b{qSf12UpwRrC+=&Wj zO#As4<>0wth>EeP=kwz~Om)WO#>33aS}dD#9R_dDT26*xGo6Yn{C&P7HX(wv@D3C?adDun$VXpd@OxcjE#;U| zsq6TX2t2{;J_6nA2F{9t5+foblC^ZjZy&YhQmO~jhD}$~30rr--yD;Xdq6kG+A9Tp zrdnpq&rG#cH4R!9zhPEK3om98Y7&(p9iuqGy4nIlX{R*Q$i=xmdlEzP2i-D7@@fn* z1#+D_J_f8kX|YOv>hXQ5 zVSllBcF?S42sDrv!RW53)~y5tm{5=l|<$JK2L z9h|2+b#})w-2933D)-jx)CBd3NQ<{40U-zcHDLK2Zzkc}nU4tZNw48|w35&|{#PrT z^~!p=qg>HBKgNvEz2XBI1b>)0+M-u8-QH(uNj)$i`knpljGScCF|-#pMKjxc@g%3$2YFfCxQ}+2H zMa@^^KWYPdRXi04D?UR7va-!iwL>iTXmQZ;xrh}xTI}V;&QZdbpI4aHGx>AM9hi@LvUJM(I;s!GB4a3QDTT>!~T4O z{!o5Bd!t9IY~TvHJ4F-^3BAV#g4@}^S3L;xsC>ISZt4__D(uV#EH1JhvjO6F7`M;_ z?KD^&5J>OxV(b@2ZFntz%deSnF;PI_f7joe-$b@#@c~m?DVF0CJDFrK^BjZ=Bwhy< zn9mKaH$bOf!RiuHbtS+M_GtT8mH>li80F;>e3|xkdLp=8!fCdaTuOaIYber6($y16 zRPU9tql*{2$tH~=q1E05wFk}IGkX|@(=gG6eTz;U*r9V!OARPttNo6|BQUlZ6^?g$ zluKj-GDz4cmWO822^-Hllt*3fb`>91)77d*`5?qJxGC|^iK7$`J8iiqBn9;eOUq2@ z^P?^legw>TxLH1&^jJ`Ej>Tt4*Klt9q=pQN?!A;A=lo{tLvv%#-QaGIG z(cgTca?xqzZ3#Z20OfNx1e_Z%Ys(Yi!BtLZtt}op?D4G@_0&k2Ucp7 z0h_MUA9Q)*{>91i5G@>^9bZ5!^BPuJnY5HQAD$fuy_wm*Ptctu<3g8{GLt@sXzENU zUCa|qC&^`nL-46mz0AUC2aO4@wk=nMy4nPzbgQ(Feceui#nTl{a_?LM1a1mP6JU&^ zD{O#w4?weL3Gy)u7>;z!=vHu15(QV=g$OnpW;m1ksPCB>jHm*dGfAc}kf^eoI@{AF z;)+e45da+^tAR?!34gr`OE**nY%y$vpz`T{#E;v&CIDTF#T!NJRk8-(7NTGD8O@4T zrj65Gyz^veOO$m|SXBhk#rX=QG-b0zhTm6D3E!A?zu5?k)blwbrD;)x_`vD4dss;_ z=7C0y(0sn2vU*O3)0=a{>8`HMjQw&iT_<`#oWNb$%?9q_c(LjRMAOiN z&8oQl`Z6@jZZCY=62C`d)4U}qO5Qr^@M1;DLPZ(#?l`jrS=kw z7c9c~G+hBjyhj5^&nGkhAurX?I*Xy5lgWPOCtuJ4Tfb(M4AT6>!zYB~o(ww&UNZq2V}1IfCl=@QYO zB!!X~C+J6_2e5MF&$uGwUK+e~Ns=eAPAx`8V`#%A*ks5bd0K2Tw|v1V1>Hi;f9+&0 zFr54C)HUaxL$9j={X8p!Sv9N+_C3L$FqTL_vxDDDzK?Pi~3?90G5j(N7K-HAkwLO5e(ZRIf#Lrxe+0? zAtC*hsCJFwg+eP<5S)Gl2G0{AT8LAl#PZ4g`fs+Kh&JAiv!!DDB?(3RX zj4fFCj9?@y>-kF^B1X3~M}E#E#g-G722&c!uU71t7z?%#dC;^HUZIH}N>MiAdxule z9xA^?z&cp`7UtrTUSEnq%*>QzP)?}-pn$C^YDWD;iEY#(x?CiU~ZF`ZCVgWdLR zn%vgg47P-8ZyJtKL5m-z&3a;>u|7n?&8m^jKIe`~jMfcAMy!WT=#*JF9{I>SpL!c z$u42eQ)9u?Mnl=d=vlNE$*d{LagLU`@){pXQh{2IR=YQyKE6F1mszF|x#S?9qFnl$ zT^*^a935U*n=L~A(CW3cM*yL$S~Qo1?v`p~8l?kM5RE4oEtnXAj9#NhIJz*=>+>Z*+CT&6KV zkGw*=OTPV6$+DOj5hV9$ZE2D`oF@vh=&WO;SJQCeFzrw=Kjx-F8*FAi^lkm<5z9W z&rc4VSjhzsAHO)`{JWdYqaW@@&#x)o<>)gZXA!SyN>*nG;h_or@H;Cri`tVpkQm)F znUY-v$F;%+U_SM}CCO3ef0{+o=VfMB&z5=z29Uf?1poG)M+dYX#+Ig0ahn2OHvRhI z2-Oj~YfC+DNIvHRdT3X#SMij0=56?!MYHhTNp4<+i+3`g!;9zNE^PZ(#TTM{xr=U1 zUMqSD0nMFjENL)aG1rzqh9P1)`uM#kwa7vbQ2}5DmL?QftgexlOyNr~hLY8V3Hmm_ zJ(+XHH_FTu`!iG8(7egiZ6Tw<2{62OC=TR~Nbq6Wh#rE=6=L z4K8Mg)8byE+Rp})mpZ&Tw#&{zp5oAft+<$}x^Tw+K&l1}PMINg$6{3&dbP|X+;l`#N5k%*5<_#GBis|L znjFvRBkE|5Am!$$gNrQA5$KJzm^{hRjXfG4&~?K&FQ(*A0TsrH61l<|smMa&GhKsQ za%A_m8VA~2QhnyAK~@_}WoT{n26;9`uHo?RksNY3Ymf8mMXhaTRxcZ<)yxX_nt^a4 zj!tN#-ZL-#P`FDc^1g^N!9b8&T3!Wy{5BVSsj%aHrQN$k)(>uA63*$UPt zNNv{>`9uw+HKB#=`nPmlA8pM)9KQZh*7D`mlPGyLoebzbAS~z1MhzK>a)`=Xkgw{_K_bE8d(7Q+vQ!RS{HJIivdf z$(^0u%cgWKbG`0_I>(Ys%eYBNxNNqICc*T0m}Z1m_|2Q{A8wc~os%)8pG1n#Oz2vu zwyQWFSR9T+Uj|`Ml)y(>Qw5m`jL6pra2v!m85wfbR0>)xb&7l?UuaQZDeJ-;VfY*j z_l0qm$duc%-N)Zk1-&p(C7*fGPsQq_(CFe{J4`u=`42%$2RcTd(drZuVcZzrtRbki zM(a|FQxp7x&Me-Xis0(-PWI-1?_z{Oxy28cOEdRit$d!e5zI z^%Xn8x1)v`)a=lQSAd8qhBj`Rhbayj2Lqzw;I=?+9yT!l>?J2l8b;=1B@3;9vq`cl z*mf~&VB_*17Fi7qw9AYVs9?mkGYj-$*ub`&0sw|t?HS=U$OzF=zh?y zt6%}Bem5o-NAU!iN3oI~0L!ccsJ1gGubMd2{^o;}WJ?OqitJ2{6jD$7dmewzOyEj5 z-5QDf!q5mMnIDbfaU80H@iBZs$8L>(A-i`qIXJXfv%#~lt+KOZ66VkrC04ch-TH3S z8n}xRiJ_m5?ts1BT5*3(3wH!J2i}C(DUR(O$8EqEJ>BjF!R%z@GB! z2`Hog_cH`XiK7gW9u{vWIiZVnh?YUFr9zmQENSFc+n)b(EeQ;+QJ0J>MFz8Jah5%LrX!i^ZjZ>uHi zf(KoTT9Zc~=&F$PH2WOM;2LMMUX`PH$&#cDrpRaCHwT|=i?Y&h-yD6!B_5mmhWo0& zu@voEF5+DGZH-o>XfZ^`yRc$3UP@$#u`j0+Z>QExzPJ7Vl0kb)jVh7dTH8$916tr(q9abN*CX_{?>a z_g?}wavGokmPs|i#S;i;)A}#SQ^BPcpru==@|1Px%kuJu3!k&sBlBaX==#ArDyH_n zf-NfWI=GTCDbEIgVm8o|z>IkaF8(|VWC((*XD3va8cSC&q+}g)oemt?Jdh?Am{>;_ zfLPY3L(akw#t*)29`)N=)2QA9%O~-3j)i6Zf`Ma9dsc>FQz4UxdIV5w? z3iP!#sw)faQ42KE`N(Z&inGG$YYxaWOs_m3|kJ>>5v_sN-1&q+I@qY8k zRws1DmAQ5o`?3y=x(_f6ihr?zC6+lGIQJ`y#fwL+yG^kQ<{%rQnc&p149ewHUNKpM zG{x9YpEha5B|!G-n$gche*}N{Z7Ly}cQ7PS7->tp2lin_+QjV{HOP{7R6&*_MYBp8 z!w?zstAyBKeV;F@n~A71c`P*Vs3dC?$EtRN$~GPKMc)&o{MHoXtI*x`(D2O{7#!s- zjM3#A-2pwsvvQVx>jT|1buWY~l_1L6(*t(zx3p>91EDoEQ=wRs&eR7e@)aQ&a1G@o z@`8F54;?WN83zZTi4zOok3;K!psp=f>GaVLL2qMQ`jR4RsjvQ*-*ezhr_9}DNKj1L zToq}zDN^kOi4+F+ObU+;8TU~42+3b_+yT@pNxC{!PG;`q^V?m|BM#M%)=P|}n{koN zrIeJ?PPU!|Sk}#6(?a7pL@22LuP{`Yuh!~%-KiEd6v)aAko6i`Aa&0uI$v@Od4V=w zfz-y(66g|^GqN<|j4Y;ZL}LPVVX#{N0K&H&Lttp#`GytluNx$#iqK}dr^wl~nGX0r zH&HHPpOAG(HOZCHLJSkKTJi={1WE%J*8l_r+#xytT z7(nVC>0b_+-++ia(d?vkY$>0p6ohuf<~_{{=?%;2r55uL7e{m=(iP(hI!tH1UinzB zD8T8Z6fG6!=7^lDw7LNTVLND|Of$c>re-a@`9@}DT~%e>qef;{VasaO%_K5}ip%eq zR2ZJ3CdU{LG{eEg4RJ*I`sIPOK(snJH4a@P(Ye_8h9dBQ zW|M0-XfqmEr((R-vE8iBEao@aGJoHtxMl%A;quuoi_m1Q@2_0lU@cKw^JxnG?rE`H zdSIrCqp=q3=7%Y>rutk0=-CI*=E$dsLMsvUV1xu1q*@Um#G?sPb+@~vkNmTCmM|F+~lffr0Vidt^9CkR^fhT)#0w- z=4FL2ZL_N^i@a!{r7(e3z4YQ;@mV9C6K19w9~t+zs92hTeodqxXmWDvF;kHz$lH-o zntc{znR}G>_K^O%A@V0GWb8H~wIfTC)ubCOL(wk|uP*+aAN;a3rLI@WnA|&&5Tov2 zPYBF>w`{LC`{DG#;pP|RJ^V@^)=NJT#rDw250b6ss8&zQLO5n}44`_b<*%s!Z85Js)-Z)+4hpDr`oYMA6h%U-&5HFUl# zxbe}I^vsm^gIkBHYocy^i^wlWwMhqq0Z?!U(ahtID3VM*?P$F1@RK1XjymzX$Mu5_ zG6x-YTR%Q`wq`=Sryr8oGF8Fon@@Xo|9`7%DAp`KFq9}e38SAcWr%}KVhy3m4a?VV zxI5R_{>oQo93bO5P>)>CG7FA@hNT((Dvh&kic1;%s*)uajk7hMn*lE7!Or#TYQ1K4 z!fMw#Rl3n6^^a0CTPhLV-THDvKN_&jBcTzU!+9wspv~{}f3^43VNrEm-ykI| z9U`D~gLEU^HFP5qL$^Zw6Cu zb4~5vti8@UYwxx8KI`Y6LbOD(3^r!X56TG62=bjt(kqSg^#1B{JngA46T3*?L}c; z#gRQf7z4#-gf(|q1n#i%P%UU@QF)v9DXaad3c3NzXdEKiK8txOywF)7q-Go8HlJ-V zXcp7TleVcRFw+!@9S0KV16MJaY)8h)b*VO=$kL#I8^Fpy^ocE3@8Gn`#5tH1v&4;W zPG-;I!bhDIdH}M|EIWsLs1-|h(w4_Y;bU_%EgIG`UxG*qX~!-Y8) z^+|VDMDv0ZDAw}HC+C7JW%*3T2!&!dA?1xfBEo6GO)h$ zo}s{IGBeH1FWYdcZ|J^YuZyh~Hi-;sP+Fk1j|hysGRj^9c$wZxF&9VrO4YA1BC6JVaZ8Fr#_DmhROB*n?E!+aKQp>#Ch zdDHA*&(A&+p@8f*HryCH8(Z##k#i?A7=2V4Ayq$8e4zt--XrCoK)t;kx=C`fP7DkQ&`)md{)J2Jra$w&_aHC&P zGQ;-L=RiMmiU4C|A-Q>A!BA9xkbA6jBp=uX4OJasDPS(Ifw6M+%73uO?N)vAwTl9< zsDJ{n`s1l$J`dF_t?;+o+}1h9czPv?7FoAQvtyJ6vvL)`(J}Rd&5H_=$haZU9OaUi z&>VHA3i1d#5^2>JIYahq%Bvd*{HT85{s~rx8(?yVm=-`pmx2{u5$x*0!r)WFmig!i z0b_XZvGU7#mD0%q=Er)5$-!A{&&M_~H&$JIvS~&S-een0yZGf!ykvf}e|%c1iU0CI z@GwqS9#m8J?K55emomBZXuy{`m{0BG?30phAK0^<7=kxq%~TZhr_~~r^lEDC>M*^4 zEtk9t4gqKI-{OS`7E@rfevMl})pTu<6Tm($9P1%K`MM$~u(giHdUTlSiNvWOtYzV* z9o0P_azh)C#^5R4J20^9v|pcVo;9iBl)36&vZ^Osjc(|eC>r`Ep1HlA3NQ3sn(-^< ziVS|uBt>C))GCIrSspp^$}C;IH6Nyz!ulP@!|RqttmcbG60@Bvb@@>6D`?qKwks0s`Ufh`Ls4kGExl+!VaK{_(_@UacgtdD9Xf+U>ghK zr#cxRJaMNt(ys%%*G86-@S%63vtwh1$o}`d3ZpHCy;hbKBk4H9)#SjNSwJ zmX<`w3u8HxD%GCJ+#Wz`qJuPIrqIW4ZVN_-x!G5jWvwSt0QK9}1J=S`qWYw-9X|}= z@=;KFL83cPsI8~+)NPYt?irWVP&x9@B5GEXjU8fMAg4i&1#wU}4gE(tPkGk6>a%Y2 z(sElpg>@gZl9%#7)M6rkN;LPL7R+I2%jLG=>r*SQC$h`)w8%NJ_NInKP8oJS90AwLvT zW;m;4ZywoiW#%~I7?t&;WjI4GN%>(s=-mi|I9N(1PF|@Zx;E}b!j^}ODS-%`)$&qgQ+M8PScv$*v{-pnw5 zky0Qmqd+*tvzhOQJEh!_X=@;XXv+?iIrz__)e~B6F(*^Ppo?L3Aoa1lav$k6{<3aa z`}};@Pfx|LYkq7G`rljMf2`5|e#_6lzc={*{+rn66RHXbi1={GKrIb8xZ7~1>a;1m zK-Exqpa2p+@Q$OSD^OC)+|K;_BgX}=^M-D}JsCG~1Jw@J*wn`jdmb>Z8_1luhNTRg z_CS9m?O}V(Vml}K#0+(JvClQchgh-(s_>zAurO;C;neu{2w=ATJ(Ie@nQDkA(GoR_ zM?UB0u+q;5NsHlcB<@fzdC*zU*h+?c;!v>?S(S_o4!miD6OHy6)k26a$x8ZxtL1JB zw*4YzTy!{PYX1`Q7s(v;w)Uaua8r`**R_=VwKrX&Opzk*m&9K8M)c;a7_sKkdU;oI z_xchaLPcb|87Usu2yCjF-R?Y(*-pF7 zuZ448hY>OuO1jLZH(6Wvm{l$N@gt zL^{iUp~f_njUl>tA5QgKW6@iF_0oWFCyb+78hQ)8 z8#9#Sx|IsW20BA`qr#C`H0X}sH>#4j*gL2g)e9fN*9jk;?H%O~e-=%uhd3Rd3U+r> z)duf|D!@KR7M>j+Q)9%kZuhSD`+hsxWxpYBRnX-CF#UF{v-$ZQ5vbN4Er-Fiy=bI* z+3XZl$V3tvHur=fleM5jN}R(}_Sv=O6UO~UY|Hq0xB>-l8DX3OcJaoVVawzfCpqzq zq;n9_UZFXj6R*@p0lz?(sB~aUJN^BQ|5fy z5OL?X&wV?vucKO>#;W({3vv+}d@UANVOYCQy4U&qS?$Oxyy-NO$L56UH{Na@S$sxa zb0#eDd`)&N+xhL?a;@Y|+~ukr)?~)SARR2Hc7tHiMn+7ADp9=Ruyq9*F#%JueY`v7 zAfK?tLKc({VF!~QUO|1iyefic38sALz0DTa$U*G~qv25~Ll{lCJRuL@!V>xt5#)S$ z7}a6A74F!)IO7A9O`~)g87B;^CXeh%QSo$3l4UgMV<+$0`O2)}_uREbhDu`v4@m0XYOPsM0c|#q&;cSaO!sN2a2O;xs z!9_Y=c(mAvyJ!#(0_6ogE;3z9fCS6dh?x z6rQgI5~9+YE@mm})Lg6lc%F|XOTwKb{aW(AceGPRoxJOp0E z&DbKN2h<7tdIuY8dmyb0G@vr&Y z-J6bZ+H!bW0L#7GrnGqeiosLx5#E{eTdEr^KDmvuP#d}h@eHzR)u@3wgw_V?9CXf)q%X~X(1XT=AGYou#Y)CN*U-HSk@rTemE}m zS}`Hc&5l3Ox={1=K7q=;@+r=~u>?cvnXT0g&ycTe9_5g@K33t4qsfA#b?cCg3EZFz zCpRn(e~#JYseTFXJ&bF0n9wbdg(&fdYsreY@N3v!WIf_6;$sNl2D4~-)du?Dc8IGb< z{w;3qhYc(vJm7{K7clpv>|>U6TCSJw=Xvi$tVzw?LiH8)3mKi6ABtOfJG3kYH3=Dv zD;?B;%(=TG_BadB^fvI4w+d2Id=?bmE8fy(3^vsAL`$sHxxS5~VZz6_`TDWH(g1BD z!<*g)pw84ES@aAWzR6VppM(QdssQ@o%YoJ%Z2s`Fisi*98?Js&xrO~kEs%BUekiMc zk6sOgDDHUgT24?+ab=F$eSM?KbRXe$Y8F+`xn;wxp1yG5D)8%s{HC;u%uf>@T_nWd zI3CRp#-Fh*&HC7{r%Bpj#(E=kI%HT0f}$Bta9Qd0O3mkQ|{Op5I@QAmxB73!f@0V(?%x*gRpu?44`TVKYf{KgYe zs1vZhDGbH-yrQr1U1wpo#p}?OzN{Jld=yI}B{?uyhW&aihCBo2w zax9qzlv?j;g5;r3c$@1TG+Pqk!~GD>`yYmsk=%IwYSQh*s+v6w!7Y=~DdrQJeG|0F zLt!f~mY?LSIh5ZyQXxLDl-5CeJrh*&sYiE6pzG`HYTNPMEvcc_e9!} z8~MxTWgg6SAbcF$!kI^`%Y2A-_zVmSdHj?dy!<#_w4XY|T}m|YIyJ{$he)N{ zVRxo6a0>r;lE=SAMCJjXQ~>{Y|2fI7ruS{lL9QOQ<}Mef8KXxID}bF?Y`!k^q1BlW zi-a0AelsQ0!!>xu*xRdOR%y74)8q-*IpmUw0iA`oaW*p9%3i+=Dl#d?#kxAh1!%|6-K%~ z%xgG~+W+&d(1jS}kVcrU?R^wM_csF)Serlb+lM9rlR-m@#~)L}Q~L@nB0VMg z&-{tiLYx8+mv8~}NFz*yEuFOLv~4(Tyq5ORR`T~1v-Zd2m9zZz&*dqi4X47~cJawD z*&1bD(|!cV?BnOG(9Ljzi~1%>X)#09!%+*G_wXQ?p}WZ+pCz2>4hY+LeIZ?2AsbCO zpF^D?F&~{HOu}7BzyrHkNHd8@(LlbeOLkl{dDO+7!0tfuIz)pHl_}mL7p+|_zUiGk zsSb41`W5wUBiz%LPGnEA>DFK$PmYr4nZ1u?7u%mI-`6GjDFUE)TcDoX4Y=#@ zzt_J7YR_Jn73Et*c>2`K5_drR|9_8kjV6G>52FryT=cq#U9hARDTY*PHnBQBKeNK4 z_tUyDD?0HksL?NYPL2n)lK2#gTb75ZjufLJW5=fzKrslg)O_6sN04Y7_8~^QgRqTa z2=TN9wWMimcD}!q`{U=+JBM52WBMo}*4Gh1@kO2Qh^G^WD4DDziOt4M(#z^`aFiOd zDwxY3dolU(nA}NHrcI%*m1IX7=}f}4U|2l8an7AnQj*UmL?9E42^w9l7!FAtVU(@Q~)khgnt}OQt&)~K*Z5KWEY{+N*MK+$i9)8 zYF6rz)`XP%4I^|swfPq>!m*&NlKtogNIen74<0(o-n=htqc6!C(0~gnD9~45(UE|i zQ%$;nUPBHt;8sJFLa9O>Ryc004aU6~k=3-Xj-%5SPo(B|z#}PW)0COMUQ{9aPIs~$ z)upu&)n{;72I3l2M}%{{V95V-J^Z&haet3%TXk(RTcKo(k zrBf`0SsDFpY%ViuYe@8C$?zwYnRQ}|W2~ETO_h~5+3)#m7Ylwq^Zc?}@}Yn^ngwQ( zcnfdFj2b>Sk;h$_wlEAVKrJT*E@w^*;+T-2j5$4K1jVWLwM^BZ36Wlxqn@DD;HJIV zDz8d9f-FzsIcS>p-nl-OIMP>%DM#cJsZ~m`px!lk7PTSdc&%GeWKnuTBW;(}Q)Dzm# zNgN3&EX~D4H~bR(ZYGdm>krr3_vuRa_wP(rM72z(#=KAC<8oG?L5@@jWB70qj;+ok zKjGN)-h`@Y>!s5BFd*`53I0mFA=mDe1s_rdm`bbFlv5|_-Xrki`q&L~ljpOh^= ze{&NjL=Kl1GC*S!yL;7dto{EEn zDVZ^EhHP(hIg*7@awSCK+X=ktB}nxjXww9Nea(7~LI#G1K{s~|?UD*$(%4{Dw~%fg z5an0^f?Xhq(tFy}`WCD_(#*c&i0Wfm9v?~^To;333p-4<5AoGZ7%4APa%|4hd0NIW zcQjfLTZ*?K#~v1Ik&nyWUYbZ7I%O7kVsx@)tdp-87HLb(J4AT0%Ed&DHoEOtDB$n< z`P1;x$9JXf)aoQ&XSdt;l41NR$~SHBOy~&-i9UqH3Gw7!=mWVaUm2-|Vapq+Sen%1 zT5XfNFW$Q=U2`{?(_AQ#{WLOcUL^5U)nHNFF1Uu2`6+{iKt*F*uGy1#F*V9ZXXr#I z#&AoD>|z<)^itWK*MlUfnjCvvx*}ys0;I<1Wo04jH6G6`Z((?xkCkn$+C74;7Nr|n zPs%QO1&>f_L!??o7+lVj$R~K;d=7BGn+Da4x>?l~k*e(9|48u}zPEyov?8h*B4ebT z=Iy=?nI>1?$E z@$O`4)v8G}iHJ}sNL{Oo63ptB6`qyWHbYCrV=Uv^B;}Lq)?J$?yj!OIJkLhZulKiZ z?Zt4bOu00yQw}~eYVmkT?LpwM&zI|=@j$ulh&t`QC(dUL_IG#uXonog8e5kird`8( zjWY^mBE~|*m8k6_lg~GKLgh_mo$-m`y%pzcAHA9^D@~4INWw8-(fDA#XXylR!+8ZoBoAU=KS>Eld(>g8ZfV>UycF#IFlFAavOy>;#o!;_sXhTk*9A2%X+M0Fm8t_nIDF zjIl4|b6F;qJu+0c)n*pMRd>qm%pdI8D^_BjXb&5@n{_-};;W-qYqG>t(^uNIMe+Nv z27iqO$kG0mx&1b%|7Z{x*lIAo#~MF8`NcDR2?xM`*t;5QcsQE77+wr?stUgXq-_My zVgdjQz{m*rrJoKoV3KLyQI_nWOqIsnNT_|c2L{y1cS6!it#cbJP%1L%;cE7-yQqC0;IwpHRQ zuLJa_98k(dI{U{_3UEnWfL(>T+PSlp>|jdZhZ{wJ3f4cG1J8spf9c$hZe4{K9gN7nrSETDQc?eD_qcki`hYUYa2Gd%H6$bTvGB@fb{%KX>Js3iXe`H}+iACXyp(lQX90_Del*T!GU z%<=Ek@hddx&3^^W{+A~D*ZQCWh(7*V>P!B%f7HjX*1>e^-yq-pFJ$Hf){XvIX3qZt zSrBjr|BTG_Um#0R{u|^=O6vbZJuBS)H^_f&c3=7UIYo2zdDiZCtvb z@8IsAW58D|12iCjXg?i!K+*l*@qY6cmv}#UimPzXm1%}j0P&E3^v|zw zOTZ8O4)@yy_p^8xr|BvjW{$&7IWX^i0gOJs!nJVzJ=~?GUWMC1wFut;q|^O_bd^A8 z_;=D>n&(wGBNly)Za}(TK)PSg#e&e^!*N_*{8#ZdqZjO~{Q)Nou;Kh2 z+@-s}3U`&?@(0{=fb8e*;4Z!8Rk*7Je?Q=603M;ggS#B&zQg@BH`Na~XT!f_s=6xE z)rI&UAYIwtWxBd7e-+{C3d0Ws&Z6%KzpOW0ZGJU({i8W?$$td+HIsc6;A(#O2f+S+ z0Dj39Uv2(thVw^bU}jkMlNSGz@4VXj*EG-1t#d1XYW-W%=W6q-A@~mf7| ZX+W;3fP{R}k{0+&1KdWy#Vio+e*mTV$`1el From 53f6bb1dfec74cbe8be9ae6a670d82eb6759cc8c Mon Sep 17 00:00:00 2001 From: raschild Date: Thu, 9 Apr 2015 07:04:18 -0400 Subject: [PATCH 684/817] SPARK-4924 addendum. Minor assembly directory fix in load-spark-env-sh Set the current dir path $FWDIR and same at $ASSEMBLY_DIR1, $ASSEMBLY_DIR2 otherwise $SPARK_HOME cannot be visible from spark-env.sh -- no SPARK_HOME variable is assigned there. I am using the Spark-1.3.0 source code package and I come across with this when trying to start the master: sbin/start-master.sh Author: raschild Closes #5261 from raschild/patch-1 and squashes the following commits: b9babcd [raschild] Update load-spark-env.sh --- bin/load-spark-env.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 2d7070c25d328..95779e9ddbb18 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -20,6 +20,7 @@ # This script loads spark-env.sh if it exists, and ensures it is only loaded once. # spark-env.sh is loaded from SPARK_CONF_DIR if set, or within the current directory's # conf/ subdirectory. +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 @@ -41,8 +42,8 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="$SPARK_HOME/assembly/target/scala-2.11" - ASSEMBLY_DIR1="$SPARK_HOME/assembly/target/scala-2.10" + ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 From 470d7453a56c56a41b2851551fe1830065f88b2c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 9 Apr 2015 07:07:50 -0400 Subject: [PATCH 685/817] [minor] [examples] Avoid packaging duplicate classes. Add exclusions and explicit dependencies so that the examples assembly does not duplicate classes already packaged in the main assembly. Also avoid relocating the commons-math3 package since it's already a dependency of spark-core, and thus is already available in the main assembly. Author: Marcelo Vanzin Closes #5379 from vanzin/examples-deps and squashes the following commits: 12c258e [Marcelo Vanzin] [minor] [examples] Avoid re-packaging unneeded classes. --- examples/pom.xml | 52 ++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 46 insertions(+), 6 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 7e93f0eec0b91..afd7c6d52f0dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -90,6 +90,12 @@ org.apache.spark spark-streaming-zeromq_${scala.binary.version} ${project.version} + + + org.spark-project.protobuf + protobuf-java + + org.apache.hbase @@ -234,6 +240,7 @@ org.apache.commons commons-math3 + provided com.twitter @@ -262,6 +269,22 @@ com.ning compress-lzf + + commons-cli + commons-cli + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-logging + commons-logging + io.netty netty @@ -270,10 +293,22 @@ jline jline + + net.jpountz.lz4 + lz4 + org.apache.cassandra.deps avro + + org.apache.commons + commons-math3 + + + org.apache.thrift + libthrift + @@ -281,6 +316,17 @@ scopt_${scala.binary.version} 3.2.0 + + + + org.scala-lang + scala-library + provided + + @@ -322,12 +368,6 @@ - - - org.apache.commons.math3 - org.spark-project.commons.math3 - - From 7d92db342e01fa694d3522fb8d2254d6297a4203 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 9 Apr 2015 17:44:08 -0400 Subject: [PATCH 686/817] [SPARK-6758]block the right jetty package in log https://issues.apache.org/jira/browse/SPARK-6758 I am not sure if it is ok to block them in test resources too (as we shade jetty in assembly?). Author: WangTaoTheTonic Closes #5406 from WangTaoTheTonic/SPARK-6758 and squashes the following commits: e09605b [WangTaoTheTonic] block the right jetty package --- bagel/src/test/resources/log4j.properties | 2 +- conf/log4j.properties.template | 4 ++-- .../main/resources/org/apache/spark/log4j-defaults.properties | 4 ++-- core/src/test/resources/log4j.properties | 4 ++-- external/flume-sink/src/test/resources/log4j.properties | 2 +- external/flume/src/test/resources/log4j.properties | 2 +- external/kafka/src/test/resources/log4j.properties | 2 +- external/mqtt/src/test/resources/log4j.properties | 2 +- external/twitter/src/test/resources/log4j.properties | 2 +- external/zeromq/src/test/resources/log4j.properties | 2 +- extras/java8-tests/src/test/resources/log4j.properties | 4 ++-- extras/kinesis-asl/src/main/resources/log4j.properties | 4 ++-- extras/kinesis-asl/src/test/resources/log4j.properties | 2 +- graphx/src/test/resources/log4j.properties | 4 ++-- launcher/src/test/resources/log4j.properties | 4 ++-- mllib/src/test/resources/log4j.properties | 2 +- repl/src/test/resources/log4j.properties | 2 +- sql/catalyst/src/test/resources/log4j.properties | 4 ++-- streaming/src/test/resources/log4j.properties | 2 +- yarn/src/test/resources/log4j.properties | 2 +- 20 files changed, 28 insertions(+), 28 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index 89eec7d4b7f61..3a2a88219818f 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 89eec7d4b7f61..3a2a88219818f 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 2a58e99817224..42df8792f147f 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 64bfc5745088f..9a3569789d2e0 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties index 97348fb5b6123..6cdc9286c5d76 100644 --- a/extras/kinesis-asl/src/main/resources/log4j.properties +++ b/extras/kinesis-asl/src/main/resources/log4j.properties @@ -31,7 +31,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties index 00c20ad69cd4d..67a6a98217118 100644 --- a/launcher/src/test/resources/log4j.properties +++ b/launcher/src/test/resources/log4j.properties @@ -27,5 +27,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index e7e4a4113174a..e2ee9c963a4da 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/sql/catalyst/src/test/resources/log4j.properties b/sql/catalyst/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/sql/catalyst/src/test/resources/log4j.properties +++ b/sql/catalyst/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index aab41fa49430f..6b8a5dbf6373e 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN log4j.logger.org.apache.hadoop=WARN From a0411aebee7c134f0426f0c2b2cb4c1c7856a291 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 9 Apr 2015 15:10:10 -0700 Subject: [PATCH 687/817] [SPARK-6264] [MLLIB] Support FPGrowth algorithm in Python API Support FPGrowth algorithm in Python API. Should we remove "Experimental" which were marked for FPGrowth and FPGrowthModel in Scala? jkbradley Author: Yanbo Liang Closes #5213 from yanboliang/spark-6264 and squashes the following commits: ed62ead [Yanbo Liang] trigger jenkins 8ce0359 [Yanbo Liang] fix docstring style 544c725 [Yanbo Liang] address comments a2d7cf7 [Yanbo Liang] add doc for FPGrowth.train() dcf7d73 [Yanbo Liang] add python doc b18fd07 [Yanbo Liang] trigger jenkins 2c951b8 [Yanbo Liang] fix typos 7f62c8f [Yanbo Liang] add fpm to __init__.py b96206a [Yanbo Liang] Support FPGrowth algorithm in Python API --- .../api/python/FPGrowthModelWrapper.scala | 33 ++++++++ .../mllib/api/python/PythonMLLibAPI.scala | 23 +++++- python/docs/pyspark.mllib.rst | 7 ++ python/pyspark/mllib/__init__.py | 2 +- python/pyspark/mllib/fpm.py | 81 +++++++++++++++++++ python/run-tests | 1 + 6 files changed, 143 insertions(+), 4 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala create mode 100644 python/pyspark/mllib/fpm.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala new file mode 100644 index 0000000000000..ee933f4cfcafd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api.python + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.rdd.RDD + +/** + * A Wrapper of FPGrowthModel to provide helper method for Python + */ +private[python] class FPGrowthModelWrapper(model: FPGrowthModel[Any]) + extends FPGrowthModel(model.freqItemsets) { + + def getFreqItemsets: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(model.freqItemsets.map(x => (x.javaItems, x.freq))) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 6c386cacfb7ca..1faa3def0e042 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -34,6 +34,7 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.feature._ +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} @@ -358,9 +359,7 @@ private[python] class PythonMLLibAPI extends Serializable { val model = new GaussianMixtureModel(weight, gaussians) model.predictSoft(data) } - - - + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -420,6 +419,24 @@ private[python] class PythonMLLibAPI extends Serializable { new MatrixFactorizationModelWrapper(model) } + /** + * Java stub for Python mllib FPGrowth.train(). This stub returns a handle + * to the Java object instead of the content of the Java object. Extra care + * needs to be taken in the Python code to ensure it gets freed on exit; see + * the Py4J documentation. + */ + def trainFPGrowthModel( + data: JavaRDD[java.lang.Iterable[Any]], + minSupport: Double, + numPartitions: Int): FPGrowthModel[Any] = { + val fpg = new FPGrowth() + .setMinSupport(minSupport) + .setNumPartitions(numPartitions) + + val model = fpg.run(data.rdd.map(_.asScala.toArray)) + new FPGrowthModelWrapper(model) + } + /** * Java stub for Normalizer.transform() */ diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index 15101470afc07..26ece4c2c389a 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -31,6 +31,13 @@ pyspark.mllib.feature module :undoc-members: :show-inheritance: +pyspark.mllib.fpm module +------------------------ + +.. automodule:: pyspark.mllib.fpm + :members: + :undoc-members: + pyspark.mllib.linalg module --------------------------- diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index 6449800d9c120..f2ef573fe9f6f 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -25,7 +25,7 @@ if numpy.version.version < '1.4': raise Exception("MLlib requires NumPy 1.4+") -__all__ = ['classification', 'clustering', 'feature', 'linalg', 'random', +__all__ = ['classification', 'clustering', 'feature', 'fpm', 'linalg', 'random', 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py new file mode 100644 index 0000000000000..3aa6d79d7093c --- /dev/null +++ b/python/pyspark/mllib/fpm.py @@ -0,0 +1,81 @@ +# +# 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. +# + +from pyspark import SparkContext +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc + +__all__ = ['FPGrowth', 'FPGrowthModel'] + + +@inherit_doc +class FPGrowthModel(JavaModelWrapper): + + """ + .. note:: Experimental + + A FP-Growth model for mining frequent itemsets + using the Parallel FP-Growth algorithm. + + >>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]] + >>> rdd = sc.parallelize(data, 2) + >>> model = FPGrowth.train(rdd, 0.6, 2) + >>> sorted(model.freqItemsets().collect()) + [([u'a'], 4), ([u'c'], 3), ([u'c', u'a'], 3)] + """ + + def freqItemsets(self): + """ + Get the frequent itemsets of this model + """ + return self.call("getFreqItemsets") + + +class FPGrowth(object): + """ + .. note:: Experimental + + A Parallel FP-growth algorithm to mine frequent itemsets. + """ + + @classmethod + def train(cls, data, minSupport=0.3, numPartitions=-1): + """ + Computes an FP-Growth model that contains frequent itemsets. + :param data: The input data set, each element + contains a transaction. + :param minSupport: The minimal support level + (default: `0.3`). + :param numPartitions: The number of partitions used by parallel + FP-growth (default: same as input data). + """ + model = callMLlibFunc("trainFPGrowthModel", data, float(minSupport), int(numPartitions)) + return FPGrowthModel(model) + + +def _test(): + import doctest + import pyspark.mllib.fpm + globs = pyspark.mllib.fpm.__dict__.copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest') + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index b7630c356cfae..f569a56fb7a9a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -77,6 +77,7 @@ function run_mllib_tests() { run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/evaluation.py" run_test "pyspark/mllib/feature.py" + run_test "pyspark/mllib/fpm.py" run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/rand.py" run_test "pyspark/mllib/recommendation.py" From 9c67049b4ef416a80803ccb958bbac1dd02cc380 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 9 Apr 2015 15:37:45 -0700 Subject: [PATCH 688/817] [Spark-6693][MLlib]add tostring with max lines and width for matrix jira: https://issues.apache.org/jira/browse/SPARK-6693 It's kind of annoying when debugging and found you cannot print out the matrix as you want. original toString of Matrix only print like following, 0.17810102596909183 0.5616906241468385 ... (10 total) 0.9692861997823815 0.015558159784155756 ... 0.8513015122819192 0.031523763918528847 ... 0.5396875653953941 0.3267864552779176 ... The def toString(maxLines : Int, maxWidth : Int) is useful when debuging, logging and saving matrix to files. Author: Yuhao Yang Closes #5344 from hhbyyh/addToString and squashes the following commits: 19a6836 [Yuhao Yang] remove extra line 6314b21 [Yuhao Yang] add exclude 736c324 [Yuhao Yang] add ut and exclude 420da39 [Yuhao Yang] Merge remote-tracking branch 'upstream/master' into addToString c22f352 [Yuhao Yang] style change 64a9e0f [Yuhao Yang] add specific to string to matrix --- .../org/apache/spark/mllib/linalg/Matrices.scala | 3 +++ .../spark/mllib/linalg/MatricesSuite.scala | 16 ++++++++++++++++ project/MimaExcludes.scala | 4 ++++ 3 files changed, 23 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index d1a174063caba..3fa5e068d16d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -87,6 +87,9 @@ sealed trait Matrix extends Serializable { /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() + /** A human readable representation of the matrix with maximum lines and width */ + def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + /** Map the values of this matrix using a function. Generates a new matrix. Performs the * function on only the backing array. For example, an operation such as addition or * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 0d2cec58e2c03..86119ec38101e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -439,4 +439,20 @@ class MatricesSuite extends FunSuite { assert(mUDT.typeName == "matrix") assert(mUDT.simpleString == "matrix") } + + test("toString") { + val empty = Matrices.ones(0, 0) + empty.toString(0, 0) + + val mat = Matrices.rand(5, 10, new Random()) + mat.toString(-1, -5) + mat.toString(0, 0) + mat.toString(Int.MinValue, Int.MinValue) + mat.toString(Int.MaxValue, Int.MaxValue) + var lines = mat.toString(6, 50).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 50)) + + lines = mat.toString(5, 100).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 100)) + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c2d828f982fe0..1564babefa62f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -64,6 +64,10 @@ object MimaExcludes { // SPARK-6492 Fix deadlock in SparkContext.stop() ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.org$" + "apache$spark$SparkContext$$SPARK_CONTEXT_CONSTRUCTOR_LOCK") + )++ Seq( + // SPARK-6693 add tostring with max lines and width for matrix + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrix.toString") ) case v if v.startsWith("1.3") => From b5c51c8df480f1a82a82e4d597d8eea631bffb4e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 9 Apr 2015 17:07:23 -0700 Subject: [PATCH 689/817] [SPARK-3074] [PySpark] support groupByKey() with single huge key This patch change groupByKey() to use external sort based approach, so it can support single huge key. For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach). During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy(). Author: Davies Liu Author: Davies Liu Closes #1977 from davies/groupby and squashes the following commits: af3713a [Davies Liu] make sure it's iterator 67772dd [Davies Liu] fix tests e78c15c [Davies Liu] address comments 0b0fde8 [Davies Liu] address comments 0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable e3b8eab [Davies Liu] fix narrow dependency 2a1857a [Davies Liu] typo d2f053b [Davies Liu] add repr for FlattedValuesSerializer c6a2f8d [Davies Liu] address comments 9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 2b9c261 [Davies Liu] fix typo in comments 70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby ab5515b [Davies Liu] Merge branch 'master' into groupby 651f891 [Davies Liu] simplify GroupByKey 1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 1f69f93 [Davies Liu] fix tests 0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 341f1e0 [Davies Liu] add comments, refactor 47918b8 [Davies Liu] remove unused code 6540948 [Davies Liu] address comments: 17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 4d4bc86 [Davies Liu] bugfix 8ef965e [Davies Liu] Merge branch 'master' into groupby fbc504a [Davies Liu] Merge branch 'master' into groupby 779ed03 [Davies Liu] fix merge conflict 2c1d05b [Davies Liu] refactor, minor turning b48cda5 [Davies Liu] Merge branch 'master' into groupby 85138e6 [Davies Liu] Merge branch 'master' into groupby acd8e1b [Davies Liu] fix memory when groupByKey().count() 905b233 [Davies Liu] Merge branch 'sort' into groupby 1f075ed [Davies Liu] Merge branch 'master' into sort 4b07d39 [Davies Liu] compress the data while spilling 0a081c6 [Davies Liu] Merge branch 'master' into groupby f157fe7 [Davies Liu] Merge branch 'sort' into groupby eb53ca6 [Davies Liu] Merge branch 'master' into sort b2dc3bf [Davies Liu] Merge branch 'sort' into groupby 644abaf [Davies Liu] add license in LICENSE 19f7873 [Davies Liu] improve tests 11ba318 [Davies Liu] typo 085aef8 [Davies Liu] Merge branch 'master' into groupby 3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data 1ea0669 [Davies Liu] choose sort based groupByKey() automatically b40bae7 [Davies Liu] bugfix efa23df [Davies Liu] refactor, add spark.shuffle.sort=False 250be4e [Davies Liu] flatten the combined values when dumping into disks d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting 083d842 [Davies Liu] sorted based groupByKey() 55602ee [Davies Liu] use external sort in sortBy() and sortByKey() --- python/pyspark/join.py | 13 +- python/pyspark/rdd.py | 48 ++- python/pyspark/resultiterable.py | 7 +- python/pyspark/serializers.py | 25 +- python/pyspark/shuffle.py | 531 ++++++++++++++++++++++++------- python/pyspark/tests.py | 50 ++- 6 files changed, 531 insertions(+), 143 deletions(-) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index efc1ef9396412..c3491defb2b29 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -48,7 +48,7 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -62,7 +62,7 @@ def dispatch(seq): wbuf.append(v) if not vbuf: vbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -76,7 +76,7 @@ def dispatch(seq): wbuf.append(v) if not wbuf: wbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -104,8 +104,9 @@ def make_mapper(i): rdd_len = len(vrdds) def dispatch(seq): - bufs = [[] for i in range(rdd_len)] - for (n, v) in seq: + bufs = [[] for _ in range(rdd_len)] + for n, v in seq: bufs[n].append(v) - return tuple(map(ResultIterable, bufs)) + return tuple(ResultIterable(vs) for vs in bufs) + return union_vrdds.groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 2d05611321ed6..1b18789040360 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -41,7 +41,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ - get_used_memory, ExternalSorter + get_used_memory, ExternalSorter, ExternalGroupBy from pyspark.traceback_utils import SCCallSiteSync from py4j.java_collections import ListConverter, MapConverter @@ -573,8 +573,8 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') - memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() serializer = self._jrdd_deserializer def sortPartition(iterator): @@ -1699,10 +1699,8 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numPartitions = self._defaultReducePartitions() serializer = self.ctx.serializer - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() - == 'true') - memory = _parse_memory(self.ctx._conf.get( - "spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() agg = Aggregator(createCombiner, mergeValue, mergeCombiners) def combineLocally(iterator): @@ -1755,21 +1753,28 @@ def createZero(): return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions) + def _can_spill(self): + return self.ctx._conf.get("spark.shuffle.spill", "True").lower() == "true" + + def _memory_limit(self): + return _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ Group the values for each key in the RDD into a single sequence. - Hash-partitions the resulting RDD with into numPartitions partitions. + Hash-partitions the resulting RDD with numPartitions partitions. Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) + >>> sorted(x.groupByKey().mapValues(len).collect()) + [('a', 2), ('b', 1)] + >>> sorted(x.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ - def createCombiner(x): return [x] @@ -1781,8 +1786,27 @@ def mergeCombiners(a, b): a.extend(b) return a - return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultIterable(x)) + spill = self._can_spill() + memory = self._memory_limit() + serializer = self._jrdd_deserializer + agg = Aggregator(createCombiner, mergeValue, mergeCombiners) + + def combine(iterator): + merger = ExternalMerger(agg, memory * 0.9, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeValues(iterator) + return merger.iteritems() + + locally_combined = self.mapPartitions(combine, preservesPartitioning=True) + shuffled = locally_combined.partitionBy(numPartitions) + + def groupByKey(it): + merger = ExternalGroupBy(agg, memory, serializer)\ + if spill else InMemoryMerger(agg) + merger.mergeCombiners(it) + return merger.iteritems() + + return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) def flatMapValues(self, f): """ diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index ef04c82866e6c..1ab5ce14c3531 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -15,15 +15,16 @@ # limitations under the License. # -__all__ = ["ResultIterable"] - import collections +__all__ = ["ResultIterable"] + class ResultIterable(collections.Iterable): """ - A special result iterable. This is used because the standard iterator can not be pickled + A special result iterable. This is used because the standard + iterator can not be pickled """ def __init__(self, data): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 0ffb41d02f6f6..4afa82f4b2973 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -220,6 +220,29 @@ def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) +class FlattenedValuesSerializer(BatchedSerializer): + + """ + Serializes a stream of list of pairs, split the list of values + which contain more than a certain number of objects to make them + have similar sizes. + """ + def __init__(self, serializer, batchSize=10): + BatchedSerializer.__init__(self, serializer, batchSize) + + def _batched(self, iterator): + n = self.batchSize + for key, values in iterator: + for i in xrange(0, len(values), n): + yield key, values[i:i + n] + + def load_stream(self, stream): + return self.serializer.load_stream(stream) + + def __repr__(self): + return "FlattenedValuesSerializer(%d)" % self.batchSize + + class AutoBatchedSerializer(BatchedSerializer): """ Choose the size of batch automatically based on the size of object @@ -251,7 +274,7 @@ def __eq__(self, other): return (isinstance(other, AutoBatchedSerializer) and other.serializer == self.serializer and other.bestSize == self.bestSize) - def __str__(self): + def __repr__(self): return "AutoBatchedSerializer(%s)" % str(self.serializer) diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 10a7ccd502000..8a6fc627eb383 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -16,28 +16,35 @@ # import os -import sys import platform import shutil import warnings import gc import itertools +import operator import random import pyspark.heapq3 as heapq -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ + CompressedSerializer, AutoBatchedSerializer + try: import psutil + process = None + def get_used_memory(): """ Return the used memory in MB """ - process = psutil.Process(os.getpid()) + global process + if process is None or process._pid != os.getpid(): + process = psutil.Process(os.getpid()) if hasattr(process, "memory_info"): info = process.memory_info() else: info = process.get_memory_info() return info.rss >> 20 + except ImportError: def get_used_memory(): @@ -46,6 +53,7 @@ def get_used_memory(): for line in open('/proc/self/status'): if line.startswith('VmRSS:'): return int(line.split()[1]) >> 10 + else: warnings.warn("Please install psutil to have better " "support with spilling") @@ -54,6 +62,7 @@ def get_used_memory(): rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss return rss >> 20 # TODO: support windows + return 0 @@ -148,10 +157,16 @@ def mergeCombiners(self, iterator): d[k] = comb(d[k], v) if k in d else v def iteritems(self): - """ Return the merged items ad iterator """ + """ Return the merged items as iterator """ return self.data.iteritems() +def _compressed_serializer(self, serializer=None): + # always use PickleSerializer to simplify implementation + ser = PickleSerializer() + return AutoBatchedSerializer(CompressedSerializer(ser)) + + class ExternalMerger(Merger): """ @@ -173,7 +188,7 @@ class ExternalMerger(Merger): dict. Repeat this again until combine all the items. - Before return any items, it will load each partition and - combine them seperately. Yield them before loading next + combine them separately. Yield them before loading next partition. - During loading a partition, if the memory goes over limit, @@ -182,7 +197,7 @@ class ExternalMerger(Merger): `data` and `pdata` are used to hold the merged items in memory. At first, all the data are merged into `data`. Once the used - memory goes over limit, the items in `data` are dumped indo + memory goes over limit, the items in `data` are dumped into disks, `data` will be cleared, all rest of items will be merged into `pdata` and then dumped into disks. Before returning, all the items in `pdata` will be dumped into disks. @@ -193,16 +208,16 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeValues(zip(xrange(N), xrange(N))) >>> assert merger.spills > 0 >>> sum(v for k,v in merger.iteritems()) - 499950000 + 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeCombiners(zip(xrange(N), xrange(N))) >>> assert merger.spills > 0 >>> sum(v for k,v in merger.iteritems()) - 499950000 + 49995000 """ # the max total partitions created recursively @@ -212,8 +227,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, localdirs=None, scale=1, partitions=59, batch=1000): Merger.__init__(self, aggregator) self.memory_limit = memory_limit - # default serializer is only used for tests - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions @@ -221,7 +235,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, self.batch = batch # scale is used to scale down the hash of key for recursive hash map self.scale = scale - # unpartitioned merged data + # un-partitioned merged data self.data = {} # partitioned merged data, list of dicts self.pdata = [] @@ -244,72 +258,63 @@ def _next_limit(self): def mergeValues(self, iterator): """ Combine the items by creator and combiner """ - iterator = iter(iterator) # speedup attribute lookup creator, comb = self.agg.createCombiner, self.agg.mergeValue - d, c, batch = self.data, 0, self.batch + c, data, pdata, hfun, batch = 0, self.data, self.pdata, self._partition, self.batch + limit = self.memory_limit for k, v in iterator: + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else creator(v) c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeValues(iterator, self._next_limit()) - break + if c >= batch: + if get_used_memory() >= limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if get_used_memory() >= limit: + self._spill() def _partition(self, key): """ Return the partition for key """ return hash((key, self._seed)) % self.partitions - def _partitioned_mergeValues(self, iterator, limit=0): - """ Partition the items by key, then combine them """ - # speedup attribute lookup - creator, comb = self.agg.createCombiner, self.agg.mergeValue - c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch - - for k, v in iterator: - d = pdata[hfun(k)] - d[k] = comb(d[k], v) if k in d else creator(v) - if not limit: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + def _object_size(self, obj): + """ How much of memory for this obj, assume that all the objects + consume similar bytes of memory + """ + return 1 - def mergeCombiners(self, iterator, check=True): + def mergeCombiners(self, iterator, limit=None): """ Merge (K,V) pair by mergeCombiner """ - iterator = iter(iterator) + if limit is None: + limit = self.memory_limit # speedup attribute lookup - d, comb, batch = self.data, self.agg.mergeCombiners, self.batch - c = 0 - for k, v in iterator: - d[k] = comb(d[k], v) if k in d else v - if not check: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeCombiners(iterator, self._next_limit()) - break - - def _partitioned_mergeCombiners(self, iterator, limit=0): - """ Partition the items by key, then merge them """ - comb, pdata = self.agg.mergeCombiners, self.pdata - c, hfun = 0, self._partition + comb, hfun, objsize = self.agg.mergeCombiners, self._partition, self._object_size + c, data, pdata, batch = 0, self.data, self.pdata, self.batch for k, v in iterator: - d = pdata[hfun(k)] + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else v if not limit: continue - c += 1 - if c % self.batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + c += objsize(v) + if c > batch: + if get_used_memory() > limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if limit and get_used_memory() >= limit: + self._spill() def _spill(self): """ @@ -335,7 +340,7 @@ def _spill(self): for k, v in self.data.iteritems(): h = self._partition(k) - # put one item in batch, make it compatitable with load_stream + # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch self.serializer.dump_stream([(k, v)], streams[h]) @@ -344,7 +349,7 @@ def _spill(self): s.close() self.data.clear() - self.pdata = [{} for i in range(self.partitions)] + self.pdata.extend([{} for i in range(self.partitions)]) else: for i in range(self.partitions): @@ -370,29 +375,12 @@ def _external_items(self): assert not self.data if any(self.pdata): self._spill() - hard_limit = self._next_limit() + # disable partitioning and spilling when merge combiners from disk + self.pdata = [] try: for i in range(self.partitions): - self.data = {} - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), - False) - - # limit the total partitions - if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS - and j < self.spills - 1 - and get_used_memory() > hard_limit): - self.data.clear() # will read from disk again - gc.collect() # release the memory as much as possible - for v in self._recursive_merged_items(i): - yield v - return - - for v in self.data.iteritems(): + for v in self._merged_items(i): yield v self.data.clear() @@ -400,53 +388,56 @@ def _external_items(self): for j in range(self.spills): path = self._get_spill_dir(j) os.remove(os.path.join(path, str(i))) - finally: self._cleanup() - def _cleanup(self): - """ Clean up all the files in disks """ - for d in self.localdirs: - shutil.rmtree(d, True) + def _merged_items(self, index): + self.data = {} + limit = self._next_limit() + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + + # limit the total partitions + if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS + and j < self.spills - 1 + and get_used_memory() > limit): + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible + return self._recursive_merged_items(index) - def _recursive_merged_items(self, start): + return self.data.iteritems() + + def _recursive_merged_items(self, index): """ merge the partitioned items and return the as iterator If one partition can not be fit in memory, then them will be partitioned and merged recursively. """ - # make sure all the data are dumps into disks. - assert not self.data - if any(self.pdata): - self._spill() - assert self.spills > 0 - - for i in range(start, self.partitions): - subdirs = [os.path.join(d, "parts", str(i)) - for d in self.localdirs] - m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions, self.partitions) - m.pdata = [{} for _ in range(self.partitions)] - limit = self._next_limit() - - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - m._partitioned_mergeCombiners( - self.serializer.load_stream(open(p))) - - if get_used_memory() > limit: - m._spill() - limit = self._next_limit() + subdirs = [os.path.join(d, "parts", str(index)) for d in self.localdirs] + m = ExternalMerger(self.agg, self.memory_limit, self.serializer, subdirs, + self.scale * self.partitions, self.partitions, self.batch) + m.pdata = [{} for _ in range(self.partitions)] + limit = self._next_limit() + + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + m.mergeCombiners(self.serializer.load_stream(open(p)), 0) + + if get_used_memory() > limit: + m._spill() + limit = self._next_limit() - for v in m._external_items(): - yield v + return m._external_items() - # remove the merged partition - for j in range(self.spills): - path = self._get_spill_dir(j) - os.remove(os.path.join(path, str(i))) + def _cleanup(self): + """ Clean up all the files in disks """ + for d in self.localdirs: + shutil.rmtree(d, True) class ExternalSorter(object): @@ -457,6 +448,7 @@ class ExternalSorter(object): The spilling will only happen when the used memory goes above the limit. + >>> sorter = ExternalSorter(1) # 1M >>> import random >>> l = range(1024) @@ -469,7 +461,7 @@ class ExternalSorter(object): def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) def _get_path(self, n): """ Choose one directory for spill by number n """ @@ -515,6 +507,7 @@ def sorted(self, iterator, key=None, reverse=False): limit = self._next_limit() MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 DiskBytesSpilled += os.path.getsize(path) + os.unlink(path) # data will be deleted after close elif not chunks: batch = min(batch * 2, 10000) @@ -529,6 +522,310 @@ def sorted(self, iterator, key=None, reverse=False): return heapq.merge(chunks, key=key, reverse=reverse) +class ExternalList(object): + """ + ExternalList can have many items which cannot be hold in memory in + the same time. + + >>> l = ExternalList(range(100)) + >>> len(l) + 100 + >>> l.append(10) + >>> len(l) + 101 + >>> for i in range(20240): + ... l.append(i) + >>> len(l) + 20341 + >>> import pickle + >>> l2 = pickle.loads(pickle.dumps(l)) + >>> len(l2) + 20341 + >>> list(l2)[100] + 10 + """ + LIMIT = 10240 + + def __init__(self, values): + self.values = values + self.count = len(values) + self._file = None + self._ser = None + + def __getstate__(self): + if self._file is not None: + self._file.flush() + f = os.fdopen(os.dup(self._file.fileno())) + f.seek(0) + serialized = f.read() + else: + serialized = '' + return self.values, self.count, serialized + + def __setstate__(self, item): + self.values, self.count, serialized = item + if serialized: + self._open_file() + self._file.write(serialized) + else: + self._file = None + self._ser = None + + def __iter__(self): + if self._file is not None: + self._file.flush() + # read all items from disks first + with os.fdopen(os.dup(self._file.fileno()), 'r') as f: + f.seek(0) + for v in self._ser.load_stream(f): + yield v + + for v in self.values: + yield v + + def __len__(self): + return self.count + + def append(self, value): + self.values.append(value) + self.count += 1 + # dump them into disk if the key is huge + if len(self.values) >= self.LIMIT: + self._spill() + + def _open_file(self): + dirs = _get_local_dirs("objects") + d = dirs[id(self) % len(dirs)] + if not os.path.exists(d): + os.makedirs(d) + p = os.path.join(d, str(id)) + self._file = open(p, "w+", 65536) + self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) + os.unlink(p) + + def _spill(self): + """ dump the values into disk """ + global MemoryBytesSpilled, DiskBytesSpilled + if self._file is None: + self._open_file() + + used_memory = get_used_memory() + pos = self._file.tell() + self._ser.dump_stream(self.values, self._file) + self.values = [] + gc.collect() + DiskBytesSpilled += self._file.tell() - pos + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + +class ExternalListOfList(ExternalList): + """ + An external list for list. + + >>> l = ExternalListOfList([[i, i] for i in range(100)]) + >>> len(l) + 200 + >>> l.append(range(10)) + >>> len(l) + 210 + >>> len(list(l)) + 210 + """ + + def __init__(self, values): + ExternalList.__init__(self, values) + self.count = sum(len(i) for i in values) + + def append(self, value): + ExternalList.append(self, value) + # already counted 1 in ExternalList.append + self.count += len(value) - 1 + + def __iter__(self): + for values in ExternalList.__iter__(self): + for v in values: + yield v + + +class GroupByKey(object): + """ + Group a sorted iterator as [(k1, it1), (k2, it2), ...] + + >>> k = [i/3 for i in range(6)] + >>> v = [[i] for i in range(6)] + >>> g = GroupByKey(iter(zip(k, v))) + >>> [(k, list(it)) for k, it in g] + [(0, [0, 1, 2]), (1, [3, 4, 5])] + """ + + def __init__(self, iterator): + self.iterator = iter(iterator) + self.next_item = None + + def __iter__(self): + return self + + def next(self): + key, value = self.next_item if self.next_item else next(self.iterator) + values = ExternalListOfList([value]) + try: + while True: + k, v = next(self.iterator) + if k != key: + self.next_item = (k, v) + break + values.append(v) + except StopIteration: + self.next_item = None + return key, values + + +class ExternalGroupBy(ExternalMerger): + + """ + Group by the items by key. If any partition of them can not been + hold in memory, it will do sort based group by. + + This class works as follows: + + - It repeatedly group the items by key and save them in one dict in + memory. + + - When the used memory goes above memory limit, it will split + the combined data into partitions by hash code, dump them + into disk, one file per partition. If the number of keys + in one partitions is smaller than 1000, it will sort them + by key before dumping into disk. + + - Then it goes through the rest of the iterator, group items + by key into different dict by hash. Until the used memory goes over + memory limit, it dump all the dicts into disks, one file per + dict. Repeat this again until combine all the items. It + also will try to sort the items by key in each partition + before dumping into disks. + + - It will yield the grouped items partitions by partitions. + If the data in one partitions can be hold in memory, then it + will load and combine them in memory and yield. + + - If the dataset in one partition cannot be hold in memory, + it will sort them first. If all the files are already sorted, + it merge them by heap.merge(), so it will do external sort + for all the files. + + - After sorting, `GroupByKey` class will put all the continuous + items with the same key as a group, yield the values as + an iterator. + """ + SORT_KEY_LIMIT = 1000 + + def flattened_serializer(self): + assert isinstance(self.serializer, BatchedSerializer) + ser = self.serializer + return FlattenedValuesSerializer(ser, 20) + + def _object_size(self, obj): + return len(obj) + + def _spill(self): + """ + dump already partitioned data into disks. + """ + global MemoryBytesSpilled, DiskBytesSpilled + path = self._get_spill_dir(self.spills) + if not os.path.exists(path): + os.makedirs(path) + + used_memory = get_used_memory() + if not self.pdata: + # The data has not been partitioned, it will iterator the + # data once, write them into different files, has no + # additional memory. It only called when the memory goes + # above limit at the first time. + + # open all the files for writing + streams = [open(os.path.join(path, str(i)), 'w') + for i in range(self.partitions)] + + # If the number of keys is small, then the overhead of sort is small + # sort them before dumping into disks + self._sorted = len(self.data) < self.SORT_KEY_LIMIT + if self._sorted: + self.serializer = self.flattened_serializer() + for k in sorted(self.data.keys()): + h = self._partition(k) + self.serializer.dump_stream([(k, self.data[k])], streams[h]) + else: + for k, v in self.data.iteritems(): + h = self._partition(k) + self.serializer.dump_stream([(k, v)], streams[h]) + + for s in streams: + DiskBytesSpilled += s.tell() + s.close() + + self.data.clear() + # self.pdata is cached in `mergeValues` and `mergeCombiners` + self.pdata.extend([{} for i in range(self.partitions)]) + + else: + for i in range(self.partitions): + p = os.path.join(path, str(i)) + with open(p, "w") as f: + # dump items in batch + if self._sorted: + # sort by key only (stable) + sorted_items = sorted(self.pdata[i].iteritems(), key=operator.itemgetter(0)) + self.serializer.dump_stream(sorted_items, f) + else: + self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.pdata[i].clear() + DiskBytesSpilled += os.path.getsize(p) + + self.spills += 1 + gc.collect() # release the memory as much as possible + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + def _merged_items(self, index): + size = sum(os.path.getsize(os.path.join(self._get_spill_dir(j), str(index))) + for j in range(self.spills)) + # if the memory can not hold all the partition, + # then use sort based merge. Because of compression, + # the data on disks will be much smaller than needed memory + if (size >> 20) >= self.memory_limit / 10: + return self._merge_sorted_items(index) + + self.data = {} + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + return self.data.iteritems() + + def _merge_sorted_items(self, index): + """ load a partition from disk, then sort and group by key """ + def load_partition(j): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + return self.serializer.load_stream(open(p, 'r', 65536)) + + disk_items = [load_partition(j) for j in range(self.spills)] + + if self._sorted: + # all the partitions are already sorted + sorted_items = heapq.merge(disk_items, key=operator.itemgetter(0)) + + else: + # Flatten the combined values, so it will not consume huge + # memory during merging sort. + ser = self.flattened_serializer() + sorter = ExternalSorter(self.memory_limit, ser) + sorted_items = sorter.sorted(itertools.chain(*disk_items), + key=operator.itemgetter(0)) + return ((k, vs) for k, vs in GroupByKey(sorted_items)) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index dd8d3b1c53733..0bd5d20f7877f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,6 +31,7 @@ import time import zipfile import random +import itertools import threading import hashlib @@ -76,7 +77,7 @@ class MergerTests(unittest.TestCase): def setUp(self): - self.N = 1 << 14 + self.N = 1 << 12 self.l = [i for i in xrange(self.N)] self.data = zip(self.l, self.l) self.agg = Aggregator(lambda x: [x], @@ -108,7 +109,7 @@ def test_small_dataset(self): sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 10) + m = ExternalMerger(self.agg, 30) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), @@ -124,10 +125,36 @@ def test_huge_dataset(self): m = ExternalMerger(self.agg, 10, partitions=3) m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), + self.assertEqual(sum(len(v) for k, v in m.iteritems()), self.N * 10) m._cleanup() + def test_group_by_key(self): + + def gen_data(N, step): + for i in range(1, N + 1, step): + for j in range(i): + yield (i, [j]) + + def gen_gs(N, step=1): + return shuffle.GroupByKey(gen_data(N, step)) + + self.assertEqual(1, len(list(gen_gs(1)))) + self.assertEqual(2, len(list(gen_gs(2)))) + self.assertEqual(100, len(list(gen_gs(100)))) + self.assertEqual(range(1, 101), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(range(k) == list(vs) for k, vs in gen_gs(100))) + + for k, vs in gen_gs(50002, 10000): + self.assertEqual(k, len(vs)) + self.assertEqual(range(k), list(vs)) + + ser = PickleSerializer() + l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) + for k, vs in l: + self.assertEqual(k, len(vs)) + self.assertEqual(range(k), list(vs)) + class SorterTests(unittest.TestCase): def test_in_memory_sort(self): @@ -702,6 +729,21 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_external_group_by_key(self): + self.sc._conf.set("spark.python.worker.memory", "5m") + N = 200001 + kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) + gkv = kv.groupByKey().cache() + self.assertEqual(3, gkv.count()) + filtered = gkv.filter(lambda (k, vs): k == 1) + self.assertEqual(1, filtered.count()) + self.assertEqual([(1, N/3)], filtered.mapValues(len).collect()) + self.assertEqual([(N/3, N/3)], + filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) + result = filtered.collect()[0][1] + self.assertEqual(N/3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalList)) + def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -752,9 +794,9 @@ def test_narrow_dependency_in_join(self): self.assertEqual(rdd.getNumPartitions() + 2, parted.union(rdd).getNumPartitions()) self.assertEqual(rdd.getNumPartitions() + 2, rdd.union(parted).getNumPartitions()) - self.sc.setJobGroup("test1", "test", True) tracker = self.sc.statusTracker() + self.sc.setJobGroup("test1", "test", True) d = sorted(parted.join(parted).collect()) self.assertEqual(10, len(d)) self.assertEqual((0, (0, 0)), d[0]) From e2360810f50de77f79d372cc9b46db117d451cfc Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 9 Apr 2015 23:10:13 -0700 Subject: [PATCH 690/817] [SPARK-6577] [MLlib] [PySpark] SparseMatrix should be supported in PySpark Supporting of SparseMatrix in PySpark. Author: MechCoder Closes #5355 from MechCoder/spark-6577 and squashes the following commits: 7492190 [MechCoder] More readable code for densifying ea2c54b [MechCoder] Check bounds for indexing 454ef2c [MechCoder] Made the following changes 1. Used convert_to_array for array conversion. 2. Used F order for toArray 3. Minor improvements in speed. db76caf [MechCoder] Add support for CSR matrix 29653e7 [MechCoder] Renamed indices to rowIndices and indptr to colPtrs b6384fe [MechCoder] [SPARK-6577] SparseMatrix should be supported in PySpark --- python/pyspark/mllib/linalg.py | 110 +++++++++++++++++++++++++++++++-- python/pyspark/mllib/tests.py | 52 +++++++++++++++- 2 files changed, 154 insertions(+), 8 deletions(-) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 51c1490b1618d..a80320c52d1d0 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -640,6 +640,15 @@ def toArray(self): """ raise NotImplementedError + @staticmethod + def _convert_to_array(array_like, dtype): + """ + Convert Matrix attributes which are array-like or buffer to array. + """ + if isinstance(array_like, basestring): + return np.frombuffer(array_like, dtype=dtype) + return np.asarray(array_like, dtype=dtype) + class DenseMatrix(Matrix): """ @@ -647,13 +656,8 @@ class DenseMatrix(Matrix): """ def __init__(self, numRows, numCols, values): Matrix.__init__(self, numRows, numCols) - if isinstance(values, basestring): - values = np.frombuffer(values, dtype=np.float64) - elif not isinstance(values, np.ndarray): - values = np.array(values, dtype=np.float64) + values = self._convert_to_array(values, np.float64) assert len(values) == numRows * numCols - if values.dtype != np.float64: - values.astype(np.float64) self.values = values def __reduce__(self): @@ -670,6 +674,17 @@ def toArray(self): """ return self.values.reshape((self.numRows, self.numCols), order='F') + def toSparse(self): + """Convert to SparseMatrix""" + indices = np.nonzero(self.values)[0] + colCounts = np.bincount(indices / self.numRows) + colPtrs = np.cumsum(np.hstack( + (0, colCounts, np.zeros(self.numCols - colCounts.size)))) + values = self.values[indices] + rowIndices = indices % self.numRows + + return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) + def __getitem__(self, indices): i, j = indices if i < 0 or i >= self.numRows: @@ -687,6 +702,82 @@ def __eq__(self, other): all(self.values == other.values)) +class SparseMatrix(Matrix): + """Sparse Matrix stored in CSC format.""" + def __init__(self, numRows, numCols, colPtrs, rowIndices, values, + isTransposed=False): + Matrix.__init__(self, numRows, numCols) + self.isTransposed = isTransposed + self.colPtrs = self._convert_to_array(colPtrs, np.int32) + self.rowIndices = self._convert_to_array(rowIndices, np.int32) + self.values = self._convert_to_array(values, np.float64) + + if self.isTransposed: + if self.colPtrs.size != numRows + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numRows + 1, self.colPtrs.size)) + else: + if self.colPtrs.size != numCols + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numCols + 1, self.colPtrs.size)) + if self.rowIndices.size != self.values.size: + raise ValueError("Expected rowIndices of length %d, got %d." + % (self.rowIndices.size, self.values.size)) + + def __reduce__(self): + return SparseMatrix, ( + self.numRows, self.numCols, self.colPtrs.tostring(), + self.rowIndices.tostring(), self.values.tostring(), + self.isTransposed) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j < 0 or j >= self.numCols: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + # If a CSR matrix is given, then the row index should be searched + # for in ColPtrs, and the column index should be searched for in the + # corresponding slice obtained from rowIndices. + if self.isTransposed: + j, i = i, j + + colStart = self.colPtrs[j] + colEnd = self.colPtrs[j + 1] + nz = self.rowIndices[colStart: colEnd] + ind = np.searchsorted(nz, i) + colStart + if ind < colEnd and self.rowIndices[ind] == i: + return self.values[ind] + else: + return 0.0 + + def toArray(self): + """ + Return an numpy.ndarray + """ + A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') + for k in xrange(self.colPtrs.size - 1): + startptr = self.colPtrs[k] + endptr = self.colPtrs[k + 1] + if self.isTransposed: + A[k, self.rowIndices[startptr:endptr]] = self.values[startptr:endptr] + else: + A[self.rowIndices[startptr:endptr], k] = self.values[startptr:endptr] + return A + + def toDense(self): + densevals = np.reshape( + self.toArray(), (self.numRows * self.numCols), order='F') + return DenseMatrix(self.numRows, self.numCols, densevals) + + # TODO: More efficient implementation: + def __eq__(self, other): + return np.all(self.toArray == other.toArray) + + class Matrices(object): @staticmethod def dense(numRows, numCols, values): @@ -695,6 +786,13 @@ def dense(numRows, numCols, values): """ return DenseMatrix(numRows, numCols, values) + @staticmethod + def sparse(numRows, numCols, colPtrs, rowIndices, values): + """ + Create a SparseMatrix + """ + return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + def _test(): import doctest diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 61ef398487c0c..3b40158c12b74 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -24,7 +24,7 @@ import tempfile import array as pyarray -from numpy import array, array_equal +from numpy import array, array_equal, zeros from py4j.protocol import Py4JJavaError if sys.version_info[:2] <= (2, 6): @@ -38,7 +38,7 @@ from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ - DenseMatrix, Vectors, Matrices + DenseMatrix, SparseMatrix, Vectors, Matrices from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics @@ -144,6 +144,54 @@ def test_matrix_indexing(self): for j in range(2): self.assertEquals(mat[i, j], expected[i][j]) + def test_sparse_matrix(self): + # Test sparse matrix creation. + sm1 = SparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self.assertEquals(sm1.numRows, 3) + self.assertEquals(sm1.numCols, 4) + self.assertEquals(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4]) + self.assertEquals(sm1.rowIndices.tolist(), [1, 2, 1, 2]) + self.assertEquals(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0]) + + # Test indexing + expected = [ + [0, 0, 0, 0], + [1, 0, 4, 0], + [2, 0, 5, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1[i, j]) + self.assertTrue(array_equal(sm1.toArray(), expected)) + + # Test conversion to dense and sparse. + smnew = sm1.toDense().toSparse() + self.assertEquals(sm1.numRows, smnew.numRows) + self.assertEquals(sm1.numCols, smnew.numCols) + self.assertTrue(array_equal(sm1.colPtrs, smnew.colPtrs)) + self.assertTrue(array_equal(sm1.rowIndices, smnew.rowIndices)) + self.assertTrue(array_equal(sm1.values, smnew.values)) + + sm1t = SparseMatrix( + 3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], + isTransposed=True) + self.assertEquals(sm1t.numRows, 3) + self.assertEquals(sm1t.numCols, 4) + self.assertEquals(sm1t.colPtrs.tolist(), [0, 2, 3, 5]) + self.assertEquals(sm1t.rowIndices.tolist(), [0, 1, 2, 0, 2]) + self.assertEquals(sm1t.values.tolist(), [3.0, 2.0, 4.0, 9.0, 8.0]) + + expected = [ + [3, 2, 0, 0], + [0, 0, 4, 0], + [9, 0, 8, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1t[i, j]) + self.assertTrue(array_equal(sm1t.toArray(), expected)) + class ListTests(PySparkTestCase): From 3290d2d13bb4bd875aec14425c8e3766f9cc644b Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 9 Apr 2015 23:14:24 -0700 Subject: [PATCH 691/817] [SPARK-6211][Streaming] Add Python Kafka API unit test Refactor the Kafka unit test and add Python API support. CC tdas davies please help to review, thanks a lot. Author: jerryshao Author: Saisai Shao Closes #4961 from jerryshao/SPARK-6211 and squashes the following commits: ee4b919 [jerryshao] Fixed newly merged issue 82c756e [jerryshao] Address the comments 92912d1 [jerryshao] Address the commits 0708bb1 [jerryshao] Fix rebase issue 40b47a3 [Saisai Shao] Style fix f889657 [Saisai Shao] Update the code according 8a2f3e2 [jerryshao] Address the issues 0f1b7ce [jerryshao] Still fix the bug 61a04f0 [jerryshao] Fix bugs and address the issues 64d9877 [jerryshao] Fix rebase bugs 8ad442f [jerryshao] Add kafka-assembly in run-tests 6020b00 [jerryshao] Add more debug info in Shell 8102d6e [jerryshao] Fix bug in Jenkins test fde1213 [jerryshao] Code style changes 5536f95 [jerryshao] Refactor the Kafka unit test and add Python Kafka unittest support --- dev/run-tests | 2 +- .../streaming/kafka/KafkaTestUtils.scala | 261 ++++++++++++++++++ .../kafka/JavaDirectKafkaStreamSuite.java | 28 +- .../streaming/kafka/JavaKafkaRDDSuite.java | 28 +- .../streaming/kafka/JavaKafkaStreamSuite.java | 34 ++- .../kafka/DirectKafkaStreamSuite.scala | 56 ++-- .../streaming/kafka/KafkaClusterSuite.scala | 29 +- .../spark/streaming/kafka/KafkaRDDSuite.scala | 40 +-- .../streaming/kafka/KafkaStreamSuite.scala | 211 ++------------ .../kafka/ReliableKafkaStreamSuite.scala | 62 +++-- python/pyspark/streaming/tests.py | 43 ++- python/run-tests | 19 +- 12 files changed, 502 insertions(+), 311 deletions(-) create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala diff --git a/dev/run-tests b/dev/run-tests index 1b6cf78b5da01..bb21ab6c9aa04 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -173,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_BUILD build/mvn $HIVE_BUILD_ARGS clean package -DskipTests else echo -e "q\n" \ - | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \ + | build/sbt $HIVE_BUILD_ARGS package assembly/assembly streaming-kafka-assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" fi } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 0000000000000..13e9475065979 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap} +import java.util.Properties +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.language.postfixOps +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZKStringSerializer +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.I0Itec.zkclient.ZkClient + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 6000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 9092 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, port) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + AdminUtils.createTopic(zkClient, topic, 1, 1) + // wait until metadata is propagated + waitUntilMetadataIsPropagated(topic, 0) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + import scala.collection.JavaConversions._ + sendMessages(topic, Map(messageToFreq.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + eventually(Time(10000), Time(100)) { + assert( + server.apis.metadataCache.containsTopicAndPartition(topic, partition), + s"Partition [$topic, $partition] metadata not propagated after timeout" + ) + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index d6ca6d58b5665..4c1d6a03eb2b8 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -41,24 +41,28 @@ public class JavaDirectKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); } @After public void tearDown() { + if (ssc != null) { ssc.stop(); ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -74,7 +78,7 @@ public void testKafkaStream() throws InterruptedException { sent.addAll(Arrays.asList(topic2data)); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); kafkaParams.put("auto.offset.reset", "smallest"); JavaDStream stream1 = KafkaUtils.createDirectStream( @@ -147,8 +151,8 @@ private HashMap topicOffsetToMap(String topic, Long off private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 4477b81827c70..a9dc6e50613ca 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -37,13 +37,12 @@ public class JavaKafkaRDDSuite implements Serializable { private transient JavaSparkContext sc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); sc = new JavaSparkContext(sparkConf); @@ -51,10 +50,15 @@ public void setUp() { @After public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -66,7 +70,7 @@ public void testKafkaRDD() throws InterruptedException { String[] topic2data = createTopicAndSendData(topic2); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), @@ -75,7 +79,7 @@ public void testKafkaRDD() throws InterruptedException { HashMap emptyLeaders = new HashMap(); HashMap leaders = new HashMap(); - String[] hostAndPort = suiteBase.brokerAddress().split(":"); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); leaders.put(new TopicAndPartition(topic1, 0), broker); leaders.put(new TopicAndPartition(topic2, 0), broker); @@ -144,8 +148,8 @@ public String call(MessageAndMetadata msgAndMd) throws Exception private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index bad0a93eb2e84..540f4ceabab47 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,9 +22,7 @@ import java.util.List; import java.util.Random; -import scala.Predef; import scala.Tuple2; -import scala.collection.JavaConverters; import kafka.serializer.StringDecoder; import org.junit.After; @@ -44,13 +42,12 @@ public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; private transient Random random = new Random(); - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); ssc = new JavaStreamingContext(sparkConf, new Duration(500)); @@ -58,10 +55,15 @@ public void setUp() { @After public void tearDown() { - ssc.stop(); - ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -75,15 +77,11 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - suiteBase.createTopic(topic); - HashMap tmp = new HashMap(sent); - suiteBase.sendMessages(topic, - JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms()) - ); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, sent); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); + kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress()); kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); @@ -126,6 +124,7 @@ public Void call(JavaPairRDD rdd) throws Exception { ); ssc.start(); + long startTime = System.currentTimeMillis(); boolean sizeMatches = false; while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { @@ -136,6 +135,5 @@ public Void call(JavaPairRDD rdd) throws Exception { for (String k : sent.keySet()) { Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); } - ssc.stop(); } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 17ca9d145d665..415730f5559c5 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -27,31 +27,41 @@ import scala.language.postfixOps import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.StringDecoder -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -class DirectKafkaStreamSuite extends KafkaStreamSuiteBase - with BeforeAndAfter with BeforeAndAfterAll with Eventually { +class DirectKafkaStreamSuite + extends FunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ - var ssc: StreamingContext = _ - var testDir: File = _ + private var sc: SparkContext = _ + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } after { @@ -72,12 +82,12 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topics = Set("basic1", "basic2", "basic3") val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } val totalSent = data.values.sum * topics.size val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) @@ -121,9 +131,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "largest" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -132,7 +142,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() > 3) } @@ -154,7 +164,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -166,9 +176,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "offset" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -177,7 +187,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() >= 10) } @@ -200,7 +210,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -210,18 +220,18 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase // Test to verify the offset ranges can be recovered from the checkpoints test("offset recovery") { val topic = "recovery" - createTopic(topic) + kafkaTestUtils.createTopic(topic) testDir = Utils.createTempDir() val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { val strings = data.map { _.toString} - sendMessages(topic, strings.map { _ -> 1}.toMap) + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) eventually(timeout(10 seconds), interval(50 milliseconds)) { assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index fc9275b7207be..2b33d2a220b2b 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -20,28 +20,35 @@ package org.apache.spark.streaming.kafka import scala.util.Random import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} -class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val topic = "kcsuitetopic" + Random.nextInt(10000) - val topicAndPartition = TopicAndPartition(topic, 0) - var kc: KafkaCluster = null +class KafkaClusterSuite extends FunSuite with BeforeAndAfterAll { + private val topic = "kcsuitetopic" + Random.nextInt(10000) + private val topicAndPartition = TopicAndPartition(topic, 0) + private var kc: KafkaCluster = null + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll() { - setupKafka() - createTopic(topic) - sendMessages(topic, Map("a" -> 1)) - kc = new KafkaCluster(Map("metadata.broker.list" -> s"$brokerAddress")) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) } override def afterAll() { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("metadata apis") { val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) val leaderAddress = s"${leader._1}:${leader._2}" - assert(leaderAddress === brokerAddress, "didn't get leader") + assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index a223da70b043f..7d26ce50875b3 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -22,18 +22,22 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext._ -class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ +class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + override def beforeAll { sc = new SparkContext(sparkConf) - - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { @@ -41,17 +45,21 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { sc.stop sc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("basic usage") { val topic = "topicbasic" - createTopic(topic) + kafkaTestUtils.createTopic(topic) val messages = Set("the", "quick", "brown", "fox") - sendMessages(topic, messages.toArray) + kafkaTestUtils.sendMessages(topic, messages.toArray) - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) @@ -67,15 +75,15 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val kc = new KafkaCluster(kafkaParams) // this is the "lots of messages" case - sendMessages(topic, sent) + kafkaTestUtils.sendMessages(topic, sent) // rdd defined from leaders after sending messages, should get the number sent val rdd = getRdd(kc, Set(topic)) @@ -92,14 +100,14 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { // shouldn't get anything, since message is sent after rdd was defined val sentOnlyOne = Map("d" -> 1) - sendMessages(topic, sentOnlyOne) + kafkaTestUtils.sendMessages(topic, sentOnlyOne) assert(rdd2.isDefined) assert(rdd2.get.count === 0, "got messages when there shouldn't be any") // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above val rdd3 = getRdd(kc, Set(topic)) // send lots of messages after rdd was defined, they shouldn't show up - sendMessages(topic, Map("extra" -> 22)) + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) assert(rdd3.isDefined) assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index e4966eebb9b34..24699dfc33adb 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -17,209 +17,38 @@ package org.apache.spark.streaming.kafka -import java.io.File -import java.net.InetSocketAddress -import java.util.Properties - import scala.collection.mutable import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import kafka.admin.AdminUtils -import kafka.common.{KafkaException, TopicAndPartition} -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.{StringDecoder, StringEncoder} -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient.ZkClient -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} -import org.scalatest.{BeforeAndAfter, FunSuite} +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils - -/** - * This is an abstract base class for Kafka testsuites. This has the functionality to set up - * and tear down local Kafka servers, and to push data using Kafka producers. - */ -abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging { - - private val zkHost = "localhost" - private var zkPort: Int = 0 - private val zkConnectionTimeout = 6000 - private val zkSessionTimeout = 6000 - private var zookeeper: EmbeddedZookeeper = _ - private val brokerHost = "localhost" - private var brokerPort = 9092 - private var brokerConf: KafkaConfig = _ - private var server: KafkaServer = _ - private var producer: Producer[String, String] = _ - private var zkReady = false - private var brokerReady = false - - protected var zkClient: ZkClient = _ - - def zkAddress: String = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") - s"$zkHost:$zkPort" - } - def brokerAddress: String = { - assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") - s"$brokerHost:$brokerPort" - } - - def setupKafka() { - // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") - // Get the actual zookeeper binding port - zkPort = zookeeper.actualPort - zkReady = true - logInfo("==================== Zookeeper Started ====================") +class KafkaStreamSuite extends FunSuite with Eventually with BeforeAndAfterAll { + private var ssc: StreamingContext = _ + private var kafkaTestUtils: KafkaTestUtils = _ - zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) - logInfo("==================== Zookeeper Client Created ====================") - - // Kafka broker startup - var bindSuccess: Boolean = false - while(!bindSuccess) { - try { - val brokerProps = getBrokerConfig() - brokerConf = new KafkaConfig(brokerProps) - server = new KafkaServer(brokerConf) - server.startup() - logInfo("==================== Kafka Broker Started ====================") - bindSuccess = true - } catch { - case e: KafkaException => - if (e.getMessage != null && e.getMessage.contains("Socket server failed to bind to")) { - brokerPort += 1 - } - case e: Exception => throw new Exception("Kafka server create failed", e) - } - } - - Thread.sleep(2000) - logInfo("==================== Kafka + Zookeeper Ready ====================") - brokerReady = true + override def beforeAll(): Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } - def tearDownKafka() { - brokerReady = false - zkReady = false - if (producer != null) { - producer.close() - producer = null - } - - if (server != null) { - server.shutdown() - server = null - } - - brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (zookeeper != null) { - zookeeper.shutdown() - zookeeper = null - } - } - - def createTopic(topic: String) { - AdminUtils.createTopic(zkClient, topic, 1, 1) - // wait until metadata is propagated - waitUntilMetadataIsPropagated(topic, 0) - logInfo(s"==================== Topic $topic Created ====================") - } - - def sendMessages(topic: String, messageToFreq: Map[String, Int]) { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - def sendMessages(topic: String, messages: Array[String]) { - producer = new Producer[String, String](new ProducerConfig(getProducerConfig())) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) - producer.close() - logInfo(s"==================== Sent Messages: ${messages.mkString(", ")} ====================") - } - - private def getBrokerConfig(): Properties = { - val props = new Properties() - props.put("broker.id", "0") - props.put("host.name", "localhost") - props.put("port", brokerPort.toString) - props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkAddress) - props.put("log.flush.interval.messages", "1") - props.put("replica.socket.timeout.ms", "1500") - props - } - - private def getProducerConfig(): Properties = { - val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - val props = new Properties() - props.put("metadata.broker.list", brokerAddr) - props.put("serializer.class", classOf[StringEncoder].getName) - props - } - - private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert( - server.apis.metadataCache.containsTopicAndPartition(topic, partition), - s"Partition [$topic, $partition] metadata not propagated after timeout" - ) - } - } - - class EmbeddedZookeeper(val zkConnect: String) { - val random = new Random() - val snapshotDir = Utils.createTempDir() - val logDir = Utils.createTempDir() - - val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) - val (ip, port) = { - val splits = zkConnect.split(":") - (splits(0), splits(1).toInt) - } - val factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress(ip, port), 16) - factory.startup(zookeeper) - - val actualPort = factory.getLocalPort - - def shutdown() { - factory.shutdown() - Utils.deleteRecursively(snapshotDir) - Utils.deleteRecursively(logDir) - } - } -} - - -class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { - var ssc: StreamingContext = _ - - before { - setupKafka() - } - - after { + override def afterAll(): Unit = { if (ssc != null) { ssc.stop() ssc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("Kafka input stream") { @@ -227,10 +56,10 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { ssc = new StreamingContext(sparkConf, Milliseconds(500)) val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) - sendMessages(topic, sent) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> zkAddress, + val kafkaParams = Map("zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}", "auto.offset.reset" -> "smallest") @@ -244,14 +73,14 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { result.put(kv._1, count) } } + ssc.start() + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert(sent.size === result.size) sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } } - ssc.stop() } } - diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 3cd960d1fd1d4..38548dd73b82c 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming.kafka - import java.io.File import scala.collection.mutable @@ -27,7 +26,7 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkConf @@ -35,47 +34,61 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils -class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { +class ReliableKafkaStreamSuite extends FunSuite + with BeforeAndAfterAll with BeforeAndAfter with Eventually { - val sparkConf = new SparkConf() + private val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private var kafkaTestUtils: KafkaTestUtils = _ - var groupId: String = _ - var kafkaParams: Map[String, String] = _ - var ssc: StreamingContext = _ - var tempDirectory: File = null + private var groupId: String = _ + private var kafkaParams: Map[String, String] = _ + private var ssc: StreamingContext = _ + private var tempDirectory: File = null + + override def beforeAll() : Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() - before { - setupKafka() groupId = s"test-consumer-${Random.nextInt(10000)}" kafkaParams = Map( - "zookeeper.connect" -> zkAddress, + "zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> groupId, "auto.offset.reset" -> "smallest" ) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) tempDirectory = Utils.createTempDir() + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDirectory) + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + before { + ssc = new StreamingContext(sparkConf, Milliseconds(500)) ssc.checkpoint(tempDirectory.getAbsolutePath) } after { if (ssc != null) { ssc.stop() + ssc = null } - Utils.deleteRecursively(tempDirectory) - tearDownKafka() } - test("Reliable Kafka input stream with single topic") { - var topic = "test-topic" - createTopic(topic) - sendMessages(topic, data) + val topic = "test-topic" + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === None) @@ -91,6 +104,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } } ssc.start() + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { // A basic process verification for ReliableKafkaReceiver. // Verify whether received message number is equal to the sent message number. @@ -100,14 +114,13 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter // Verify the offset number whether it is equal to the total message number. assert(getCommitOffset(groupId, topic, 0) === Some(29L)) } - ssc.stop() } test("Reliable Kafka input stream with multiple topics") { val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } // Before started, verify all the group/topic/partition offsets are 0. @@ -118,19 +131,18 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() + eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { // Verify the offset for each group/topic to see whether they are equal to the expected one. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } } - ssc.stop() } /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { - assert(zkClient != null, "Zookeeper client is not initialized") val topicDirs = new ZKGroupTopicDirs(groupId, topic) val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) + ZkUtils.readDataMaybeNull(kafkaTestUtils.zookeeperClient, zkPath)._1.map(_.toLong) } } diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 608f8e26473a6..9b4635e49020b 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -23,13 +23,16 @@ import tempfile import struct +from py4j.java_collections import MapConverter + from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext +from pyspark.streaming.kafka import KafkaUtils class PySparkStreamingTestCase(unittest.TestCase): - timeout = 10 # seconds + timeout = 20 # seconds duration = 1 def setUp(self): @@ -556,5 +559,43 @@ def check_output(n): check_output(3) +class KafkaStreamTests(PySparkStreamingTestCase): + + def setUp(self): + super(KafkaStreamTests, self).setUp() + + kafkaTestUtilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ + .loadClass("org.apache.spark.streaming.kafka.KafkaTestUtils") + self._kafkaTestUtils = kafkaTestUtilsClz.newInstance() + self._kafkaTestUtils.setup() + + def tearDown(self): + if self._kafkaTestUtils is not None: + self._kafkaTestUtils.teardown() + self._kafkaTestUtils = None + + super(KafkaStreamTests, self).tearDown() + + def test_kafka_stream(self): + """Test the Python Kafka stream API.""" + topic = "topic1" + sendData = {"a": 3, "b": 5, "c": 10} + jSendData = MapConverter().convert(sendData, + self.ssc.sparkContext._gateway._gateway_client) + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, jSendData) + + stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), + "test-streaming-consumer", {topic: 1}, + {"auto.offset.reset": "smallest"}) + + result = {} + for i in chain.from_iterable(self._collect(stream.map(lambda x: x[1]), + sum(sendData.values()))): + result[i] = result.get(i, 0) + 1 + + self.assertEqual(sendData, result) + if __name__ == "__main__": unittest.main() diff --git a/python/run-tests b/python/run-tests index f569a56fb7a9a..f3a07d8aba562 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,6 +21,8 @@ # Figure out where the Spark framework is installed FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" +. "$FWDIR"/bin/load-spark-env.sh + # CD into the python directory to find things on the right path cd "$FWDIR/python" @@ -57,7 +59,7 @@ function run_core_tests() { PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py" run_test "pyspark/serializers.py" - run_test "pyspark/profiler.py" + run_test "pyspark/profiler.py" run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" } @@ -97,6 +99,21 @@ function run_ml_tests() { function run_streaming_tests() { echo "Run streaming tests ..." + + KAFKA_ASSEMBLY_DIR="$FWDIR"/external/kafka-assembly + JAR_PATH="${KAFKA_ASSEMBLY_DIR}/target/scala-${SPARK_SCALA_VERSION}" + for f in "${JAR_PATH}"/spark-streaming-kafka-assembly-*.jar; do + if [[ ! -e "$f" ]]; then + echo "Failed to find Spark Streaming Kafka assembly jar in $KAFKA_ASSEMBLY_DIR" 1>&2 + echo "You need to build Spark with " \ + "'build/sbt assembly/assembly streaming-kafka-assembly/assembly' or" \ + "'build/mvn package' before running this program" 1>&2 + exit 1 + fi + KAFKA_ASSEMBLY_JAR="$f" + done + + export PYSPARK_SUBMIT_ARGS="--jars ${KAFKA_ASSEMBLY_JAR} pyspark-shell" run_test "pyspark/streaming/util.py" run_test "pyspark/streaming/tests.py" } From 18ca089bed41ce3e87deeb14206317863518c12c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 10 Apr 2015 01:51:42 -0700 Subject: [PATCH 692/817] [SPARK-6766][Streaming] Fix issue about StreamingListenerBatchSubmitted and StreamingListenerBatchStarted This PR includes: 1. Send `StreamingListenerBatchSubmitted` when `JobSet` is submitted 1. Fix `StreamingListenerBatchStarted.batchInfo.processingStartTime` 1. Fix a type: `completedaBatchInfos` -> `completedBatchInfos` Author: zsxwing Closes #5414 from zsxwing/SPARK-6766 and squashes the following commits: 2f85060 [zsxwing] Update tests ca0955b [zsxwing] Combine unit tests 79b4fed [zsxwing] Add StreamingJobProgressListenerSuite to test StreamingJobProgressListener fc3a2a1 [zsxwing] Add unit tests for SPARK-6766 74aed99 [zsxwing] Refactor as per TD's suggestion 493f978 [zsxwing] Send StreamingListenerBatchSubmitted when JobSet is submitted; fix StreamingListenerBatchStarted.batchInfo.processingStartTime; fix a typo --- .../streaming/scheduler/JobScheduler.scala | 8 +- .../ui/StreamingJobProgressListener.scala | 16 +-- .../streaming/StreamingListenerSuite.scala | 55 ++++++-- .../StreamingJobProgressListenerSuite.scala | 119 ++++++++++++++++++ 4 files changed, 180 insertions(+), 18 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index d6a93acbe711b..95f1857b4c377 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -105,6 +105,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { if (jobSet.jobs.isEmpty) { logInfo("No jobs added for time " + jobSet.time) } else { + listenerBus.post(StreamingListenerBatchSubmitted(jobSet.toBatchInfo)) jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) logInfo("Added jobs for time " + jobSet.time) @@ -134,10 +135,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private def handleJobStart(job: Job) { val jobSet = jobSets.get(job.time) - if (!jobSet.hasStarted) { + val isFirstJobOfJobSet = !jobSet.hasStarted + jobSet.handleJobStart(job) + if (isFirstJobOfJobSet) { + // "StreamingListenerBatchStarted" should be posted after calling "handleJobStart" to get the + // correct "jobSet.processingStartTime". listenerBus.post(StreamingListenerBatchStarted(jobSet.toBatchInfo)) } - jobSet.handleJobStart(job) logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index e4bd067cacb77..84f80e638f638 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -33,7 +33,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] - private val completedaBatchInfos = new Queue[BatchInfo] + private val completedBatchInfos = new Queue[BatchInfo] private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L private var totalReceivedRecords = 0L @@ -62,7 +62,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { synchronized { - runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + waitingBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo } } @@ -79,8 +79,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) synchronized { waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) - completedaBatchInfos.enqueue(batchCompleted.batchInfo) - if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + completedBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedBatchInfos.size > batchInfoLimit) completedBatchInfos.dequeue() totalCompletedBatches += 1L batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => @@ -118,7 +118,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def retainedCompletedBatches: Seq[BatchInfo] = synchronized { - completedaBatchInfos.toSeq + completedBatchInfos.toSeq } def processingDelayDistribution: Option[Distribution] = synchronized { @@ -165,7 +165,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def lastCompletedBatch: Option[BatchInfo] = { - completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + completedBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption } def lastReceivedBatch: Option[BatchInfo] = { @@ -174,10 +174,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private def retainedBatches: Seq[BatchInfo] = synchronized { (waitingBatchInfos.values.toSeq ++ - runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + runningBatchInfos.values.toSeq ++ completedBatchInfos).sortBy(_.batchTime)(Time.ordering) } private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { - Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + Distribution(completedBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 852e8bb71d4f6..7210439509541 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -46,10 +46,38 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { val collector = new BatchInfoCollector ssc.addStreamingListener(collector) runStreams(ssc, input.size, input.size) - val batchInfos = collector.batchInfos - batchInfos should have size 4 - batchInfos.foreach(info => { + // SPARK-6766: batch info should be submitted + val batchInfosSubmitted = collector.batchInfosSubmitted + batchInfosSubmitted should have size 4 + + batchInfosSubmitted.foreach(info => { + info.schedulingDelay should be (None) + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + isInIncreasingOrder(batchInfosSubmitted.map(_.submissionTime)) should be (true) + + // SPARK-6766: processingStartTime of batch info should not be None when starting + val batchInfosStarted = collector.batchInfosStarted + batchInfosStarted should have size 4 + + batchInfosStarted.foreach(info => { + info.schedulingDelay should not be None + info.schedulingDelay.get should be >= 0L + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + isInIncreasingOrder(batchInfosStarted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosStarted.map(_.processingStartTime.get)) should be (true) + + // test onBatchCompleted + val batchInfosCompleted = collector.batchInfosCompleted + batchInfosCompleted should have size 4 + + batchInfosCompleted.foreach(info => { info.schedulingDelay should not be None info.processingDelay should not be None info.totalDelay should not be None @@ -58,9 +86,9 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { info.totalDelay.get should be >= 0L }) - isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingEndTime.get)) should be (true) } test("receiver info reporting") { @@ -99,9 +127,20 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { /** Listener that collects information on processed batches */ class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] + val batchInfosCompleted = new ArrayBuffer[BatchInfo] + val batchInfosStarted = new ArrayBuffer[BatchInfo] + val batchInfosSubmitted = new ArrayBuffer[BatchInfo] + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { + batchInfosSubmitted += batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { + batchInfosStarted += batchStarted.batchInfo + } + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo + batchInfosCompleted += batchCompleted.batchInfo } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala new file mode 100644 index 0000000000000..2b9d164500b72 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import org.scalatest.Matchers + +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.{Time, Milliseconds, TestSuiteBase} + +class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { + + val input = (1 to 4).map(Seq(_)).toSeq + val operation = (d: DStream[Int]) => d.map(x => x) + + override def batchDuration = Milliseconds(100) + + test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + + "onReceiverStarted, onReceiverError, onReceiverStopped") { + val ssc = setupStreams(input, operation) + val listener = new StreamingJobProgressListener(ssc) + + val receivedBlockInfo = Map( + 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), + 1 -> Array(ReceivedBlockInfo(1, 300, null)) + ) + + // onBatchSubmitted + val batchInfoSubmitted = BatchInfo(Time(1000), receivedBlockInfo, 1000, None, None) + listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) + listener.waitingBatches should be (List(batchInfoSubmitted)) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (0) + + // onBatchStarted + val batchInfoStarted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + listener.onBatchStarted(StreamingListenerBatchStarted(batchInfoStarted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (List(batchInfoStarted)) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (600) + + // onBatchCompleted + val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (List(batchInfoCompleted)) + listener.lastCompletedBatch should be (Some(batchInfoCompleted)) + listener.numUnprocessedBatches should be (0) + listener.numTotalCompletedBatches should be (1) + listener.numTotalProcessedRecords should be (600) + listener.numTotalReceivedRecords should be (600) + + // onReceiverStarted + val receiverInfoStarted = ReceiverInfo(0, "test", null, true, "localhost") + listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (None) + + // onReceiverError + val receiverInfoError = ReceiverInfo(1, "test", null, true, "localhost") + listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (None) + + // onReceiverStopped + val receiverInfoStopped = ReceiverInfo(2, "test", null, true, "localhost") + listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (Some(receiverInfoStopped)) + listener.receiverInfo(3) should be (None) + } + + test("Remove the old completed batches when exceeding the limit") { + val ssc = setupStreams(input, operation) + val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + val listener = new StreamingJobProgressListener(ssc) + + val receivedBlockInfo = Map( + 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), + 1 -> Array(ReceivedBlockInfo(1, 300, null)) + ) + val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + + for(_ <- 0 until (limit + 10)) { + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + } + + listener.retainedCompletedBatches.size should be (limit) + listener.numTotalCompletedBatches should be(limit + 10) + } +} From 9f5ed99d644949443d19c4895de6e0ece4be24d0 Mon Sep 17 00:00:00 2001 From: "June.He" Date: Fri, 10 Apr 2015 20:02:35 +0100 Subject: [PATCH 693/817] [SPARK-6773][Tests]Fix RAT checks still passed issue when download rat jar failed check -license will passed in next time when rat jar download failed. Add 2 step to fix this: 1. Clean the rat.jar if download failed. 2. Add a check logic after run rat checking. Author: June.He Closes #5421 from sisihj/rat-issue and squashes the following commits: 4958302 [June.He] delete redundant check and add quoted 66f7088 [June.He] Fix RAT checks still passed issue when download rat jar failed 7311e83 [June.He] Fix RAT checks still passed issue when download rat jar failed --- dev/check-license | 45 ++++++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/dev/check-license b/dev/check-license index 39943f882b6ca..10740cfdc5242 100755 --- a/dev/check-license +++ b/dev/check-license @@ -24,29 +24,27 @@ acquire_rat_jar () { JAR="$rat_jar" - if [[ ! -f "$rat_jar" ]]; then - # Download rat launch jar if it hasn't been downloaded yet - if [ ! -f "$JAR" ]; then - # Download - printf "Attempting to fetch rat\n" - JAR_DL="${JAR}.part" - if [ $(command -v curl) ]; then - curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" - elif [ $(command -v wget) ]; then - wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" - else - printf "You do not have curl or wget installed, please install rat manually.\n" - exit -1 - fi - fi - - unzip -tq $JAR &> /dev/null - if [ $? -ne 0 ]; then - # We failed to download - printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + # Download rat launch jar if it hasn't been downloaded yet + if [ ! -f "$JAR" ]; then + # Download + printf "Attempting to fetch rat\n" + JAR_DL="${JAR}.part" + if [ $(command -v curl) ]; then + curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" + elif [ $(command -v wget) ]; then + wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" + else + printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi - printf "Launching rat from ${JAR}\n" + fi + + unzip -tq "$JAR" &> /dev/null + if [ $? -ne 0 ]; then + # We failed to download + rm "$JAR" + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 fi } @@ -71,6 +69,11 @@ mkdir -p "$FWDIR"/lib $java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt +if [ $? -ne 0 ]; then + echo "RAT exited abnormally" + exit 1 +fi + ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then From b9baa4cd9f6e9fc58161f79744b6b7729894d920 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Fri, 10 Apr 2015 12:09:54 -0700 Subject: [PATCH 694/817] [SQL] [SPARK-6794] Use kryo-based SparkSqlSerializer for GeneralHashedRelation Benchmarking results: http://pastie.org/private/1dneo1mta5zpsw6gmsoeq Author: Volodymyr Lyubinets Closes #5433 from vlyubin/joins and squashes the following commits: d70c829 [Volodymyr Lyubinets] Addressed review feedback 527eac6 [Volodymyr Lyubinets] Use kryo-based SparkSqlSerializer for GeneralHashedRelation --- .../sql/execution/SparkSqlSerializer.scala | 3 +- .../sql/execution/joins/HashedRelation.scala | 47 +++++++++++++++++-- 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 967bd76b302d8..347e2f4a1a1af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -26,14 +26,13 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} -import com.twitter.chill.{AllScalaRegistrar, ResourcePool} +import com.twitter.chill.ResourcePool import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair -import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 2fa1cf5add3b5..ab84c123e0c0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution.joins +import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.util.collection.CompactBuffer @@ -29,16 +31,43 @@ import org.apache.spark.util.collection.CompactBuffer */ private[joins] sealed trait HashedRelation { def get(key: Row): CompactBuffer[Row] + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def writeBytes(out: ObjectOutput, serialized: Array[Byte]): Unit = { + out.writeInt(serialized.length) // Write the length of serialized bytes first + out.write(serialized) + } + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def readBytes(in: ObjectInput): Array[Byte] = { + val serializedSize = in.readInt() // Read the length of serialized bytes first + val bytes = new Array[Byte](serializedSize) + in.readFully(bytes) + bytes + } } /** * A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values. */ -private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) - extends HashedRelation with Serializable { +private[joins] final class GeneralHashedRelation( + private var hashTable: JavaHashMap[Row, CompactBuffer[Row]]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } @@ -46,8 +75,10 @@ private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, Com * A specialized [[HashedRelation]] that maps key into a single value. This implementation * assumes the key is unique. */ -private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) - extends HashedRelation with Serializable { +private[joins] final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[Row, Row]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = { val v = hashTable.get(key) @@ -55,6 +86,14 @@ private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, R } def getValue(key: Row): Row = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } From 0375134f42197f2e29aa865a513cda381f0a1445 Mon Sep 17 00:00:00 2001 From: Milan Straka Date: Fri, 10 Apr 2015 13:50:32 -0700 Subject: [PATCH 695/817] [SPARK-5969][PySpark] Fix descending pyspark.rdd.sortByKey. The samples should always be sorted in ascending order, because bisect.bisect_left is used on it. The reverse order of the result is already achieved in rangePartitioner by reversing the found index. The current implementation also work, but always uses only two partitions -- the first one and the last one (because the bisect_left return returns either "beginning" or "end" for a descending sequence). Author: Milan Straka This patch had conflicts when merged, resolved by Committer: Josh Rosen Closes #4761 from foxik/fix-descending-sort and squashes the following commits: 95896b5 [Milan Straka] Add regression test for SPARK-5969. 5757490 [Milan Straka] Fix descending pyspark.rdd.sortByKey. --- python/pyspark/rdd.py | 2 +- python/pyspark/tests.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1b18789040360..c8e54ed5c6b2c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -595,7 +595,7 @@ def sortPartition(iterator): maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() - samples = sorted(samples, reverse=(not ascending), key=keyfunc) + samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0bd5d20f7877f..0e3721b55adf7 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -829,6 +829,17 @@ def test_take_on_jrdd(self): rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) rdd._jrdd.first() + def test_sortByKey_uses_all_partitions_not_only_first_and_last(self): + # Regression test for SPARK-5969 + seq = [(i * 59 % 101, i) for i in range(101)] # unsorted sequence + rdd = self.sc.parallelize(seq) + for ascending in [True, False]: + sort = rdd.sortByKey(ascending=ascending, numPartitions=5) + self.assertEqual(sort.collect(), sorted(seq, reverse=not ascending)) + sizes = sort.glom().map(len).collect() + for size in sizes: + self.assertGreater(size, 0) + class ProfilerTests(PySparkTestCase): From 4740d6a158cb4d35408a95265c5b950b9e9628a3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Apr 2015 14:04:53 -0700 Subject: [PATCH 696/817] [SPARK-6216] [PySpark] check the python version in worker Author: Davies Liu Closes #5404 from davies/check_version and squashes the following commits: e559248 [Davies Liu] add tests ec33b5f [Davies Liu] check the python version in worker --- python/pyspark/rdd.py | 2 +- python/pyspark/tests.py | 16 ++++++++++++++++ python/pyspark/worker.py | 6 +++++- 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c8e54ed5c6b2c..c9ac95d117574 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2233,7 +2233,7 @@ def toLocalIterator(self): def _prepare_for_python_RDD(sc, command, obj=None): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() - pickled_command = ser.dumps(command) + pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0e3721b55adf7..b938b9ce12395 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -35,6 +35,8 @@ import threading import hashlib +from py4j.protocol import Py4JJavaError + if sys.version_info[:2] <= (2, 6): try: import unittest2 as unittest @@ -1494,6 +1496,20 @@ def count(): self.assertTrue(not t.isAlive()) self.assertEqual(100000, rdd.count()) + def test_with_different_versions_of_python(self): + rdd = self.sc.parallelize(range(10)) + rdd.count() + version = sys.version_info + sys.version_info = (2, 0, 0) + log4j = self.sc._jvm.org.apache.log4j + old_level = log4j.LogManager.getRootLogger().getLevel() + log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) + try: + self.assertRaises(Py4JJavaError, lambda: rdd.count()) + finally: + sys.version_info = version + log4j.LogManager.getRootLogger().setLevel(old_level) + class SparkSubmitTests(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8a93c320ec5d3..452d6fabdcc17 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -88,7 +88,11 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, profiler, deserializer, serializer) = command + (func, profiler, deserializer, serializer), version = command + if version != sys.version_info[:2]: + raise Exception(("Python in worker has different version %s than that in " + + "driver %s, PySpark cannot run with different minor versions") % + (sys.version_info[:2], version)) init_time = time.time() def process(): From 68ecdb7f99ae30f7c04c33a47ab7f59a3836f2a4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Apr 2015 15:35:45 -0700 Subject: [PATCH 697/817] [SPARK-6850] [SparkR] use one partition when we need to compare the whole result Author: Davies Liu Closes #5460 from davies/r_test and squashes the following commits: 0a593ce [Davies Liu] use one partition when we need to compare the whole result --- R/pkg/inst/tests/test_binaryFile.R | 4 ++-- R/pkg/inst/tests/test_textFile.R | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/inst/tests/test_binaryFile.R index 4bb5f58d83dc9..ca4218f3819f8 100644 --- a/R/pkg/inst/tests/test_binaryFile.R +++ b/R/pkg/inst/tests/test_binaryFile.R @@ -27,7 +27,7 @@ test_that("saveAsObjectFile()/objectFile() following textFile() works", { fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName1) - rdd <- textFile(sc, fileName1) + rdd <- textFile(sc, fileName1, 1) saveAsObjectFile(rdd, fileName2) rdd <- objectFile(sc, fileName2) expect_equal(collect(rdd), as.list(mockFile)) @@ -40,7 +40,7 @@ test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { fileName <- tempfile(pattern="spark-test", fileext=".tmp") l <- list(1, 2, 3) - rdd <- parallelize(sc, l) + rdd <- parallelize(sc, l, 1) saveAsObjectFile(rdd, fileName) rdd <- objectFile(sc, fileName) expect_equal(collect(rdd), l) diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R index 7bb3e8003131d..6b87b4b3e0b08 100644 --- a/R/pkg/inst/tests/test_textFile.R +++ b/R/pkg/inst/tests/test_textFile.R @@ -81,7 +81,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName1) - rdd <- textFile(sc, fileName1) + rdd <- textFile(sc, fileName1, 1L) saveAsTextFile(rdd, fileName2) rdd <- textFile(sc, fileName2) expect_equal(collect(rdd), as.list(mockFile)) @@ -93,7 +93,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", test_that("saveAsTextFile() on a parallelized list works as expected", { fileName <- tempfile(pattern="spark-test", fileext=".tmp") l <- list(1, 2, 3) - rdd <- parallelize(sc, l) + rdd <- parallelize(sc, l, 1L) saveAsTextFile(rdd, fileName) rdd <- textFile(sc, fileName) expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) From 23d5f8864f7d665a74b1d38118700139854dbb1c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Apr 2015 16:05:14 -0700 Subject: [PATCH 698/817] [SPARK-6851][SQL] Create new instance for each converted parquet relation Otherwise we end up rewriting predicates to be trivially equal (i.e. `a#1 = a#2` -> `a#3 = a#3`), at which point the query is no longer valid. Author: Michael Armbrust Closes #5458 from marmbrus/selfJoinParquet and squashes the following commits: 22df77c [Michael Armbrust] [SPARK-6851][SQL] Create new instance for each converted parquet relation --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 78 ++++++++++++++++++- 2 files changed, 80 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 315fab673da5c..3ed5c5b031736 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -279,7 +279,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } - if (metastoreRelation.hiveQlTable.isPartitioned) { + val result = if (metastoreRelation.hiveQlTable.isPartitioned) { val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) val partitionColumnDataTypes = partitionSchema.map(_.dataType) val partitions = metastoreRelation.hiveQlPartitions.map { p => @@ -314,6 +314,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with parquetRelation } + + result.newInstance() } override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 7811bd2e9effb..4c369c0634bda 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -34,6 +34,17 @@ case class Nested3(f3: Int) case class NestedArray2(b: Seq[Int]) case class NestedArray1(a: NestedArray2) +case class Order( + id: Int, + make: String, + `type`: String, + price: Int, + pdate: String, + customer: String, + city: String, + state: String, + month: Int) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -41,6 +52,72 @@ case class NestedArray1(a: NestedArray2) */ class SQLQuerySuite extends QueryTest { + test("SPARK-6851: Self-joined converted parquet tables") { + val orders = Seq( + Order(1, "Atlas", "MTB", 234, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(3, "Swift", "MTB", 285, "2015-01-17", "John S", "Redwood City", "CA", 20151), + Order(4, "Atlas", "Hybrid", 303, "2015-01-23", "Jones S", "San Mateo", "CA", 20151), + Order(7, "Next", "MTB", 356, "2015-01-04", "Jane D", "Daly City", "CA", 20151), + Order(10, "Next", "YFlikr", 187, "2015-01-09", "John D", "Fremont", "CA", 20151), + Order(11, "Swift", "YFlikr", 187, "2015-01-23", "John D", "Hayward", "CA", 20151), + Order(2, "Next", "Hybrid", 324, "2015-02-03", "Jane D", "Daly City", "CA", 20152), + Order(5, "Next", "Street", 187, "2015-02-08", "John D", "Fremont", "CA", 20152), + Order(6, "Atlas", "Street", 154, "2015-02-09", "John D", "Pacifica", "CA", 20152), + Order(8, "Swift", "Hybrid", 485, "2015-02-19", "John S", "Redwood City", "CA", 20152), + Order(9, "Atlas", "Split", 303, "2015-02-28", "Jones S", "San Mateo", "CA", 20152)) + + val orderUpdates = Seq( + Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151)) + + orders.toDF.registerTempTable("orders1") + orderUpdates.toDF.registerTempTable("orderupdates1") + + sql( + """CREATE TABLE orders( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql( + """CREATE TABLE orderupdates( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("INSERT INTO TABLE orders PARTITION(state, month) SELECT * FROM orders1") + sql("INSERT INTO TABLE orderupdates PARTITION(state, month) SELECT * FROM orderupdates1") + + checkAnswer( + sql( + """ + |select orders.state, orders.month + |from orders + |join ( + | select distinct orders.state,orders.month + | from orders + | join orderupdates + | on orderupdates.id = orders.id) ao + | on ao.state = orders.state and ao.month = orders.month + """.stripMargin), + (1 to 6).map(_ => Row("CA", 20151))) + } + test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") df.registerTempTable("table1") @@ -478,5 +555,4 @@ class SQLQuerySuite extends QueryTest { sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed } - } From 67d06880e47e0324409cf7e5b21db1dcb0107b82 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Fri, 10 Apr 2015 16:27:56 -0700 Subject: [PATCH 699/817] [SQL] [SPARK-6620] Speed up toDF() and rdd() functions by constructing converters in ScalaReflection cc marmbrus Author: Volodymyr Lyubinets Closes #5279 from vlyubin/speedup and squashes the following commits: e75a387 [Volodymyr Lyubinets] Changes to ScalaUDF 11a20ec [Volodymyr Lyubinets] Avoid creating a tuple c327bc9 [Volodymyr Lyubinets] Moved the only remaining function from DataTypeConversions to DateUtils dec6802 [Volodymyr Lyubinets] Addresed review feedback 74301fa [Volodymyr Lyubinets] Addressed review comments afa3aa5 [Volodymyr Lyubinets] Minor refactoring, added license, removed debug output 881dc60 [Volodymyr Lyubinets] Moved to a separate module; addressed review comments; one extra place of usage; changed behaviour for Java 8cad6e2 [Volodymyr Lyubinets] Addressed review commments 41b2aa9 [Volodymyr Lyubinets] Creating converters for ScalaReflection stuff, and more --- .../spark/ml/feature/TokenizerSuite.scala | 17 +- .../sql/catalyst/CatalystTypeConverters.scala | 295 +++++++ .../spark/sql/catalyst/ScalaReflection.scala | 55 -- .../sql/catalyst/expressions/ScalaUdf.scala | 819 ++++++++++++------ .../plans/logical/LocalRelation.scala | 7 +- .../spark/sql/types/DataTypeConversions.scala | 77 -- .../apache/spark/sql/types/DateUtils.scala | 29 + .../sql/catalyst/ScalaReflectionSuite.scala | 4 +- .../org/apache/spark/sql/DataFrame.scala | 11 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../spark/sql/execution/ExistingRDD.scala | 14 +- .../spark/sql/execution/LocalTableScan.scala | 16 +- .../spark/sql/execution/SparkPlan.scala | 11 +- .../spark/sql/execution/basicOperators.scala | 9 +- .../org/apache/spark/sql/json/JsonRDD.scala | 4 +- .../apache/spark/sql/JavaDataFrameSuite.java | 10 +- .../org/apache/spark/sql/json/JsonSuite.scala | 3 +- 17 files changed, 929 insertions(+), 461 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala index bf862b912d326..d186ead8f542f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -25,10 +25,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row, SQLContext} @BeanInfo -case class TokenizerTestData(rawText: String, wantedTokens: Seq[String]) { - /** Constructor used in [[org.apache.spark.ml.feature.JavaTokenizerSuite]] */ - def this(rawText: String, wantedTokens: Array[String]) = this(rawText, wantedTokens.toSeq) -} +case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { import org.apache.spark.ml.feature.RegexTokenizerSuite._ @@ -46,14 +43,14 @@ class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { .setOutputCol("tokens") val dataset0 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization", ".")), - TokenizerTestData("Te,st. punct", Seq("Te", ",", "st", ".", "punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization", ".")), + TokenizerTestData("Te,st. punct", Array("Te", ",", "st", ".", "punct")) )) testRegexTokenizer(tokenizer, dataset0) val dataset1 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization")), - TokenizerTestData("Te,st. punct", Seq("punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization")), + TokenizerTestData("Te,st. punct", Array("punct")) )) tokenizer.setMinTokenLength(3) @@ -64,8 +61,8 @@ class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { .setGaps(true) .setMinTokenLength(0) val dataset2 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization.")), - TokenizerTestData("Te,st. punct", Seq("Te,st.", "", "punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization.")), + TokenizerTestData("Te,st. punct", Array("Te,st.", "", "punct")) )) testRegexTokenizer(tokenizer, dataset2) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala new file mode 100644 index 0000000000000..91976fef6dc0d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import java.util.{Map => JavaMap} + +import scala.collection.mutable.HashMap + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +/** + * Functions to convert Scala types to Catalyst types and vice versa. + */ +object CatalystTypeConverters { + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + + /** + * Converts Scala objects to catalyst rows / types. This method is slow, and for batch + * conversion you should be using converter produced by createToCatalystConverter. + * Note: This is always called after schemaFor has been called. + * This ordering is important for UDT registration. + */ + def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (obj, udt: UserDefinedType[_]) => + udt.serialize(obj) + + case (o: Option[_], _) => + o.map(convertToCatalyst(_, dataType)).orNull + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToCatalyst(_, arrayType.elementType)) + + case (s: Array[_], arrayType: ArrayType) => + s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) + } + + case (jmap: JavaMap[_, _], mapType: MapType) => + val iter = jmap.entrySet.iterator + var listOfEntries: List[(Any, Any)] = List() + while (iter.hasNext) { + val entry = iter.next() + listOfEntries :+= (convertToCatalyst(entry.getKey, mapType.keyType), + convertToCatalyst(entry.getValue, mapType.valueType)) + } + listOfEntries.toMap + + case (p: Product, structType: StructType) => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = convertToCatalyst(iter.next(), structType.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case (d: BigDecimal, _) => + Decimal(d) + + case (d: java.math.BigDecimal, _) => + Decimal(d) + + case (d: java.sql.Date, _) => + DateUtils.fromJavaDate(d) + + case (r: Row, structType: StructType) => + val converters = structType.fields.map { + f => (item: Any) => convertToCatalyst(item, f.dataType) + } + convertRowWithConverters(r, structType, converters) + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Scala objects to the specified catalyst type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToCatalystConverter(dataType: DataType): Any => Any = { + def extractOption(item: Any): Any = item match { + case opt: Option[_] => opt.orNull + case other => other + } + + dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item) => extractOption(item) match { + case null => null + case other => udt.serialize(other) + } + + case arrayType: ArrayType => + val elementConverter = createToCatalystConverter(arrayType.elementType) + (item: Any) => { + extractOption(item) match { + case a: Array[_] => a.toSeq.map(elementConverter) + case s: Seq[_] => s.map(elementConverter) + case null => null + } + } + + case mapType: MapType => + val keyConverter = createToCatalystConverter(mapType.keyType) + val valueConverter = createToCatalystConverter(mapType.valueType) + (item: Any) => { + extractOption(item) match { + case m: Map[_, _] => + m.map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + + case jmap: JavaMap[_, _] => + val iter = jmap.entrySet.iterator + val convertedMap: HashMap[Any, Any] = HashMap() + while (iter.hasNext) { + val entry = iter.next() + convertedMap(keyConverter(entry.getKey)) = valueConverter(entry.getValue) + } + convertedMap + + case null => null + } + } + + case structType: StructType => + val converters = structType.fields.map(f => createToCatalystConverter(f.dataType)) + (item: Any) => { + extractOption(item) match { + case r: Row => + convertRowWithConverters(r, structType, converters) + + case p: Product => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = converters(idx)(iter.next()) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case null => + null + } + } + + case dateType: DateType => (item: Any) => extractOption(item) match { + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case other => other + } + + case _ => + (item: Any) => extractOption(item) match { + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case other => other + } + } + } + + /** + * Converts Catalyst types used internally in rows to standard Scala types + * This method is slow, and for batch conversion you should be using converter + * produced by createToScalaConverter. + */ + def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (d, udt: UserDefinedType[_]) => + udt.deserialize(d) + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToScala(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) + } + + case (r: Row, s: StructType) => + convertRowToScala(r, s) + + case (d: Decimal, _: DecimalType) => + d.toJavaBigDecimal + + case (i: Int, DateType) => + DateUtils.toJavaDate(i) + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Catalyst types to Scala type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToScalaConverter(dataType: DataType): Any => Any = dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item: Any) => if (item == null) null else udt.deserialize(item) + + case arrayType: ArrayType => + val elementConverter = createToScalaConverter(arrayType.elementType) + (item: Any) => if (item == null) null else item.asInstanceOf[Seq[_]].map(elementConverter) + + case mapType: MapType => + val keyConverter = createToScalaConverter(mapType.keyType) + val valueConverter = createToScalaConverter(mapType.valueType) + (item: Any) => if (item == null) { + null + } else { + item.asInstanceOf[Map[_, _]].map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + } + + case s: StructType => + val converters = s.fields.map(f => createToScalaConverter(f.dataType)) + (item: Any) => { + if (item == null) { + null + } else { + convertRowWithConverters(item.asInstanceOf[Row], s, converters) + } + } + + case _: DecimalType => + (item: Any) => item match { + case d: Decimal => d.toJavaBigDecimal + case other => other + } + + case DateType => + (item: Any) => item match { + case i: Int => DateUtils.toJavaDate(i) + case other => other + } + + case other => + (item: Any) => item + } + + def convertRowToScala(r: Row, schema: StructType): Row = { + val ar = new Array[Any](r.size) + var idx = 0 + while (idx < r.size) { + ar(idx) = convertToScala(r(idx), schema.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } + + /** + * Converts a row by applying the provided set of converter functions. It is used for both + * toScala and toCatalyst conversions. + */ + private[sql] def convertRowWithConverters( + row: Row, + schema: StructType, + converters: Array[Any => Any]): Row = { + val ar = new Array[Any](row.size) + var idx = 0 + while (idx < row.size) { + ar(idx) = converters(idx)(row(idx)) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 8bfd0471d9c7a..01d5c1512201a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -46,61 +46,6 @@ trait ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) - /** - * Converts Scala objects to catalyst rows / types. - * Note: This is always called after schemaFor has been called. - * This ordering is important for UDT registration. - */ - def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (obj, udt: UserDefinedType[_]) => udt.serialize(obj) - case (o: Option[_], _) => o.map(convertToCatalyst(_, dataType)).orNull - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, arrayType.elementType)) - case (s: Array[_], arrayType: ArrayType) => if (arrayType.elementType.isPrimitive) { - s.toSeq - } else { - s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) - } - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) - } - case (p: Product, structType: StructType) => - new GenericRow( - p.productIterator.toSeq.zip(structType.fields).map { case (elem, field) => - convertToCatalyst(elem, field.dataType) - }.toArray) - case (d: BigDecimal, _) => Decimal(d) - case (d: java.math.BigDecimal, _) => Decimal(d) - case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) - case (r: Row, structType: StructType) => - new GenericRow( - r.toSeq.zip(structType.fields).map { case (elem, field) => - convertToCatalyst(elem, field.dataType) - }.toArray) - case (other, _) => other - } - - /** Converts Catalyst types used internally in rows to standard Scala types */ - def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (d, udt: UserDefinedType[_]) => udt.deserialize(d) - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToScala(_, arrayType.elementType)) - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) - } - case (r: Row, s: StructType) => convertRowToScala(r, s) - case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal - case (i: Int, DateType) => DateUtils.toJavaDate(i) - case (other, _) => other - } - - def convertRowToScala(r: Row, schema: StructType): Row = { - // TODO: This is very slow!!! - new GenericRowWithSchema( - r.toSeq.zip(schema.fields.map(_.dataType)) - .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray, schema) - } - /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 389dc4f745723..9a77ca624ebe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types.DataType /** @@ -39,12 +39,14 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) - val evals = (0 to x - 1).map(x => s"ScalaReflection.convertToScala(child$x.eval(input), child$x.dataType)").reduce(_ + ",\n " + _) + val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) + lazy val converters = (0 to x - 1).map(x => s"lazy val converter$x = CatalystTypeConverters.createToScalaConverter(child$x.dataType)").reduce(_ + "\n " + _) + val evals = (0 to x - 1).map(x => s"converter$x(child$x.eval(input))").reduce(_ + ",\n " + _) - s""" case $x => + s"""case $x => val func = function.asInstanceOf[($anys) => Any] $childs + $converters (input: Row) => { func( $evals) @@ -60,51 +62,61 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (input: Row) => { func() } - + case 1 => val func = function.asInstanceOf[(Any) => Any] val child0 = children(0) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType)) + converter0(child0.eval(input))) } - + case 2 => val func = function.asInstanceOf[(Any, Any) => Any] val child0 = children(0) val child1 = children(1) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input))) } - + case 3 => val func = function.asInstanceOf[(Any, Any, Any) => Any] val child0 = children(0) val child1 = children(1) val child2 = children(2) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input))) } - + case 4 => val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] val child0 = children(0) val child1 = children(1) val child2 = children(2) val child3 = children(3) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input))) } - + case 5 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -112,15 +124,20 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child2 = children(2) val child3 = children(3) val child4 = children(4) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input))) } - + case 6 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -129,16 +146,22 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child3 = children(3) val child4 = children(4) val child5 = children(5) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input))) } - + case 7 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -148,17 +171,24 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child4 = children(4) val child5 = children(5) val child6 = children(6) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input))) } - + case 8 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -169,18 +199,26 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child5 = children(5) val child6 = children(6) val child7 = children(7) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input))) } - + case 9 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -192,19 +230,28 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child6 = children(6) val child7 = children(7) val child8 = children(8) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input))) } - + case 10 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -217,20 +264,30 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child7 = children(7) val child8 = children(8) val child9 = children(9) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input))) } - + case 11 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -244,21 +301,32 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child8 = children(8) val child9 = children(9) val child10 = children(10) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input))) } - + case 12 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -273,22 +341,34 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child9 = children(9) val child10 = children(10) val child11 = children(11) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input))) } - + case 13 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -304,23 +384,36 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child10 = children(10) val child11 = children(11) val child12 = children(12) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input))) } - + case 14 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -337,24 +430,38 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child11 = children(11) val child12 = children(12) val child13 = children(13) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input))) } - + case 15 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -372,25 +479,40 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child12 = children(12) val child13 = children(13) val child14 = children(14) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input))) } - + case 16 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -409,26 +531,42 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child13 = children(13) val child14 = children(14) val child15 = children(15) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input))) } - + case 17 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -448,27 +586,44 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child14 = children(14) val child15 = children(15) val child16 = children(16) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input))) } - + case 18 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -489,28 +644,46 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child15 = children(15) val child16 = children(16) val child17 = children(17) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input))) } - + case 19 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -532,29 +705,48 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child16 = children(16) val child17 = children(17) val child18 = children(18) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input))) } - + case 20 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -577,30 +769,50 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child17 = children(17) val child18 = children(18) val child19 = children(19) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input))) } - + case 21 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -624,31 +836,52 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child18 = children(18) val child19 = children(19) val child20 = children(20) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType), - ScalaReflection.convertToScala(child20.eval(input), child20.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input))) } - + case 22 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -673,35 +906,57 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child19 = children(19) val child20 = children(20) val child21 = children(21) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) + lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType), - ScalaReflection.convertToScala(child20.eval(input), child20.dataType), - ScalaReflection.convertToScala(child21.eval(input), child21.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input)), + converter21(child21.eval(input))) } } - + // scalastyle:on - - override def eval(input: Row): Any = ScalaReflection.convertToCatalyst(f(input), dataType) + + override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index bb79dc340553b..e3e070f0ff307 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, analysis} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.types.{DataTypeConversions, StructType, StructField} +import org.apache.spark.sql.types.{StructType, StructField} object LocalRelation { def apply(output: Attribute*): LocalRelation = new LocalRelation(output) @@ -31,7 +31,8 @@ object LocalRelation { def fromProduct(output: Seq[Attribute], data: Seq[Product]): LocalRelation = { val schema = StructType.fromAttributes(output) - LocalRelation(output, data.map(row => DataTypeConversions.productToRow(row, schema))) + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + LocalRelation(output, data.map(converter(_).asInstanceOf[Row])) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala deleted file mode 100644 index a9d63e784963d..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala +++ /dev/null @@ -1,77 +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.types - -import java.text.SimpleDateFormat - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow - - -private[sql] object DataTypeConversions { - - def productToRow(product: Product, schema: StructType): Row = { - val mutableRow = new GenericMutableRow(product.productArity) - val schemaFields = schema.fields.toArray - - var i = 0 - while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(product.productElement(i), schemaFields(i).dataType) - i += 1 - } - - mutableRow - } - - def stringToTime(s: String): java.util.Date = { - if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - java.sql.Timestamp.valueOf(s) - } else { - java.sql.Date.valueOf(s) - } - } else if (s.endsWith("Z")) { - // this is zero timezone of ISO8601 - stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") - } else if (s.indexOf("GMT") == -1) { - // timezone with ISO8601 - val inset = "+00.00".length - val s0 = s.substring(0, s.length - inset) - val s1 = s.substring(s.length - inset, s.length) - if (s0.substring(s0.lastIndexOf(':')).contains('.')) { - stringToTime(s0 + "GMT" + s1) - } else { - stringToTime(s0 + ".0GMT" + s1) - } - } else { - // ISO8601 with GMT insert - val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) - ISO8601GMT.parse(s) - } - } - - /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type - case (d: java.math.BigDecimal, _) => Decimal(d) - case (other, _) => other - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 8a1a3b81b3d2c..504fb05842505 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.types import java.sql.Date +import java.text.SimpleDateFormat import java.util.{Calendar, TimeZone} import org.apache.spark.sql.catalyst.expressions.Cast @@ -57,4 +58,32 @@ object DateUtils { } def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) + + def stringToTime(s: String): java.util.Date = { + if (!s.contains('T')) { + // JDBC escape string + if (s.contains(' ')) { + java.sql.Timestamp.valueOf(s) + } else { + java.sql.Date.valueOf(s) + } + } else if (s.endsWith("Z")) { + // this is zero timezone of ISO8601 + stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") + } else if (s.indexOf("GMT") == -1) { + // timezone with ISO8601 + val inset = "+00.00".length + val s0 = s.substring(0, s.length - inset) + val s1 = s.substring(s.length - inset, s.length) + if (s0.substring(s0.lastIndexOf(':')).contains('.')) { + stringToTime(s0 + "GMT" + s1) + } else { + stringToTime(s0 + ".0GMT" + s1) + } + } else { + // ISO8601 with GMT insert + val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) + ISO8601GMT.parse(s) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index eee00e3f7ea76..bbc0b661a0c0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -260,7 +260,7 @@ class ScalaReflectionSuite extends FunSuite { val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) val convertedData = Row(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) val dataType = schemaFor[PrimitiveData].dataType - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("convert Option[Product] to catalyst") { @@ -270,7 +270,7 @@ class ScalaReflectionSuite extends FunSuite { val dataType = schemaFor[OptionalData].dataType val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, Row(1, 1, 1, 1, 1, 1, true)) - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("infer schema from case class with multiple constructors") { 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 5c6016a4a2ce2..9b9adf855077a 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 @@ -33,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} @@ -713,7 +713,7 @@ class DataFrame private[sql]( val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributes = schema.toAttributes val rowFunction = - f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) + f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema).asInstanceOf[Row])) val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) Generate(generator, join = true, outer = false, None, logicalPlan) @@ -734,7 +734,7 @@ class DataFrame private[sql]( val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil def rowFunction(row: Row): TraversableOnce[Row] = { - f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) + f(row(0).asInstanceOf[A]).map(o => Row(CatalystTypeConverters.convertToCatalyst(o, dataType))) } val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) @@ -961,7 +961,10 @@ class DataFrame private[sql]( lazy val rdd: RDD[Row] = { // use a local variable to make sure the map closure doesn't capture the whole DataFrame val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + queryExecution.executedPlan.execute().mapPartitions { rows => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 39dd14e796f06..c25ef58e6f62a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -31,9 +31,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, OneRowRelation} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.{ScalaReflection, expressions} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, expressions} import org.apache.spark.sql.execution.{Filter, _} import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.json._ @@ -404,7 +404,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val catalystRows = if (needsConversion) { - rowRDD.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row]) + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + rowRDD.map(converter(_).asInstanceOf[Row]) } else { rowRDD } @@ -459,7 +460,7 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { row => new GenericRow( extractors.zip(attributeSeq).map { case (e, attr) => - DataTypeConversions.convertJavaToCatalyst(e.invoke(row), attr.dataType) + CatalystTypeConverters.convertToCatalyst(e.invoke(row), attr.dataType) }.toArray[Any] ) : Row } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index d8955725e59b1..656bdd7212f56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -20,14 +20,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType -import scala.collection.immutable - /** * :: DeveloperApi :: */ @@ -39,13 +37,15 @@ object RDDConversions { Iterator.empty } else { val bufferedIterator = iterator.buffered - val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) + val mutableRow = new SpecificMutableRow(schema.fields.map(_.dataType)) val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(r.productElement(i), schemaFields(i).dataType) + mutableRow(i) = converters(i)(r.productElement(i)) i += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala index 5bd699a2fa949..8a8c3a404323a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.Attribute @@ -32,9 +32,15 @@ case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNo override def execute(): RDD[Row] = rdd - override def executeCollect(): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray - override def executeTake(limit: Int): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).toArray + } + + + override def executeTake(limit: Int): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).take(limit).toArray + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index d239637cd4b4e..fabcf6b4a0570 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.{ScalaReflection, trees} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan @@ -80,8 +80,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ + def executeCollect(): Array[Row] = { - execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() + execute().mapPartitions { iter => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + iter.map(converter(_).asInstanceOf[Row]) + }.collect() } /** @@ -125,7 +129,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ partsScanned += numPartsToTry } - buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) + val converter = CatalystTypeConverters.createToScalaConverter(schema) + buf.toArray.map(converter(_).asInstanceOf[Row]) } protected def newProjection( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 1f5251a20376f..6eec520abff53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -21,7 +21,7 @@ import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -139,9 +139,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) private def collectData(): Array[Row] = child.execute().map(_.copy()).takeOrdered(limit)(ord) - // TODO: Is this copying for no reason? - override def executeCollect(): Array[Row] = - collectData().map(ScalaReflection.convertRowToScala(_, this.schema)) + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + collectData().map(converter(_).asInstanceOf[Row]) + } // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 0b770f2251943..b1e8521383756 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -391,7 +391,7 @@ private[sql] object JsonRDD extends Logging { value match { // only support string as date case value: java.lang.String => - DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) + DateUtils.millisToDays(DateUtils.stringToTime(value).getTime) case value: java.sql.Date => DateUtils.fromJavaDate(value) } } @@ -400,7 +400,7 @@ private[sql] object JsonRDD extends Logging { value match { case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => toTimestamp(DataTypeConversions.stringToTime(value).getTime) + case value: java.lang.String => toTimestamp(DateUtils.stringToTime(value).getTime) } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 1ff2d5a190521..6d0fbe83c2f36 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.Arrays; +import scala.collection.Seq; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -127,6 +129,12 @@ public void testCreateDataFrameFromJavaBeans() { schema.apply("b")); Row first = df.select("a", "b").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); - Assert.assertArrayEquals(bean.getB(), first.getAs(1)); + // Now Java lists and maps are converetd to Scala Seq's and Map's. Once we get a Seq below, + // verify that it has the expected length, and contains expected elements. + Seq result = first.getAs(1); + Assert.assertEquals(bean.getB().length, result.length()); + for (int i = 0; i < result.length(); i++) { + Assert.assertEquals(bean.getB()[i], result.apply(i)); + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1fe0b76c00be3..fd0e2746dc045 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -895,8 +895,7 @@ class JsonSuite extends QueryTest { ) } - test("SPARK-4228 DataFrame to JSON") - { + test("SPARK-4228 DataFrame to JSON") { val schema1 = StructType( StructField("f1", IntegerType, false) :: StructField("f2", StringType, false) :: From 95a07591b3e23782a7021ef2bbf07a67a1e4a83a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 11 Apr 2015 13:07:41 +0100 Subject: [PATCH 700/817] [Minor][Core] Fix typo Author: Liang-Chi Hsieh Closes #5466 from viirya/fix_ShuffleMapTask_typo and squashes the following commits: 2789fd5 [Liang-Chi Hsieh] fix typo. --- .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index fd0d484b45460..6c7d00069acb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -33,7 +33,7 @@ import org.apache.spark.shuffle.ShuffleWriter * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param taskBinary broadcast version of of the RDD and the ShuffleDependency. Once deserialized, + * @param taskBinary broadcast version of the RDD and the ShuffleDependency. Once deserialized, * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling From 694aef0d71d2683eaf63cbd1d8e95c2da423b72e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 11 Apr 2015 13:10:01 +0100 Subject: [PATCH 701/817] [hotfix] [build] Make sure JAVA_HOME is set for tests. This is needed at least for YARN integration tests, since `$JAVA_HOME` is used to launch the executors. Author: Marcelo Vanzin Closes #5441 from vanzin/yarn-test-test and squashes the following commits: 3eeec30 [Marcelo Vanzin] Use JAVA_HOME when available, java.home otherwise. d71f1bb [Marcelo Vanzin] And sbt too. 6bda399 [Marcelo Vanzin] WIP: Testing to see whether this fixes the yarn test issue. --- pom.xml | 14 ++++++++++++++ project/SparkBuild.scala | 10 +++++++--- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 70e297c4f082a..d8881c213bf07 100644 --- a/pom.xml +++ b/pom.xml @@ -159,6 +159,8 @@ 1.1.1.6 1.1.2 + ${java.home} + ${test_classpath} + ${test.java.home} true @@ -1224,6 +1227,7 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + ${test.java.home} true @@ -1716,6 +1720,16 @@ + + test-java-home + + env.JAVA_HOME + + + ${env.JAVA_HOME} + + + scala-2.11 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d3faa551a4b14..5f51f4b58f97a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -119,7 +119,9 @@ object SparkBuild extends PomBuild { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") lazy val sharedSettings = graphSettings ++ genjavadocSettings ++ Seq ( - javaHome := Properties.envOrNone("JAVA_HOME").map(file), + javaHome := sys.env.get("JAVA_HOME") + .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) + .map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -426,8 +428,10 @@ object TestSettings { fork := true, // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. - envVars in Test += ("SPARK_DIST_CLASSPATH" -> - (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":")), + envVars in Test ++= Map( + "SPARK_DIST_CLASSPATH" -> + (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), + "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", From 3ceb810aa8e69bc4abb69cbe713a624cb351cb35 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sat, 11 Apr 2015 22:11:03 +0800 Subject: [PATCH 702/817] [SPARK-6835] [SQL] Fix bug of Hive UDTF in Lateral View (ClassNotFound) ```SQL select key, v from src lateral view stack(3, 1+1, 2+2, 3) d as v; ``` Will cause exception ``` java.lang.ClassNotFoundException: stack at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at org.apache.spark.sql.hive.HiveFunctionWrapper.createFunction(Shim13.scala:148) at org.apache.spark.sql.hive.HiveGenericUdtf.function$lzycompute(hiveUdfs.scala:274) at org.apache.spark.sql.hive.HiveGenericUdtf.function(hiveUdfs.scala:274) at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector$lzycompute(hiveUdfs.scala:280) at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector(hiveUdfs.scala:280) at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes$lzycompute(hiveUdfs.scala:285) at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes(hiveUdfs.scala:285) at org.apache.spark.sql.hive.HiveGenericUdtf.makeOutput(hiveUdfs.scala:291) at org.apache.spark.sql.catalyst.expressions.Generator.output(generators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60) at scala.Option.map(Option.scala:145) at org.apache.spark.sql.catalyst.plans.logical.Generate.generatorOutput(basicOperators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate.output(basicOperators.scala:70) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117) ``` Author: Cheng Hao Closes #5444 from chenghao-intel/hive_udtf and squashes the following commits: 065a98c [Cheng Hao] fix bug of Hive UDTF in Lateral View (ClassNotFound) --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 9 ++++++++- .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 077e64133faad..0bdaf5f7ef8ef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.hive import java.sql.Date +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf @@ -1284,8 +1286,13 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Explode(attributes, nodeToExpr(child)) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $functionName")) + val functionClassName = functionInfo.getFunctionClass.getName + HiveGenericUdtf( - new HiveFunctionWrapper(functionName), + new HiveFunctionWrapper(functionClassName), attributes, children.map(nodeToExpr)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 4c369c0634bda..47b4cb9ca61ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -51,6 +51,12 @@ case class Order( * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("SPARK-6835: udtf in lateral view") { + val df = Seq((1, 1)).toDF("c1", "c2") + df.registerTempTable("table1") + val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, c1 + 2) d AS v") + checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) + } test("SPARK-6851: Self-joined converted parquet tables") { val orders = Seq( From 198cf2a3fa9babb5e8b7b44da0471b63b9f3ec04 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 11 Apr 2015 14:50:50 -0700 Subject: [PATCH 703/817] [SPARK-6858][SQL] Register Java HashMap for SparkSqlSerializer Since now kyro serializer is used for `GeneralHashedRelation` whether kyro is enabled or not, it is better to register Java `HashMap` in `SparkSqlSerializer`. Author: Liang-Chi Hsieh Closes #5465 from viirya/register_hashmap and squashes the following commits: 9062601 [Liang-Chi Hsieh] Register Java HashMap for SparkSqlSerializer. --- .../org/apache/spark/sql/execution/SparkSqlSerializer.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 347e2f4a1a1af..914f387dec78f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.types.Decimal @@ -54,6 +55,7 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], new OpenHashSetSerializer) kryo.register(classOf[Decimal]) + kryo.register(classOf[JavaHashMap[_, _]]) kryo.setReferences(false) kryo From 5f7b7cdab41002ad291f0a1ba7777cfff21d0019 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Sat, 11 Apr 2015 14:52:49 -0700 Subject: [PATCH 704/817] [SPARK-6611][SQL] Add support for INTEGER as synonym of INT. https://issues.apache.org/jira/browse/SPARK-6611 Author: Santiago M. Mola Closes #5271 from smola/features/integer-parse and squashes the following commits: f5c1c64 [Santiago M. Mola] [SPARK-6611] Add support for INTEGER as synonym of INT. --- .../main/scala/org/apache/spark/sql/types/DataTypeParser.scala | 2 +- .../scala/org/apache/spark/sql/types/DataTypeParserSuite.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala index 34270d0ca7cd7..5163f05879e42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala @@ -40,7 +40,7 @@ private[sql] trait DataTypeParser extends StandardTokenParsers { protected lazy val primitiveType: Parser[DataType] = "(?i)string".r ^^^ StringType | "(?i)float".r ^^^ FloatType | - "(?i)int".r ^^^ IntegerType | + "(?i)(?:int|integer)".r ^^^ IntegerType | "(?i)tinyint".r ^^^ ByteType | "(?i)smallint".r ^^^ ShortType | "(?i)double".r ^^^ DoubleType | diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala index 1ba21b64603ac..169125264a803 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala @@ -34,10 +34,12 @@ class DataTypeParserSuite extends FunSuite { } checkDataType("int", IntegerType) + checkDataType("integer", IntegerType) checkDataType("BooLean", BooleanType) checkDataType("tinYint", ByteType) checkDataType("smallINT", ShortType) checkDataType("INT", IntegerType) + checkDataType("INTEGER", IntegerType) checkDataType("bigint", LongType) checkDataType("float", FloatType) checkDataType("dOUBle", DoubleType) From 6437e7cc3bd405ebd4ad7d8f9c7a5e703652ad36 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Sat, 11 Apr 2015 15:42:03 -0700 Subject: [PATCH 705/817] [SPARK-6863] Fix formatting on SQL programming guide. https://issues.apache.org/jira/browse/SPARK-6863 Author: Santiago M. Mola Closes #5472 from smola/fix/sql-docs and squashes the following commits: 42503d4 [Santiago M. Mola] [SPARK-6863] Fix formatting on SQL programming guide. --- docs/sql-programming-guide.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 663f656883721..332618edf0c55 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1784,6 +1784,7 @@ in Hive deployments. **Esoteric Hive Features** + * `UNION` type * Unique join * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at From 7dbd37160ff57f80cc7abdcaef95f8c6df20a0f0 Mon Sep 17 00:00:00 2001 From: "Guancheng (G.C.) Chen" Date: Sat, 11 Apr 2015 15:43:12 -0700 Subject: [PATCH 706/817] [Minor][SQL] Fix typo in sql In this PR, "analyser" is changed to "analyzer" to keep a consistent naming. Some other typos are also fixed. Author: Guancheng (G.C.) Chen Closes #5474 from gchen/sql-typo and squashes the following commits: 70e6e76 [Guancheng (G.C.) Chen] Merge branch 'sql-typo' of github.com:gchen/spark into sql-typo fb7a6e2 [Guancheng (G.C.) Chen] fix typo in sql 37e3da1 [Guancheng (G.C.) Chen] fix type in sql --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../org/apache/spark/sql/catalyst/analysis/Catalog.scala | 4 ++-- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b3aba4f68ddf9..524c73c31bbe1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types._ /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing - * when all relations are already filled in and the analyser needs only to resolve attribute + * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 5eb7dff0cede8..b2f8157a1a61f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} /** - * Thrown by a catalog when a table cannot be found. The analzyer will rethrow the exception + * Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception * as an AnalysisException with the correct position information. */ class NoSuchTableException extends Exception @@ -201,7 +201,7 @@ trait OverrideCatalog extends Catalog { /** * A trivial catalog that returns an error when a relation is requested. Used for testing when all - * relations are already filled in and the analyser needs only to resolve attribute references. + * relations are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c43ea55899695..16ca5bcd57a72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -57,8 +57,8 @@ class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistr } /** - * A trivial catalog that returns an error when a function is requested. Used for testing when all - * functions are already filled in and the analyser needs only to resolve attribute references. + * A trivial catalog that returns an error when a function is requested. Used for testing when all + * functions are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { override def registerFunction(name: String, builder: FunctionBuilder): Unit = { From 2f53588738e95a2191f9844818e47f0d2ebbfd54 Mon Sep 17 00:00:00 2001 From: haiyang Date: Sat, 11 Apr 2015 18:30:17 -0700 Subject: [PATCH 707/817] [SPARK-6199] [SQL] Support CTE in HiveContext and SQLContext Author: haiyang Closes #4929 from haiyangsea/cte and squashes the following commits: 220b67d [haiyang] add golden files for cte test d3c7681 [haiyang] Merge branch 'master' into cte-repair 0ba2070 [haiyang] modify code style 9ce6b58 [haiyang] fix conflict ff74741 [haiyang] add comment for With plan 0d56af4 [haiyang] code indention 776a440 [haiyang] add comments for resolve relation strategy 2fccd7e [haiyang] add comments for resolve relation strategy 241bbe2 [haiyang] fix cte problem of view e9e1237 [haiyang] fix test case problem 614182f [haiyang] add test cases for CTE feature 32e415b [haiyang] add comment 1cc8c15 [haiyang] support with 03f1097 [haiyang] support with e960099 [haiyang] support with 9aaa874 [haiyang] support with 0566978 [haiyang] support with a99ecd2 [haiyang] support with c3fa4c2 [haiyang] support with 3b6077f [haiyang] support with 5f8abe3 [haiyang] support with 4572b05 [haiyang] support with f801f54 [haiyang] support with --- .../apache/spark/sql/catalyst/SqlParser.scala | 7 +++++ .../sql/catalyst/analysis/Analyzer.scala | 31 ++++++++++++++----- .../plans/logical/basicOperators.scala | 12 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 14 +++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 27 ++++++++++++---- ...ture #1-0-eedabbfe6ba8799f7b7782fb47a82768 | 3 ++ ...ture #2-0-aa03d104251f97e36bc52279cb9931c9 | 4 +++ ...ture #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++ 9 files changed, 100 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 create mode 100644 sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 create mode 100644 sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 89f4a19add1c6..ee04cb579deb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -111,6 +111,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected val UPPER = Keyword("UPPER") protected val WHEN = Keyword("WHEN") protected val WHERE = Keyword("WHERE") + protected val WITH = Keyword("WITH") protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { exprs.zipWithIndex.map { @@ -127,6 +128,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) | insert + | cte ) protected lazy val select: Parser[LogicalPlan] = @@ -156,6 +158,11 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) } + protected lazy val cte: Parser[LogicalPlan] = + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start <~ ")"), ",") ~ start ^^ { + case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) + } + protected lazy val projection: Parser[Expression] = expression ~ (AS.? ~> ident.?) ^^ { case e ~ a => a.fold(e)(Alias(e, _)()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 524c73c31bbe1..b83f18abdd239 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -169,21 +169,36 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation): LogicalPlan = { + def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]) = { try { - catalog.lookupRelation(u.tableIdentifier, u.alias) + // In hive, if there is same table name in database and CTE definition, + // hive will use the table in database, not the CTE one. + // Taking into account the reasonableness and the implementation complexity, + // here use the CTE definition first, check table name only and ignore database name + cteRelations.get(u.tableIdentifier.last) + .map(relation => u.alias.map(Subquery(_, relation)).getOrElse(relation)) + .getOrElse(catalog.lookupRelation(u.tableIdentifier, u.alias)) } catch { case _: NoSuchTableException => u.failAnalysis(s"no such table ${u.tableName}") } } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _) => - i.copy( - table = EliminateSubQueries(getTable(u))) - case u: UnresolvedRelation => - getTable(u) + def apply(plan: LogicalPlan): LogicalPlan = { + val (realPlan, cteRelations) = plan match { + // TODO allow subquery to define CTE + // Add cte table to a temp relation map,drop `with` plan and keep its child + case With(child, relations) => (child, relations) + case other => (other, Map.empty[String, LogicalPlan]) + } + + realPlan transform { + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _) => + i.copy( + table = EliminateSubQueries(getTable(u, cteRelations))) + case u: UnresolvedRelation => + getTable(u, cteRelations) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 8633e06093cf3..3bd5aa5964221 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -147,6 +147,18 @@ case class CreateTableAsSelect[T]( override lazy val resolved: Boolean = databaseName != None && childrenResolved } +/** + * A container for holding named common table expressions (CTEs) and a query plan. + * This operator will be removed during analysis and the relations will be substituted into child. + * @param child The final query of this CTE. + * @param cteRelations Queries that this CTE defined, + * key is the alias of the CTE definition, + * value is the CTE definition. + */ +case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends UnaryNode { + override def output = child.output +} + case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1392b4819131b..fb8fc6dbd1e1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -407,6 +407,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("CTE feature") { + checkAnswer( + sql("with q1 as (select * from testData limit 10) select * from q1"), + testData.take(10).toSeq) + + checkAnswer( + sql(""" + |with q1 as (select * from testData where key= '5'), + |q2 as (select * from testData where key = '4') + |select * from q1 union all select * from q2""".stripMargin), + Row(5, "5") :: Row(4, "4") :: Nil) + + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 0bdaf5f7ef8ef..2fb2e7c4a5370 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -576,11 +576,23 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_QUERY", queryArgs) if Seq("TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) => - val (fromClause: Option[ASTNode], insertClauses) = queryArgs match { - case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => - (Some(args.head), insertClauses) - case Token("TOK_INSERT", _) :: Nil => (None, queryArgs) - } + val (fromClause: Option[ASTNode], insertClauses, cteRelations) = + queryArgs match { + case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => + // check if has CTE + insertClauses.last match { + case Token("TOK_CTE", cteClauses) => + val cteRelations = cteClauses.map(node => { + val relation = nodeToRelation(node).asInstanceOf[Subquery] + (relation.alias, relation) + }).toMap + (Some(args.head), insertClauses.init, Some(cteRelations)) + + case _ => (Some(args.head), insertClauses, None) + } + + case Token("TOK_INSERT", _) :: Nil => (None, queryArgs, None) + } // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => @@ -794,7 +806,10 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } // If there are multiple INSERTS just UNION them together into on query. - queries.reduceLeft(Union) + val query = queries.reduceLeft(Union) + + // return With plan if there is CTE + cteRelations.map(With(query, _)).getOrElse(query) case Token("TOK_UNION", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right)) diff --git a/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 new file mode 100644 index 0000000000000..f6ba75da254ca --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 @@ -0,0 +1,3 @@ +5 +5 +5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 new file mode 100644 index 0000000000000..ca7b591095e28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 @@ -0,0 +1,4 @@ +val_4 +val_5 +val_5 +val_5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 new file mode 100644 index 0000000000000..b8626c4cff284 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 @@ -0,0 +1 @@ +4 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index af781a502e9f3..1222fbabd8b33 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -542,6 +542,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") + createQueryTest("CTE feature #1", + "with q1 as (select key from src) select * from q1 where key = 5") + + createQueryTest("CTE feature #2", + """with q1 as (select * from src where key= 5), + |q2 as (select * from src s2 where key = 4) + |select value from q1 union all select value from q2 + """.stripMargin) + + createQueryTest("CTE feature #3", + """with q1 as (select key from src) + |from q1 + |select * where key = 4 + """.stripMargin) + test("predicates contains an empty AttributeSet() references") { sql( """ From 1f39a61118184e136f38381a9f3ba0b2d5d589d9 Mon Sep 17 00:00:00 2001 From: lazymam500 Date: Sat, 11 Apr 2015 18:33:14 -0700 Subject: [PATCH 708/817] [Spark-5068][SQL]Fix bug query data when path doesn't exist for HiveContext This PR follow up PR #3907 & #3891 & #4356. According to marmbrus liancheng 's comments, I try to use fs.globStatus to retrieve all FileStatus objects under path(s), and then do the filtering locally. [1]. get pathPattern by path, and put it into pathPatternSet. (hdfs://cluster/user/demo/2016/08/12 -> hdfs://cluster/user/demo/*/*/*) [2]. retrieve all FileStatus objects ,and cache them by undating existPathSet. [3]. do the filtering locally [4]. if we have new pathPattern,do 1,2 step again. (external table maybe have more than one partition pathPattern) chenghao-intel jeanlyn Author: lazymam500 Author: lazyman Closes #5059 from lazyman500/SPARK-5068 and squashes the following commits: 5bfcbfd [lazyman] move spark.sql.hive.verifyPartitionPath to SQLConf,fix scala style e1d6386 [lazymam500] fix scala style f23133f [lazymam500] bug fix 47e0023 [lazymam500] fix scala style,add config flag,break the chaining 04c443c [lazyman] SPARK-5068: fix bug when partition path doesn't exists #2 41f60ce [lazymam500] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/sql/SQLConf.scala | 6 ++ .../apache/spark/sql/hive/TableReader.scala | 41 +++++++++++- .../spark/sql/hive/QueryPartitionSuite.scala | 64 +++++++++++++++++++ 3 files changed, 110 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4815620c6fe57..ee641bdfeb2d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -39,6 +39,8 @@ private[spark] object SQLConf { val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" val PARQUET_USE_DATA_SOURCE_API = "spark.sql.parquet.useDataSourceApi" + val HIVE_VERIFY_PARTITIONPATH = "spark.sql.hive.verifyPartitionPath" + val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" val BROADCAST_TIMEOUT = "spark.sql.broadcastTimeout" @@ -119,6 +121,10 @@ private[sql] class SQLConf extends Serializable { private[spark] def parquetUseDataSourceApi = getConf(PARQUET_USE_DATA_SOURCE_API, "true").toBoolean + /** When true uses verifyPartitionPath to prune the path which is not exists. */ + private[spark] def verifyPartitionPath = + getConf(HIVE_VERIFY_PARTITIONPATH, "true").toBoolean + /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 3563472c7ae81..d35291543c9f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -142,7 +142,46 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => + + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists + def verifyPartitionPath( + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): + Map[HivePartition, Class[_ <: Deserializer]] = { + if (!sc.conf.verifyPartitionPath) { + partitionToDeserializer + } else { + var existPathSet = collection.mutable.Set[String]() + var pathPatternSet = collection.mutable.Set[String]() + partitionToDeserializer.filter { + case (partition, partDeserializer) => + def updateExistPathSetByPathPattern(pathPatternStr: String) { + val pathPattern = new Path(pathPatternStr) + val fs = pathPattern.getFileSystem(sc.hiveconf) + val matches = fs.globStatus(pathPattern) + matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) + } + // convert /demo/data/year/month/day to /demo/data/*/*/*/ + def getPathPatternByPath(parNum: Int, tempPath: Path): String = { + var path = tempPath + for (i <- (1 to parNum)) path = path.getParent + val tails = (1 to parNum).map(_ => "*").mkString("/", "/", "/") + path.toString + tails + } + + val partPath = HiveShim.getDataLocationPath(partition) + val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size(); + var pathPatternStr = getPathPatternByPath(partNum, partPath) + if (!pathPatternSet.contains(pathPatternStr)) { + pathPatternSet += pathPatternStr + updateExistPathSetByPathPattern(pathPatternStr) + } + existPathSet.contains(partPath.toString) + } + } + } + + val hivePartitionRDDs = verifyPartitionPath(partitionToDeserializer) + .map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = HiveShim.getDataLocationPath(partition) val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala new file mode 100644 index 0000000000000..83f97128c5e83 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.File + +import com.google.common.io.Files +import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.util.Utils +/* Implicits */ +import org.apache.spark.sql.hive.test.TestHive._ + + + +class QueryPartitionSuite extends QueryTest { + import org.apache.spark.sql.hive.test.TestHive.implicits._ + + test("SPARK-5068: query data when path doesn't exists"){ + val testData = TestHive.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") + + val tmpDir = Files.createTempDir() + //create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') SELECT key,value FROM testData") + + //test for the exist path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) + + //delect the path of one partition + val folders = tmpDir.listFiles.filter(_.isDirectory) + Utils.deleteRecursively(folders(0)) + + //test for affter delete the path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect) + + sql("DROP TABLE table_with_partition") + sql("DROP TABLE createAndInsertTest") + } +} From 48cc840021c43fcb4c5bb365d2c80512678cf120 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Sat, 11 Apr 2015 18:34:17 -0700 Subject: [PATCH 709/817] [SPARK-6179][SQL] Add token for "SHOW PRINCIPALS role_name" and "SHOW TRANSACTIONS" and "SHOW COMPACTIONS" [SHOW PRINCIPALS role_name] Lists all roles and users who belong to this role. Only the admin role has privilege for this. [SHOW COMPACTIONS] It returns a list of all tables and partitions currently being compacted or scheduled for compaction when Hive transactions are being used. [SHOW TRANSACTIONS] It is for use by administrators when Hive transactions are being used. It returns a list of all currently open and aborted transactions in the system. Author: DoingDone9 <799203320@qq.com> Author: Zhongshuai Pei <799203320@qq.com> Author: Xu Tingjun Closes #4902 from DoingDone9/SHOW_PRINCIPALS and squashes the following commits: 4add42f [Zhongshuai Pei] for test 311f806 [Zhongshuai Pei] for test 0c7550a [DoingDone9] Update HiveQl.scala c8aeb1c [Xu Tingjun] aa 802261c [DoingDone9] Merge pull request #7 from apache/master d00303b [DoingDone9] Merge pull request #6 from apache/master 98b134f [DoingDone9] Merge pull request #5 from apache/master 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 2fb2e7c4a5370..b2ae74efeb097 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -113,13 +113,16 @@ private[hive] object HiveQl { "TOK_REVOKE", + "TOK_SHOW_COMPACTIONS", "TOK_SHOW_CREATETABLE", "TOK_SHOW_GRANT", "TOK_SHOW_ROLE_GRANT", + "TOK_SHOW_ROLE_PRINCIPALS", "TOK_SHOW_ROLES", "TOK_SHOW_SET_ROLE", "TOK_SHOW_TABLESTATUS", "TOK_SHOW_TBLPROPERTIES", + "TOK_SHOW_TRANSACTIONS", "TOK_SHOWCOLUMNS", "TOK_SHOWDATABASES", "TOK_SHOWFUNCTIONS", From 352a5da421d61379f2a8bcd7548ccc5d2647120a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Sat, 11 Apr 2015 18:41:12 -0700 Subject: [PATCH 710/817] [SPARK-6379][SQL] Support a functon to call user-defined functions registered in SQLContext This is useful for using pre-defined UDFs in SQLContext; val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") val sqlctx = df.sqlContext sqlctx.udf.register("simpleUdf", (v: Int) => v * v) df.select($"id", sqlctx.callUdf("simpleUdf", $"value")) Author: Takeshi YAMAMURO Closes #5061 from maropu/SupportUDFConversionInSparkContext and squashes the following commits: f858aff [Takeshi YAMAMURO] Move the function into functions.scala afd0380 [Takeshi YAMAMURO] Add a return type of callUDF 599b76c [Takeshi YAMAMURO] Remove the implicit conversion and add SqlContext#callUdf 8b56f10 [Takeshi YAMAMURO] Support an implicit conversion from udf"name" to an UDF defined in SQLContext --- .../org/apache/spark/sql/functions.scala | 21 ++++++++++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 9 ++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 111e751588a8b..ff91e1d74bc2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,7 +22,7 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -605,4 +605,23 @@ object functions { } // scalastyle:on + + /** + * Call an user-defined function. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + * val sqlContext = df.sqlContext + * sqlContext.udf.register("simpleUdf", (v: Int) => v * v) + * df.select($"id", callUdf("simpleUdf", $"value")) + * }}} + * + * @group udf_funcs + */ + def callUdf(udfName: String, cols: Column*): Column = { + UnresolvedFunction(udfName, cols.map(_.expr)) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f5df8c6a59f10..b26e22f6229fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -440,6 +440,15 @@ class DataFrameSuite extends QueryTest { ) } + test("call udf in SQLContext") { + val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + val sqlctx = df.sqlContext + sqlctx.udf.register("simpleUdf", (v: Int) => v * v) + checkAnswer( + df.select($"id", callUdf("simpleUdf", $"value")), + Row("id1", 1) :: Row("id2", 16) :: Row("id3", 25) :: Nil) + } + test("withColumn") { val df = testData.toDF().withColumn("newCol", col("key") + 1) checkAnswer( From d2383fb5ffafd6b3a56b1ee6e0e035594473e2c8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 11 Apr 2015 18:44:54 -0700 Subject: [PATCH 711/817] [SQL] Handle special characters in the authority of a Path's URI. Author: Yin Huai Closes #5381 from yhuai/parquetPath2 and squashes the following commits: fe296b4 [Yin Huai] Create new Path to take care special characters in the authority of a Path's URI. --- .../apache/spark/sql/parquet/newParquet.scala | 30 +++++++++++++++++-- 1 file changed, 27 insertions(+), 3 deletions(-) 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 0dce3623a66df..20fdf5e58ef82 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 @@ -432,7 +432,10 @@ private[sql] case class ParquetRelation2( // FileInputFormat cannot handle empty lists. if (selectedFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, selectedFiles.map(_.getPath): _*) + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val selectedPaths = selectedFiles.map(status => new Path(status.getPath.toUri.toString)) + FileInputFormat.setInputPaths(job, selectedPaths: _*) } // Try to push down filters when filter push-down is enabled. @@ -484,10 +487,31 @@ private[sql] case class ParquetRelation2( val cacheMetadata = useCache @transient - val cachedStatus = selectedFiles + val cachedStatus = selectedFiles.map { st => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val newPath = new Path(st.getPath.toUri.toString) + + new FileStatus( + st.getLen, + st.isDir, + st.getReplication, + st.getBlockSize, + st.getModificationTime, + st.getAccessTime, + st.getPermission, + st.getOwner, + st.getGroup, + newPath) + } @transient - val cachedFooters = selectedFooters + val cachedFooters = selectedFooters.map { f => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + new Footer(new Path(f.getFile.toUri.toString), f.getParquetMetadata) + } + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { From 6d4e854ffbd7dee9a3cd7b44a00fd9c0e551f5b8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 11 Apr 2015 19:26:15 -0700 Subject: [PATCH 712/817] [SPARK-6367][SQL] Use the proper data type for those expressions that are hijacking existing data types. This PR adds internal UDTs for expressions that are hijacking existing data types. The following UDTs are added: * `HyperLogLogUDT` (`BinaryType` as the SQL type) for `ApproxCountDistinctPartition` * `OpenHashSetUDT` (`ArrayType` as the SQL type) for `CollectHashSet`, `NewSet`, `AddItemToSet`, and `CombineSets`. I am also adding more unit tests for aggregation with code gen enabled. JIRA: https://issues.apache.org/jira/browse/SPARK-6367 Author: Yin Huai Closes #5094 from yhuai/expressionType and squashes the following commits: 8bcd11a [Yin Huai] Return types. 61a1d66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType e8b4599 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType 2753156 [Yin Huai] Ignore aggregations having sum functions for now. b5eb259 [Yin Huai] Case object for HyperLogLog type. 00ebdbd [Yin Huai] deserialize/serialize. 54b87ae [Yin Huai] Add UDTs for expressions that return HyperLogLog and OpenHashSet. --- .../sql/catalyst/expressions/aggregates.scala | 24 +++++++++++-- .../expressions/codegen/CodeGenerator.scala | 4 +-- .../spark/sql/catalyst/expressions/sets.scala | 35 ++++++++++++++++--- .../sql/execution/GeneratedAggregate.scala | 12 ++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 12 ++++--- .../spark/sql/UserDefinedTypeSuite.scala | 24 ++++++++++++- 6 files changed, 91 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 406de38d1c483..14a855054b94d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -189,9 +189,10 @@ case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpress override def children: Seq[Expression] = expressions override def nullable: Boolean = false - override def dataType: ArrayType = ArrayType(expressions.head.dataType) + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(expressions.head.dataType) override def toString: String = s"AddToHashSet(${expressions.mkString(",")})" - override def newInstance(): CollectHashSetFunction = new CollectHashSetFunction(expressions, this) + override def newInstance(): CollectHashSetFunction = + new CollectHashSetFunction(expressions, this) } case class CollectHashSetFunction( @@ -250,11 +251,28 @@ case class CombineSetsAndCountFunction( override def eval(input: Row): Any = seen.size.toLong } +/** The data type of ApproxCountDistinctPartition since its output is a HyperLogLog object. */ +private[sql] case object HyperLogLogUDT extends UserDefinedType[HyperLogLog] { + + override def sqlType: DataType = BinaryType + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def serialize(obj: Any): Array[Byte] = + obj.asInstanceOf[HyperLogLog].getBytes + + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def deserialize(datum: Any): HyperLogLog = + HyperLogLog.Builder.build(datum.asInstanceOf[Array[Byte]]) + + override def userClass: Class[HyperLogLog] = classOf[HyperLogLog] +} + case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { override def nullable: Boolean = false - override def dataType: DataType = child.dataType + override def dataType: DataType = HyperLogLogUDT override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" override def newInstance(): ApproxCountDistinctPartitionFunction = { new ApproxCountDistinctPartitionFunction(child, this, relativeSD) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index d1abf3c0b64a5..aac56e1568332 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -464,7 +464,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val itemEval = expressionEvaluator(item) val setEval = expressionEvaluator(set) - val ArrayType(elementType, _) = set.dataType + val elementType = set.dataType.asInstanceOf[OpenHashSetUDT].elementType itemEval.code ++ setEval.code ++ q""" @@ -482,7 +482,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val leftEval = expressionEvaluator(left) val rightEval = expressionEvaluator(right) - val ArrayType(elementType, _) = left.dataType + val elementType = left.dataType.asInstanceOf[OpenHashSetUDT].elementType leftEval.code ++ rightEval.code ++ q""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index 35faa00782e80..4c44182278207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -20,6 +20,33 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet +/** The data type for expressions returning an OpenHashSet as the result. */ +private[sql] class OpenHashSetUDT( + val elementType: DataType) extends UserDefinedType[OpenHashSet[Any]] { + + override def sqlType: DataType = ArrayType(elementType) + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def serialize(obj: Any): Seq[Any] = { + obj.asInstanceOf[OpenHashSet[Any]].iterator.toSeq + } + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def deserialize(datum: Any): OpenHashSet[Any] = { + val iterator = datum.asInstanceOf[Seq[Any]].iterator + val set = new OpenHashSet[Any] + while(iterator.hasNext) { + set.add(iterator.next()) + } + + set + } + + override def userClass: Class[OpenHashSet[Any]] = classOf[OpenHashSet[Any]] + + private[spark] override def asNullable: OpenHashSetUDT = this +} + /** * Creates a new set of the specified type */ @@ -28,9 +55,7 @@ case class NewSet(elementType: DataType) extends LeafExpression { override def nullable: Boolean = false - // We are currently only using these Expressions internally for aggregation. However, if we ever - // expose these to users we'll want to create a proper type instead of hijacking ArrayType. - override def dataType: DataType = ArrayType(elementType) + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(elementType) override def eval(input: Row): Any = { new OpenHashSet[Any]() @@ -50,7 +75,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { override def nullable: Boolean = set.nullable - override def dataType: DataType = set.dataType + override def dataType: OpenHashSetUDT = set.dataType.asInstanceOf[OpenHashSetUDT] override def eval(input: Row): Any = { val itemEval = item.eval(input) @@ -80,7 +105,7 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres override def nullable: Boolean = left.nullable || right.nullable - override def dataType: DataType = left.dataType + override def dataType: OpenHashSetUDT = left.dataType.asInstanceOf[OpenHashSetUDT] override def symbol: String = "++=" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 861a2c21ad9a0..3c58e93b45e9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -68,6 +68,8 @@ case class GeneratedAggregate( a.collect { case agg: AggregateExpression => agg} } + // If you add any new function support, please add tests in org.apache.spark.sql.SQLQuerySuite + // (in test "aggregation with codegen"). val computeFunctions = aggregatesToCompute.map { case c @ Count(expr) => // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its @@ -208,7 +210,8 @@ case class GeneratedAggregate( currentMax) case CollectHashSet(Seq(expr)) => - val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val set = + AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() val initialValue = NewSet(expr.dataType) val addToSet = AddItemToSet(expr, set) @@ -219,9 +222,10 @@ case class GeneratedAggregate( set) case CombineSetsAndCount(inputSet) => - val ArrayType(inputType, _) = inputSet.dataType - val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() - val initialValue = NewSet(inputType) + val elementType = inputSet.dataType.asInstanceOf[OpenHashSetUDT].elementType + val set = + AttributeReference("hashSet", new OpenHashSetUDT(elementType), nullable = false)() + val initialValue = NewSet(elementType) val collectSets = CombineSets(set, inputSet) AggregateEvaluation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index fb8fc6dbd1e1e..5e453e05e2ac7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll @@ -151,10 +152,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { "SELECT count(distinct key) FROM testData3x", Row(100) :: Nil) // SUM - testCodeGen( - "SELECT value, sum(key) FROM testData3x GROUP BY value", - (1 to 100).map(i => Row(i.toString, 3 * i))) - testCodeGen( + testCodeGen( + "SELECT value, sum(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 3 * i))) + testCodeGen( "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x", Row(5050 * 3, 5050 * 3.0) :: Nil) // AVERAGE @@ -192,10 +193,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { testCodeGen( "SELECT sum('a'), avg('a'), count(null) FROM testData", Row(0, null, 0) :: Nil) - + dropTempTable("testData3x") setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } + test("Add Parser of SQL COALESCE()") { checkAnswer( sql("""SELECT COALESCE(1, 2)"""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 902da5c3ba6d6..2672e20deadc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -23,13 +23,16 @@ import org.apache.spark.util.Utils import scala.beans.{BeanInfo, BeanProperty} +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{OpenHashSetUDT, HyperLogLogUDT} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ - +import org.apache.spark.util.collection.OpenHashSet @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { @@ -119,4 +122,23 @@ class UserDefinedTypeSuite extends QueryTest { df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) } + + test("HyperLogLogUDT") { + val hyperLogLogUDT = HyperLogLogUDT + val hyperLogLog = new HyperLogLog(0.4) + (1 to 10).foreach(i => hyperLogLog.offer(Row(i))) + + val actual = hyperLogLogUDT.deserialize(hyperLogLogUDT.serialize(hyperLogLog)) + assert(actual.cardinality() === hyperLogLog.cardinality()) + assert(java.util.Arrays.equals(actual.getBytes, hyperLogLog.getBytes)) + } + + test("OpenHashSetUDT") { + val openHashSetUDT = new OpenHashSetUDT(IntegerType) + val set = new OpenHashSet[Int] + (1 to 10).foreach(i => set.add(i)) + + val actual = openHashSetUDT.deserialize(openHashSetUDT.serialize(set)) + assert(actual.iterator.toSet === set.iterator.toSet) + } } From 5c2844c51aca6a0da9251a3fd346a6f872cf17f8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 11 Apr 2015 19:35:56 -0700 Subject: [PATCH 713/817] [SQL][minor] move `resolveGetField` into a object The method `resolveGetField` isn't belong to `LogicalPlan` logically and didn't access any members of it. Author: Wenchen Fan Closes #5435 from cloud-fan/tmp and squashes the following commits: 9a66c83 [Wenchen Fan] code clean up --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../catalyst/expressions/complexTypes.scala | 37 +++++++++++++++++++ .../catalyst/plans/logical/LogicalPlan.scala | 35 +----------------- 3 files changed, 39 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b83f18abdd239..fd1ceb1f77931 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -308,7 +308,7 @@ class Analyzer( logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => - q.resolveGetField(child, fieldName, resolver) + GetField(child, fieldName, resolver) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 3b2b9211268a9..fc1f69655963d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types._ /** @@ -81,6 +83,41 @@ trait GetField extends UnaryExpression { def field: StructField } +object GetField { + /** + * Returns the resolved `GetField`, and report error if no desired field or over one + * desired fields are found. + */ + def apply( + expr: Expression, + fieldName: String, + resolver: Resolver): GetField = { + def findField(fields: Array[StructField]): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + throw new AnalysisException( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } + expr.dataType match { + case StructType(fields) => + val ordinal = findField(fields) + StructGetField(expr, fields(ordinal), ordinal) + case ArrayType(StructType(fields), containsNull) => + val ordinal = findField(fields) + ArrayGetField(expr, fields(ordinal), ordinal, containsNull) + case otherType => + throw new AnalysisException(s"GetField is not valid on fields of type $otherType") + } + } +} + /** * Returns the value of fields in the Struct `child`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index d8f5858f5033e..579a0fb8d3f93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -210,7 +210,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // For example, consider "a.b.c", where "a" is resolved to an existing attribute. // Then this will add GetField("c", GetField("b", a)), and alias // the final expression as "c". - val fieldExprs = nestedFields.foldLeft(a: Expression)(resolveGetField(_, _, resolver)) + val fieldExprs = nestedFields.foldLeft(a: Expression)(GetField(_, _, resolver)) val aliasName = nestedFields.last Some(Alias(fieldExprs, aliasName)()) } catch { @@ -229,39 +229,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { s"Reference '$name' is ambiguous, could be: $referenceNames.") } } - - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - def resolveGetField( - expr: Expression, - fieldName: String, - resolver: Resolver): Expression = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } } /** From dea5dacc5d701c3dfe433360b2e17e50edec31e8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 11 Apr 2015 20:12:40 -0700 Subject: [PATCH 714/817] [HOTFIX] Add explicit return types to fix lint errors --- .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fd1ceb1f77931..50702ac6832ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -169,7 +169,7 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]) = { + def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { try { // In hive, if there is same table name in database and CTE definition, // hive will use the table in database, not the CTE one. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3bd5aa5964221..5d31a6eecfce2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -156,7 +156,7 @@ case class CreateTableAsSelect[T]( * value is the CTE definition. */ case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class WriteToFile( From 1205f7ea6165089985edf46ea4d2d53975d0f1f4 Mon Sep 17 00:00:00 2001 From: Michael Malak Date: Sat, 11 Apr 2015 21:01:23 -0700 Subject: [PATCH 715/817] SPARK-6710 GraphX Fixed Wrong initial bias in GraphX SVDPlusPlus Author: Michael Malak Closes #5464 from michaelmalak/master and squashes the following commits: 9d942ba [Michael Malak] SPARK-6710 GraphX Fixed Wrong initial bias in GraphX SVDPlusPlus --- .../main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 1a7178b82e3af..3b0e1628d86b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -93,7 +93,7 @@ object SVDPlusPlus { val gJoinT0 = g.outerJoinVertices(t0) { (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => - (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + (vd._1, vd._2, msg.get._2 / msg.get._1 - u, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() materialize(gJoinT0) g.unpersist() From 0cc8fcb4cd20cb90a1fac50b1a3ffed833ce5eac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 Apr 2015 22:12:56 -0700 Subject: [PATCH 716/817] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #4994 (close requested by 'marmbrus') Closes #4995 (close requested by 'marmbrus') Closes #4491 (close requested by 'srowen') Closes #3597 (close requested by 'srowen') Closes #4693 (close requested by 'marmbrus') Closes #3855 (close requested by 'marmbrus') Closes #4398 (close requested by 'marmbrus') Closes #4246 (close requested by 'marmbrus') Closes #5153 (close requested by 'srowen') Closes #3626 (close requested by 'srowen') Closes #5166 (close requested by 'marmbrus') Closes #5040 (close requested by 'marmbrus') Closes #5044 (close requested by 'marmbrus') Closes #5440 (close requested by 'JoshRosen') Closes #4039 (close requested by 'marmbrus') Closes #1237 (close requested by 'srowen') Closes #216 (close requested by 'mengxr') Closes #5092 (close requested by 'srowen') Closes #5100 (close requested by 'marmbrus') Closes #4469 (close requested by 'marmbrus') Closes #5246 (close requested by 'srowen') Closes #5013 (close requested by 'marmbrus') From 5d8f7b9e87e8066d54717a1a78b06e8531d8b0d4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 11 Apr 2015 22:33:23 -0700 Subject: [PATCH 717/817] [SPARK-6677] [SQL] [PySpark] fix cached classes It's possible to have two DataType object with same id (memory address) at different time, we should check the cached classes to verify that it's generated by given datatype. This PR also change `__FIELDS__` and `__DATATYPE__` to lower case to match Python code style. Author: Davies Liu Closes #5445 from davies/fix_type_cache and squashes the following commits: 63b3238 [Davies Liu] typo 47bdede [Davies Liu] fix cached classes --- python/pyspark/sql/types.py | 39 +++++++++++++++++++------------------ 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 7e0124b13671b..ef76d84c00481 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -567,8 +567,8 @@ def _infer_schema(row): elif isinstance(row, (tuple, list)): if hasattr(row, "_fields"): # namedtuple items = zip(row._fields, tuple(row)) - elif hasattr(row, "__FIELDS__"): # Row - items = zip(row.__FIELDS__, tuple(row)) + elif hasattr(row, "__fields__"): # Row + items = zip(row.__fields__, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -647,7 +647,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): + if hasattr(obj, "_fields") or hasattr(obj, "__fields__"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -997,12 +997,13 @@ def _restore_object(dataType, obj): # same object in most cases. k = id(dataType) cls = _cached_cls.get(k) - if cls is None: + if cls is None or cls.__datatype is not dataType: # use dataType as key to avoid create multiple class cls = _cached_cls.get(dataType) if cls is None: cls = _create_cls(dataType) _cached_cls[dataType] = cls + cls.__datatype = dataType _cached_cls[k] = cls return cls(obj) @@ -1119,8 +1120,8 @@ def Dict(d): class Row(tuple): """ Row in DataFrame """ - __DATATYPE__ = dataType - __FIELDS__ = tuple(f.name for f in dataType.fields) + __datatype = dataType + __fields__ = tuple(f.name for f in dataType.fields) __slots__ = () # create property for fast access @@ -1128,22 +1129,22 @@ class Row(tuple): def asDict(self): """ Return as a dict """ - return dict((n, getattr(self, n)) for n in self.__FIELDS__) + return dict((n, getattr(self, n)) for n in self.__fields__) def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) - for n in self.__FIELDS__)) + for n in self.__fields__)) def __reduce__(self): - return (_restore_object, (self.__DATATYPE__, tuple(self))) + return (_restore_object, (self.__datatype, tuple(self))) return Row def _create_row(fields, values): row = Row(*values) - row.__FIELDS__ = fields + row.__fields__ = fields return row @@ -1183,7 +1184,7 @@ def __new__(self, *args, **kwargs): # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) - row.__FIELDS__ = names + row.__fields__ = names return row else: @@ -1193,11 +1194,11 @@ def asDict(self): """ Return as an dict """ - if not hasattr(self, "__FIELDS__"): + if not hasattr(self, "__fields__"): raise TypeError("Cannot convert a Row class into dict") - return dict(zip(self.__FIELDS__, self)) + return dict(zip(self.__fields__, self)) - # let obect acs like class + # let object acts like class def __call__(self, *args): """create new Row object""" return _create_row(self, args) @@ -1208,21 +1209,21 @@ def __getattr__(self, item): try: # it will be slow when it has many fields, # but this will not be used in normal cases - idx = self.__FIELDS__.index(item) + idx = self.__fields__.index(item) return self[idx] except IndexError: raise AttributeError(item) def __reduce__(self): - if hasattr(self, "__FIELDS__"): - return (_create_row, (self.__FIELDS__, tuple(self))) + if hasattr(self, "__fields__"): + return (_create_row, (self.__fields__, tuple(self))) else: return tuple.__reduce__(self) def __repr__(self): - if hasattr(self, "__FIELDS__"): + if hasattr(self, "__fields__"): return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) + for k, v in zip(self.__fields__, tuple(self))) else: return "" % ", ".join(self) From e9445b187e8f5c3703771b775e60164166309570 Mon Sep 17 00:00:00 2001 From: "Guancheng (G.C.) Chen" Date: Sun, 12 Apr 2015 11:36:41 +0100 Subject: [PATCH 718/817] [SPARK-6866][Build] Remove duplicated dependency in launcher/pom.xml JIRA: https://issues.apache.org/jira/browse/SPARK-6866 Remove duplicated dependency of scalatest in launcher/pom.xml since it already inherited the dependency from the parent pom.xml. Author: Guancheng (G.C.) Chen Closes #5476 from gchen/SPARK-6866 and squashes the following commits: 1ab484b [Guancheng (G.C.) Chen] remove duplicated dependency in launcher/pom.xml --- launcher/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 0fe2814135d88..182e5f60218db 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -52,11 +52,6 @@ mockito-all test - - org.scalatest - scalatest_${scala.binary.version} - test - org.slf4j slf4j-api From ddc17431a4108ab6efe0cd329d69e1f2fca5ac12 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Sun, 12 Apr 2015 13:41:44 +0100 Subject: [PATCH 719/817] [SPARK-6843][core]Add volatile for the "state" Fix potential visibility problem for the "state" of Executor The field of "state" is shared and modified by multiple threads. i.e: ```scala Within ExecutorRunner.scala (1) workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } workerThread.start() // Shutdown hook that kills actors on shutdown. (2)shutdownHook = new Thread() { override def run() { killProcess(Some("Worker shutting down")) } } (3)and also the "Actor thread" for worker. ``` I think we should at lease add volatile to ensure the visibility among threads otherwise the worker might send an out-of-date status to the master. https://issues.apache.org/jira/browse/SPARK-6843 Author: lisurprise Closes #5448 from zhichao-li/state and squashes the following commits: a2386e7 [lisurprise] add volatile for state field --- .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 83e24a7a1f80c..7d5acabb95a48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -50,7 +50,7 @@ private[deploy] class ExecutorRunner( val workerUrl: String, conf: SparkConf, val appLocalDirs: Seq[String], - var state: ExecutorState.Value) + @volatile var state: ExecutorState.Value) extends Logging { private val fullId = appId + "/" + execId From 6ac8eea2fc6b782015236e4e7106e59d0d9e1b38 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 12 Apr 2015 17:37:30 +0100 Subject: [PATCH 720/817] [SPARK-6431][Streaming][Kafka] Error message for partition metadata requ... ...ests The original reported problem was misdiagnosed; the topic just didn't exist yet. Agreed upon solution was to improve error handling / message Author: cody koeninger Closes #5454 from koeninger/spark-6431-master and squashes the following commits: 44300f8 [cody koeninger] [SPARK-6431][Streaming][Kafka] Error message for partition metadata requests --- .../spark/streaming/kafka/KafkaCluster.scala | 14 +++++++++++--- .../spark/streaming/kafka/KafkaClusterSuite.scala | 3 +++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 2f7e0ab39fefd..bd767031c1849 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -123,9 +123,17 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { val errs = new Err withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => val resp: TopicMetadataResponse = consumer.send(req) - // error codes here indicate missing / just created topic, - // repeating on a different broker wont be useful - return Right(resp.topicsMetadata.toSet) + val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) + + if (respErrs.isEmpty) { + return Right(resp.topicsMetadata.toSet) + } else { + respErrs.foreach { m => + val cause = ErrorMapping.exceptionFor(m.errorCode) + val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" + errs.append(new SparkException(msg, cause)) + } + } } Left(errs) } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index 2b33d2a220b2b..7fb841b79cb65 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -52,6 +52,9 @@ class KafkaClusterSuite extends FunSuite with BeforeAndAfterAll { val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") + + val err = kc.getPartitions(Set(topic + "BAD")) + assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") } test("leader offset apis") { From 04bcd67cfc50f847559a9ff59a31aa93028b3628 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Sun, 12 Apr 2015 18:58:53 +0100 Subject: [PATCH 721/817] [MINOR] a typo: coalesce Author: Daoyuan Wang Closes #5482 from adrian-wang/typo and squashes the following commits: e65ef6f [Daoyuan Wang] typo --- .../org/apache/spark/sql/execution/GeneratedAggregate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 3c58e93b45e9d..95176e425132d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -97,7 +97,7 @@ case class GeneratedAggregate( val currentSum = AttributeReference("currentSum", calcType, nullable = true)() val initialValue = Literal.create(null, calcType) - // Coalasce avoids double calculation... + // Coalesce avoids double calculation... // but really, common sub expression elimination would be better.... val zero = Cast(Literal(0), calcType) val updateFunction = Coalesce( From a1fe59dae50f551d02dd18676308eca054ff6b07 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 12 Apr 2015 20:50:49 -0700 Subject: [PATCH 722/817] [SPARK-6765] Fix test code style for core. Author: Reynold Xin Closes #5484 from rxin/test-style-core and squashes the following commits: e0b0100 [Reynold Xin] [SPARK-6765] Fix test code style for core. --- .../org/apache/spark/AccumulatorSuite.scala | 30 ++--- .../org/apache/spark/CacheManagerSuite.scala | 7 +- .../org/apache/spark/CheckpointSuite.scala | 15 ++- .../apache/spark/ContextCleanerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 26 +++-- .../apache/spark/ImplicitOrderingSuite.scala | 6 +- .../apache/spark/JobCancellationSuite.scala | 2 +- .../org/apache/spark/LocalSparkContext.scala | 4 +- .../org/apache/spark/PartitioningSuite.scala | 30 ++--- .../org/apache/spark/SSLOptionsSuite.scala | 15 ++- .../org/apache/spark/SSLSampleConfigs.scala | 7 +- .../scala/org/apache/spark/ShuffleSuite.scala | 8 +- .../org/apache/spark/SparkContextSuite.scala | 6 +- .../org/apache/spark/StatusTrackerSuite.scala | 5 +- .../spark/broadcast/BroadcastSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 6 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 2 +- .../deploy/history/HistoryServerSuite.scala | 4 +- .../rest/StandaloneRestSubmitSuite.scala | 4 +- .../deploy/rest/SubmitRestProtocolSuite.scala | 3 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- .../deploy/worker/WorkerArgumentsTest.scala | 4 +- .../spark/deploy/worker/WorkerSuite.scala | 6 +- .../metrics/InputOutputMetricsSuite.scala | 2 +- .../spark/metrics/MetricsConfigSuite.scala | 15 ++- .../org/apache/spark/rdd/JdbcRDDSuite.scala | 4 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 41 +++---- .../rdd/ParallelCollectionSplitSuite.scala | 20 ++-- .../spark/rdd/PartitionPruningRDDSuite.scala | 7 +- .../rdd/PartitionwiseSampledRDDSuite.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 22 ++-- .../org/apache/spark/rdd/RDDSuiteUtils.scala | 4 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 40 +++---- .../spark/scheduler/DAGSchedulerSuite.scala | 103 ++++++++++-------- .../scheduler/EventLoggingListenerSuite.scala | 4 +- .../scheduler/NotSerializableFakeTask.scala | 4 +- .../spark/scheduler/SparkListenerSuite.scala | 21 ++-- .../scheduler/TaskSchedulerImplSuite.scala | 10 +- .../spark/scheduler/TaskSetManagerSuite.scala | 26 +++-- .../mesos/MesosSchedulerBackendSuite.scala | 14 ++- .../serializer/KryoSerializerSuite.scala | 15 ++- .../ProactiveClosureSerializationSuite.scala | 8 +- .../spark/serializer/TestSerializer.scala | 5 +- .../hash/HashShuffleManagerSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 32 +++--- .../apache/spark/storage/LocalDirsSuite.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- .../spark/ui/storage/StorageTabSuite.scala | 3 +- .../spark/util/ClosureCleanerSuite.scala | 6 +- .../apache/spark/util/FileAppenderSuite.scala | 13 ++- .../apache/spark/util/NextIteratorSuite.scala | 2 +- .../spark/util/SizeEstimatorSuite.scala | 6 +- .../spark/util/TimeStampedHashMapSuite.scala | 4 +- .../org/apache/spark/util/UtilsSuite.scala | 5 +- .../org/apache/spark/util/VectorSuite.scala | 2 +- .../ExternalAppendOnlyMapSuite.scala | 2 +- .../util/collection/ExternalSorterSuite.scala | 35 +++--- .../util/random/XORShiftRandomSuite.scala | 2 +- 59 files changed, 386 insertions(+), 304 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index bd0f8bdefa171..75399461f2a5f 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -27,19 +27,20 @@ import org.scalatest.Matchers class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { - implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { - def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[A]) : mutable.Set[A] = { - new mutable.HashSet[A]() + implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = + new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } } - } test ("basic accumulation"){ sc = new SparkContext("local", "test") @@ -49,11 +50,10 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { d.foreach{x => acc += x} acc.value should be (210) - - val longAcc = sc.accumulator(0l) + val longAcc = sc.accumulator(0L) val maxInt = Integer.MAX_VALUE.toLong d.foreach{x => longAcc += maxInt + x} - longAcc.value should be (210l + maxInt * 20) + longAcc.value should be (210L + maxInt * 20) } test ("value not assignable from tasks") { diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 4b25c200a695a..70529d9216591 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -45,16 +45,17 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(split) override val getDependencies = List[Dependency[_]]() - override def compute(split: Partition, context: TaskContext) = Array(1, 2, 3, 4).iterator + override def compute(split: Partition, context: TaskContext): Iterator[Int] = + Array(1, 2, 3, 4).iterator } rdd2 = new RDD[Int](sc, List(new OneToOneDependency(rdd))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() rdd3 = new RDD[Int](sc, List(new OneToOneDependency(rdd2))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() } diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 32abc65385267..e1faddeabec79 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -75,7 +75,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result) assert(parCollection.dependencies != Nil) assert(parCollection.partitions.length === numPartitions) - assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList) + assert(parCollection.partitions.toList === + parCollection.checkpointData.get.getPartitions.toList) assert(parCollection.collect() === result) } @@ -102,13 +103,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("UnionRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(_.union(otherRDD)) testRDDPartitions(_.union(otherRDD)) } test("CartesianRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(new CartesianRDD(sc, _, otherRDD)) testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) @@ -223,7 +224,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val partitionAfterCheckpoint = serializeDeserialize( unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition]) assert( - partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass, + partitionBeforeCheckpoint.parents.head.getClass != + partitionAfterCheckpoint.parents.head.getClass, "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed" ) } @@ -358,7 +360,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * Generate an pair RDD (with partitioner) such that both the RDD and its partitions * have large size. */ - def generateFatPairRDD() = { + def generateFatPairRDD(): RDD[(Int, Int)] = { new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) } @@ -445,7 +447,8 @@ class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like // the PairRDDFunctions.cogroup() - def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { + def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) + : RDD[(K, Array[Iterable[V]])] = { new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index cdfaacee7da40..1de169d964d23 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -64,7 +64,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha } } - //------ Helper functions ------ + // ------ Helper functions ------ protected def newRDD() = sc.makeRDD(1 to 10) protected def newPairRDD() = newRDD().map(_ -> 1) @@ -370,7 +370,7 @@ class CleanerTester( val cleanerListener = new CleanerListener { def rddCleaned(rddId: Int): Unit = { toBeCleanedRDDIds -= rddId - logInfo("RDD "+ rddId + " cleaned") + logInfo("RDD " + rddId + " cleaned") } def shuffleCleaned(shuffleId: Int): Unit = { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 7acd27c735727..c8f08eed47c76 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -222,7 +222,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) val output = - sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) + sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } @@ -451,7 +451,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } @@ -459,8 +460,10 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) @@ -471,16 +474,20 @@ class FileSuite extends FunSuite with LocalSparkContext { val sf = new SparkConf() sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") sc = new SparkContext(sf) - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) @@ -492,7 +499,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index d895230ecf330..51348c039b5c9 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -51,7 +51,7 @@ private object ImplicitOrderingSuite { override def compare(o: OrderedClass): Int = ??? } - def basicMapExpectations(rdd: RDD[Int]) = { + def basicMapExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.map(x => (x, x)).keyOrdering.isDefined, "rdd.map(x => (x, x)).keyOrdering.isDefined"), (rdd.map(x => (1, x)).keyOrdering.isDefined, @@ -68,7 +68,7 @@ private object ImplicitOrderingSuite { "rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined")) } - def otherRDDMethodExpectations(rdd: RDD[Int]) = { + def otherRDDMethodExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.groupBy(x => x).keyOrdering.isDefined, "rdd.groupBy(x => x).keyOrdering.isDefined"), (rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty, @@ -82,4 +82,4 @@ private object ImplicitOrderingSuite { (rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined, "rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined")) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 21487bc24d58a..4d3e09793faff 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -188,7 +188,7 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter val rdd = sc.parallelize(1 to 10, 2).map { i => JobCancellationSuite.twoJobsSharingStageSemaphore.acquire() (i, i) - }.reduceByKey(_+_) + }.reduceByKey(_ + _) val f1 = rdd.collectAsync() val f2 = rdd.countAsync() diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 53e367a61715b..8bf2e55defd02 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -37,7 +37,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self super.afterEach() } - def resetSparkContext() = { + def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) sc = null } @@ -54,7 +54,7 @@ object LocalSparkContext { } /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ - def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { + def withSpark[T](sc: SparkContext)(f: SparkContext => T): T = { try { f(sc) } finally { diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index b7532314ada01..47e3bf6e1ac41 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -92,7 +92,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet test("RangePartitioner for keys that are not Comparable (but with Ordering)") { // Row does not extend Comparable, but has an implicit Ordering defined. implicit object RowOrdering extends Ordering[Row] { - override def compare(x: Row, y: Row) = x.value - y.value + override def compare(x: Row, y: Row): Int = x.value - y.value } val rdd = sc.parallelize(1 to 4500).map(x => (Row(x), Row(x))) @@ -212,20 +212,24 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet val arrPairs: RDD[(Array[Int], Int)] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => (Array(x), x)) - assert(intercept[SparkException]{ arrs.distinct() }.getMessage.contains("array")) + def verify(testFun: => Unit): Unit = { + intercept[SparkException](testFun).getMessage.contains("array") + } + + verify(arrs.distinct()) // We can't catch all usages of arrays, since they might occur inside other collections: // assert(fails { arrPairs.distinct() }) - assert(intercept[SparkException]{ arrPairs.partitionBy(new HashPartitioner(2)) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.rightOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.fullOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.groupByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKeyApprox(1) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.cogroup(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) + verify(arrPairs.partitionBy(new HashPartitioner(2))) + verify(arrPairs.join(arrPairs)) + verify(arrPairs.leftOuterJoin(arrPairs)) + verify(arrPairs.rightOuterJoin(arrPairs)) + verify(arrPairs.fullOuterJoin(arrPairs)) + verify(arrPairs.groupByKey()) + verify(arrPairs.countByKey()) + verify(arrPairs.countByKeyApprox(1)) + verify(arrPairs.cogroup(arrPairs)) + verify(arrPairs.reduceByKeyLocally(_ + _)) + verify(arrPairs.reduceByKey(_ + _)) } test("zero-length partitions should be correctly handled") { diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 444a33371bd71..93f46ef11c0e2 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -36,7 +36,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val opts = SSLOptions.parse(conf, "spark.ssl") @@ -52,7 +53,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test resolving property with defaults specified ") { @@ -66,7 +68,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) @@ -83,7 +86,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test whether defaults can be overridden ") { @@ -99,7 +103,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") conf.set("spark.ssl.protocol", "SSLv3") diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala index ace8123a8961f..308b9ea17708d 100644 --- a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -21,10 +21,11 @@ import java.io.File object SSLSampleConfigs { val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath - val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath + val untrustedKeyStorePath = new File( + this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath - def sparkSSLConfig() = { + def sparkSSLConfig(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", keyStorePath) @@ -38,7 +39,7 @@ object SSLSampleConfigs { conf } - def sparkSSLConfigUntrusted() = { + def sparkSSLConfigUntrusted(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", untrustedKeyStorePath) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 30b6184c77839..d7180516029d5 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -142,7 +142,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("shuffle on mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new ShuffledRDD[Int, Int, Int](pairs, @@ -155,7 +155,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // This is not in SortingSuite because of the local cluster setup. // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) @@ -169,7 +169,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("cogroup using mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) @@ -196,7 +196,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("subtract mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index c7301a30d8b11..94be1c6d6397c 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -114,11 +114,13 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { if (length1 != gotten1.length()) { throw new SparkException( - s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + s"file has different length $length1 than added file ${gotten1.length()} : " + + absolutePath1) } if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) + s"file has different length $length2 than added file ${gotten2.length()} : " + + absolutePath2) } if (absolutePath1 == gotten1.getAbsolutePath) { diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 41d6ea29d5b06..084eb237d70d1 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -82,7 +82,8 @@ class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext { secondJobFuture.jobIds.head } eventually(timeout(10 seconds)) { - sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId)) + sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be ( + Set(firstJobId, secondJobId)) } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index af3272692d7a1..c8fdfa693912e 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -33,7 +33,7 @@ class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { val broadcast = rdd.context.broadcast(list) val bid = broadcast.id - def doSomething() = { + def doSomething(): Set[(Int, Boolean)] = { rdd.map { x => val bm = SparkEnv.get.blockManager // Check if broadcast block was fetched diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 68b5776fc6515..2071701b313db 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -100,13 +100,13 @@ class JsonProtocolSuite extends FunSuite { appInfo } - def createDriverCommand() = new Command( + def createDriverCommand(): Command = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) - def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, - false, createDriverCommand()) + def createDriverDesc(): DriverDescription = + new DriverDescription("hdfs://some-dir/some.jar", 100, 3, false, createDriverCommand()) def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date()) diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 54dd7c9c45c61..9cdb42814ca32 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -56,7 +56,7 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") { val SPARK_PUBLIC_DNS = "public_dns" class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS else super.getenv(name) } 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 3a9963a5ce7b7..20de46fdab909 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 @@ -42,10 +42,10 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) val page = new HistoryPage(historyServer) - //when + // when val response = page.render(request) - //then + // then val links = response \\ "a" val justHrefs = for { l <- links diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 2fa90e3bd1c63..8e09976636386 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -508,7 +508,7 @@ private class DummyMaster( exception: Option[Exception] = None) extends Actor { - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage) case RequestKillDriver(driverId) => @@ -531,7 +531,7 @@ private class SmarterMaster extends Actor { private var counter: Int = 0 private val submittedDrivers = new mutable.HashMap[String, DriverState] - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => val driverId = s"driver-$counter" submittedDrivers(driverId) = RUNNING diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 1d64ec201e647..61071ee17256c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -129,7 +129,8 @@ class SubmitRestProtocolSuite extends FunSuite { assert(newMessage.sparkProperties("spark.files") === "fireball.png") assert(newMessage.sparkProperties("spark.driver.memory") === "512m") assert(newMessage.sparkProperties("spark.driver.cores") === "180") - assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red") + assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === + " -Dslices=5 -Dcolor=mostly_red") assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar") assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") assert(newMessage.sparkProperties("spark.driver.supervise") === "false") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 6fca6321e5a1b..a8b9df227c996 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -35,7 +35,8 @@ class ExecutorRunnerTest extends FunSuite { val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) - val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) + val builder = CommandUtils.buildProcessBuilder( + appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala index 372d7aa453008..7cc2104281464 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala @@ -37,7 +37,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "50000" else super.getenv(name) } @@ -56,7 +56,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "5G" else super.getenv(name) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index 84e2fd7ad936d..450fba21f4b5c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -24,8 +24,10 @@ import org.scalatest.{Matchers, FunSuite} class WorkerSuite extends FunSuite with Matchers { - def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) - def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts) + def cmd(javaOpts: String*): Command = { + Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) + } + def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts) test("test isUseLocalNodeSSLConfig") { Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 78fa98a3b9065..190b08d950a02 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -238,7 +238,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext sc.textFile(tmpFilePath, 4) .map(key => (key, 1)) - .reduceByKey(_+_) + .reduceByKey(_ + _) .saveAsTextFile("file://" + tmpFile.getAbsolutePath) sc.listenerBus.waitUntilEmpty(500) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index 37e528435aa5d..100ac77dec1f7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -35,7 +35,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val property = conf.getInstance("random") assert(property.size() === 2) - assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(property.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(property.getProperty("sink.servlet.path") === "/metrics/json") } @@ -47,16 +48,20 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(masterProp.size() === 5) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minutes") - assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(masterProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(masterProp.getProperty("sink.servlet.path") === "/metrics/master/json") val workerProp = conf.getInstance("worker") assert(workerProp.size() === 5) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "seconds") - assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(workerProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(workerProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(workerProp.getProperty("sink.servlet.path") === "/metrics/json") } diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 0dc59888f7304..be8467354b222 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -80,7 +80,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 10100) + assert(rdd.reduce(_ + _) === 10100) } test("large id overflow") { @@ -92,7 +92,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { 1131544775L, 567279358897692673L, 20, (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 5050) + assert(rdd.reduce(_ + _) === 5050) } after { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 108f70af43f37..ca0d953d306d8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -168,13 +168,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() + val sums = pairs.reduceByKey(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("reduceByKey with collectAsMap") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() + val sums = pairs.reduceByKey(_ + _).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) assert(sums(2) === 1) @@ -182,7 +182,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey with many output partitons") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() + val sums = pairs.reduceByKey(_ + _, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -192,7 +192,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def getPartition(key: Any) = key.asInstanceOf[Int] } val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) + val sums = pairs.reduceByKey(_ + _) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) // count the dependencies to make sure there is only 1 ShuffledRDD @@ -208,7 +208,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("countApproxDistinctByKey") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is * only a statistical bound, the tests can fail for large values of relativeSD. We will be using @@ -465,7 +465,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("foldByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() + val sums = pairs.foldByKey(0)(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -505,7 +505,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { conf.setOutputCommitter(classOf[FakeOutputCommitter]) FakeOutputCommitter.ran = false - pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + pairs.saveAsHadoopFile( + "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) assert(FakeOutputCommitter.ran, "OutputCommitter was never called") } @@ -552,7 +553,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } private object StratifiedAuxiliary { - def stratifier (fractionPositive: Double) = { + def stratifier (fractionPositive: Double): (Int) => String = { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } @@ -572,7 +573,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSampleExact(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, true, samplingRate, seed, n) testPoisson(stratifiedData, true, samplingRate, seed, n) } @@ -580,7 +581,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSample(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, false, samplingRate, seed, n) testPoisson(stratifiedData, false, samplingRate, seed, n) } @@ -590,7 +591,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey() .mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -612,7 +613,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -701,27 +702,27 @@ class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { */ class NewFakeWriter extends NewRecordWriter[Integer, Integer] { - def close(p1: NewTaskAttempContext) = () + def close(p1: NewTaskAttempContext): Unit = () - def write(p1: Integer, p2: Integer) = () + def write(p1: Integer, p2: Integer): Unit = () } class NewFakeCommitter extends NewOutputCommitter { - def setupJob(p1: NewJobContext) = () + def setupJob(p1: NewJobContext): Unit = () def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: NewTaskAttempContext) = () + def setupTask(p1: NewTaskAttempContext): Unit = () - def commitTask(p1: NewTaskAttempContext) = () + def commitTask(p1: NewTaskAttempContext): Unit = () - def abortTask(p1: NewTaskAttempContext) = () + def abortTask(p1: NewTaskAttempContext): Unit = () } class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: NewJobContext) = () + def checkOutputSpecs(p1: NewJobContext): Unit = () def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { new NewFakeWriter() @@ -735,7 +736,7 @@ class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false - def setConf(p1: Configuration) = { + def setConf(p1: Configuration): Unit = { setConfCalled = true () } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index cd193ae4f5238..1880364581c1a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -100,7 +100,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[Range])) } @@ -108,7 +108,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[Range])) } @@ -139,7 +139,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(i).isInstanceOf[Range]) val range = slices(i).asInstanceOf[Range] assert(range.start === i * (N / 40), "slice " + i + " start") - assert(range.end === (i+1) * (N / 40), "slice " + i + " end") + assert(range.end === (i + 1) * (N / 40), "slice " + i + " end") assert(range.step === 1, "slice " + i + " step") } } @@ -156,7 +156,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size /n + 1)) } check(prop) } @@ -174,7 +174,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -192,7 +192,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -209,7 +209,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -217,7 +217,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -225,7 +225,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala index 8408d7e785c65..465068c6cbb16 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.{Partition, SharedSparkContext, TaskContext} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { - test("Pruned Partitions inherit locality prefs correctly") { val rdd = new RDD[Int](sc, Nil) { @@ -74,8 +73,6 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { } class TestPartition(i: Int, value: Int) extends Partition with Serializable { - def index = i - - def testValue = this.value - + def index: Int = i + def testValue: Int = this.value } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index a0483886f8db3..0d1369c19c69e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -35,7 +35,7 @@ class MockSampler extends RandomSampler[Long, Long] { Iterator(s) } - override def clone = new MockSampler + override def clone: MockSampler = new MockSampler } class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index bede1ffb3e2d0..df42faab64505 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -82,7 +82,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("countApproxDistinct") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble val size = 1000 val uniformDistro = for (i <- 1 to 5000) yield i % size @@ -100,7 +100,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("partitioner aware union") { - def makeRDDWithPartitioner(seq: Seq[Int]) = { + def makeRDDWithPartitioner(seq: Seq[Int]): RDD[Int] = { sc.makeRDD(seq, 1) .map(x => (x, null)) .partitionBy(new HashPartitioner(2)) @@ -159,8 +159,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("treeAggregate") { val rdd = sc.makeRDD(-1000 until 1000, 10) - def seqOp = (c: Long, x: Int) => c + x - def combOp = (c1: Long, c2: Long) => c1 + c2 + def seqOp: (Long, Int) => Long = (c: Long, x: Int) => c + x + def combOp: (Long, Long) => Long = (c1: Long, c2: Long) => c1 + c2 for (depth <- 1 until 10) { val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) assert(sum === -1000L) @@ -204,7 +204,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(empty.collect().size === 0) val thrown = intercept[UnsupportedOperationException]{ - empty.reduce(_+_) + empty.reduce(_ + _) } assert(thrown.getMessage.contains("empty")) @@ -321,7 +321,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 - val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) + val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i + 2)).map{ j => "m" + (j%6)}))) val coalesced1 = data.coalesce(3) assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") @@ -921,15 +921,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("task serialization exception should not hang scheduler") { class BadSerializable extends Serializable { @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream): Unit = throw new KryoException("Bad serialization") + private def writeObject(out: ObjectOutputStream): Unit = + throw new KryoException("Bad serialization") @throws(classOf[IOException]) private def readObject(in: ObjectInputStream): Unit = {} } - // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if there were - // more threads in the Spark Context than there were number of objects in this sequence. + // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if + // there were more threads in the Spark Context than there were number of objects in this + // sequence. intercept[Throwable] { - sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect + sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect() } // Check that the context has not crashed sc.parallelize(1 to 100).map(x => x*2).collect diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala index 4762fc17855ce..fe695d85e29dd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala @@ -21,11 +21,11 @@ object RDDSuiteUtils { case class Person(first: String, last: String, age: Int) object AgeOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = a.age compare b.age + def compare(a:Person, b:Person): Int = a.age.compare(b.age) } object NameOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = + def compare(a:Person, b:Person): Int = implicitly[Ordering[Tuple2[String,String]]].compare((a.last, a.first), (b.last, b.first)) } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 5a734ec5ba5ec..ada07ef11cd7a 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -70,7 +70,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("send-remotely", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case msg: String => message = msg } }) @@ -109,7 +109,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { context.reply(msg) } @@ -123,7 +123,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("ask-remotely", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { context.reply(msg) } @@ -146,7 +146,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("ask-timeout", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { Thread.sleep(100) context.reply(msg) @@ -182,7 +182,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { calledMethods += "start" } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case msg: String => } @@ -206,7 +206,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { throw new RuntimeException("Oops!") } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -225,7 +225,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -250,8 +250,8 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint { override val rpcEnv = env - override def receive = { - case m => throw new RuntimeException("Oops!") + override def receive: PartialFunction[Any, Unit] = { + case m => throw new RuntimeException("Oops!") } override def onError(cause: Throwable): Unit = { @@ -277,7 +277,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { callSelfSuccessfully = true } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } }) @@ -294,7 +294,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => { self callSelfSuccessfully = true @@ -316,7 +316,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -343,7 +343,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => result += 1 } @@ -372,7 +372,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -394,7 +394,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.reply("ack") } }) @@ -410,7 +410,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.reply("ack") } }) @@ -432,7 +432,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.sendFailure(new SparkException("Oops")) } }) @@ -450,7 +450,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => context.sendFailure(new SparkException("Oops")) } }) @@ -476,7 +476,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("network-events", new ThreadSafeRpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case "hello" => case m => events += "receive" -> m } @@ -519,7 +519,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => context.sendFailure(new UnserializableException) } }) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index eb759f0807a17..3c52a8c4460c6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -57,20 +57,18 @@ class MyRDD( locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new RuntimeException("should not be reached") - override def getPartitions = (0 until numPartitions).map(i => new Partition { - override def index = i + override def getPartitions: Array[Partition] = (0 until numPartitions).map(i => new Partition { + override def index: Int = i }).toArray override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil + if (locations.isDefinedAt(split.index)) locations(split.index) else Nil override def toString: String = "DAGSchedulerSuiteRDD " + id } class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite + extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -209,7 +207,8 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) } } } @@ -269,21 +268,23 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job") { val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator - override def getPartitions = Array( new Partition { override def index = 0 } ) - override def getPreferredLocations(split: Partition) = Nil - override def toString = "DAGSchedulerSuite Local RDD" + override def getPartitions: Array[Partition] = + Array( new Partition { override def index: Int = 0 } ) + override def getPreferredLocations(split: Partition): List[String] = Nil + override def toString: String = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job oom") { @@ -295,9 +296,10 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results.size == 0) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial job w/ dependency") { @@ -306,7 +308,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cache location preferences w/ dependency") { @@ -319,7 +321,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("regression test for getCacheLocs") { @@ -335,7 +337,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("avoid exponential blowup when getting preferred locs list") { - // Build up a complex dependency graph with repeated zip operations, without preferred locations. + // Build up a complex dependency graph with repeated zip operations, without preferred locations var rdd: RDD[_] = new MyRDD(sc, 1, Nil) (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. @@ -357,7 +359,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job failure") { @@ -367,7 +369,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job cancellation") { @@ -378,7 +380,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("job cancellation no-kill backend") { @@ -387,18 +389,20 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar val noKillTaskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE - override def start() = {} - override def stop() = {} - override def submitTasks(taskSet: TaskSet) = { + override def start(): Unit = {} + override def stop(): Unit = {} + override def submitTasks(taskSet: TaskSet): Unit = { taskSets += taskSet } override def cancelTasks(stageId: Int, interruptThread: Boolean) { throw new UnsupportedOperationException } - override def setDAGScheduler(dagScheduler: DAGScheduler) = {} - override def defaultParallelism() = 2 - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} + override def defaultParallelism(): Int = 2 + override def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } val noKillScheduler = new DAGScheduler( @@ -422,7 +426,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // When the task set completes normally, state should be correctly updated. complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.isEmpty) @@ -442,7 +446,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with fetch failure") { @@ -465,10 +469,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial shuffle with multiple fetch failures") { @@ -521,19 +526,23 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(newEpoch > oldEpoch) val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run shuffle with map stage failure") { @@ -552,7 +561,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.toSet === Set(0)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -586,7 +595,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar class FailureRecordingJobListener() extends JobListener { var failureMessage: String = _ override def taskSucceeded(index: Int, result: Any) {} - override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + override def jobFailed(exception: Exception): Unit = { failureMessage = exception.getMessage } } val listener1 = new FailureRecordingJobListener() val listener2 = new FailureRecordingJobListener() @@ -606,7 +615,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with out-of-band failure and retry") { @@ -629,7 +638,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("recursive shuffle failures") { @@ -658,7 +667,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cached post-shuffle") { @@ -690,7 +699,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { @@ -742,7 +751,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("accumulator not calculated for resubmitted result stage") { - //just for register + // just for register val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) @@ -754,7 +763,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(accVal === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -774,7 +783,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) - private def assertDataStructuresEmpty = { + private def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 30ee63e78d9d8..6d25edb7d20dc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -268,7 +268,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef object EventLoggingListenerSuite { /** Get a SparkConf with event logging enabled. */ - def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { val conf = new SparkConf conf.set("spark.eventLog.enabled", "true") conf.set("spark.eventLog.testing", "true") @@ -280,5 +280,5 @@ object EventLoggingListenerSuite { conf } - def getUniqueApplicationId = "test-" + System.currentTimeMillis + def getUniqueApplicationId: String = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 6b75c98839e03..9b92f8de56759 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -24,7 +24,9 @@ import org.apache.spark.TaskContext /** * A Task implementation that fails to serialize. */ -private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) extends Task[Array[Byte]](stageId, 0) { +private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) + extends Task[Array[Byte]](stageId, 0) { + override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 627c9a4ddfffc..825c616c0c3e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -85,7 +85,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val stopperReturned = new Semaphore(0) class BlockingListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { listenerStarted.release() listenerWait.acquire() drained = true @@ -206,8 +206,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time val w = { i: Int => - if (i == 0) + if (i == 0) { Thread.sleep(100) + } i } @@ -247,12 +248,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => - taskMetrics.resultSize should be > (0l) + taskMetrics.resultSize should be > (0L) if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.inputMetrics should not be ('defined) taskMetrics.outputMetrics should not be ('defined) taskMetrics.shuffleWriteMetrics should be ('defined) - taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) + taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0L) } if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) @@ -260,7 +261,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sm.totalBlocksFetched should be (128) sm.localBlocksFetched should be (128) sm.remoteBlocksFetched should be (0) - sm.remoteBytesRead should be (0l) + sm.remoteBytesRead should be (0L) } } } @@ -406,12 +407,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { startedTasks += taskStart.taskInfo.index notify() } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { endedTasks += taskEnd.taskInfo.index notify() } @@ -425,7 +426,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers * A simple listener that throws an exception on job end. */ private class BadListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { throw new Exception } } } @@ -438,10 +439,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ private class BasicJobCounter extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index add13f5b21765..ffa4381969b68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import java.util.Properties - import org.scalatest.FunSuite import org.apache.spark._ @@ -27,7 +25,7 @@ class FakeSchedulerBackend extends SchedulerBackend { def start() {} def stop() {} def reviveOffers() {} - def defaultParallelism() = 1 + def defaultParallelism(): Int = 1 } class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { @@ -115,7 +113,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } val numFreeCores = 1 taskScheduler.setDAGScheduler(dagScheduler) - var taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -123,7 +122,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(0 === taskDescriptions.length) // Now check that we can still submit tasks - // Even if one of the tasks has not-serializable tasks, the other task set should still be processed without error + // Even if one of the tasks has not-serializable tasks, the other task set should + // still be processed without error taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(FakeTask.createTaskSet(1)) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 12330d8f63c40..716d12c0762cf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -67,7 +67,7 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRackForHost(host: String) = { + def getRackForHost(host: String): Option[String] = { hostToRack.get(host) } } @@ -327,8 +327,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - // After this, nothing should get chosen, because we have separated tasks with unavailable preference - // from the noPrefPendingTasks + // After this, nothing should get chosen, because we have separated tasks with unavailable + // preference from the noPrefPendingTasks assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead @@ -499,7 +499,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sched.addExecutor("execC", "host2") manager.executorAdded() // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL and ANY - assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) + assert(manager.myLocalityLevels.sameElements( + Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) // test if the valid locality is recomputed when the executor is lost sched.removeExecutor("execC") manager.executorLost("execC", "host2") @@ -569,7 +570,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { @@ -582,7 +584,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") sc = new SparkContext("local", "test", conf) - def genBytes(size: Int) = { (x: Int) => + def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => val bytes = Array.ofDim[Byte](size) scala.util.Random.nextBytes(bytes) bytes @@ -605,7 +607,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("speculative and noPref task should be scheduled after node-local") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2"), TaskLocation("host1")), @@ -629,9 +632,11 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } - test("node-local tasks should be scheduled right away when there are only node-local and no-preference tasks") { + test("node-local tasks should be scheduled right away " + + "when there are only node-local and no-preference tasks") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -650,7 +655,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } - test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") { + test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") + { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) val taskSet = FakeTask.createTaskSet(4, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index f1a4380d349b3..a311512e82c5e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -63,16 +63,18 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo // uri is null. val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo.getCommand.getValue === + s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") // uri exists. conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") val executorInfo1 = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo1.getCommand.getValue === + s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") } test("mesos resource offers result in launching tasks") { - def createOffer(id: Int, mem: Int, cpu: Int) = { + def createOffer(id: Int, mem: Int, cpu: Int): Offer = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -82,8 +84,10 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) + .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) + .setHostname(s"host${id.toString}").build() } val driver = mock[SchedulerDriver] diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 6198df84fab3d..b070a54aa989b 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -106,7 +106,9 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1->"one",2->"two",3->"three"))) } test("ranges") { @@ -169,7 +171,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("kryo with collect") { val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + val result = sc.parallelize(control, 2) + .map(new ClassWithoutNoArgConstructor(_)) + .collect() + .map(_.x) assert(control === result.toSeq) } @@ -237,7 +242,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { // Set a special, broken ClassLoader and make sure we get an exception on deserialization ser.setDefaultClassLoader(new ClassLoader() { - override def loadClass(name: String) = throw new UnsupportedOperationException + override def loadClass(name: String): Class[_] = throw new UnsupportedOperationException }) intercept[UnsupportedOperationException] { ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) @@ -287,14 +292,14 @@ object KryoTest { class ClassWithNoArgConstructor { var x: Int = 0 - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithNoArgConstructor => x == c.x case _ => false } } class ClassWithoutNoArgConstructor(val x: Int) { - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithoutNoArgConstructor => x == c.x case _ => false } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index d037e2c19a64d..433fd6bb4a11d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -24,14 +24,16 @@ import org.apache.spark.rdd.RDD /* A trivial (but unserializable) container for trivial functions */ class UnserializableClass { - def op[T](x: T) = x.toString + def op[T](x: T): String = x.toString - def pred[T](x: T) = x.toString.length % 2 == 0 + def pred[T](x: T): Boolean = x.toString.length % 2 == 0 } class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + def fixture: (RDD[String], UnserializableClass) = { + (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + } test("throws expected serialization exceptions on actions") { val (data, uc) = fixture diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala index 0ade1bab18d7e..963264cef3a71 100644 --- a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag * A serializer implementation that always return a single element in a deserialization stream. */ class TestSerializer extends Serializer { - override def newInstance() = new TestSerializerInstance + override def newInstance(): TestSerializerInstance = new TestSerializerInstance } @@ -36,7 +36,8 @@ class TestSerializerInstance extends SerializerInstance { override def serializeStream(s: OutputStream): SerializationStream = ??? - override def deserializeStream(s: InputStream) = new TestDeserializationStream + override def deserializeStream(s: InputStream): TestDeserializationStream = + new TestDeserializationStream override def deserialize[T: ClassTag](bytes: ByteBuffer): T = ??? diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index b834dc0e735eb..7d76435cd75e7 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -85,8 +85,8 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { // Now comes the test : // Write to shuffle 3; and close it, but before registering it, check if the file lengths for // previous task (forof shuffle1) is the same as 'segments'. Earlier, we were inferring length - // of block based on remaining data in file : which could mess things up when there is concurrent read - // and writes happening to the same shuffle group. + // of block based on remaining data in file : which could mess things up when there is + // concurrent read and writes happening to the same shuffle group. val shuffle3 = shuffleBlockManager.forMapTask(1, 3, 1, new JavaSerializer(testConf), new ShuffleWriteMetrics) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 6dc5bc4cb08c4..545722b050ee8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -60,7 +60,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + def rdd(rddId: Int, splitId: Int): RDDBlockId = RDDBlockId(rddId, splitId) private def makeBlockManager( maxMem: Long, @@ -107,8 +107,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, false, 3) - val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object + // this should return the same object as level1 + val level2 = StorageLevel(false, false, false, false, 3) + // this should return a different object + val level3 = StorageLevel(false, false, false, false, 2) assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -802,7 +804,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // Create a non-trivial (not all zeros) byte array var counter = 0.toByte - def incr = {counter = (counter + 1).toByte; counter;} + def incr: Byte = {counter = (counter + 1).toByte; counter;} val bytes = Array.fill[Byte](1000)(incr) val byteBuffer = ByteBuffer.wrap(bytes) @@ -956,8 +958,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) - assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size + === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size + === 1) // insert some more blocks store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) @@ -965,8 +969,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size + === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size + === 3) val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => @@ -1090,8 +1096,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with plenty of space. This should succeed and cache both blocks. @@ -1144,8 +1150,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) store.putIterator("b1", smallIterator, memAndDisk) @@ -1187,7 +1193,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memOnly = StorageLevel.MEMORY_ONLY val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // All unroll memory used is released because unrollSafely returned an array diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index 82a82e23eecf2..b47157f8331cc 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -47,7 +47,7 @@ class LocalDirsSuite extends FunSuite with BeforeAndAfter { assert(!new File("/NONEXISTENT_DIR").exists()) // SPARK_LOCAL_DIRS is a valid directory: class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir") else super.getenv(name) } 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 0d155982a8c54..1cb594633f331 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -137,7 +137,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before new SparkContext(conf) } - def hasKillLink = find(className("kill-link")).isDefined + def hasKillLink: Boolean = find(className("kill-link")).isDefined def runSlowJob(sc: SparkContext) { sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index c0c28cb60e21d..21d8267114133 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -269,7 +269,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) val execId = "exe-1" - def makeTaskMetrics(base: Int) = { + def makeTaskMetrics(base: Int): TaskMetrics = { val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() val shuffleWriteMetrics = new ShuffleWriteMetrics() @@ -291,7 +291,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics } - def makeTaskInfo(taskId: Long, finishTime: Int = 0) = { + def makeTaskInfo(taskId: Long, finishTime: Int = 0): TaskInfo = { val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = finishTime diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index e1bc1379b5d80..3744e479d2f05 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -107,7 +107,8 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + val stageInfo0 = new StageInfo( + 0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) 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 054ef54e746a5..c47162779bbba 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -83,7 +83,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -95,7 +95,7 @@ class TestClass extends Serializable { } class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -164,7 +164,7 @@ object TestObjectWithNesting { } class TestClassWithNesting(val y: Int) extends Serializable { - def getY = y + def getY: Int = y def run(): Int = { var nonSer = new NonSerializable diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 43b6a405cb68c..c05317534cddf 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -109,7 +109,8 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // verify whether the earliest file has been deleted val rolledOverFiles = allGeneratedFiles.filter { _ != testFile.toString }.toArray.sorted - logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + rolledOverFiles.mkString("\n")) + logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + + rolledOverFiles.mkString("\n")) assert(rolledOverFiles.size > 2) val earliestRolledOverFile = rolledOverFiles.head val existingRolledOverFiles = RollingFileAppender.getSortedRolledOverFiles( @@ -135,7 +136,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream) val appender = FileAppender(testInputStream, testFile, conf) - //assert(appender.getClass === classTag[ExpectedAppender].getClass) + // assert(appender.getClass === classTag[ExpectedAppender].getClass) assert(appender.getClass.getSimpleName === classTag[ExpectedAppender].runtimeClass.getSimpleName) if (appender.isInstanceOf[RollingFileAppender]) { @@ -153,9 +154,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { import RollingFileAppender._ - def rollingStrategy(strategy: String) = Seq(STRATEGY_PROPERTY -> strategy) - def rollingSize(size: String) = Seq(SIZE_PROPERTY -> size) - def rollingInterval(interval: String) = Seq(INTERVAL_PROPERTY -> interval) + def rollingStrategy(strategy: String): Seq[(String, String)] = + Seq(STRATEGY_PROPERTY -> strategy) + def rollingSize(size: String): Seq[(String, String)] = Seq(SIZE_PROPERTY -> size) + def rollingInterval(interval: String): Seq[(String, String)] = + Seq(INTERVAL_PROPERTY -> interval) val msInDay = 24 * 60 * 60 * 1000L val msInHour = 60 * 60 * 1000L diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 72e81f3f1a884..403dcb03bd6e5 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -71,7 +71,7 @@ class NextIteratorSuite extends FunSuite with Matchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - override def getNext() = { + override def getNext(): Int = { if (ints.size == 0) { finished = true 0 diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 7424c2e91d4f2..67a9f75ff2187 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -98,8 +98,10 @@ class SizeEstimatorSuite // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + // 10 pointers plus 8-byte object + assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) + // 100 pointers plus 8-byte object + assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index c1c605cdb487b..8b72fe665c214 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -63,7 +63,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } @@ -93,7 +93,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 5d93086082189..449fb87f111c4 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -106,7 +106,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { val second = 1000 val minute = second * 60 val hour = minute * 60 - def str = Utils.msDurationToString(_) + def str: (Long) => String = Utils.msDurationToString(_) val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator() @@ -199,7 +199,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() - val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + // The parent directory has two child directories + val child1: File = Utils.createTempDir(parent.getCanonicalPath) val child2: File = Utils.createTempDir(parent.getCanonicalPath) val child3: File = Utils.createTempDir(child1.getCanonicalPath) // set the last modified time of child1 to 30 secs old diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala index 794a55d61750b..ce2968728a996 100644 --- a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite @deprecated("suppress compile time deprecation warning", "1.0.0") class VectorSuite extends FunSuite { - def verifyVector(vector: Vector, expectedLength: Int) = { + def verifyVector(vector: Vector, expectedLength: Int): Unit = { assert(vector.length == expectedLength) assert(vector.elements.min > 0.0) assert(vector.elements.max < 1.0) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 48f79ea651018..dff8f3ddc816f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -185,7 +185,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { // reduceByKey val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) - val result1 = rdd.reduceByKey(_+_).collect() + val result1 = rdd.reduceByKey(_ + _).collect() assert(result1.toSet === Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 72d96798b1141..9ff067f86af44 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -553,10 +553,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = - buffer1 ++= buffer2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) + : ArrayBuffer[String] = buffer1 ++= buffer2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner _, mergeValue _, mergeCombiners _) @@ -633,14 +633,17 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: Int) = ArrayBuffer[Int](i) - def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + def createCombiner(i: Int): ArrayBuffer[Int] = ArrayBuffer[Int](i) + def mergeValue(buffer: ArrayBuffer[Int], i: Int): ArrayBuffer[Int] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]): ArrayBuffer[Int] = { + buf1 ++= buf2 + } val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) - sorter.insertAll((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + sorter.insertAll( + (1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) val it = sorter.iterator while (it.hasNext) { @@ -654,9 +657,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]) = buf1 ++= buf2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]): ArrayBuffer[String] = + buf1 ++= buf2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) @@ -720,7 +724,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using wrongOrdering to show integer overflow introduced exception. val rand = new Random(100L) val wrongOrdering = new Ordering[String] { - override def compare(a: String, b: String) = { + override def compare(a: String, b: String): Int = { val h1 = if (a == null) 0 else a.hashCode() val h2 = if (b == null) 0 else b.hashCode() h1 - h2 @@ -742,9 +746,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using aggregation and external spill to make sure ExternalSorter using // partitionKeyComparator. - def createCombiner(i: String) = ArrayBuffer(i) - def mergeValue(c: ArrayBuffer[String], i: String) = c += i - def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]) = c1 ++= c2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer(i) + def mergeValue(c: ArrayBuffer[String], i: String): ArrayBuffer[String] = c += i + def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]): ArrayBuffer[String] = + c1 ++= c2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index ef7178bcdf5c2..03f5f2d1b8528 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -28,7 +28,7 @@ import scala.language.reflectiveCalls class XORShiftRandomSuite extends FunSuite with Matchers { - def fixture = new { + def fixture: Object {val seed: Long; val hundMil: Int; val xorRand: XORShiftRandom} = new { val seed = 1L val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt From fc17661475443d9f0a8d28e3439feeb7a7bca67b Mon Sep 17 00:00:00 2001 From: lewuathe Date: Sun, 12 Apr 2015 22:17:16 -0700 Subject: [PATCH 723/817] [SPARK-6643][MLLIB] Implement StandardScalerModel missing methods This is the sub-task of SPARK-6254. Wrap missing method for `StandardScalerModel`. Author: lewuathe Closes #5310 from Lewuathe/SPARK-6643 and squashes the following commits: fafd690 [lewuathe] Fix for lint-python bd31a64 [lewuathe] Merge branch 'master' into SPARK-6643 578f5ee [lewuathe] Remove unnecessary class a38f155 [lewuathe] Merge master 66bb2ab [lewuathe] Fix typos 82683a0 [lewuathe] [SPARK-6643] Implement StandardScalerModel missing methods --- .../mllib/api/python/PythonMLLibAPI.scala | 4 ++-- python/pyspark/mllib/feature.py | 16 +++++++++++++ python/pyspark/mllib/tests.py | 24 +++++++++++++++++++ 3 files changed, 42 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 1faa3def0e042..ab15f0f36a14b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -450,9 +450,9 @@ private[python] class PythonMLLibAPI extends Serializable { def normalizeVector(p: Double, rdd: JavaRDD[Vector]): JavaRDD[Vector] = { new Normalizer(p).transform(rdd) } - + /** - * Java stub for IDF.fit(). This stub returns a + * Java stub for StandardScaler.fit(). This stub returns a * handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 3cda1205e1391..8be819aceec24 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -132,6 +132,22 @@ def transform(self, vector): """ return JavaVectorTransformer.transform(self, vector) + def setWithMean(self, withMean): + """ + Setter of the boolean which decides + whether it uses mean or not + """ + self.call("setWithMean", withMean) + return self + + def setWithStd(self, withStd): + """ + Setter of the boolean which decides + whether it uses std or not + """ + self.call("setWithStd", withStd) + return self + class StandardScaler(object): """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 3b40158c12b74..8eaddcf8b9b5e 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -44,6 +44,7 @@ from pyspark.mllib.stat import Statistics from pyspark.mllib.feature import Word2Vec from pyspark.mllib.feature import IDF +from pyspark.mllib.feature import StandardScaler from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase @@ -745,6 +746,29 @@ def test_word2vec_get_vectors(self): model = Word2Vec().fit(self.sc.parallelize(data)) self.assertEquals(len(model.getVectors()), 3) + +class StandardScalerTests(PySparkTestCase): + def test_model_setters(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertIsNotNone(model.setWithMean(True)) + self.assertIsNotNone(model.setWithStd(True)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([-1.0, -1.0, -1.0])) + + def test_model_transform(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([1.0, 2.0, 3.0])) + + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" From d3792f54974e16cbe8f10b3091d248e0bdd48986 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 12 Apr 2015 22:38:27 -0700 Subject: [PATCH 724/817] [SPARK-4081] [mllib] VectorIndexer **Ready for review!** Since the original PR, I moved the code to the spark.ml API and renamed this to VectorIndexer. This introduces a VectorIndexer class which does the following: * VectorIndexer.fit(): collect statistics about how many values each feature in a dataset (RDD[Vector]) can take (limited by maxCategories) * Feature which exceed maxCategories are declared continuous, and the Model will treat them as such. * VectorIndexerModel.transform(): Convert categorical feature values to corresponding 0-based indices Design notes: * This maintains sparsity in vectors by ensuring that categorical feature value 0.0 gets index 0. * This does not yet support transforming data with new (unknown) categorical feature values. That can be added later. * This is necessary for DecisionTree and tree ensembles. Reviewers: Please check my use of metadata and my unit tests for it; I'm not sure if I covered everything in the tests. Other notes: * This also adds a public toMetadata method to AttributeGroup (for simpler construction of metadata). CC: mengxr Author: Joseph K. Bradley Closes #3000 from jkbradley/indexer and squashes the following commits: 5956d91 [Joseph K. Bradley] minor cleanups f5c57a8 [Joseph K. Bradley] added Java test suite 643b444 [Joseph K. Bradley] removed FeatureTests 02236c3 [Joseph K. Bradley] Updated VectorIndexer, ready for PR 286d221 [Joseph K. Bradley] Reworked DatasetIndexer for spark.ml API, and renamed it to VectorIndexer 12e6cf2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into indexer 6d8f3f1 [Joseph K. Bradley] Added partly done DatasetIndexer to spark.ml 6a2f553 [Joseph K. Bradley] Updated TODO for allowUnknownCategories 3f041f8 [Joseph K. Bradley] Final cleanups for DatasetIndexer 038b9e3 [Joseph K. Bradley] DatasetIndexer now maintains sparsity in SparseVector 3a4a0bd [Joseph K. Bradley] Added another test for DatasetIndexer 2006923 [Joseph K. Bradley] DatasetIndexer now passes tests f409987 [Joseph K. Bradley] partly done with DatasetIndexerSuite 5e7c874 [Joseph K. Bradley] working on DatasetIndexer --- .../scala/org/apache/spark/ml/Pipeline.scala | 3 + .../spark/ml/attribute/AttributeGroup.scala | 21 +- .../spark/ml/feature/VectorIndexer.scala | 393 ++++++++++++++++++ .../org/apache/spark/ml/param/params.scala | 20 +- .../ml/feature/JavaVectorIndexerSuite.java | 70 ++++ .../ml/attribute/AttributeGroupSuite.scala | 8 +- .../spark/ml/feature/NormalizerSuite.scala | 7 +- .../spark/ml/feature/VectorIndexerSuite.scala | 255 ++++++++++++ .../apache/spark/ml/util/TestingUtils.scala | 60 +++ 9 files changed, 818 insertions(+), 19 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index c4a36103303a2..a455341a1f723 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -47,6 +47,9 @@ abstract class PipelineStage extends Serializable with Logging { /** * Derives the output schema from the input schema and parameters, optionally with logging. + * + * This should be optimistic. If it is unclear whether the schema will be valid, then it should + * be assumed valid until proven otherwise. */ protected def transformSchema( schema: StructType, diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 970e6ad5514d1..aa27a668f1695 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -106,7 +106,7 @@ class AttributeGroup private ( def getAttr(attrIndex: Int): Attribute = this(attrIndex) /** Converts to metadata without name. */ - private[attribute] def toMetadata: Metadata = { + private[attribute] def toMetadataImpl: Metadata = { import AttributeKeys._ val bldr = new MetadataBuilder() if (attributes.isDefined) { @@ -142,17 +142,24 @@ class AttributeGroup private ( bldr.build() } - /** Converts to a StructField with some existing metadata. */ - def toStructField(existingMetadata: Metadata): StructField = { - val newMetadata = new MetadataBuilder() + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, toMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl) .build() - StructField(name, new VectorUDT, nullable = false, newMetadata) + } + + /** Converts to ML metadata */ + def toMetadata: Metadata = toMetadata(Metadata.empty) + + /** Converts to a StructField with some existing metadata. */ + def toStructField(existingMetadata: Metadata): StructField = { + StructField(name, new VectorUDT, nullable = false, toMetadata(existingMetadata)) } /** Converts to a StructField. */ - def toStructField(): StructField = toStructField(Metadata.empty) + def toStructField: StructField = toStructField(Metadata.empty) override def equals(other: Any): Boolean = { other match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala new file mode 100644 index 0000000000000..8760960e19272 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -0,0 +1,393 @@ +/* + * 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.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.{BinaryAttribute, NumericAttribute, NominalAttribute, + Attribute, AttributeGroup} +import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap, Params} +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, VectorUDT} +import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.functions.callUDF +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.collection.OpenHashSet + + +/** Private trait for params for VectorIndexer and VectorIndexerModel */ +private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOutputCol { + + /** + * Threshold for the number of values a categorical feature can take. + * If a feature is found to have > maxCategories values, then it is declared continuous. + * + * (default = 20) + */ + val maxCategories = new IntParam(this, "maxCategories", + "Threshold for the number of values a categorical feature can take." + + " If a feature is found to have > maxCategories values, then it is declared continuous.", + Some(20)) + + /** @group getParam */ + def getMaxCategories: Int = get(maxCategories) +} + +/** + * :: AlphaComponent :: + * + * Class for indexing categorical feature columns in a dataset of [[Vector]]. + * + * This has 2 usage modes: + * - Automatically identify categorical features (default behavior) + * - This helps process a dataset of unknown vectors into a dataset with some continuous + * features and some categorical features. The choice between continuous and categorical + * is based upon a maxCategories parameter. + * - Set maxCategories to the maximum number of categorical any categorical feature should have. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories = 2, then feature 0 will be declared categorical and use indices {0, 1}, + * and feature 1 will be declared continuous. + * - Index all features, if all features are categorical + * - If maxCategories is set to be very large, then this will build an index of unique + * values for all features. + * - Warning: This can cause problems if features are continuous since this will collect ALL + * unique values to the driver. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories >= 3, then both features will be declared categorical. + * + * This returns a model which can transform categorical features to use 0-based indices. + * + * Index stability: + * - This is not guaranteed to choose the same category index across multiple runs. + * - If a categorical feature includes value 0, then this is guaranteed to map value 0 to index 0. + * This maintains vector sparsity. + * - More stability may be added in the future. + * + * TODO: Future extensions: The following functionality is planned for the future: + * - Preserve metadata in transform; if a feature's metadata is already present, do not recompute. + * - Specify certain features to not index, either via a parameter or via existing metadata. + * - Add warning if a categorical feature has only 1 category. + * - Add option for allowing unknown categories. + */ +@AlphaComponent +class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerParams { + + /** @group setParam */ + def setMaxCategories(value: Int): this.type = { + require(value > 1, + s"DatasetIndexer given maxCategories = value, but requires maxCategories > 1.") + set(maxCategories, value) + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def fit(dataset: DataFrame, paramMap: ParamMap): VectorIndexerModel = { + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val firstRow = dataset.select(map(inputCol)).take(1) + require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") + val numFeatures = firstRow(0).getAs[Vector](0).size + val vectorDataset = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } + val maxCats = map(maxCategories) + val categoryStats: VectorIndexer.CategoryStats = vectorDataset.mapPartitions { iter => + val localCatStats = new VectorIndexer.CategoryStats(numFeatures, maxCats) + iter.foreach(localCatStats.addVector) + Iterator(localCatStats) + }.reduce((stats1, stats2) => stats1.merge(stats2)) + val model = new VectorIndexerModel(this, map, numFeatures, categoryStats.getCategoryMaps) + Params.inheritValues(map, this, model) + model + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + // We do not transfer feature metadata since we do not know what types of features we will + // produce in transform(). + val map = this.paramMap ++ paramMap + val dataType = new VectorUDT + require(map.contains(inputCol), s"VectorIndexer requires input column parameter: $inputCol") + require(map.contains(outputCol), s"VectorIndexer requires output column parameter: $outputCol") + checkInputColumn(schema, map(inputCol), dataType) + addOutputColumn(schema, map(outputCol), dataType) + } +} + +private object VectorIndexer { + + /** + * Helper class for tracking unique values for each feature. + * + * TODO: Track which features are known to be continuous already; do not update counts for them. + * + * @param numFeatures This class fails if it encounters a Vector whose length is not numFeatures. + * @param maxCategories This class caps the number of unique values collected at maxCategories. + */ + class CategoryStats(private val numFeatures: Int, private val maxCategories: Int) + extends Serializable { + + /** featureValueSets[feature index] = set of unique values */ + private val featureValueSets = + Array.fill[OpenHashSet[Double]](numFeatures)(new OpenHashSet[Double]()) + + /** Merge with another instance, modifying this instance. */ + def merge(other: CategoryStats): CategoryStats = { + featureValueSets.zip(other.featureValueSets).foreach { case (thisValSet, otherValSet) => + otherValSet.iterator.foreach { x => + // Once we have found > maxCategories values, we know the feature is continuous + // and do not need to collect more values for it. + if (thisValSet.size <= maxCategories) thisValSet.add(x) + } + } + this + } + + /** Add a new vector to this index, updating sets of unique feature values */ + def addVector(v: Vector): Unit = { + require(v.size == numFeatures, s"VectorIndexer expected $numFeatures features but" + + s" found vector of size ${v.size}.") + v match { + case dv: DenseVector => addDenseVector(dv) + case sv: SparseVector => addSparseVector(sv) + } + } + + /** + * Based on stats collected, decide which features are categorical, + * and choose indices for categories. + * + * Sparsity: This tries to maintain sparsity by treating value 0.0 specially. + * If a categorical feature takes value 0.0, then value 0.0 is given index 0. + * + * @return Feature value index. Keys are categorical feature indices (column indices). + * Values are mappings from original features values to 0-based category indices. + */ + def getCategoryMaps: Map[Int, Map[Double, Int]] = { + // Filter out features which are declared continuous. + featureValueSets.zipWithIndex.filter(_._1.size <= maxCategories).map { + case (featureValues: OpenHashSet[Double], featureIndex: Int) => + var sortedFeatureValues = featureValues.iterator.filter(_ != 0.0).toArray.sorted + val zeroExists = sortedFeatureValues.length + 1 == featureValues.size + if (zeroExists) { + sortedFeatureValues = 0.0 +: sortedFeatureValues + } + val categoryMap: Map[Double, Int] = sortedFeatureValues.zipWithIndex.toMap + (featureIndex, categoryMap) + }.toMap + } + + private def addDenseVector(dv: DenseVector): Unit = { + var i = 0 + while (i < dv.size) { + if (featureValueSets(i).size <= maxCategories) { + featureValueSets(i).add(dv(i)) + } + i += 1 + } + } + + private def addSparseVector(sv: SparseVector): Unit = { + // TODO: This might be able to handle 0's more efficiently. + var vecIndex = 0 // index into vector + var k = 0 // index into non-zero elements + while (vecIndex < sv.size) { + val featureValue = if (k < sv.indices.length && vecIndex == sv.indices(k)) { + k += 1 + sv.values(k - 1) + } else { + 0.0 + } + if (featureValueSets(vecIndex).size <= maxCategories) { + featureValueSets(vecIndex).add(featureValue) + } + vecIndex += 1 + } + } + } +} + +/** + * :: AlphaComponent :: + * + * Transform categorical features to use 0-based indices instead of their original values. + * - Categorical features are mapped to indices. + * - Continuous features (columns) are left unchanged. + * This also appends metadata to the output column, marking features as Numeric (continuous), + * Nominal (categorical), or Binary (either continuous or categorical). + * + * This maintains vector sparsity. + * + * @param numFeatures Number of features, i.e., length of Vectors which this transforms + * @param categoryMaps Feature value index. Keys are categorical feature indices (column indices). + * Values are maps from original features values to 0-based category indices. + * If a feature is not in this map, it is treated as continuous. + */ +@AlphaComponent +class VectorIndexerModel private[ml] ( + override val parent: VectorIndexer, + override val fittingParamMap: ParamMap, + val numFeatures: Int, + val categoryMaps: Map[Int, Map[Double, Int]]) + extends Model[VectorIndexerModel] with VectorIndexerParams { + + /** + * Pre-computed feature attributes, with some missing info. + * In transform(), set attribute name and other info, if available. + */ + private val partialFeatureAttributes: Array[Attribute] = { + val attrs = new Array[Attribute](numFeatures) + var categoricalFeatureCount = 0 // validity check for numFeatures, categoryMaps + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (categoryMaps.contains(featureIndex)) { + // categorical feature + val featureValues: Array[String] = + categoryMaps(featureIndex).toArray.sortBy(_._1).map(_._1).map(_.toString) + if (featureValues.length == 2) { + attrs(featureIndex) = new BinaryAttribute(index = Some(featureIndex), + values = Some(featureValues)) + } else { + attrs(featureIndex) = new NominalAttribute(index = Some(featureIndex), + isOrdinal = Some(false), values = Some(featureValues)) + } + categoricalFeatureCount += 1 + } else { + // continuous feature + attrs(featureIndex) = new NumericAttribute(index = Some(featureIndex)) + } + featureIndex += 1 + } + require(categoricalFeatureCount == categoryMaps.size, "VectorIndexerModel given categoryMaps" + + s" with keys outside expected range [0,...,numFeatures), where numFeatures=$numFeatures") + attrs + } + + // TODO: Check more carefully about whether this whole class will be included in a closure. + + private val transformFunc: Vector => Vector = { + val sortedCategoricalFeatureIndices = categoryMaps.keys.toArray.sorted + val localVectorMap = categoryMaps + val f: Vector => Vector = { + case dv: DenseVector => + val tmpv = dv.copy + localVectorMap.foreach { case (featureIndex: Int, categoryMap: Map[Double, Int]) => + tmpv.values(featureIndex) = categoryMap(tmpv(featureIndex)) + } + tmpv + case sv: SparseVector => + // We use the fact that categorical value 0 is always mapped to index 0. + val tmpv = sv.copy + var catFeatureIdx = 0 // index into sortedCategoricalFeatureIndices + var k = 0 // index into non-zero elements of sparse vector + while (catFeatureIdx < sortedCategoricalFeatureIndices.length && k < tmpv.indices.length) { + val featureIndex = sortedCategoricalFeatureIndices(catFeatureIdx) + if (featureIndex < tmpv.indices(k)) { + catFeatureIdx += 1 + } else if (featureIndex > tmpv.indices(k)) { + k += 1 + } else { + tmpv.values(k) = localVectorMap(featureIndex)(tmpv.values(k)) + catFeatureIdx += 1 + k += 1 + } + } + tmpv + } + f + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val newField = prepOutputField(dataset.schema, map) + val newCol = callUDF(transformFunc, new VectorUDT, dataset(map(inputCol))) + // For now, just check the first row of inputCol for vector length. + val firstRow = dataset.select(map(inputCol)).take(1) + if (firstRow.length != 0) { + val actualNumFeatures = firstRow(0).getAs[Vector](0).size + require(numFeatures == actualNumFeatures, "VectorIndexerModel expected vector of length" + + s" $numFeatures but found length $actualNumFeatures") + } + dataset.withColumn(map(outputCol), newCol.as(map(outputCol), newField.metadata)) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + val dataType = new VectorUDT + require(map.contains(inputCol), + s"VectorIndexerModel requires input column parameter: $inputCol") + require(map.contains(outputCol), + s"VectorIndexerModel requires output column parameter: $outputCol") + checkInputColumn(schema, map(inputCol), dataType) + + val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) + val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { + Some(origAttrGroup.attributes.get.length) + } else { + origAttrGroup.numAttributes + } + require(origNumFeatures.forall(_ == numFeatures), "VectorIndexerModel expected" + + s" $numFeatures features, but input column ${map(inputCol)} had metadata specifying" + + s" ${origAttrGroup.numAttributes.get} features.") + + val newField = prepOutputField(schema, map) + val outputFields = schema.fields :+ newField + StructType(outputFields) + } + + /** + * Prepare the output column field, including per-feature metadata. + * @param schema Input schema + * @param map Parameter map (with this class' embedded parameter map folded in) + * @return Output column field + */ + private def prepOutputField(schema: StructType, map: ParamMap): StructField = { + val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) + val featureAttributes: Array[Attribute] = if (origAttrGroup.attributes.nonEmpty) { + // Convert original attributes to modified attributes + val origAttrs: Array[Attribute] = origAttrGroup.attributes.get + origAttrs.zip(partialFeatureAttributes).map { + case (origAttr: Attribute, featAttr: BinaryAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NominalAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NumericAttribute) => + origAttr.withIndex(featAttr.index.get) + } + } else { + partialFeatureAttributes + } + val newAttributeGroup = new AttributeGroup(map(outputCol), featureAttributes) + newAttributeGroup.toStructField(schema(map(inputCol)).metadata) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 17ece897a6c55..7d5178d0abb2d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -198,23 +198,31 @@ trait Params extends Identifiable with Serializable { /** * Check whether the given schema contains an input column. - * @param colName Parameter name for the input column. - * @param dataType SQL DataType of the input column. + * @param colName Input column name + * @param dataType Input column DataType */ protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), - s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + require(actualDataType.equals(dataType), s"Input column $colName must be of type $dataType" + + s" but was actually $actualDataType. Column param description: ${getParam(colName)}") } + /** + * Add an output column to the given schema. + * This fails if the given output column already exists. + * @param schema Initial schema (not modified) + * @param colName Output column name. If this column name is an empy String "", this method + * returns the initial schema, unchanged. This allows users to disable output + * columns. + * @param dataType Output column DataType + */ protected def addOutputColumn( schema: StructType, colName: String, dataType: DataType): StructType = { if (colName.length == 0) return schema val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Prediction column $colName already exists.") + require(!fieldNames.contains(colName), s"Output column $colName already exists.") val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) StructType(outputFields) } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java new file mode 100644 index 0000000000000..161100134c92d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -0,0 +1,70 @@ +/* + * 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.feature; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; + + +public class JavaVectorIndexerSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaVectorIndexerSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void vectorIndexerAPI() { + // The tests are to check Java compatibility. + List points = Lists.newArrayList( + new FeatureData(Vectors.dense(0.0, -2.0)), + new FeatureData(Vectors.dense(1.0, 3.0)), + new FeatureData(Vectors.dense(1.0, 4.0)) + ); + SQLContext sqlContext = new SQLContext(sc); + DataFrame data = sqlContext.createDataFrame(sc.parallelize(points, 2), FeatureData.class); + VectorIndexer indexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexed") + .setMaxCategories(2); + VectorIndexerModel model = indexer.fit(data); + Assert.assertEquals(model.numFeatures(), 2); + Assert.assertEquals(model.categoryMaps().size(), 1); + DataFrame indexedData = model.transform(data); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala index 3fb6e2ec46468..0dcfe5a2002dc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -43,8 +43,8 @@ class AttributeGroupSuite extends FunSuite { intercept[NoSuchElementException] { group("abc") } - assert(group === AttributeGroup.fromMetadata(group.toMetadata, group.name)) - assert(group === AttributeGroup.fromStructField(group.toStructField())) + assert(group === AttributeGroup.fromMetadata(group.toMetadataImpl, group.name)) + assert(group === AttributeGroup.fromStructField(group.toStructField)) } test("attribute group without attributes") { @@ -53,8 +53,8 @@ class AttributeGroupSuite extends FunSuite { assert(group0.numAttributes === Some(10)) assert(group0.size === 10) assert(group0.attributes.isEmpty) - assert(group0 === AttributeGroup.fromMetadata(group0.toMetadata, group0.name)) - assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) + assert(group0 === AttributeGroup.fromMetadata(group0.toMetadataImpl, group0.name)) + assert(group0 === AttributeGroup.fromStructField(group0.toStructField)) val group1 = new AttributeGroup("item") assert(group1.name === "item") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index a18c335952b96..9d09f24709e23 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} -private case class DataSet(features: Vector) class NormalizerSuite extends FunSuite with MLlibTestSparkContext { @@ -63,7 +62,7 @@ class NormalizerSuite extends FunSuite with MLlibTestSparkContext { ) val sqlContext = new SQLContext(sc) - dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(DataSet)) + dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normalized_features") @@ -107,3 +106,7 @@ class NormalizerSuite extends FunSuite with MLlibTestSparkContext { assertValues(result, l1Normalized) } } + +private object NormalizerSuite { + case class FeatureData(features: Vector) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala new file mode 100644 index 0000000000000..61c46c85a78b5 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -0,0 +1,255 @@ +/* + * 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.feature + +import scala.beans.{BeanInfo, BeanProperty} + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.util.TestingUtils +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} + + +class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { + + import VectorIndexerSuite.FeatureData + + @transient var sqlContext: SQLContext = _ + + // identical, of length 3 + @transient var densePoints1: DataFrame = _ + @transient var sparsePoints1: DataFrame = _ + @transient var point1maxes: Array[Double] = _ + + // identical, of length 2 + @transient var densePoints2: DataFrame = _ + @transient var sparsePoints2: DataFrame = _ + + // different lengths + @transient var badPoints: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val densePoints1Seq = Seq( + Vectors.dense(1.0, 2.0, 0.0), + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(0.0, 0.0, -1.0), + Vectors.dense(1.0, 3.0, 2.0)) + val sparsePoints1Seq = Seq( + Vectors.sparse(3, Array(0, 1), Array(1.0, 2.0)), + Vectors.sparse(3, Array(1, 2), Array(1.0, 2.0)), + Vectors.sparse(3, Array(2), Array(-1.0)), + Vectors.sparse(3, Array(0, 1, 2), Array(1.0, 3.0, 2.0))) + point1maxes = Array(1.0, 3.0, 2.0) + + val densePoints2Seq = Seq( + Vectors.dense(1.0, 1.0, 0.0, 1.0), + Vectors.dense(0.0, 1.0, 1.0, 1.0), + Vectors.dense(-1.0, 1.0, 2.0, 0.0)) + val sparsePoints2Seq = Seq( + Vectors.sparse(4, Array(0, 1, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(1, 2, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(0, 1, 2), Array(-1.0, 1.0, 2.0))) + + val badPointsSeq = Seq( + Vectors.sparse(2, Array(0, 1), Array(1.0, 1.0)), + Vectors.sparse(3, Array(2), Array(-1.0))) + + // Sanity checks for assumptions made in tests + assert(densePoints1Seq.head.size == sparsePoints1Seq.head.size) + assert(densePoints2Seq.head.size == sparsePoints2Seq.head.size) + assert(densePoints1Seq.head.size != densePoints2Seq.head.size) + def checkPair(dvSeq: Seq[Vector], svSeq: Seq[Vector]): Unit = { + assert(dvSeq.zip(svSeq).forall { case (dv, sv) => dv.toArray === sv.toArray }, + "typo in unit test") + } + checkPair(densePoints1Seq, sparsePoints1Seq) + checkPair(densePoints2Seq, sparsePoints2Seq) + + sqlContext = new SQLContext(sc) + densePoints1 = sqlContext.createDataFrame(sc.parallelize(densePoints1Seq, 2).map(FeatureData)) + sparsePoints1 = sqlContext.createDataFrame(sc.parallelize(sparsePoints1Seq, 2).map(FeatureData)) + densePoints2 = sqlContext.createDataFrame(sc.parallelize(densePoints2Seq, 2).map(FeatureData)) + sparsePoints2 = sqlContext.createDataFrame(sc.parallelize(sparsePoints2Seq, 2).map(FeatureData)) + badPoints = sqlContext.createDataFrame(sc.parallelize(badPointsSeq, 2).map(FeatureData)) + } + + private def getIndexer: VectorIndexer = + new VectorIndexer().setInputCol("features").setOutputCol("indexed") + + test("Cannot fit an empty DataFrame") { + val rdd = sqlContext.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) + val vectorIndexer = getIndexer + intercept[IllegalArgumentException] { + vectorIndexer.fit(rdd) + } + } + + test("Throws error when given RDDs with different size vectors") { + val vectorIndexer = getIndexer + val model = vectorIndexer.fit(densePoints1) // vectors of length 3 + model.transform(densePoints1) // should work + model.transform(sparsePoints1) // should work + intercept[IllegalArgumentException] { + model.transform(densePoints2) + println("Did not throw error when fit, transform were called on vectors of different lengths") + } + intercept[SparkException] { + vectorIndexer.fit(badPoints) + println("Did not throw error when fitting vectors of different lengths in same RDD.") + } + } + + test("Same result with dense and sparse vectors") { + def testDenseSparse(densePoints: DataFrame, sparsePoints: DataFrame): Unit = { + val denseVectorIndexer = getIndexer.setMaxCategories(2) + val sparseVectorIndexer = getIndexer.setMaxCategories(2) + val denseModel = denseVectorIndexer.fit(densePoints) + val sparseModel = sparseVectorIndexer.fit(sparsePoints) + val denseMap = denseModel.categoryMaps + val sparseMap = sparseModel.categoryMaps + assert(denseMap.keys.toSet == sparseMap.keys.toSet, + "Categorical features chosen from dense vs. sparse vectors did not match.") + assert(denseMap == sparseMap, + "Categorical feature value indexes chosen from dense vs. sparse vectors did not match.") + } + testDenseSparse(densePoints1, sparsePoints1) + testDenseSparse(densePoints2, sparsePoints2) + } + + test("Builds valid categorical feature value index, transform correctly, check metadata") { + def checkCategoryMaps( + data: DataFrame, + maxCategories: Int, + categoricalFeatures: Set[Int]): Unit = { + val collectedData = data.collect().map(_.getAs[Vector](0)) + val errMsg = s"checkCategoryMaps failed for input with maxCategories=$maxCategories," + + s" categoricalFeatures=${categoricalFeatures.mkString(", ")}" + try { + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val categoryMaps = model.categoryMaps + assert(categoryMaps.keys.toSet === categoricalFeatures) // Chose correct categorical features + val transformed = model.transform(data).select("indexed") + val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) + val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) + assert(featureAttrs.name === "indexed") + assert(featureAttrs.attributes.get.length === model.numFeatures) + categoricalFeatures.foreach { feature: Int => + val origValueSet = collectedData.map(_(feature)).toSet + val targetValueIndexSet = Range(0, origValueSet.size).toSet + val catMap = categoryMaps(feature) + assert(catMap.keys.toSet === origValueSet) // Correct categories + assert(catMap.values.toSet === targetValueIndexSet) // Correct category indices + if (origValueSet.contains(0.0)) { + assert(catMap(0.0) === 0) // value 0 gets index 0 + } + // Check transformed data + assert(indexedRDD.map(_(feature)).collect().toSet === targetValueIndexSet) + // Check metadata + val featureAttr = featureAttrs(feature) + assert(featureAttr.index.get === feature) + featureAttr match { + case attr: BinaryAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + case attr: NominalAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + assert(attr.isOrdinal.get === false) + case _ => + throw new RuntimeException(errMsg + s". Categorical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + // Check numerical feature metadata. + Range(0, model.numFeatures).filter(feature => !categoricalFeatures.contains(feature)) + .foreach { feature: Int => + val featureAttr = featureAttrs(feature) + featureAttr match { + case attr: NumericAttribute => + assert(featureAttr.index.get === feature) + case _ => + throw new RuntimeException(errMsg + s". Numerical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + } catch { + case e: org.scalatest.exceptions.TestFailedException => + println(errMsg) + throw e + } + } + checkCategoryMaps(densePoints1, maxCategories = 2, categoricalFeatures = Set(0)) + checkCategoryMaps(densePoints1, maxCategories = 3, categoricalFeatures = Set(0, 2)) + checkCategoryMaps(densePoints2, maxCategories = 2, categoricalFeatures = Set(1, 3)) + } + + test("Maintain sparsity for sparse vectors") { + def checkSparsity(data: DataFrame, maxCategories: Int): Unit = { + val points = data.collect().map(_.getAs[Vector](0)) + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val indexedPoints = model.transform(data).select("indexed").map(_.getAs[Vector](0)).collect() + points.zip(indexedPoints).foreach { + case (orig: SparseVector, indexed: SparseVector) => + assert(orig.indices.length == indexed.indices.length) + case _ => throw new UnknownError("Unit test has a bug in it.") // should never happen + } + } + checkSparsity(sparsePoints1, maxCategories = 2) + checkSparsity(sparsePoints2, maxCategories = 2) + } + + test("Preserve metadata") { + // For continuous features, preserve name and stats. + val featureAttributes: Array[Attribute] = point1maxes.zipWithIndex.map { case (maxVal, i) => + NumericAttribute.defaultAttr.withName(i.toString).withMax(maxVal) + } + val attrGroup = new AttributeGroup("features", featureAttributes) + val densePoints1WithMeta = + densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata)) + val vectorIndexer = getIndexer.setMaxCategories(2) + val model = vectorIndexer.fit(densePoints1WithMeta) + // Check that ML metadata are preserved. + val indexedPoints = model.transform(densePoints1WithMeta) + val transAttributes: Array[Attribute] = + AttributeGroup.fromStructField(indexedPoints.schema("indexed")).attributes.get + featureAttributes.zip(transAttributes).foreach { case (orig, trans) => + assert(orig.name === trans.name) + (orig, trans) match { + case (orig: NumericAttribute, trans: NumericAttribute) => + assert(orig.max.nonEmpty && orig.max === trans.max) + case _ => + // do nothing + // TODO: Once input features marked as categorical are handled correctly, check that here. + } + } + // Check that non-ML metadata are preserved. + TestingUtils.testPreserveMetadata(densePoints1WithMeta, model, "features", "indexed") + } +} + +private[feature] object VectorIndexerSuite { + @BeanInfo + case class FeatureData(@BeanProperty features: Vector) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala new file mode 100644 index 0000000000000..c44cb61b34171 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.ml.Transformer +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.MetadataBuilder +import org.scalatest.FunSuite + +private[ml] object TestingUtils extends FunSuite { + + /** + * Test whether unrelated metadata are preserved for this transformer. + * This attaches extra metadata to a column, transforms the column, and check to ensure the + * extra metadata have not changed. + * @param data Input dataset + * @param transformer Transformer to test + * @param inputCol Unique input column for Transformer. This must be the ONLY input column. + * @param outputCol Output column to test for metadata presence. + */ + def testPreserveMetadata( + data: DataFrame, + transformer: Transformer, + inputCol: String, + outputCol: String): Unit = { + // Create some fake metadata + val origMetadata = data.schema(inputCol).metadata + val metaKey = "__testPreserveMetadata__fake_key" + val metaValue = 12345 + assert(!origMetadata.contains(metaKey), + s"Unit test with testPreserveMetadata will fail since metadata key was present: $metaKey") + val newMetadata = + new MetadataBuilder().withMetadata(origMetadata).putLong(metaKey, metaValue).build() + // Add metadata to the inputCol + val withMetadata = data.select(data(inputCol).as(inputCol, newMetadata)) + // Transform, and ensure extra metadata was not affected + val transformed = transformer.transform(withMetadata) + val transMetadata = transformed.schema(outputCol).metadata + assert(transMetadata.contains(metaKey), + "Unit test with testPreserveMetadata failed; extra metadata key was not present.") + assert(transMetadata.getLong(metaKey) === metaValue, + "Unit test with testPreserveMetadata failed; extra metadata value was wrong." + + s" Expected $metaValue but found ${transMetadata.getLong(metaKey)}") + } +} From 685ddcf5253c0ecb39853802431e22b0c7b61dee Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 12 Apr 2015 22:41:05 -0700 Subject: [PATCH 725/817] [SPARK-5886][ML] Add StringIndexer as a feature transformer This PR adds string indexer, which takes a column of string labels and outputs a double column with labels indexed by their frequency. TODOs: - [x] store feature to index map in output metadata Author: Xiangrui Meng Closes #4735 from mengxr/SPARK-5886 and squashes the following commits: d82575f [Xiangrui Meng] fix test 700e70f [Xiangrui Meng] rename LabelIndexer to StringIndexer 16a6f8c [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5886 457166e [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5886 f8b30f4 [Xiangrui Meng] update label indexer to output metadata e81ec28 [Xiangrui Meng] Merge branch 'openhashmap-contains' into SPARK-5886-2 d6e6f1f [Xiangrui Meng] add contains to primitivekeyopenhashmap 748a69b [Xiangrui Meng] add contains to OpenHashMap def3c5c [Xiangrui Meng] add LabelIndexer --- .../spark/ml/feature/StringIndexer.scala | 126 ++++++++++++++++++ .../spark/ml/feature/StringIndexerSuite.scala | 52 ++++++++ 2 files changed, 178 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala new file mode 100644 index 0000000000000..61e6742e880d8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -0,0 +1,126 @@ +/* + * 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.feature + +import org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.param._ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.collection.OpenHashMap + +/** + * Base trait for [[StringIndexer]] and [[StringIndexerModel]]. + */ +private[feature] trait StringIndexerBase extends Params with HasInputCol with HasOutputCol { + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + checkInputColumn(schema, map(inputCol), StringType) + val inputFields = schema.fields + val outputColName = map(outputCol) + require(inputFields.forall(_.name != outputColName), + s"Output column $outputColName already exists.") + val attr = NominalAttribute.defaultAttr.withName(map(outputCol)) + val outputFields = inputFields :+ attr.toStructField() + StructType(outputFields) + } +} + +/** + * :: AlphaComponent :: + * A label indexer that maps a string column of labels to an ML column of label indices. + * The indices are in [0, numLabels), ordered by label frequencies. + * So the most frequent label gets index 0. + */ +@AlphaComponent +class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + // TODO: handle unseen labels + + override def fit(dataset: DataFrame, paramMap: ParamMap): StringIndexerModel = { + val map = this.paramMap ++ paramMap + val counts = dataset.select(map(inputCol)).map(_.getString(0)).countByValue() + val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray + val model = new StringIndexerModel(this, map, labels) + Params.inheritValues(map, this, model) + model + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap) + } +} + +/** + * :: AlphaComponent :: + * Model fitted by [[StringIndexer]]. + */ +@AlphaComponent +class StringIndexerModel private[ml] ( + override val parent: StringIndexer, + override val fittingParamMap: ParamMap, + labels: Array[String]) extends Model[StringIndexerModel] with StringIndexerBase { + + private val labelToIndex: OpenHashMap[String, Double] = { + val n = labels.length + val map = new OpenHashMap[String, Double](n) + var i = 0 + while (i < n) { + map.update(labels(i), i) + i += 1 + } + map + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + val map = this.paramMap ++ paramMap + val indexer = udf { label: String => + if (labelToIndex.contains(label)) { + labelToIndex(label) + } else { + // TODO: handle unseen labels + throw new SparkException(s"Unseen label: $label.") + } + } + val outputColName = map(outputCol) + val metadata = NominalAttribute.defaultAttr + .withName(outputColName).withValues(labels).toStructField().metadata + dataset.select(col("*"), indexer(dataset(map(inputCol))).as(outputColName, metadata)) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala new file mode 100644 index 0000000000000..00b5d094d82f1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -0,0 +1,52 @@ +/* + * 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.feature + +import org.scalatest.FunSuite + +import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.SQLContext + +class StringIndexerSuite extends FunSuite with MLlibTestSparkContext { + private var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("StringIndexer") { + val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) + val df = sqlContext.createDataFrame(data).toDF("id", "label") + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + .fit(df) + val transformed = indexer.transform(df) + val attr = Attribute.fromStructField(transformed.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attr.values.get === Array("a", "c", "b")) + val output = transformed.select("id", "labelIndex").map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 0, b -> 2, c -> 1 + val expected = Set((0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)) + assert(output === expected) + } +} From 929404498506c34180e2eaaa1a4d4a3c4ed51daa Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 12 Apr 2015 22:42:01 -0700 Subject: [PATCH 726/817] [SPARK-5885][MLLIB] Add VectorAssembler as a feature transformer VectorAssembler merges multiple columns into a vector column. This PR contains content from #5195. ~~carry ML attributes~~ (moved to a follow-up PR) Author: Xiangrui Meng Closes #5196 from mengxr/SPARK-5885 and squashes the following commits: a52b101 [Xiangrui Meng] recognize more types 35daac2 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5885 bb5e64b [Xiangrui Meng] add TODO for null 976a3d6 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5885 0859311 [Xiangrui Meng] Revert "add CreateStruct" 29fb6ac [Xiangrui Meng] use CreateStruct adb71c4 [Xiangrui Meng] Merge branch 'SPARK-6542' into SPARK-5885 85f3106 [Xiangrui Meng] add CreateStruct 4ff16ce [Xiangrui Meng] add VectorAssembler --- .../org/apache/spark/ml/Identifiable.scala | 2 +- .../spark/ml/feature/VectorAssembler.scala | 111 ++++++++++++++++++ .../apache/spark/ml/param/sharedParams.scala | 10 ++ .../ml/feature/VectorAssemblerSuite.scala | 63 ++++++++++ 4 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala index cd84b05bfb496..a50090671ae48 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala @@ -29,5 +29,5 @@ private[ml] trait Identifiable extends Serializable { * random hex chars. */ private[ml] val uid: String = - this.getClass.getSimpleName + "-" + UUID.randomUUID().toString.take(8) + this.getClass.getSimpleName + "_" + UUID.randomUUID().toString.take(8) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala new file mode 100644 index 0000000000000..d1b8f7e6e9295 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -0,0 +1,111 @@ +/* + * 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.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.param.{HasInputCols, HasOutputCol, ParamMap} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, CreateStruct} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * :: AlphaComponent :: + * A feature transformer than merge multiple columns into a vector column. + */ +@AlphaComponent +class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { + + /** @group setParam */ + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + val map = this.paramMap ++ paramMap + val assembleFunc = udf { r: Row => + VectorAssembler.assemble(r.toSeq: _*) + } + val schema = dataset.schema + val inputColNames = map(inputCols) + val args = inputColNames.map { c => + schema(c).dataType match { + case DoubleType => UnresolvedAttribute(c) + case t if t.isInstanceOf[VectorUDT] => UnresolvedAttribute(c) + case _: NativeType => Alias(Cast(UnresolvedAttribute(c), DoubleType), s"${c}_double_$uid")() + } + } + dataset.select(col("*"), assembleFunc(new Column(CreateStruct(args))).as(map(outputCol))) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + val inputColNames = map(inputCols) + val outputColName = map(outputCol) + val inputDataTypes = inputColNames.map(name => schema(name).dataType) + inputDataTypes.foreach { + case _: NativeType => + case t if t.isInstanceOf[VectorUDT] => + case other => + throw new IllegalArgumentException(s"Data type $other is not supported.") + } + if (schema.fieldNames.contains(outputColName)) { + throw new IllegalArgumentException(s"Output column $outputColName already exists.") + } + StructType(schema.fields :+ new StructField(outputColName, new VectorUDT, false)) + } +} + +@AlphaComponent +object VectorAssembler { + + private[feature] def assemble(vv: Any*): Vector = { + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + var cur = 0 + vv.foreach { + case v: Double => + if (v != 0.0) { + indices += cur + values += v + } + cur += 1 + case vec: Vector => + vec.foreachActive { case (i, v) => + if (v != 0.0) { + indices += cur + i + values += v + } + } + cur += vec.size + case null => + // TODO: output Double.NaN? + throw new SparkException("Values to assemble cannot be null.") + case o => + throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") + } + Vectors.sparse(cur, indices.result(), values.result()) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 0739fdbfcbaae..07e6eb417763d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -140,6 +140,16 @@ private[ml] trait HasInputCol extends Params { def getInputCol: String = get(inputCol) } +private[ml] trait HasInputCols extends Params { + /** + * Param for input column names. + */ + val inputCols: Param[Array[String]] = new Param(this, "inputCols", "input column names") + + /** @group getParam */ + def getInputCols: Array[String] = get(inputCols) +} + private[ml] trait HasOutputCol extends Params { /** * param for output column name diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala new file mode 100644 index 0000000000000..57d0278e03639 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Row, SQLContext} + +class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("assemble") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + assert(assemble(0.0) === Vectors.sparse(1, Array.empty, Array.empty)) + assert(assemble(0.0, 1.0) === Vectors.sparse(2, Array(1), Array(1.0))) + val dv = Vectors.dense(2.0, 0.0) + assert(assemble(0.0, dv, 1.0) === Vectors.sparse(4, Array(1, 3), Array(2.0, 1.0))) + val sv = Vectors.sparse(2, Array(0, 1), Array(3.0, 4.0)) + assert(assemble(0.0, dv, 1.0, sv) === + Vectors.sparse(6, Array(1, 3, 4, 5), Array(2.0, 1.0, 3.0, 4.0))) + for (v <- Seq(1, "a", null)) { + intercept[SparkException](assemble(v)) + intercept[SparkException](assemble(1.0, v)) + } + } + + test("VectorAssembler") { + val df = sqlContext.createDataFrame(Seq( + (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) + )).toDF("id", "x", "y", "name", "z", "n") + val assembler = new VectorAssembler() + .setInputCols(Array("x", "y", "z", "n")) + .setOutputCol("features") + assembler.transform(df).select("features").collect().foreach { + case Row(v: Vector) => + assert(v === Vectors.sparse(6, Array(1, 2, 4, 5), Array(1.0, 2.0, 3.0, 10.0))) + } + } +} From 68d1faa3c04e9412bbc2b60421dc12bd19c396b2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 12 Apr 2015 22:56:12 -0700 Subject: [PATCH 727/817] [SPARK-6562][SQL] DataFrame.replace Supports replacing values with other values in DataFrames. Python support should be in a separate pull request. Author: Reynold Xin Closes #5282 from rxin/df-na-replace and squashes the following commits: 4b72434 [Reynold Xin] Removed println. c8d9946 [Reynold Xin] col -> cols fbb3c21 [Reynold Xin] [SPARK-6562][SQL] DataFrame.replace --- .../spark/sql/DataFrameNaFunctions.scala | 144 ++++++++++++++++++ .../spark/sql/DataFrameNaFunctionsSuite.scala | 34 +++++ 2 files changed, 178 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index bf3c3fe876873..481ed4924857e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -192,6 +192,127 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { */ def fill(valueMap: Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", ImmutableMap.of("UNKNOWN", "unnamed")); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: java.util.Map[T, T]): DataFrame = { + replace[T](col, replacement.toMap : Map[T, T]) + } + + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace(new String[] {"height", "weight"}, ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace(new String[] {"firstname", "lastname"}, ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { + replace(cols.toSeq, replacement.toMap) + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", Map(1.0 -> 2.0)) + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", Map("UNKNOWN" -> "unnamed") + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", Map("UNKNOWN" -> "unnamed") + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: Map[T, T]): DataFrame = { + if (col == "*") { + replace0(df.columns, replacement) + } else { + replace0(Seq(col), replacement) + } + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace("height" :: "weight" :: Nil, Map(1.0 -> 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace("firstname" :: "lastname" :: Nil, Map("UNKNOWN" -> "unnamed"); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = replace0(cols, replacement) + + private def replace0[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = { + if (replacement.isEmpty || cols.isEmpty) { + return df + } + + // replacementMap is either Map[String, String] or Map[Double, Double] + val replacementMap: Map[_, _] = replacement.head._2 match { + case v: String => replacement + case _ => replacement.map { case (k, v) => (convertToDouble(k), convertToDouble(v)) } + } + + // targetColumnType is either DoubleType or StringType + val targetColumnType = replacement.head._1 match { + case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long => DoubleType + case _: String => StringType + } + + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + val shouldReplace = cols.exists(colName => columnEquals(colName, f.name)) + if (f.dataType.isInstanceOf[NumericType] && targetColumnType == DoubleType && shouldReplace) { + replaceCol(f, replacementMap) + } else if (f.dataType == targetColumnType && shouldReplace) { + replaceCol(f, replacementMap) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + private def fill0(values: Seq[(String, Any)]): DataFrame = { // Error handling values.foreach { case (colName, replaceValue) => @@ -228,4 +349,27 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { private def fillCol[T](col: StructField, replacement: T): Column = { coalesce(df.col(col.name), lit(replacement).cast(col.dataType)).as(col.name) } + + /** + * Returns a [[Column]] expression that replaces value matching key in `replacementMap` with + * value in `replacementMap`, using [[CaseWhen]]. + * + * TODO: This can be optimized to use broadcast join when replacementMap is large. + */ + private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { + val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => + df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: + lit(target).cast(col.dataType).expr :: Nil + }.toSeq + new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + } + + private def convertToDouble(v: Any): Double = v match { + case v: Float => v.toDouble + case v: Double => v + case v: Long => v.toDouble + case v: Int => v.toDouble + case v => throw new IllegalArgumentException( + s"Unsupported value type ${v.getClass.getName} ($v).") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 0896f175c056f..41b4f02e6a294 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -154,4 +154,38 @@ class DataFrameNaFunctionsSuite extends QueryTest { ))), Row("test", null, 1, 2.2)) } + + test("replace") { + val input = createDF() + + // Replace two numeric columns: age and height + val out = input.na.replace(Seq("age", "height"), Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 461.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + + // Replace only the age column + val out1 = input.na.replace("age", Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out1, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 164.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + } } From 950645d597dbc5a8c5010bcb1a9b51c6abad86ea Mon Sep 17 00:00:00 2001 From: Dean Chen Date: Mon, 13 Apr 2015 12:08:55 +0100 Subject: [PATCH 728/817] [SPARK-6868][YARN] Fix broken container log link on executor page when HTTPS_ONLY. Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY. Uses the same logic as the YARN jobtracker webapp. Entry point is [JobBlock](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java#L108) and logic is in [MRWebAppUtil](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java#L75). I chose to migrate the logic over instead of importing MRWebAppUtil(but can update the PR to do so) since the class is designated as private and the logic was straightforward. Author: Dean Chen Closes #5477 from deanchen/container-url and squashes the following commits: 91d3090 [Dean Chen] Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY. --- .../apache/spark/deploy/yarn/ExecutorRunnable.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 1ce10d906ab23..b06069c07f451 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -290,10 +290,19 @@ class ExecutorRunnable( YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) } + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + // Add log urls sys.env.get("SPARK_USER").foreach { user => - val baseUrl = "http://%s/node/containerlogs/%s/%s" - .format(container.getNodeHttpAddress, ConverterUtils.toString(container.getId), user) + val containerId = ConverterUtils.toString(container.getId) + val address = container.getNodeHttpAddress + val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0" env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0" } From cadd7d72c52ccc8d2def405a77dcf807fb5c17c2 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Mon, 13 Apr 2015 12:18:05 +0100 Subject: [PATCH 729/817] [SPARK-6762]Fix potential resource leaks in CheckPoint CheckpointWriter and CheckpointReader The close action should be placed within finally block to avoid the potential resource leaks Author: lisurprise Closes #5407 from zhichao-li/master and squashes the following commits: 065999f [lisurprise] add guard for null ef862d6 [lisurprise] remove fs.close a754adc [lisurprise] refactor with tryWithSafeFinally 824adb3 [lisurprise] close before validation c877da7 [lisurprise] Fix potential resource leaks --- .../apache/spark/streaming/Checkpoint.scala | 47 ++++++++++++------- .../spark/streaming/util/RawTextSender.scala | 3 +- 2 files changed, 31 insertions(+), 19 deletions(-) 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 28703ef8129b3..0a50485118588 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkException, SparkConf, Logging} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.MetadataCleaner +import org.apache.spark.util.{MetadataCleaner, Utils} import org.apache.spark.streaming.scheduler.JobGenerator @@ -139,8 +139,11 @@ class CheckpointWriter( // Write checkpoint to temp file fs.delete(tempFile, true) // just in case it exists val fos = fs.create(tempFile) - fos.write(bytes) - fos.close() + Utils.tryWithSafeFinally { + fos.write(bytes) + } { + fos.close() + } // If the checkpoint file exists, back it up // If the backup exists as well, just delete it, otherwise rename will fail @@ -187,9 +190,11 @@ class CheckpointWriter( val bos = new ByteArrayOutputStream() val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) - oos.writeObject(checkpoint) - oos.close() - bos.close() + Utils.tryWithSafeFinally { + oos.writeObject(checkpoint) + } { + oos.close() + } try { executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) @@ -248,18 +253,24 @@ object CheckpointReader extends Logging { checkpointFiles.foreach(file => { logInfo("Attempting to load checkpoint from file " + file) try { - 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) - val ois = new ObjectInputStreamWithLoader(zis, - Thread.currentThread().getContextClassLoader) - val cp = ois.readObject.asInstanceOf[Checkpoint] - ois.close() - fs.close() + 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() logInfo("Checkpoint successfully loaded from file " + file) logInfo("Checkpoint was generated at time " + cp.checkpointTime) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index a7850812bd612..ca2f319f174a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -72,7 +72,8 @@ object RawTextSender extends Logging { } catch { case e: IOException => logError("Client disconnected") - socket.close() + } finally { + socket.close() } } } From 14ce3ea2c9546c58203af85aceb76b1bfc1f650a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 13 Apr 2015 12:21:29 +0100 Subject: [PATCH 730/817] [SPARK-6860][Streaming][WebUI] Fix the possible inconsistency of StreamingPage Because `StreamingPage.render` doesn't hold the `listener` lock when generating the content, the different parts of content may have some inconsistent values if `listener` updates its status at the same time. And it will confuse people. This PR added `listener.synchronized` to make sure we have a consistent view of StreamingJobProgressListener when creating the content. Author: zsxwing Closes #5470 from zsxwing/SPARK-6860 and squashes the following commits: cec6f92 [zsxwing] Add missing 'synchronized' in StreamingJobProgressListener 7182498 [zsxwing] Add synchronized to make sure we have a consistent view of StreamingJobProgressListener when creating the content --- .../streaming/ui/StreamingJobProgressListener.scala | 10 +++++----- .../org/apache/spark/streaming/ui/StreamingPage.scala | 3 ++- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 84f80e638f638..be1e8686cf9fa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -149,7 +149,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) }.toMap } - def lastReceivedBatchRecords: Map[Int, Long] = { + def lastReceivedBatchRecords: Map[Int, Long] = synchronized { val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => (0 until numReceivers).map { receiverId => @@ -160,19 +160,19 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } - def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = synchronized { receiverInfos.get(receiverId) } - def lastCompletedBatch: Option[BatchInfo] = { + def lastCompletedBatch: Option[BatchInfo] = synchronized { completedBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption } - def lastReceivedBatch: Option[BatchInfo] = { + def lastReceivedBatch: Option[BatchInfo] = synchronized { retainedBatches.lastOption } - private def retainedBatches: Seq[BatchInfo] = synchronized { + private def retainedBatches: Seq[BatchInfo] = { (waitingBatchInfos.values.toSeq ++ runningBatchInfos.values.toSeq ++ completedBatchInfos).sortBy(_.batchTime)(Time.ordering) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index bfe8086fcf8fe..b6dcb62bfeec8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -37,11 +37,12 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = + val content = listener.synchronized { generateBasicStats() ++

    ++

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

    ++ generateReceiverStats() ++ generateBatchStatsTable() + } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } From 9d117cee0be2c73a25702d98f78211055d50babe Mon Sep 17 00:00:00 2001 From: nyaapa Date: Mon, 13 Apr 2015 12:55:25 +0100 Subject: [PATCH 731/817] [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI Author: nyaapa Closes #5424 from nyaapa/master and squashes the following commits: 6b717aa [nyaapa] [SPARK-6440][CORE] Remove Utils.localIpAddressHostname, Utils.localIpAddressURI and Utils.getAddressHostName; make Utils.localIpAddress private; rename Utils.localHostURI into Utils.localHostNameForURI; use Utils.localHostName in org.apache.spark.streaming.kinesis.KinesisReceiver and org.apache.spark.sql.hive.thriftserver.SparkSQLEnv 2098081 [nyaapa] [SPARK-6440][CORE] style fixes and use getHostAddress instead of getHostName 84763d7 [nyaapa] [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI --- .../scala/org/apache/spark/HttpServer.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 2 +- .../spark/deploy/client/TestClient.scala | 2 +- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 34 ++++++++++++------- .../streaming/kinesis/KinesisReceiver.scala | 3 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 3 +- 7 files changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 09a9ccc226721..8de3a6c04df34 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -160,7 +160,7 @@ private[spark] class HttpServer( throw new ServerStateException("Server is not started") } else { val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" - s"$scheme://${Utils.localIpAddress}:$port" + s"$scheme://${Utils.localHostNameForURI()}:$port" } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 3ab425aab84c8..f0e77c2ba982b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -53,7 +53,7 @@ class LocalSparkCluster( /* Start the Master */ val (masterSystem, masterPort, _, _) = Master.startSystemAndActor(localHostname, 0, 0, _conf) masterActorSystems += masterSystem - val masterUrl = "spark://" + localHostname + ":" + masterPort + val masterUrl = "spark://" + Utils.localHostNameForURI() + ":" + masterPort val masters = Array(masterUrl) /* Start the Workers */ diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index c1c4812f17fbe..40835b9550586 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -46,7 +46,7 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val conf = new SparkConf - val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localHostName(), 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription("TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored") diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ea548f23120d9..f9860d1a5ce76 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -48,7 +48,7 @@ private[spark] abstract class WebUI( protected val handlers = ArrayBuffer[ServletContextHandler]() protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None - protected val localHostName = Utils.localHostName() + protected val localHostName = Utils.localHostNameForURI() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0fdfaf300e95d..a541d660cd5c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -34,6 +34,7 @@ import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.{ByteStreams, Files} +import com.google.common.net.InetAddresses import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -789,13 +790,12 @@ private[spark] object Utils extends Logging { * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). * Note, this is typically not used from within core spark. */ - lazy val localIpAddress: String = findLocalIpAddress() - lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) + private lazy val localIpAddress: InetAddress = findLocalInetAddress() - private def findLocalIpAddress(): String = { + private def findLocalInetAddress(): InetAddress = { val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") if (defaultIpOverride != null) { - defaultIpOverride + InetAddress.getByName(defaultIpOverride) } else { val address = InetAddress.getLocalHost if (address.isLoopbackAddress) { @@ -806,15 +806,20 @@ private[spark] object Utils extends Logging { // It's more proper to pick ip address following system output order. val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.toList val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse + for (ni <- reOrderedNetworkIFs) { - for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && - !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { + val addresses = ni.getInetAddresses.toList + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress) + if (addresses.nonEmpty) { + val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) + // because of Inet6Address.toHostName may add interface at the end if it knows about it + val strippedAddress = InetAddress.getByAddress(addr.getAddress) // We've found an address that looks reasonable! logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + - " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + - " instead (on interface " + ni.getName + ")") + " a loopback address: " + address.getHostAddress + "; using " + + strippedAddress.getHostAddress + " instead (on interface " + ni.getName + ")") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") - return addr.getHostAddress + return strippedAddress } } logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + @@ -822,7 +827,7 @@ private[spark] object Utils extends Logging { " external IP address!") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") } - address.getHostAddress + address } } @@ -842,11 +847,14 @@ private[spark] object Utils extends Logging { * Get the local machine's hostname. */ def localHostName(): String = { - customHostname.getOrElse(localIpAddressHostname) + customHostname.getOrElse(localIpAddress.getHostAddress) } - def getAddressHostName(address: String): String = { - InetAddress.getByName(address).getHostName + /** + * Get the local machine's URI. + */ + def localHostNameForURI(): String = { + customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) } def checkHost(host: String, message: String = "") { diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 1bd1f324298e7..a7fe4476cacb8 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -23,6 +23,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.Duration import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils import com.amazonaws.auth.AWSCredentialsProvider import com.amazonaws.auth.DefaultAWSCredentialsProviderChain @@ -118,7 +119,7 @@ private[kinesis] class KinesisReceiver( * method. */ override def onStart() { - workerId = InetAddress.getLocalHost.getHostAddress() + ":" + UUID.randomUUID() + workerId = Utils.localHostName() + ":" + UUID.randomUUID() credentialsProvider = new DefaultAWSCredentialsProviderChain() kinesisClientLibConfiguration = new KinesisClientLibConfiguration(appName, streamName, credentialsProvider, workerId).withKinesisEndpoint(endpointUrl) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 158c225159720..97b46a01ba5b4 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.{HiveShim, HiveContext} import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.util.Utils /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -37,7 +38,7 @@ private[hive] object SparkSQLEnv extends Logging { val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") sparkConf - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") + .setAppName(s"SparkSQL::${Utils.localHostName()}") .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", From 240ea03faea005306e73c10253716b95487325ff Mon Sep 17 00:00:00 2001 From: Pradeep Chanumolu Date: Mon, 13 Apr 2015 13:02:55 +0100 Subject: [PATCH 732/817] [SPARK-6671] Add status command for spark daemons SPARK-6671 Currently using the spark-daemon.sh script we can start and stop the spark demons. But we cannot get the status of the daemons. It will be nice to include the status command in the spark-daemon.sh script, through which we can know if the spark demon is alive or not. Author: Pradeep Chanumolu Closes #5327 from pchanumolu/master and squashes the following commits: d3a1f05 [Pradeep Chanumolu] Make status command check consistent with Stop command 5062926 [Pradeep Chanumolu] Fix indentation in spark-daemon.sh 3e66bc8 [Pradeep Chanumolu] SPARK-6671 : Add status command to spark daemons 1ac3918 [Pradeep Chanumolu] Add status command to spark-daemon --- sbin/spark-daemon.sh | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 92e76a3fe6ca2..d8e0facb81169 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -29,7 +29,7 @@ # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. ## -usage="Usage: spark-daemon.sh [--config ] (start|stop) " +usage="Usage: spark-daemon.sh [--config ] (start|stop|status) " # if no args specified, show usage if [ $# -le 1 ]; then @@ -195,6 +195,23 @@ case $option in fi ;; + (status) + + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo $command is running. + exit 0 + else + echo $pid file is present but $command not running + exit 1 + fi + else + echo $command not running. + exit 2 + fi + ;; + (*) echo $usage exit 1 From 202ebf06e0f2d5df8b712e604fd95fa58e34ea20 Mon Sep 17 00:00:00 2001 From: linweizhong Date: Mon, 13 Apr 2015 13:06:54 +0100 Subject: [PATCH 733/817] [SPARK-6870][Yarn] Catch InterruptedException when yarn application state monitor thread been interrupted On PR #5305 we interrupt the monitor thread but forget to catch the InterruptedException, then in the log will print the stack info, so we need to catch it. Author: linweizhong Closes #5479 from Sephiroth-Lin/SPARK-6870 and squashes the following commits: f775f93 [linweizhong] Update, don't need to call Thread.currentThread() on monitor thread 0e2ef1f [linweizhong] Update 0d8958a [linweizhong] Update 3513fdb [linweizhong] Catch InterruptedException --- .../cluster/YarnClientSchedulerBackend.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 407dc1ac4d37d..99c05329b4d73 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -128,10 +128,13 @@ private[spark] class YarnClientSchedulerBackend( assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - val (state, _) = client.monitorApplication(appId, logApplicationReport = false) - logError(s"Yarn application has already exited with state $state!") - sc.stop() - Thread.currentThread().interrupt() + try { + val (state, _) = client.monitorApplication(appId, logApplicationReport = false) + logError(s"Yarn application has already exited with state $state!") + sc.stop() + } catch { + case e: InterruptedException => logInfo("Interrupting monitor thread") + } } } t.setName("Yarn application state monitor") From b29663eeea440b1d1a288d41b5ddf67e77c5bd54 Mon Sep 17 00:00:00 2001 From: Pei-Lun Lee Date: Mon, 13 Apr 2015 21:52:00 +0800 Subject: [PATCH 734/817] [SPARK-6352] [SQL] Add DirectParquetOutputCommitter Add a DirectParquetOutputCommitter class that skips _temporary directory when saving to s3. Add new config value "spark.sql.parquet.useDirectParquetOutputCommitter" (default false) to choose between the default output committer. Author: Pei-Lun Lee Closes #5042 from ypcat/spark-6352 and squashes the following commits: e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352 9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer. 0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case 0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter 769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter --- .../DirectParquetOutputCommitter.scala | 66 +++++++++++++++++++ .../sql/parquet/ParquetTableOperations.scala | 22 +++++++ .../spark/sql/parquet/ParquetIOSuite.scala | 21 ++++++ 3 files changed, 109 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala new file mode 100644 index 0000000000000..25a66cb488103 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + +import parquet.Log +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter} + +private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) + extends ParquetOutputCommitter(outputPath, context) { + val LOG = Log.getLog(classOf[ParquetOutputCommitter]) + + override def getWorkPath(): Path = outputPath + override def abortTask(taskContext: TaskAttemptContext): Unit = {} + override def commitTask(taskContext: TaskAttemptContext): Unit = {} + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + override def setupJob(jobContext: JobContext): Unit = {} + override def setupTask(taskContext: TaskAttemptContext): Unit = {} + + override def commitJob(jobContext: JobContext) { + try { + val configuration = ContextUtil.getConfiguration(jobContext) + val fileSystem = outputPath.getFileSystem(configuration) + val outputStatus = fileSystem.getFileStatus(outputPath) + val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) + try { + ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) + if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { + val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) + fileSystem.create(successPath).close() + } + } catch { + case e: Exception => { + LOG.warn("could not write summary file for " + outputPath, e) + val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) + if (fileSystem.exists(metadataPath)) { + fileSystem.delete(metadataPath, true) + } + } + } + } catch { + case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) + } + } + +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 1c868da23e060..3724bda829d30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -379,6 +379,8 @@ private[sql] case class InsertIntoParquetTable( */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { + var committer: OutputCommitter = null + // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} @@ -403,6 +405,26 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } + + // override to create output committer from configuration + override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { + if (committer == null) { + val output = getOutputPath(context) + val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", + classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) + val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] + } + committer + } + + // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 + private def getOutputPath(context: TaskAttemptContext): Path = { + context.getConfiguration().get("mapred.output.dir") match { + case null => null + case name => new Path(name) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 97c0f439acf13..4d0bf7cf99cdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -381,6 +381,27 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } + + test("SPARK-6352 DirectParquetOutputCommitter") { + // Write to a parquet file and let it fail. + // _temporary should be missing if direct output committer works. + try { + configuration.set("spark.sql.parquet.output.committer.class", + "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") + sqlContext.udf.register("div0", (x: Int) => x / 0) + withTempPath { dir => + intercept[org.apache.spark.SparkException] { + sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) + } + val path = new Path(dir.getCanonicalPath, "_temporary") + val fs = path.getFileSystem(configuration) + assert(!fs.exists(path)) + } + } + finally { + configuration.unset("spark.sql.parquet.output.committer.class") + } + } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From 77620be76e82b6cdaae406cd752d3272656f5fe0 Mon Sep 17 00:00:00 2001 From: Doug Balog Date: Mon, 13 Apr 2015 09:49:58 -0500 Subject: [PATCH 735/817] [SPARK-6207] [YARN] [SQL] Adds delegation tokens for metastore to conf. Adds hive2-metastore delegation token to conf when running in secure mode. Without this change, running on YARN in cluster mode fails with a GSS exception. This is a rough patch that adds a dependency to spark/yarn on hive-exec. I'm looking for suggestions on how to make this patch better. This contribution is my original work and that I licenses the work to the Apache Spark project under the project's open source licenses. Author: Doug Balog Author: Doug Balog Closes #5031 from dougb/SPARK-6207 and squashes the following commits: 3e9ac16 [Doug Balog] [SPARK-6207] Fixes minor code spacing issues. e260765 [Doug Balog] [SPARK-6207] Second pass at adding Hive delegation token to conf. - Use reflection instead of adding dependency on hive. - Tested on Hive 0.13 and Hadoop 2.4.1 1ab1729 [Doug Balog] Merge branch 'master' of git://github.com/apache/spark into SPARK-6207 bf356d2 [Doug Balog] [SPARK-6207] [YARN] [SQL] Adds delegation tokens for metastore to conf. Adds hive2-metastore delagations token to conf when running in securemode. Without this change, runing on YARN in cluster mode fails with a GSS exception. --- .../org/apache/spark/deploy/yarn/Client.scala | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index c1effd3c8a718..1091ff54b0463 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -22,17 +22,21 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} +import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} import com.google.common.base.Objects import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -220,6 +224,7 @@ private[spark] class Client( val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = getNameNodesToAccess(sparkConf) + dst obtainTokensForNamenodes(nns, hadoopConf, credentials) + obtainTokenForHiveMetastore(hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -936,6 +941,64 @@ object Client extends Logging { } } + /** + * Obtains token for the Hive metastore and adds them to the credentials. + */ + private def obtainTokenForHiveMetastore(conf: Configuration, credentials: Credentials) { + if (UserGroupInformation.isSecurityEnabled) { + val mirror = universe.runtimeMirror(getClass.getClassLoader) + + try { + val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive") + val hive = hiveClass.getMethod("get").invoke(null) + + val hiveConf = hiveClass.getMethod("getConf").invoke(hive) + val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") + + val hiveConfGet = (param:String) => Option(hiveConfClass + .getMethod("get", classOf[java.lang.String]) + .invoke(hiveConf, param)) + + val metastore_uri = hiveConfGet("hive.metastore.uris") + + // Check for local metastore + if (metastore_uri != None && metastore_uri.get.toString.size > 0) { + val metastore_kerberos_principal_conf_var = mirror.classLoader + .loadClass("org.apache.hadoop.hive.conf.HiveConf$ConfVars") + .getField("METASTORE_KERBEROS_PRINCIPAL").get("varname").toString + + val principal = hiveConfGet(metastore_kerberos_principal_conf_var) + + val username = Option(UserGroupInformation.getCurrentUser().getUserName) + if (principal != None && username != None) { + val tokenStr = hiveClass.getMethod("getDelegationToken", + classOf[java.lang.String], classOf[java.lang.String]) + .invoke(hive, username.get, principal.get).asInstanceOf[java.lang.String] + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + credentials.addToken(new Text("hive.server2.delegation.token"),hive2Token) + logDebug("Added hive.Server2.delegation.token to conf.") + hiveClass.getMethod("closeCurrent").invoke(null) + } else { + logError("Username or principal == NULL") + logError(s"""username=${username.getOrElse("(NULL)")}""") + logError(s"""principal=${principal.getOrElse("(NULL)")}""") + throw new IllegalArgumentException("username and/or principal is equal to null!") + } + } else { + logDebug("HiveMetaStore configured in localmode") + } + } catch { + case e:java.lang.NoSuchMethodException => { logInfo("Hive Method not found " + e); return } + case e:java.lang.ClassNotFoundException => { logInfo("Hive Class not found " + e); return } + case e:Exception => { logError("Unexpected Exception " + e) + throw new RuntimeException("Unexpected exception", e) + } + } + } + } + /** * Return whether the two file systems are the same. */ From c5b0b296b842926b5c07531a5affe8984bc799c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Apr 2015 09:29:04 -0700 Subject: [PATCH 736/817] [SPARK-6765] Enable scalastyle on test code. Turn scalastyle on for all test code. Most of the violations have been resolved in my previous pull requests: Core: https://github.com/apache/spark/pull/5484 SQL: https://github.com/apache/spark/pull/5412 MLlib: https://github.com/apache/spark/pull/5411 GraphX: https://github.com/apache/spark/pull/5410 Streaming: https://github.com/apache/spark/pull/5409 Author: Reynold Xin Closes #5486 from rxin/test-style-enable and squashes the following commits: 01683de [Reynold Xin] Fixed new code. a4ab46e [Reynold Xin] Fixed tests. 20adbc8 [Reynold Xin] Missed one violation. 5e36521 [Reynold Xin] [SPARK-6765] Enable scalastyle on test code. --- .../org/apache/spark/deploy/ClientSuite.scala | 1 - dev/scalastyle | 5 ++-- .../spark/ml/feature/VectorIndexerSuite.scala | 3 +- .../sql/hive/thriftserver/CliSuite.scala | 15 +++++----- .../HiveThriftServer2Suites.scala | 7 +++-- .../spark/sql/hive/QueryPartitionSuite.scala | 30 ++++++++++--------- .../StreamingJobProgressListenerSuite.scala | 4 +-- .../spark/deploy/yarn/ClientSuite.scala | 19 ++++++++---- .../deploy/yarn/YarnAllocatorSuite.scala | 6 ++-- .../yarn/YarnSparkHadoopUtilSuite.scala | 2 +- 10 files changed, 52 insertions(+), 40 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 518073dcbb64e..745f9eeee7536 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -46,5 +46,4 @@ class ClientSuite extends FunSuite with Matchers { // Invalid syntax. ClientArguments.isValidJarUrl("hdfs:") should be (false) } - } diff --git a/dev/scalastyle b/dev/scalastyle index 86919227ed1ab..4e03f89ed5d5d 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,9 +18,10 @@ # echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver test:scalastyle >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ - >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 test:scalastyle >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') rm scalastyle.txt diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 61c46c85a78b5..81ef831c42e55 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -150,7 +150,8 @@ class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { val vectorIndexer = getIndexer.setMaxCategories(maxCategories) val model = vectorIndexer.fit(data) val categoryMaps = model.categoryMaps - assert(categoryMaps.keys.toSet === categoricalFeatures) // Chose correct categorical features + // Chose correct categorical features + assert(categoryMaps.keys.toSet === categoricalFeatures) val transformed = model.transform(data).select("indexed") val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 75738fa22b572..6d1d7c3a4e698 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index bf20acecb1f32..4cf95e7bdfb2b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File +import java.net.URL import java.sql.{Date, DriverManager, Statement} import scala.collection.mutable.ArrayBuffer @@ -41,7 +42,7 @@ import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils object TestData { - def getTestDataFilePath(name: String) = { + def getTestDataFilePath(name: String): URL = { Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") } @@ -50,7 +51,7 @@ object TestData { } class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.binary private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logics below mimics HiveConnection.createBinaryTransport @@ -337,7 +338,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.http + override def mode: ServerMode.Value = ServerMode.http test("JDBC query execution") { withJdbcStatement { statement => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index 83f97128c5e83..a787fa5546e76 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.sql.hive -import java.io.File - import com.google.common.io.Files + import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.util.Utils -/* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ - +import org.apache.spark.util.Utils class QueryPartitionSuite extends QueryTest { @@ -37,23 +34,28 @@ class QueryPartitionSuite extends QueryTest { testData.registerTempTable("testData") val tmpDir = Files.createTempDir() - //create the table for test - sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') SELECT key,value FROM testData") + // create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) " + + s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') " + + "SELECT key,value FROM testData") - //test for the exist path + // test for the exist path checkAnswer(sql("select key,value from table_with_partition"), testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) - //delect the path of one partition + // delete the path of one partition val folders = tmpDir.listFiles.filter(_.isDirectory) Utils.deleteRecursively(folders(0)) - //test for affter delete the path + // test for after delete the path checkAnswer(sql("select key,value from table_with_partition"), testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 2b9d164500b72..94b1985116feb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -21,14 +21,14 @@ import org.scalatest.Matchers import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.{Time, Milliseconds, TestSuiteBase} +import org.apache.spark.streaming.{Duration, Time, Milliseconds, TestSuiteBase} class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { val input = (1 to 4).map(Seq(_)).toSeq val operation = (d: DStream[Int]) => d.map(x => x) - override def batchDuration = Milliseconds(100) + override def batchDuration: Duration = Milliseconds(100) test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + "onReceiverStarted, onReceiverError, onReceiverStopped") { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 92f04b4b859b3..c1b94ac9c5bdd 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -232,19 +232,26 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { testCode(conf) } - def newEnv = MutableHashMap[String, String]() + def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]() - def classpath(env: MutableHashMap[String, String]) = env(Environment.CLASSPATH.name).split(":|;|") + def classpath(env: MutableHashMap[String, String]): Array[String] = + env(Environment.CLASSPATH.name).split(":|;|") - def flatten(a: Option[Seq[String]], b: Option[Seq[String]]) = (a ++ b).flatten.toArray + def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] = + (a ++ b).flatten.toArray - def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = - Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults) + def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = { + Try(clazz.getField(field)) + .map(_.get(null).asInstanceOf[A]) + .toOption + .map(mapTo) + .getOrElse(defaults) + } def getFieldValue2[A: ClassTag, A1: ClassTag, B]( clazz: Class[_], field: String, - defaults: => B)(mapTo: A => B)(mapTo1: A1 => B) : B = { + defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = { Try(clazz.getField(field)).map(_.get(null)).map { case v: A => mapTo(v) case v1: A1 => mapTo1(v1) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index c09b01bafce37..455f1019d86dd 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -79,7 +79,7 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach } class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) { - override def equals(other: Any) = false + override def equals(other: Any): Boolean = false } def createAllocator(maxExecutors: Int = 5): YarnAllocator = { @@ -118,7 +118,9 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size should be (0) + + val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size + size should be (0) } test("some containers allocated") { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 4194f36499e66..9395316b71ff4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -46,7 +46,7 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { logWarning("Cannot execute bash, skipping bash tests.") } - def bashTest(name: String)(fn: => Unit) = + def bashTest(name: String)(fn: => Unit): Unit = if (hasBash) test(name)(fn) else ignore(name)(fn) bashTest("shell script escaping") { From 6cc5b3ed3c0c729f97956fa017d8eb7d6b43f90f Mon Sep 17 00:00:00 2001 From: Cheolsoo Park Date: Mon, 13 Apr 2015 13:45:10 -0500 Subject: [PATCH 737/817] [SPARK-6662][YARN] Allow variable substitution in spark.yarn.historyServer.address In Spark on YARN, explicit hostname and port number need to be set for "spark.yarn.historyServer.address" in SparkConf to make the HISTORY link. If the history server address is known and static, this is usually not a problem. But in cloud, that is usually not true. Particularly in EMR, the history server always runs on the same node as with RM. So I could simply set it to ${yarn.resourcemanager.hostname}:18080 if variable substitution is allowed. In fact, Hadoop configuration already implements variable substitution, so if this property is read via YarnConf, this can be easily achievable. Author: Cheolsoo Park Closes #5321 from piaozhexiu/SPARK-6662 and squashes the following commits: e37de75 [Cheolsoo Park] Preserve the space between the Hadoop and Spark imports 79757c6 [Cheolsoo Park] Incorporate review comments 10e2917 [Cheolsoo Park] Add helper function that substitutes hadoop vars to SparkHadoopUtil 589b52c [Cheolsoo Park] Revert "Allow variable substitution for spark.yarn. properties" ff9c35d [Cheolsoo Park] Allow variable substitution for spark.yarn. properties --- .../apache/spark/deploy/SparkHadoopUtil.scala | 38 +++++++++++++++++-- docs/running-on-yarn.md | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 1 + 3 files changed, 37 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index c2568eb4b60ac..cfaebf9ea5050 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,11 +24,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -201,6 +200,37 @@ class SparkHadoopUtil extends Logging { val baseStatus = fs.getFileStatus(basePath) if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) } + + private val HADOOP_CONF_PATTERN = "(\\$\\{hadoopconf-[^\\}\\$\\s]+\\})".r.unanchored + + /** + * Substitute variables by looking them up in Hadoop configs. Only variables that match the + * ${hadoopconf- .. } pattern are substituted. + */ + def substituteHadoopVariables(text: String, hadoopConf: Configuration): String = { + text match { + case HADOOP_CONF_PATTERN(matched) => { + logDebug(text + " matched " + HADOOP_CONF_PATTERN) + val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } + val eval = Option[String](hadoopConf.get(key)) + .map { value => + logDebug("Substituted " + matched + " with " + value) + text.replace(matched, value) + } + if (eval.isEmpty) { + // The variable was not found in Hadoop configs, so return text as is. + text + } else { + // Continue to substitute more variables. + substituteHadoopVariables(eval.get, hadoopConf) + } + } + case _ => { + logDebug(text + " didn't match " + HADOOP_CONF_PATTERN) + text + } + } + } } object SparkHadoopUtil { diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b7e68d4f71714..ed5bb263a5809 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -87,7 +87,8 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.historyServer.address (none) - The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + For this property, YARN properties can be used as variables, and these are substituted by Spark at runtime. For eg, if the Spark history server runs on the same node as the YARN ResourceManager, it can be set to `${hadoopconf-yarn.resourcemanager.hostname}:18080`. 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 32bc4e5663062..26259cee77151 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 @@ -223,6 +223,7 @@ private[spark] class ApplicationMaster( val appId = client.getAttemptId().getApplicationId().toString() val historyAddress = sparkConf.getOption("spark.yarn.historyServer.address") + .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) } .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } .getOrElse("") From 1e340c3ae4d5361d048a3d6990f144cfc923666f Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 13 Apr 2015 11:53:17 -0700 Subject: [PATCH 738/817] [SPARK-5988][MLlib] add save/load for PowerIterationClusteringModel See JIRA issue [SPARK-5988](https://issues.apache.org/jira/browse/SPARK-5988). Author: Xusen Yin Closes #5450 from yinxusen/SPARK-5988 and squashes the following commits: cb1ecfa [Xusen Yin] change Assignment into case class b1dd24c [Xusen Yin] add test suite 63c3923 [Xusen Yin] add save load for power iteration clustering --- .../clustering/PowerIterationClustering.scala | 68 +++++++++++++++++-- .../PowerIterationClusteringSuite.scala | 34 ++++++++++ 2 files changed, 97 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 180023922a9b0..aa53e88d59856 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -17,15 +17,20 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.{Logging, SparkException} +import org.json4s.JsonDSL._ +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.{Logging, SparkContext, SparkException} /** * :: Experimental :: @@ -38,7 +43,60 @@ import org.apache.spark.util.random.XORShiftRandom @Experimental class PowerIterationClusteringModel( val k: Int, - val assignments: RDD[PowerIterationClustering.Assignment]) extends Serializable + val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { + + override def save(sc: SparkContext, path: String): Unit = { + PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) + } + + override protected def formatVersion: String = "1.0" +} + +object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { + override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) + } + + private[clustering] + object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" + + def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val dataRDD = model.assignments.toDF() + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + implicit val formats = DefaultFormats + val sqlContext = new SQLContext(sc) + + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + + val k = (metadata \ "k").extract[Int] + val assignments = sqlContext.parquetFile(Loader.dataPath(path)) + Loader.checkSchema[PowerIterationClustering.Assignment](assignments.schema) + + val assignmentsRDD = assignments.map { + case Row(id: Long, cluster: Int) => PowerIterationClustering.Assignment(id, cluster) + } + + new PowerIterationClusteringModel(k, assignmentsRDD) + } + } +} /** * :: Experimental :: @@ -135,7 +193,7 @@ class PowerIterationClustering private[clustering] ( val v = powerIter(w, maxIterations) val assignments = kMeans(v, k).mapPartitions({ iter => iter.map { case (id, cluster) => - new Assignment(id, cluster) + Assignment(id, cluster) } }, preservesPartitioning = true) new PowerIterationClusteringModel(k, assignments) @@ -152,7 +210,7 @@ object PowerIterationClustering extends Logging { * @param cluster assigned cluster id */ @Experimental - class Assignment(val id: Long, val cluster: Int) extends Serializable + case class Assignment(id: Long, cluster: Int) /** * Normalizes the affinity matrix (A) by row sums and returns the normalized affinity matrix (W). diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 6315c03a700f1..6d6fe6fe46bab 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -18,12 +18,15 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable +import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkContext import org.apache.spark.graphx.{Edge, Graph} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext { @@ -110,4 +113,35 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext assert(x ~== u1(i.toInt) absTol 1e-14) } } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val model = PowerIterationClusteringSuite.createModel(sc, 3, 10) + try { + model.save(sc, path) + val sameModel = PowerIterationClusteringModel.load(sc, path) + PowerIterationClusteringSuite.checkEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} + +object PowerIterationClusteringSuite extends FunSuite { + def createModel(sc: SparkContext, k: Int, nPoints: Int): PowerIterationClusteringModel = { + val assignments = sc.parallelize( + (0 until nPoints).map(p => PowerIterationClustering.Assignment(p, Random.nextInt(k)))) + new PowerIterationClusteringModel(k, assignments) + } + + def checkEqual(a: PowerIterationClusteringModel, b: PowerIterationClusteringModel): Unit = { + assert(a.k === b.k) + + val aAssignments = a.assignments.map(x => (x.id, x.cluster)) + val bAssignments = b.assignments.map(x => (x.id, x.cluster)) + val unequalElements = aAssignments.join(bAssignments).filter { + case (id, (c1, c2)) => c1 != c2 }.count() + assert(unequalElements === 0L) + } } From 85ee0cabe87a27b6947c2d3e8525f04c77f80f6f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 14:29:07 -0700 Subject: [PATCH 739/817] [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Standard syntax: INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement; INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;   Hive extension (multiple inserts): FROM from_statement INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...; FROM from_statement INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;   Hive extension (dynamic partition inserts): INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; Author: Daoyuan Wang Closes #4865 from adrian-wang/insertoverwrite and squashes the following commits: 2fce94f [Daoyuan Wang] add assert 10ea6f3 [Daoyuan Wang] add name for boolean parameter 0bbe9b9 [Daoyuan Wang] fix failure 4391154 [Daoyuan Wang] support if not exists for insert overwrite into partition in hiveQl --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../plans/logical/basicOperators.scala | 4 ++- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../sql/sources/DataSourceStrategy.scala | 2 +- .../org/apache/spark/sql/sources/rules.scala | 7 +++-- .../execution/HiveCompatibilitySuite.scala | 1 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 19 ++++++------ .../org/apache/spark/sql/hive/HiveQl.scala | 22 ++++++++++++- .../spark/sql/hive/HiveStrategies.scala | 10 +++--- .../hive/execution/CreateTableAsSelect.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 31 +++++++++++++------ ...titions-0-d5edc0daa94b33915df794df3b710774 | 0 ...titions-1-9eb9372f4855928fae16f5fa554b3a62 | 0 ...itions-10-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-11-8854d6001200fc11529b2e2da755e5a2 | 0 ...itions-12-71ff68fda0aa7a36cb50d8fab0d70d25 | 0 ...titions-13-7e4e7d7003fc6ef17bc19c3461ad899 | 0 ...itions-14-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-15-a3b2e230efde74e970ae8a3b55f383fc | 0 ...titions-2-8396c17a66e3d9a374d4361873b9bfe3 | 0 ...titions-3-3876bb356dd8af7e78d061093d555457 | 0 ...rtitions-4-528e23afb272c2e69004c86ddaa70ee | 0 ...titions-5-de5d56456c28d63775554e56355911d2 | 0 ...titions-6-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ ...titions-7-92f6af82704504968de078c133f222f8 | 0 ...titions-8-316cad7c63ddd4fb043be2affa5b0a67 | 0 ...titions-9-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ 30 files changed, 84 insertions(+), 35 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ee04cb579deb6..bc8d3751f6616 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -155,7 +155,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val insert: Parser[LogicalPlan] = INSERT ~> (OVERWRITE ^^^ true | INTO ^^^ false) ~ (TABLE ~> relation) ~ select ^^ { - case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o, false) } protected lazy val cte: Parser[LogicalPlan] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 50702ac6832ec..8b68b0df35f48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -193,7 +193,7 @@ class Analyzer( } realPlan transform { - case i@InsertIntoTable(u: UnresolvedRelation, _, _, _) => + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => i.copy( table = EliminateSubQueries(getTable(u, cteRelations))) case u: UnresolvedRelation => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 145f062dd6817..21c15ad14fd19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -293,7 +293,7 @@ package object dsl { def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( - analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) + analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite, false) def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer(logicalPlan)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 5d31a6eecfce2..17522976dc2c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -125,12 +125,14 @@ case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = child.output + assert(overwrite || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) 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 9b9adf855077a..94ae2d65fd0e4 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 @@ -1209,7 +1209,7 @@ class DataFrame private[sql]( @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit = { sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd + Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 23f7e5609414b..5268b7334051a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -211,7 +211,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil - case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = 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 e13759b7feb7b..34d048e426d10 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 @@ -56,7 +56,7 @@ private[sql] object DataSourceStrategy extends Strategy { execution.PhysicalRDD(l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), part, query, overwrite) if part.isEmpty => + l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 5a78001117d1b..6ed68d179edc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -37,7 +37,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation. case i @ InsertIntoTable( - l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite) => { + l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite, ifNotExists) => { // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -84,7 +84,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => + l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") @@ -102,7 +102,8 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite) if !l.isInstanceOf[InsertableRelation] => + l: LogicalRelation, partition, query, overwrite, ifNotExists) + if !l.isInstanceOf[InsertableRelation] => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 2ae9d018e1b1b..81ee48ef4152f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -532,6 +532,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl7", "inputddl8", "insert1", + "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", "join0", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 3ed5c5b031736..f1c0bd92aa23d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -527,7 +527,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Collects all `MetastoreRelation`s which should be replaced val toBeReplaced = plan.collect { // Write path - case InsertIntoTable(relation: MetastoreRelation, _, _, _) + case InsertIntoTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -538,7 +538,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with (relation, parquetRelation, attributedRewrites) // Write path - case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) + case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -569,15 +569,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val alias = r.alias.getOrElse(r.tableName) Subquery(alias, parquetRelation) - case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) - case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) @@ -698,7 +698,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => castChildOutput(p, table, child) } @@ -715,7 +715,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with .forall { case (left, right) => left.sameType(right) }) { // If both types ignoring nullability of ArrayType, MapType, StructType are the same, // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite) + InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite, p.ifNotExists) } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { @@ -753,7 +753,8 @@ private[hive] case class InsertIntoHiveTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b2ae74efeb097..53a204b8c2932 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1002,7 +1002,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite) + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.getChildren.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5f7e897295117..1ccb0c279c60e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -184,12 +184,14 @@ private[hive] trait HiveStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil - case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) => + table, partition, planLater(child), overwrite, ifNotExists) :: Nil + case hive.InsertIntoHiveTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil + table, partition, planLater(child), overwrite, ifNotExists) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index fade9e5852eaa..76a1965f3cb25 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -67,7 +67,7 @@ case class CreateTableAsSelect( new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") } } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 6c96747439683..89995a91b1a92 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -45,7 +45,8 @@ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan, - overwrite: Boolean) extends UnaryNode with HiveInspectors { + overwrite: Boolean, + ifNotExists: Boolean) extends UnaryNode with HiveInspectors { @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @@ -219,15 +220,25 @@ case class InsertIntoHiveTable( isSkewedStoreAsSubdir) } } else { - catalog.synchronized { - catalog.client.loadPartition( - outputPath, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + // scalastyle:off + // ifNotExists is only valid with static partition, refer to + // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries + // scalastyle:on + val oldPart = catalog.synchronized { + catalog.client.getPartition( + catalog.client.getTable(qualifiedTableName), partitionSpec, false) + } + if (oldPart == null || !ifNotExists) { + catalog.synchronized { + catalog.client.loadPartition( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } } } else { diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 From 3a205bbd9e352668a020c3146391e1e4441467af Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Mon, 13 Apr 2015 14:43:07 -0700 Subject: [PATCH 740/817] [SQL][SPARK-6742]: Don't push down predicates which reference partition column(s) cc liancheng Author: Yash Datta Closes #5390 from saucam/fpush and squashes the following commits: 3f026d6 [Yash Datta] SPARK-6742: Fix scalastyle ce3d702 [Yash Datta] SPARK-6742: Add test case, fix scalastyle 8592acc [Yash Datta] SPARK-6742: Don't push down predicates which reference partition column(s) --- .../spark/sql/execution/SparkStrategies.scala | 11 ++++++++- .../sql/parquet/ParquetFilterSuite.scala | 24 ++++++++++++++++++- 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5268b7334051a..f0d92ffffcda3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -215,6 +215,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => + val partitionColNames = relation.partitioningAttributes.map(_.name).toSet + val filtersToPush = filters.filter { pred => + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } val prunePushedDownFilters = if (sqlContext.conf.parquetFilterPushDown) { (predicates: Seq[Expression]) => { @@ -226,6 +231,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // "A AND B" in the higher-level filter, not just "B". predicates.map(p => p -> ParquetFilters.createFilter(p)).collect { case (predicate, None) => predicate + // Filter needs to be applied above when it contains partitioning + // columns + case (predicate, _) if(!predicate.references.map(_.name).toSet + .intersect (partitionColNames).isEmpty) => predicate } } } else { @@ -238,7 +247,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetTableScan( _, relation, - if (sqlContext.conf.parquetFilterPushDown) filters else Nil)) :: Nil + if (sqlContext.conf.parquetFilterPushDown) filtersToPush else Nil)) :: Nil case _ => Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index 6a2c2a7c4080a..10d0ede4dc0dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -22,7 +22,7 @@ import parquet.filter2.predicate.Operators._ import parquet.filter2.predicate.{FilterPredicate, Operators} import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, Predicate, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext @@ -350,4 +350,26 @@ class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with Before override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6742: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + val df = DataFrame(sqlContext, org.apache.spark.sql.parquet.ParquetRelation( + path, + Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext, + Seq(AttributeReference("part", IntegerType, false)()) )) + + checkAnswer( + df.filter("a = 1 or part = 1"), + (1 to 3).map(i => Row(1, i, i.toString))) + } + } + } } From 2a55cb41bf7da1786be2c76b8af398da8fedb44b Mon Sep 17 00:00:00 2001 From: MechCoder Date: Mon, 13 Apr 2015 15:36:33 -0700 Subject: [PATCH 741/817] [SPARK-5972] [MLlib] Cache residuals and gradient in GBT during training and validation The previous PR https://github.com/apache/spark/pull/4906 helped to extract the learning curve giving the error for each iteration. This continues the work refactoring some code and extending the same logic during training and validation. Author: MechCoder Closes #5330 from MechCoder/spark-5972 and squashes the following commits: 0b5d659 [MechCoder] minor 32d409d [MechCoder] EvaluateeachIteration and training cache should follow different paths d542bb0 [MechCoder] Remove unused imports and docs 58f4932 [MechCoder] Remove unpersist 70d3b4c [MechCoder] Broadcast for each tree 5869533 [MechCoder] Access broadcasted values locally and other minor changes 923dbf6 [MechCoder] [SPARK-5972] Cache residuals and gradient in GBT during training and validation --- .../mllib/tree/GradientBoostedTrees.scala | 42 ++++++---- .../spark/mllib/tree/loss/AbsoluteError.scala | 10 +-- .../spark/mllib/tree/loss/LogLoss.scala | 11 +-- .../apache/spark/mllib/tree/loss/Loss.scala | 8 +- .../spark/mllib/tree/loss/SquaredError.scala | 10 +-- .../mllib/tree/model/treeEnsembleModels.scala | 77 +++++++++++++++---- 6 files changed, 105 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index a9c93e181e3ce..c02c79f094b66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -157,7 +157,6 @@ object GradientBoostedTrees extends Logging { validationInput: RDD[LabeledPoint], boostingStrategy: BoostingStrategy, validate: Boolean): GradientBoostedTreesModel = { - val timer = new TimeTracker() timer.start("total") timer.start("init") @@ -192,20 +191,29 @@ object GradientBoostedTrees extends Logging { // Initialize tree timer.start("building tree 0") val firstTreeModel = new DecisionTree(treeStrategy).run(data) + val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel - baseLearnerWeights(0) = 1.0 - val startingModel = new GradientBoostedTreesModel(Regression, Array(firstTreeModel), Array(1.0)) - logDebug("error of gbt = " + loss.computeError(startingModel, input)) + baseLearnerWeights(0) = firstTreeWeight + val startingModel = new GradientBoostedTreesModel( + Regression, Array(firstTreeModel), baseLearnerWeights.slice(0, 1)) + + var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + logDebug("error of gbt = " + predError.values.mean()) // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") - var bestValidateError = if (validate) loss.computeError(startingModel, validationInput) else 0.0 + var validatePredError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) + var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 var bestM = 1 - // psuedo-residual for second iteration - data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), - point.features)) + // pseudo-residual for second iteration + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } + var m = 1 while (m < numIterations) { timer.start(s"building tree $m") @@ -222,15 +230,22 @@ object GradientBoostedTrees extends Logging { baseLearnerWeights(m) = learningRate // Note: A model of type regression is used since we require raw prediction val partialModel = new GradientBoostedTreesModel( - Regression, baseLearners.slice(0, m + 1), baseLearnerWeights.slice(0, m + 1)) - logDebug("error of gbt = " + loss.computeError(partialModel, input)) + Regression, baseLearners.slice(0, m + 1), + baseLearnerWeights.slice(0, m + 1)) + + predError = GradientBoostedTreesModel.updatePredictionError( + input, predError, baseLearnerWeights(m), baseLearners(m), loss) + logDebug("error of gbt = " + predError.values.mean()) if (validate) { // Stop training early if // 1. Reduction in error is less than the validationTol or // 2. If the error increases, that is if the model is overfit. // We want the model returned corresponding to the best validation error. - val currentValidateError = loss.computeError(partialModel, validationInput) + + validatePredError = GradientBoostedTreesModel.updatePredictionError( + validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) + val currentValidateError = validatePredError.values.mean() if (bestValidateError - currentValidateError < validationTol) { return new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, @@ -242,8 +257,9 @@ object GradientBoostedTrees extends Logging { } } // Update data with pseudo-residuals - data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), - point.features)) + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } m += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 793dd664c5d5a..6f570b4e09c79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -37,14 +37,12 @@ object AbsoluteError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. * The gradient with respect to F(x) is: sign(F(x) - y) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 + override def gradient(prediction: Double, label: Double): Double = { + if (label - prediction < 0) 1.0 else -1.0 } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 51b1aed167b66..24ee9f3d51293 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -39,15 +39,12 @@ object LogLoss extends Loss { * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification * The gradient with respect to F(x) is: - 4 y / (1 + exp(2 y F(x))) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - val prediction = model.predict(point.features) - - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) + override def gradient(prediction: Double, label: Double): Double = { + - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 357869ff6b333..d3b82b752fa0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -31,13 +31,11 @@ trait Loss extends Serializable { /** * Method to calculate the gradients for the gradient boosting calculation. - * @param model Model of the weak learner. - * @param point Instance of the training dataset. + * @param prediction Predicted feature + * @param label true label. * @return Loss gradient. */ - def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double + def gradient(prediction: Double, label: Double): Double /** * Method to calculate error of the base learner for the gradient boosting calculation. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index b990707ca4525..58857ae15e93e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -37,14 +37,12 @@ object SquaredError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. * The gradient with respect to F(x) is: - 2 (y - F(x)) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - 2.0 * (model.predict(point.features) - point.label) + override def gradient(prediction: Double, label: Double): Double = { + 2.0 * (prediction - label) } override def computeError(prediction: Double, label: Double): Double = { 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 1950254b2aa6d..fef3d2acb202a 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 @@ -130,35 +130,28 @@ class GradientBoostedTreesModel( val numIterations = trees.length val evaluationArray = Array.fill(numIterations)(0.0) + val localTreeWeights = treeWeights + + var predictionAndError = GradientBoostedTreesModel.computeInitialPredictionAndError( + remappedData, localTreeWeights(0), trees(0), loss) - var predictionAndError: RDD[(Double, Double)] = remappedData.map { i => - val pred = treeWeights(0) * trees(0).predict(i.features) - val error = loss.computeError(pred, i.label) - (pred, error) - } evaluationArray(0) = predictionAndError.values.mean() - // Avoid the model being copied across numIterations. val broadcastTrees = sc.broadcast(trees) - val broadcastWeights = sc.broadcast(treeWeights) - (1 until numIterations).map { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) - val currentTreeWeight = broadcastWeights.value(nTree) - iter.map { - case (point, (pred, error)) => { - val newPred = pred + currentTree.predict(point.features) * currentTreeWeight - val newError = loss.computeError(newPred, point.label) - (newPred, newError) - } + val currentTreeWeight = localTreeWeights(nTree) + iter.map { case (point, (pred, error)) => + val newPred = pred + currentTree.predict(point.features) * currentTreeWeight + val newError = loss.computeError(newPred, point.label) + (newPred, newError) } } evaluationArray(nTree) = predictionAndError.values.mean() } broadcastTrees.unpersist() - broadcastWeights.unpersist() evaluationArray } @@ -166,6 +159,58 @@ class GradientBoostedTreesModel( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { + /** + * Compute the initial predictions and errors for a dataset for the first + * iteration of gradient boosting. + * @param data: training data. + * @param initTreeWeight: learning rate assigned to the first tree. + * @param initTree: first DecisionTreeModel. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to every sample. + */ + def computeInitialPredictionAndError( + data: RDD[LabeledPoint], + initTreeWeight: Double, + initTree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + data.map { lp => + val pred = initTreeWeight * initTree.predict(lp.features) + val error = loss.computeError(pred, lp.label) + (pred, error) + } + } + + /** + * Update a zipped predictionError RDD + * (as obtained with computeInitialPredictionAndError) + * @param data: training data. + * @param predictionAndError: predictionError RDD + * @param treeWeight: Learning rate. + * @param tree: Tree using which the prediction and error should be updated. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to each sample. + */ + def updatePredictionError( + data: RDD[LabeledPoint], + predictionAndError: RDD[(Double, Double)], + treeWeight: Double, + tree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + + val newPredError = data.zip(predictionAndError).mapPartitions { iter => + iter.map { + case (lp, (pred, error)) => { + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) + } + } + } + newPredError + } + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName From e63a86abe2794332cdad71d87b72a7c56327a43d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 16:00:58 -0700 Subject: [PATCH 742/817] [SPARK-6872] [SQL] add copy in external sort We need add copy before call externalsort. Author: Daoyuan Wang Closes #5481 from adrian-wang/extsort and squashes the following commits: 9611586 [Daoyuan Wang] fix bug in external sort --- .../scala/org/apache/spark/sql/execution/basicOperators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 6eec520abff53..f8221f41bc6c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -194,7 +194,7 @@ case class ExternalSort( child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) - sorter.insertAll(iterator.map(r => (r, null))) + sorter.insertAll(iterator.map(r => (r.copy, null))) val baseIterator = sorter.iterator.map(_._1) // TODO(marmbrus): The complex type signature below thwarts inference for no reason. CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop()) From c5602bdc310cc8f82dc304500bebe40217cba785 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 13 Apr 2015 16:02:18 -0700 Subject: [PATCH 743/817] [SPARK-5941] [SQL] Unit Test loads the table `src` twice for leftsemijoin.q In `leftsemijoin.q`, there is a data loading command for table `sales` already, but in `TestHive`, it also created the table `sales`, which causes duplicated records inserted into the `sales`. Author: Cheng Hao Closes #4506 from chenghao-intel/df_table and squashes the following commits: 0be05f7 [Cheng Hao] Remove the table `sales` creating from TestHive --- .../sql/columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../org/apache/spark/sql/hive/test/TestHive.scala | 6 ------ ...ftsemijoin-10-89737a8857b5b61cc909e0c797f86aea | 2 -- .../leftsemijoin-8-73cad58a10a1483ccb15e94a857013 | 2 -- .../spark/sql/hive/execution/HiveSerDeSuite.scala | 15 +++++++++++---- 5 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 27dfabca90217..479210d1c9c43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { .toDF().registerTempTable("sizeTst") cacheTable("sizeTst") assert( - table("sizeTst").queryExecution.logical.statistics.sizeInBytes > + table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > conf.autoBroadcastJoinThreshold) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index a3497eadd67f6..6570fa1043900 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -262,12 +262,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |WITH SERDEPROPERTIES ('field.delim'='\\t') """.stripMargin.cmd, "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("sales", - s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) - |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea +++ b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 +++ b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index d05e11fcf281b..5586a793618bd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -25,18 +25,25 @@ import org.apache.spark.sql.hive.test.TestHive * A set of tests that validates support for Hive SerDe. */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - override def beforeAll(): Unit = { + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + super.beforeAll() TestHive.cacheTables = false - super.beforeAll() + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") } + // table sales is not a cache table, and will be clear after reset + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales", false) + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") - createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") - createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") From c4ab255e94366ba9b9023d5431f9d2412e0d6dc7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 16:28:07 -0700 Subject: [PATCH 744/817] [SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin Author: Ilya Ganelin Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time --- .../spark/ExecutorAllocationManager.scala | 34 ++++---- .../org/apache/spark/HeartbeatReceiver.scala | 15 ++-- .../scala/org/apache/spark/SparkConf.scala | 36 ++++++++ .../org/apache/spark/executor/Executor.scala | 6 +- .../spark/network/nio/ConnectionManager.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 10 +-- .../spark/scheduler/TaskSetManager.scala | 21 +++-- .../CoarseGrainedSchedulerBackend.scala | 13 +-- .../org/apache/spark/util/AkkaUtils.scala | 14 +-- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 26 +++++- .../ExecutorAllocationManagerSuite.scala | 7 +- .../network/nio/ConnectionManagerSuite.scala | 8 +- .../spark/scheduler/TaskSetManagerSuite.scala | 25 +++--- .../BlockManagerReplicationSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 44 ++++++++++ docs/configuration.md | 86 ++++++++++--------- docs/running-on-yarn.md | 4 +- .../apache/spark/network/util/JavaUtils.java | 66 ++++++++++++++ .../spark/network/util/TransportConf.java | 15 +++- .../streaming/receiver/BlockGenerator.scala | 8 +- .../streaming/scheduler/JobGenerator.scala | 12 ++- .../spark/streaming/ReceiverSuite.scala | 14 +-- .../streaming/StreamingContextSuite.scala | 22 ++--- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +- 25 files changed, 345 insertions(+), 157 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 9385f557c4614..4e7bf51fc0622 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,16 +80,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 5) + private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") - // Same as above, but used only after `schedulerBacklogTimeout` is exceeded - private val sustainedSchedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = conf.getLong( - "spark.dynamicAllocation.executorIdleTimeout", 600) + private val executorIdleTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -150,14 +150,14 @@ private[spark] class ExecutorAllocationManager( throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } - if (schedulerBacklogTimeout <= 0) { + if (schedulerBacklogTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") } - if (sustainedSchedulerBacklogTimeout <= 0) { + if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") } - if (executorIdleTimeout <= 0) { + if (executorIdleTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") } // Require external shuffle service for dynamic allocation @@ -262,8 +262,8 @@ private[spark] class ExecutorAllocationManager( } else if (addTime != NOT_SET && now >= addTime) { val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime += sustainedSchedulerBacklogTimeoutS * 1000 delta } else { 0 @@ -351,7 +351,7 @@ private[spark] class ExecutorAllocationManager( val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + - s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") + s"$executorIdleTimeoutS seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -407,8 +407,8 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerBacklogged(): Unit = synchronized { if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + - s"are building up (to expire in $schedulerBacklogTimeout seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 } } @@ -431,8 +431,8 @@ private[spark] class ExecutorAllocationManager( if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeoutS seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeoutS * 1000 } } else { logWarning(s"Attempted to mark unknown executor $executorId idle") diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 5871b8c869f03..e3bd16f1cbf24 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -62,14 +62,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val executorTimeoutMs = sc.conf.getOption("spark.network.timeout").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120000)) - + private val slaveTimeoutMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") + private val executorTimeoutMs = + sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 + // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val checkTimeoutIntervalMs = - sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) + private val timeoutIntervalMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + private val checkTimeoutIntervalMs = + sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0c123c96b8d7b..390e631647bd6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -174,6 +174,42 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsSeconds(key: String): Long = { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsMs(key: String): Long = { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String, defaultValue: String): Long = { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { Option(settings.get(key)) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 14f99a464b6e9..516f619529c48 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -436,14 +436,14 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) while (!isStopped) { reportHeartBeat() - Thread.sleep(interval) + Thread.sleep(intervalMs) } } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 741fe3e1ea750..8e3c30fc3d781 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,8 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) + conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. // diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 076b36e86c0ce..2362cc7240039 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,10 +62,10 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) + val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) @@ -143,8 +143,8 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, + SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } @@ -173,7 +173,7 @@ private[spark] class TaskSchedulerImpl( this.cancel() } } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + }, STARVATION_TIMEOUT_MS, STARVATION_TIMEOUT_MS) } hasReceivedTask = true } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d509881c74fef..7dc325283d961 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -848,15 +848,18 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000") - level match { - case TaskLocality.PROCESS_LOCAL => - conf.get("spark.locality.wait.process", defaultWait).toLong - case TaskLocality.NODE_LOCAL => - conf.get("spark.locality.wait.node", defaultWait).toLong - case TaskLocality.RACK_LOCAL => - conf.get("spark.locality.wait.rack", defaultWait).toLong - case _ => 0L + val defaultWait = conf.get("spark.locality.wait", "3s") + val localityWaitKey = level match { + case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" + case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" + case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + case _ => null + } + + if (localityWaitKey != null) { + conf.getTimeAsMs(localityWaitKey, defaultWait) + } else { + 0L } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4c49da87af9dc..63987dfb32695 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -52,8 +52,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val maxRegisteredWaitingTimeMs = + conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -77,12 +77,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onStart() { // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) + val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") + reviveThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { Option(self).foreach(_.send(ReviveOffers)) } - }, 0, reviveInterval, TimeUnit.MILLISECONDS) + }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) } override def receive: PartialFunction[Any, Unit] = { @@ -301,9 +302,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") return true } false diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 6c2c5261306e7..8e8cc7cc6389e 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -66,7 +65,8 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) + val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", + conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -78,8 +78,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") + val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() @@ -102,14 +102,14 @@ private[spark] object AkkaUtils extends Logging { |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 375ed430bde45..2bbfc988a99a8 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - conf.getInt("spark.cleaner.ttl", -1) + conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a541d660cd5c6..1029b0f9fce1e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ @@ -47,6 +47,7 @@ import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -612,9 +613,10 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 - uc.setConnectTimeout(timeout) - uc.setReadTimeout(timeout) + val timeoutMs = + conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 + uc.setConnectTimeout(timeoutMs) + uc.setReadTimeout(timeoutMs) uc.connect() val in = uc.getInputStream() downloadFile(url, in, targetFile, fileOverwrite) @@ -1018,6 +1020,22 @@ private[spark] object Utils extends Logging { ) } + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + def timeStringAsMs(str: String): Long = { + JavaUtils.timeStringAsMs(str) + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + def timeStringAsSeconds(str: String): Long = { + JavaUtils.timeStringAsSec(str) + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3ded1e4af8742..6b3049b28cd5e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -684,10 +684,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", + s"${schedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", - sustainedSchedulerBacklogTimeout.toString) - .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + s"${sustainedSchedulerBacklogTimeout.toString}s") + .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") val sc = new SparkContext(conf) contexts += sc diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 716f875d30b8a..02424c59d6831 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -260,8 +260,8 @@ class ConnectionManagerSuite extends FunSuite { test("sendMessageReliably timeout") { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") - val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + val ackTimeoutS = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeoutS}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) @@ -272,7 +272,7 @@ class ConnectionManagerSuite extends FunSuite { val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { // sleep 60 sec > ack timeout for simulating server slow down or hang up - Thread.sleep(ackTimeout * 3 * 1000) + Thread.sleep(ackTimeoutS * 3 * 1000) None }) @@ -287,7 +287,7 @@ class ConnectionManagerSuite extends FunSuite { // Otherwise TimeoutExcepton is thrown from Await.result. // We expect TimeoutException is not thrown. intercept[IOException] { - Await.result(future, (ackTimeout * 2) second) + Await.result(future, (ackTimeoutS * 2) second) } manager.stop() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 716d12c0762cf..6198cea46ddf8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -27,7 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -152,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) + val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 override def beforeEach() { @@ -240,7 +239,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) @@ -251,7 +250,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } @@ -292,7 +291,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host1 again: nothing should get chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -306,7 +305,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Now that we've launched a local task, we should no longer launch the task for host3 assert(manager.resourceOffer("exec2", "host2", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) @@ -338,7 +337,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT * 2) + clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -528,7 +527,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) @@ -622,12 +621,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } @@ -716,13 +715,13 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) assert(manager.resourceOffer("execA", "host1", ANY) !== None) - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1") manager.executorLost("execB.2", "host2") - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index b4de90b65d545..ffa5162a31841 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -76,7 +76,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd conf.set("spark.storage.unrollMemoryThreshold", "512") // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1") + conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 449fb87f111c4..fb97e650ff95c 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -23,6 +23,7 @@ import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStr import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols +import java.util.concurrent.TimeUnit import java.util.Locale import com.google.common.base.Charsets.UTF_8 @@ -35,7 +36,50 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { + + test("timeConversion") { + // Test -1 + assert(Utils.timeStringAsSeconds("-1") === -1) + + // Test zero + assert(Utils.timeStringAsSeconds("0") === 0) + + assert(Utils.timeStringAsSeconds("1") === 1) + assert(Utils.timeStringAsSeconds("1s") === 1) + assert(Utils.timeStringAsSeconds("1000ms") === 1) + assert(Utils.timeStringAsSeconds("1000000us") === 1) + assert(Utils.timeStringAsSeconds("1m") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) + + assert(Utils.timeStringAsMs("1") === 1) + assert(Utils.timeStringAsMs("1ms") === 1) + assert(Utils.timeStringAsMs("1000us") === 1) + assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1m") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) + assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) + + // Test invalid strings + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600s") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600ds") + } + intercept[NumberFormatException] { + Utils.timeStringAsMs("600s This breaks") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This 123s breaks") + } + } + test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.0 B") diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..7169ec295ef7f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,9 +35,19 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually -require one to prevent any sort of starvation issues. +Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may +actually require one to prevent any sort of starvation issues. +Properties that specify some time duration should be configured with a unit of time. +The following format is accepted: + + 25ms (milliseconds) + 5s (seconds) + 10m or 10min (minutes) + 3h (hours) + 5d (days) + 1y (years) + ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For instance, if you'd like to run the same application with different masters or different @@ -429,10 +439,10 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5 + 5s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is 15 seconds by default, calculated as maxRetries * retryWait. @@ -732,17 +742,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000 - Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10s + Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60 + 60s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver. @@ -853,11 +863,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value reduces network overhead and a smaller value ( ~ 1 s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case @@ -868,11 +878,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 6000 + 6000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -886,9 +896,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. @@ -938,10 +948,10 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + Default timeout for all network interactions. This config will be used in place of + spark.core.connection.ack.wait.timeout, spark.akka.timeout, spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -989,9 +999,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3s - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1024,10 +1034,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30s - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. @@ -1054,10 +1063,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1s - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. @@ -1070,9 +1078,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. @@ -1127,10 +1135,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this
    description. @@ -1157,10 +1165,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration (in seconds), new executors will be requested. For more detail, see this + this duration, new executors will be requested. For more detail, see this description. @@ -1215,18 +1223,18 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. spark.core.connection.auth.wait.timeout - 30 + 30s - Number of seconds for the connection to wait for authentication to occur before timing + How long for the connection to wait for authentication to occur before timing out and giving up. @@ -1347,9 +1355,9 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index ed5bb263a5809..853c9f26b0ec9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,9 +48,9 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000 + 100s - In yarn-cluster mode, time in milliseconds for the application master to wait for the + In yarn-cluster mode, time for the application master to wait for the SparkContext to be initialized. In yarn-client mode, time for the application master to wait for the driver to connect to it. diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 73da9b7346f4d..b6fbace509a0e 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -21,9 +21,13 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,4 +125,66 @@ private static boolean isSymlink(File file) throws IOException { } return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + + private static ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for + * internal use. If no suffix is provided a direct conversion is attempted. + */ + private static long parseTimeString(String str, TimeUnit unit) { + String lower = str.toLowerCase().trim(); + + try { + String suffix; + long val; + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (m.matches()) { + val = Long.parseLong(m.group(1)); + suffix = m.group(2); + } else { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min) hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return parseTimeString(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return parseTimeString(str, TimeUnit.SECONDS); + } + } diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 2eaf3b71d9a49..0aef7f1987315 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -37,8 +37,11 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int defaultTimeout = conf.getInt("spark.network.timeout", 120); - return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; } /** Number of concurrent connections between two nodes for fetching data. */ @@ -68,7 +71,9 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } + public int saslRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -80,7 +85,9 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + } /** * Minimum size of a block that we should start using memory map rather than reading in through diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 42514d8b47dcf..f4963a78e1d18 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.SystemClock +import org.apache.spark.util.{SystemClock, Utils} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -79,9 +79,9 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms") private val blockIntervalTimer = - new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } @@ -132,7 +132,7 @@ private[streaming] class BlockGenerator( val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(receiverId, time - blockInterval) + val blockId = StreamBlockId(receiverId, time - blockIntervalMs) val newBlock = new Block(blockId, newBlockBuffer) listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 4946806d2ee95..58e56638a2dca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -24,7 +24,7 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock} +import org.apache.spark.util.{Clock, ManualClock, Utils} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -104,17 +104,15 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = conf.getLong( - "spark.streaming.gracefulStopTimeout", - 10 * ssc.graph.batchDuration.milliseconds - ) + val stopTimeoutMs = conf.getTimeAsMs( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut: Boolean = { - val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeout + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs if (timedOut) { - logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } timedOut } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 10c35cba8dc53..91261a9db7360 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,11 +131,11 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockIntervalMs = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 - val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) val generatedData = new ArrayBuffer[Int] // Generate blocks @@ -157,15 +157,15 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 100 + val blockIntervalMs = 100 val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 - val waitTime = expectedBlocks * blockInterval + val waitTime = expectedBlocks * blockIntervalMs val expectedMessages = maxRate * waitTime / 1000 - val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000 val generatedData = new ArrayBuffer[Int] // Generate blocks 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 d1bbf39dc7897..58353a5f97c8a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -73,9 +73,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -85,24 +85,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") + assert( + Utils.timeStringAsSeconds(cp.sparkConfPairs + .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { @@ -176,7 +178,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600") + conf.set("spark.cleaner.ttl", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -207,7 +209,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000") + conf.set("spark.streaming.gracefulStopTimeout", "20000s") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) 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 26259cee77151..c357b7ae9d4da 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 @@ -296,7 +296,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) @@ -379,7 +379,8 @@ private[spark] class ApplicationMaster( logWarning( "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") } - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", + s"${waitTries.getOrElse(100000L)}ms") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { @@ -404,8 +405,8 @@ private[spark] class ApplicationMaster( // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) - val deadline = System.currentTimeMillis + totalWaitTime + val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") + val deadline = System.currentTimeMillis + totalWaitTimeMs while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { From d7f2c198678d1f5ffc1c6da3a6fb189b0a4ef070 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 13 Apr 2015 16:53:50 -0700 Subject: [PATCH 745/817] [SPARK-6881][SparkR] Changes the checkpoint directory name. Author: hlin09 Closes #5493 from hlin09/fixCheckpointDir and squashes the following commits: e67fc40 [hlin09] Change to temp dir. 1f7ed9e [hlin09] Change the checkpoint dir name. --- R/pkg/R/RDD.R | 2 +- R/pkg/R/context.R | 2 +- R/pkg/inst/tests/test_rdd.R | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 604ad03c407b9..d6a75007a6ad7 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -279,7 +279,7 @@ setMethod("unpersist", #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoints") +#' setCheckpointDir(sc, "checkpoint") #' rdd <- parallelize(sc, 1:10, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 2fc0bb294bcce..ebbb8fba1052d 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -216,7 +216,7 @@ broadcast <- function(sc, object) { #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "~/checkpoints") +#' setCheckpointDir(sc, "~/checkpoint") #' rdd <- parallelize(sc, 1:2, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index f75e0817b9406..b76e4db03e715 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -141,7 +141,8 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp unpersist(rdd2) expect_false(rdd2@env$isCached) - setCheckpointDir(sc, "checkpoints") + tempDir <- tempfile(pattern = "checkpoint") + setCheckpointDir(sc, tempDir) checkpoint(rdd2) expect_true(rdd2@env$isCheckpointed) @@ -152,7 +153,7 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp # make sure the data is collectable collect(rdd2) - unlink("checkpoints") + unlink(tempDir) }) test_that("reduce on RDD", { From 5b8b324f33e857b95de65031334846a7ca26fa60 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:15:29 -0700 Subject: [PATCH 746/817] [SPARK-6303][SQL] Remove unnecessary Average in GeneratedAggregate Because `Average` is a `PartialAggregate`, we never get a `Average` node when reaching `HashAggregation` to prepare `GeneratedAggregate`. That is why in SQLQuerySuite there is already a test for `avg` with codegen. And it works. But we can find a case in `GeneratedAggregate` to deal with `Average`. Based on the above, we actually never execute this case. So we can remove this case from `GeneratedAggregate`. Author: Liang-Chi Hsieh Closes #4996 from viirya/add_average_codegened and squashes the following commits: 621c12f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 368cfbc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 74926d1 [Liang-Chi Hsieh] Add Average in canBeCodeGened lists. --- .../sql/execution/GeneratedAggregate.scala | 45 ------------------- 1 file changed, 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 95176e425132d..b510cf033c4a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -153,51 +153,6 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) - case a @ Average(expr) => - val calcType = - expr.dataType match { - case DecimalType.Fixed(_, _) => - DecimalType.Unlimited - case _ => - expr.dataType - } - - val currentCount = AttributeReference("currentCount", LongType, nullable = false)() - val currentSum = AttributeReference("currentSum", calcType, nullable = false)() - val initialCount = Literal(0L) - val initialSum = Cast(Literal(0L), calcType) - - // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its - // UnscaledValue will be null if and only if x is null; helps with Average on decimals - val toCount = expr match { - case UnscaledValue(e) => e - case _ => expr - } - - val updateCount = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) - val updateSum = Coalesce(Add(Cast(expr, calcType), currentSum) :: currentSum :: Nil) - - val result = - expr.dataType match { - case DecimalType.Fixed(_, _) => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Cast(Divide( - Cast(currentSum, DecimalType.Unlimited), - Cast(currentCount, DecimalType.Unlimited)), a.dataType)) - case _ => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Divide(Cast(currentSum, a.dataType), Cast(currentCount, a.dataType))) - } - - AggregateEvaluation( - currentCount :: currentSum :: Nil, - initialCount :: initialSum :: Nil, - updateCount :: updateSum :: Nil, - result - ) - case m @ Max(expr) => val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() val initialValue = Literal.create(null, expr.dataType) From 4898dfa464be55772e3f9db10c48adcb3cfc9a3d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:16:33 -0700 Subject: [PATCH 747/817] [SPARK-6877][SQL] Add code generation support for Min Currently `min` is not supported in code generation. This pr adds the support for it. Author: Liang-Chi Hsieh Closes #5487 from viirya/add_min_codegen and squashes the following commits: 0ddec23 [Liang-Chi Hsieh] Add code generation support for Min. --- .../sql/catalyst/expressions/arithmetic.scala | 45 +++++++++++++++++++ .../expressions/codegen/CodeGenerator.scala | 24 ++++++++++ .../ExpressionEvaluationSuite.scala | 10 +++++ .../sql/execution/GeneratedAggregate.scala | 13 ++++++ .../spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 14 ++++-- 6 files changed, 104 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 1f6526ef66c56..566b34f7c3a6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -369,6 +369,51 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString: String = s"MaxOf($left, $right)" } +case class MinOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def foldable: Boolean = left.foldable && right.foldable + + override def nullable: Boolean = left.nullable && right.nullable + + override def children: Seq[Expression] = left :: right :: Nil + + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + lazy val ordering = left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + val evalE2 = right.eval(input) + if (evalE1 == null) { + evalE2 + } else if (evalE2 == null) { + evalE1 + } else { + if (ordering.compare(evalE1, evalE2) < 0) { + evalE1 + } else { + evalE2 + } + } + } + + override def toString: String = s"MinOf($left, $right)" +} + /** * A function that get the absolute value of the numeric value. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index aac56e1568332..d141354a0f427 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -524,6 +524,30 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case MinOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + if (${eval1.primitiveTerm} < ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + case UnscaledValue(child) => val childEval = expressionEvaluator(child) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index d2b1090a0cdd5..d4362a91d992c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -233,6 +233,16 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) } + test("MinOf") { + checkEvaluation(MinOf(1, 2), 1) + checkEvaluation(MinOf(2, 1), 1) + checkEvaluation(MinOf(1L, 2L), 1L) + checkEvaluation(MinOf(2L, 1L), 1L) + + checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) + checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal.create(null, StringType).like("a"), null) checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index b510cf033c4a4..b1ef6556de1e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -164,6 +164,17 @@ case class GeneratedAggregate( updateMax :: Nil, currentMax) + case m @ Min(expr) => + val currentMin = AttributeReference("currentMin", expr.dataType, nullable = true)() + val initialValue = Literal.create(null, expr.dataType) + val updateMin = MinOf(currentMin, expr) + + AggregateEvaluation( + currentMin :: Nil, + initialValue :: Nil, + updateMin :: Nil, + currentMin) + case CollectHashSet(Seq(expr)) => val set = AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() @@ -188,6 +199,8 @@ case class GeneratedAggregate( initialValue :: Nil, collectSets :: Nil, CountSet(set)) + + case o => sys.error(s"$o can't be codegened.") } val computationSchema = computeFunctions.flatMap(_.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f0d92ffffcda3..5b99e40c2f491 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -155,7 +155,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: CombineSum | _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: Min | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5e453e05e2ac7..73fb791c3ead7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -172,6 +172,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { testCodeGen( "SELECT max(key) FROM testData3x", Row(100) :: Nil) + // MIN + testCodeGen( + "SELECT value, min(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT min(key) FROM testData3x", + Row(1) :: Nil) // Some combinations. testCodeGen( """ @@ -179,16 +186,17 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { | value, | sum(key), | max(key), + | min(key), | avg(key), | count(key), | count(distinct key) |FROM testData3x |GROUP BY value """.stripMargin, - (1 to 100).map(i => Row(i.toString, i*3, i, i, 3, 1))) + (1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1))) testCodeGen( - "SELECT max(key), avg(key), count(key), count(distinct key) FROM testData3x", - Row(100, 50.5, 300, 100) :: Nil) + "SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 1, 50.5, 300, 100) :: Nil) // Aggregate with Code generation handling all null values testCodeGen( "SELECT sum('a'), avg('a'), count(null) FROM testData", From 435b8779df01a7477addecb1023605957bca4e9b Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Mon, 13 Apr 2015 18:21:16 -0700 Subject: [PATCH 748/817] [Spark-4848] Allow different Worker configurations in standalone cluster This refixes #3699 with the latest code. This fixes SPARK-4848 I've changed the stand-alone cluster scripts to allow different workers to have different numbers of instances, with both port and web-ui port following allong appropriately. I did this by moving the loop over instances from start-slaves and stop-slaves (on the master) to start-slave and stop-slave (on the worker). Wile I was at it, I changed SPARK_WORKER_PORT to work the same way as SPARK_WORKER_WEBUI_PORT, since the new methods work fine for both. Author: Nathan Kronenfeld Closes #5140 from nkronenfeld/feature/spark-4848 and squashes the following commits: cf5f47e [Nathan Kronenfeld] Merge remote branch 'upstream/master' into feature/spark-4848 044ca6f [Nathan Kronenfeld] Documentation and formatting as requested by by andrewor14 d739640 [Nathan Kronenfeld] Move looping through instances from the master to the workers, so that each worker respects its own number of instances and web-ui port --- sbin/start-slave.sh | 59 +++++++++++++++++++++++++++++++++++++++++--- sbin/start-slaves.sh | 11 +-------- sbin/stop-slave.sh | 43 ++++++++++++++++++++++++++++++++ sbin/stop-slaves.sh | 12 +++------ 4 files changed, 103 insertions(+), 22 deletions(-) create mode 100755 sbin/stop-slave.sh diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 5a6de11afdd3d..4c919ff76a8f5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -18,15 +18,68 @@ # # Starts a slave on the machine this script is executed on. +# +# Environment Variables +# +# SPARK_WORKER_INSTANCES The number of worker instances to run on this +# slave. Default is 1. +# SPARK_WORKER_PORT The base port number for the first worker. If set, +# subsequent workers will increment this number. If +# unset, Spark will find a valid port number, but +# with no guarantee of a predictable pattern. +# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first +# worker. Subsequent workers will increment this +# number. Default is 8081. -usage="Usage: start-slave.sh where is like spark://localhost:7077" +usage="Usage: start-slave.sh where is like spark://localhost:7077" -if [ $# -lt 2 ]; then +if [ $# -lt 1 ]; then echo $usage + echo Called as start-slave.sh $* exit 1 fi sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +# Determine desired worker port +if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 +fi + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$SPARK_WORKER_PORT" = "" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) + fi + WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) + + "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker $WORKER_NUM \ + --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" +} + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi + diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4356c03657109..24d6268815ed3 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -59,13 +59,4 @@ if [ "$START_TACHYON" == "true" ]; then fi # Launch the slaves -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" -else - if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then - SPARK_WORKER_WEBUI_PORT=8081 - fi - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" diff --git a/sbin/stop-slave.sh b/sbin/stop-slave.sh new file mode 100755 index 0000000000000..3d1da5b254f2a --- /dev/null +++ b/sbin/stop-slave.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A shell script to stop all workers on a single slave +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this slave. Default is 1. + +# Usage: stop-slave.sh +# Stops all slaves on this worker machine + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index 7c2201100ef97..54c9bd46803a9 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -29,10 +29,4 @@ if [ -e "$sbin"/../tachyon/bin/tachyon ]; then "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon killAll tachyon.worker.Worker fi -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/stop-slave.sh From 3782e1f2bec07b5ffbc8503e40591e96fce67256 Mon Sep 17 00:00:00 2001 From: Fei Wang Date: Mon, 13 Apr 2015 18:23:35 -0700 Subject: [PATCH 749/817] [SQL] [Minor] Fix for SqlApp.scala SqlApp.scala is out of date. Author: Fei Wang Closes #5485 from scwf/patch-1 and squashes the following commits: 6f731c2 [Fei Wang] SqlApp.scala compile error --- dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index d888de929fdda..cc86ef45858c9 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -36,8 +36,10 @@ object SparkSqlExample { val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ import sqlContext._ - val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF() people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() From b45059d0d7809a986ba07a447deb71f11ec6afe4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 18:26:00 -0700 Subject: [PATCH 750/817] [SPARK-5794] [SQL] fix add jar Author: Daoyuan Wang Closes #4586 from adrian-wang/addjar and squashes the following commits: efdd602 [Daoyuan Wang] move jar to another place 6c707e8 [Daoyuan Wang] restrict hive version for test 32c4fb8 [Daoyuan Wang] fix style and add a test 9957d87 [Daoyuan Wang] use sessionstate classloader in makeRDDforTable 0810e71 [Daoyuan Wang] remove variable substitution 1898309 [Daoyuan Wang] fix classnotfound 95a40da [Daoyuan Wang] support env argus in add jar, and set add jar ret to 0 --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 5 +++-- .../apache/spark/sql/hive/TableReader.scala | 5 ++++- .../spark/sql/hive/execution/commands.scala | 2 +- .../resources/hive-hcatalog-core-0.13.1.jar | Bin 0 -> 468533 bytes .../sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++++++++ 5 files changed, 23 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 6272cdedb3e48..62c061bef690a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket @@ -264,7 +264,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { - if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || + proc.isInstanceOf[AddResourceProcessor]) { val driver = new SparkSQLDriver driver.init() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index d35291543c9f9..e556c74ffb015 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -35,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils +import org.apache.spark.util.Utils /** * A trait for subclasses that handle table scans. @@ -76,7 +77,9 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + Class.forName( + relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + .asInstanceOf[Class[Deserializer]], filterOpt = None) /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 99dc58646ddd6..902a12785e3e9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -80,7 +80,7 @@ case class AddJar(path: String) extends RunnableCommand { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) - Seq.empty[Row] + Seq(Row(0)) } } diff --git a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..37af9aafad8a41d3007b63a610b1af37c324544a GIT binary patch literal 468533 zcmb5V1DIt`l0IDSvTfV8ZQHkO+qP{RUG74cZQHKuGP>AR|DN5M{l1yq>EG^s&U5kH zc;iGy<{KFqc~VgZ6bu^ZZ!ex2Q?dW}^7l8W-=A_~szP*9^5P7Ne;0!U;{GkhtKaH6 z|NHN+z(7E#|9>$#A$cirF%?yMIq}ehDXRfS#E@%{?*rWdA##PlTgW0((1C)&bw-KH z4HdLlTJrO4^%oCcjHT;Xnpg-r#q6!mlR1?yY;T~YwH#VwIvOyd{O_jcY)|(_c59+S zB4adtsWsgyaDwRcN1a=-Yh;r1CUjifha%kXnU)z$9KEJ+-r|~TYll<54EB*rKrgrB zE~AmV?7^xmE-DJYx}BsKlY z6aa~cRYKg?+8-nwDIXKxiT-T&gr8c&_=gXGfRd{I*;Qb_ePHio{$F1D?-DS7moRcL z0$7;-K@jnu1uZPy{v?g{KS)~uj9iRt?Ehp9-k)Fq8_Pe~NB>t)6;oRyI~PlUke!i@ zr?aK=A9UgWNtem*R{Q}=@t0T^CnGy&GgGHOD53k`D{*!)a`{tF(EjgLS=u_-{J|)) zztrUXdkES7$vFgnQekiJW%;L;5dH}au(!3fxBG+jlz&1yyVyG!nVVV|*_qhH<8^oV^Nc9KQ z|AB)48bJOLJN|-w#Q)~*{{j175bs|{$v@rpzwm#>xqpoT|Ff3Apc~Eii^Q1Y+S!#|+^ zoraMAF(Vs|gi$DB00F@;0|8P0i+}$%js6!G|Hdpy5hE878%uhCjghl+iw=~J>e78~?K5^-~K6QO3SY3m=rTomEnenjvI};c;P7LA` zaCK?%*vT5t7Q_aWz1V>@DRa`n40n5g)S9YCC%hG$uJsYPF7b$3v{7y>wydsg2_BE& zJ~@8qJK^N)qb=fi^<<9mw|(R7L0gEnw+dO0tw|l(tAr;qa`*8!Wlx*bGh1?dju+-+ z_Y9wQshMotr1o6$@FP5?HTG{4hjz){vT?5!VeVL68xqwaPBGqaac_ja{U#G5zFiXh zxCEYDRQXh%C34#|!q>^|G_u z4vQ%XSfAMmE{_o=4oJxTRRh{4MlSoWMnpdeFt~qPUOwiI<8HpD12o+`K1V zU?`vc1wZ&Y)wG4bBkDR?qNgwac8;xa7yF}z_lo1$nZJV`l#8_Z$9GL-lHxBW&GUy} zhVt>5UyKM4Zj6Gz_&(rwuQ#0^tC9N4`iauMR*Z>2_-{)P95P_OXT-xt9TB8IVSqoA zCHK(M@*CZ0__$#BFCCIV2w2{!@PA)|WFyyKAjt8DKijkNi`^iui~-)amktqWpd+H7-q9+cSwfLxo|@Py zqv*hhcc4V;7gyWUY_`(#rBow`HfGVVr^GhT(r<0C)UdFxphgEFu%-HPu{e4Kn`ItQ zFa8x3Zm38jOnFYUYcJ~S76mAIH4}bY?yDwcW^;q1@Ahz^B=KDOngB_4#5XRqSTdQB zGA)t|YqO%rs;OI(j(Uc@k_$-@-kgYldfma7&_ya_J6J9$9t`JY$T7CLjIAWeTB(wQ zMBSJqq?qAl-ETC9E+UPU;H-lsNf}j`Di#hPd~IuEq!ulu!c;ntVJNqC2ev8YRUR5k z+%e}&h~Zxk91v`8L2x~?IW8?Mv^G}fdre4L7si<0Wmhey3?bL}=MqV(xH6IP3b6OK zC6mnxU1&GGVniHcL&5pqpm}nP62~r(W5c}~Hn+@qz8lt8Y@7kIfMP@3Il^C?@1?N{ z%3bbyBu_qZ5FdvGO3z%1VwM(c{DA<1w%EH$7EzN9$8t zaJMN<{)1mwp`WRj44IspQ!KToO#tI=!j&PEBU@jzca6P_?wAuXD&F40f?LXF?P#20 zHanct@#L3sAhomrH}RQK4X&8esSR>cT7j5$A%!T8o!uNod~1QYbs2ag(Kx0R8t7bZ z(lt2^L^X`k+C|`y-bp-}AQZFpG7&NpkDTOO0bi4D`mH5X z9yikHVC3NQpe&mk1wm(0*g}1SG$J~v;L-Vpm~IO8i&(l=#V}>(gSG6apZy@6wk8bv zaBKa-j<-@3WGuHjuDZCL$(0G9-*3I>x|3yFC|K1L2K%JHe%oEN^k@;Uonx0g<}s{2 z2>Hn&mQ0n)(i|Ce;UPMTE~xETbUP%IR+LrbQ$%QmCVo&kHKxcR+tGEQN@YAKS$n2x zzztsHR-q9dEgkdvc6*<3xsjeN$1z@pC(l|sl+%HMB>)0$CrTb+MJ`=BoP|i%CJbouQ3Z{Yeoc*(o4*D_vp_)2JLqRzomWuTN?DwY=hvZ2X6FF1OiiT z+Za_y39V#@ejqTq4}>22ctFICAvIDn>TPtGu?&^kAnX0%tvO|S>E?Lwkg^<|DJ6;( zl4a1EEZ#+!&#hs4@+ls$lz3Z;MrPVwrLl~b6$qt^r2*ZTUJk5ygX+N6mOOFnH_f7* zzPb<;*~its(dnub)hffr8F-1#A~(^rMSH6XTYE_Iq4Do-xLTE7tPV-`4h^N`RZ_Aj zZ_MD%pP`ZnmNx=ghk42J>wPix;B2}mGo!p+0hYs3OA!y7pfBDRn1|VQcnp*Th7f~Z zuFZXyi>ME>07PvZ1#@m9R+6;4>PlJXktH{6-eB`0--bn2HRCz^(>8PkfxsnusH#*lc?umV_ z7^^Fk1^W4)P0#df14Zd)Xwh-V#Kyx?ItI^90NfXyH(|xri=-E^K4zJz6xv}JCt;3q z4m0X%g%SYLcZlT(k#Z83L1pDDB*+4j3avEgswOzL4P|Flstyd9DYbiAHUlFkli+vt zQRSnm7|XlcLbJ+H5Q?wrfJ$h!$Ub%vx<=JhZJSEx#r8Dr_;_ubsD}>}ZzQH?Er;-a zy@*NrhDBw%&Kc#2)32q|8W@^Z?NzO+vAxl)WElIIDMdchtC>t2O{6)b$O={mH|eG7 z6)o7(s?5G+g>Y)tyPjV13NmnJ))691nGfvaLDV1?gW^gS7#b zEiRbS+9*NW#C^VVstU9Ct$a>v+@TOj)o?A_$~dYh3!b0hez1>Cc$b=tWWO|9N@SN} zP_bp#DFqS_Z`nu|#3P#NUP5U&AXkXGR!&#trTDvbDZYdDN%L@e59KvqUKnukJeL`o zV(<^Cy$M(13)Vr~!Ku4}mH{9GE6h0# zY;+pCY^qfn>i46f#xK958MFWeD9hL;74;#>v{br1n?Tq)9=&WJBQH@&x2IUT0d(Cp`{rB+UPx+^@ltA9t8CNO>kwJewXDKBaK%_O@kCo) zgW4;*IC2+X$~pE{Nl?6vYVs6$GpNwGIyI?FTA7xD=Wm_M2~!2XJKRZKsv4Ea2%dux z^AXV|2W7~wc{a__=Q<<|1-}i--I0sP+hOC$r>`xX9v;kS05DQx*I4c^R0x8@SroQC z9ajq<%`W-mjSY?|IqiLNiKi#$gQYbfW|Q*H{|zl z`m((}7FW2~VB~!@*lK&JS=V5OJqa;3H!r#cBI^X@H*ULrWO!Vi6;US2=Q?y@X4X>D z_)xbu-&humy5VjahzTvIEmJZU^EeL#+^M%3cDb{r9`w+iR$A$1$0LGcB*(Fci)TzOMSp35wvg?F#*#=zm7cq5F1{ap zme(ro#87t_>lmhkm04anIb_j_2g|K!4ZaseJYrnR#ZBuM)#&J> z93?WA-KDjHW4I%A9jRGN40lpe3-$@gD4-q{x>1hmBO`p*VPg8X4sVLi+OZ0Hx; z&?$#|kbUPaw5hmk@ju0U9qEq4$$?cr@rt{}qn`xe4 z+X*$ANc3Ed?76FL9>>ld4PzjbLmtl;8|4kixQad6F*ailpn_D$o`*qH>6h4wXwu~S z?_qOn8#G^ee4%Z{nzT9Ouerzg-1R}%mXi|*ICznA28U>l^Qd*f4g4Ub#|g9D5s7V+ z`Zq$}pd13GIQf>sSk$X2G4n?5)BEbiR0CR^zT)Z|u>Z7`I9?>hz~5(=Y#Hd`W-Y$o z;B6c)!2<&I;ayQs95~&*l%~@oVP%}36Pp~_3Sm0|=?o%Pu84`7JlVC;9bVafStvrnBcq%qrzi>!L4g$g z=ivbBkNA>@XMNhHGQ#w?*&yo&yOzjC&ve6y++EaClteSKv(Sqy%M6nzG1*E ziGue(BBVRdut88>0;0PbmCr3MpI%bYdr>(#t0eQP;_yKD!LA3{n$xk=|`Brvb*z&2lr!XEToWii~3a*wG7e8p=L| z1zm}YO^GJ;=BO*$yHbBl?p3Qm-LUt&R^t0WzIKgPteUHdGN@c(3v2jtV82&Y zAzZ-%h_qaDVpb*IjQqN&-lDjp4_9+0e876kqR`WPX++KCgKgC8wIxUA*dF2KiJA*s z+rowikKRQ3!iFct7Hr)17*t2#)0~`7!1l<>LhU>KE8>pymlwB=*z1yw7yXW4f+C(5 zrjF?RlFk>N2PnTZ{wao8zIW%2bbpDb`-ulGzf|~JCTvA|?iyNbWtG!=YC(g*=J?9Y zhi_G-3{?<5^~DU8j}N9xG71%Z4<#K*dCK->^=Gr)2+G+W0`dmplJ+6|_~jImff6vq z^$AeY_e`(US_MSKBpE`z3&{_t%Tr1lf@eQ8H!6?k79QYA;yF2A$bDh5+!nzsj&urM z5e4LS3e88SHayvV)4L_`H7(HjWOgV(kp*M~fbyW)^UF>c?}XQR&OBp$5+220iNuE< zNG?`Il2@=F0Nmo%l2{KY!TN;kr{#4kmeE2C*bgl4?lIk>W@b)cX0tL7zcvdDAYU0M zCf^WplQTJ`7ap3x6eg-zPBl8=UR|;7j((?wYqXvSC60T({7K3uBq3niy0?(FrreVE zjH|;BV(Z(=`Y(`r&DI z3ABeVuOz6Em(9{r@gJi85G?*<#|N+<27wIHROGHHJ`ley*|Z@+7#3)-x;?2zF$GsO zn)?Z?=%21}3mjyQ028`m6PsjL$Sb)nTBeb2n^p?F2l22;z6oaL!S%w%Gc1yN$lg3G z4J(@pFn485yHP^t{E~Toq#;7*iMe&I&YulD(_IKUKLUblDo`ANz!y4=t_WfE;Z^Db z6LE+J++LyjPr%R1@|9xri@+S_zowp=r-ZP3~Gk;XN3=;R^(TSU_37~sc zMomL_#}yVbiVJ~Z<9lLqn_$+w9fo(r2w>!bV&X>H6O{1D1s^UJQId0{ys6y$C~{`1 za|74?VC4=DW(7q2?1z0D7dz8{o%NGNm`w$B+2!Buk~+xY6V-4PKCtJP)i^o$E_M@k zimm8T{Q$jsY70bnpW*>OKN#R0P0JRVIYb5V{#0v%G8oSQtI_-w}%c(dk|GpT9X%7RI8x>V6 zN4)x=DP;HSZKB6@25Gb6#3b^Jtw~D81YwSIw|fVN$HZkbB*R;n(AKXqv4O7$$QkB)NdGnSeeSqwCbK_uM~~ zF?3RPBVEW8%Bnq~^+x@(xHUJgAlR-_JV{X0 z$sIm-H7)iw2;_Um_z3!PahEfEt5VFl=7-e&yzuNjL%8LQGer~fqv!e={Au^vT)uI) ziQ58Ds2oP%ybY zZoEBj`e13X&)Nc^zLd3IVy7LL&jiIcNoUr#g;X7WFXAeRY zmNpmnEU_7{Mq=hiza^pB@m-fqCsSffv8vt8dlXhQ#!Lp>H60F@!t`vo7%m1@d9TRp zE|*tcjktBY;Cs}>Tp>~}aFCd8;NWTLr4Q(@>4Q*C@=sf}U4P(z?3Dj!UH-m@WwZ<) z2uOtp2uS3w>hk|>?@GkU^!LuIs*$mcsieKV^?w%ZYqelJaF<*^;#=0&Mphe-FsyrN z#am6a=GP%`x|kufk`ir*pk$a=fpc!JG_kunn=Ynbm&h$uBysMnkS&;O@rUG=j_20l zN73y9?!M~0Le8>+A!8#awOUGBKw9De96otS*0epzCi3t1a^JY0a-3&gX1JcBPPV-- z9D!&|%ptQK31b9?(pld#!vcXmc)D)v;2cw@~MPqg4hM83gNfIM#? zvhQBA@?r?1egS#*Uit{u_Mr{%R}J=uKFIdacl0C*!ML3P_K_Lrv3Pa^VxVf>nY(d< zWa=u~>m;Hbc@p+O`>fh~Bhn29wYte#e*$%#CaBu`aehzo_~Hop8RoxoLtu4-Qu^dy zwXMMRt^pbAK^nqw(*(4wdg&j2>)eGl%y5$u);3@<#K86*Z}q8h`63{8Uri{#pNlBJ zw?();aE9mwu8RgWnv#$bsgUd>_a0OLuu;gw6JJMFZQU`wU^b>AS-|+==A>R$FSKg_ zFr_uGEvWGtGYhOAlTT_c+8WQm3w>~7St-V>m~!ivv}Jcn!$wee&GDmQ_*S?z{Wug; zlX%Jj__a*$ha3hdx#-p;HXZ7iYEE>l5d{oOUQX$}XsS%J;>Jr<+g_KT7urm_q(3G8 z09_J$k14S==WBH@H`XFa=K$2=qBU%pAEQ>2^BiP&=Y-ueTU=b7?MmUK$z~}NF4_)( zJyrLc^WV+Jw6x&>I2Zm)`c^2$yR%&*px_@IgmGA?pi!$AAa3;s7UG-=NTRYc4)X zPx7X(6iYrLZT6_&!`mtB(j*>rQO;#0J$0VOSc9Y(J$s6B2Uv_WXti}?Z;{(EJ`0jc zxK`6cjAy5uFt_QYGs5>})AmNqLaC}EDNU4IQ|aW&W}UXBbfz5PIQ5>-mAG%w{3XIR z6&v)2$j4qXM1bXtM&=7$xE?C_sve4vV+U#$`E`s)`|F zPwBIGv05ZrJJ0pX#*R8BHI(amZG+C8*<^8QPR2A5er>z4i&!Hs9Z72uYO;M4s&<@C zFnZdtK2`n7Eq3B&DW%ttLQ^TBhp{BKYLfh27O9;;t~6R-G_71bSEhJ8c=OtPsHqV< zdiUhOcZnQh?%Ql|Y^>`&uRK4^JOa-RF9j@vxRo{5L-`u97IojtR+jge8Q^KQbC8@sJw|P z8Gg!qDx_sV+T&a)20F3AeCYKQy%SLm<4!&$(<42VwYWG(6$j7OF{{B(*ud6GIcGyW zeQ%YN-349k+{0brq%7^&0`P=66e+Uc^g4wray+?aW1ljCRQcF382Vy2=b0qs?A$p{ z(NUB5gQCQ(Nubic#koYEhJ%sf+^Luq+AK%TuvWY~-%>5zbsR>nQ8wMoa?bQS_I3$> z?YNw6lwq);S)_2qgt*uWJ;#svLPHELjY_;1&Gfwo7MvgRRC+bM18814JsPI2?PBaq zra$KC__7}2BB<#_J8aZ%8GUJ!T-((wdN)dx(&_qXU-;Nh;0bMGi!!f>jstIWH}n_b9n za_@Z9SlX^coiEWxu5ra0@fEo9M8wQQ6Rq9|3pLMCrB5SCO=SnjPqPB}-W7M*PFT9{ zjx`TZ#!%<83sYliG9R{ESMooDoPLgJ>|@nDTs637(Qc=!rz>yhk?w6ertwx+Yk*&W zCA+AQRp;REx^!8e{uV%9x5(DFC)U*1d6c%D7FU+`XnRSwvCiG$<&C)~mYn;Y@5*LK zEAUEfU1g+cs^&aNqE7X-17X3{JJT#e6~Cw|<@}^u+t*WuPn5nA9|NBV9ISp4YFT!h zM@L2yM$^3CLjEyno? zO+}jKZ7@S^U-LGZA{2Yd=rOfW_j{(yif)^PCm&DRN70R@0;#Uw-CD=yO}OLL&yYPR zI3-r;i`k5+&X%`dfpQIvS$os}0JKhYeqUY0r))8GeSQ%Ed zem?(PNwI^YkF{!30$rxD6A@d&1v^&xlGu>QG9UDVyj-?YvKD6~gq}Ag9#ynz(VWGZ zq=kDUcV)e`-Bbr)uV8&r)><3OXPb++q0NoG($dCyf6SL>FU?X$u9m#EJ2b|Xg>RVTPLH$20O?95UG9v`pVk@p{VWzfhJooQJNS4Mph1 z&)zKMU=qn&8wtIF`OP3^5veABDj;y<;6& z5{USN7ao+z+;LHo7Ld+h`4XO&eZW2p_{y{-g(MkoqNJQYvyH{yL=zE+gcZX{_J}4? zE=I(cI#q8->7#!j`68xkOcM;EArO^i;EK3RAn9tg8zO2{4pFzJ!i0ZGZ^9cS*0^5? zhe)^yfir5jXf(BC)%kh`zxouCNB+ED>pf1pQn0SRq-F%n5v5NOTe zyG>3Y2&2v(iu!|op5Qo#Y(7HUgBV8|&q*-9>Wv{Dmm6{$eN(uD@8m)#84jmIDQ(%@ z3J|!3zC~tfL8?TLIHt?E6U;OxpFjbMeW0mIkOr-!XL`j2!KWP*EA`Rcs|5iJ5*<{t zhqi}%vw8PNYN+1^!f7gx=^M>#*N^_qQ=Wwqc9^T=`pIBeytlog@ zY=<}Du=zcpRoHw^S&Ja8p6TFvYWD_m5aD_T{F^D{I-{NYsJu*Hx?DGr{4ku^HjDSR z-De$Ebt{1y+_(9jQSXQ=p@U*F83JCtBI{kc##YQ>byk91{^!U@HXTF)aHyNl@ibEeIJ zj`2PQ>`dDh;u~q162y&?)p0p0nkE6@+EkSCL;9oCN^ zKP-P!@@wOg3YD{XZ{*|MtAvKdp_uOr3;{oK638Jv^lOrh&VJ_qn+h8ffZq2s~ES5{8{h zo4`b_1R6mk2u)=dLCDm=j+ZvZY_zsE2_I4-XX{&nz_z}uS=_p;VJX*aoC@ZxeaT>} z@L7Vd^VaU3K6N+RXu?Gi-o&!=viBHeTcALLy^3tq_XVBl)IrDZ(Q@51u2i^l$CY&`5e~R-zsr@-Nvm` z-Ij{ZgaYG=9$`{C;|}1mDw(i*%x;}PTw#1&$+@oUKg66j?hG5sYAT3UEUaZ^*5LW1 zQXR8W*0ncYC=f65qO_$fjg6@Yz5<{YvK`u&C#>y8i{_*yPVWPaU0olUf)seJG6eN& z99fJU?vSQ!QkiDdz^-V)V9VbwY(a3G-R6I2)tzxMBl|9=^wyzGvDz2alv1r{Nmy;( zO(GrVHxPn4m0B-&d~3)^oIsF719?HFuh!*`C%zow5*whw_N$fNs>Y@_ttLC*UQv=r zNvE$9l0@rh5(X6>R`o=$>TM4j(J-ic)$^W@(_yM^p)`s-+7@=t`0-SbD=F`xjFkM` zSH$==j~ivXi#^W?Boe1X4n&#>gBE^W9HS6!0#Kld^DG_)9-~G-eyX4xE!fSh()+Wz zu`hwD!~9JFo!8{!e{is1S8OMtkeD$cr2?g9ckp=px1;R_86PpS~O1HW_lPD0se z9w#KDzcs~)vsyljju`P0JIq+okfWR*14QUwsp9$Yn!Zur^iG+TEutjHTi-EJI2hVy zIiUBZzArN6-08A(i0a}ugScZ@$g3xp>LvDyh~eMX^ulb2ySaxhW13m>kCZ9TCxf1G z*dsMIzj_+&Rd)81nO2r`MVMT1pMTw*X7h7Kp@8(za_th%VLSxmVc*Njw3_n?HeE8Y zq^9~R$L~R~d#3M}yXcJqHE|h}O$CCwI={fP=-YKL*Ow;}|A7Ak^MVhRYa8!ub!TNO z^@LT=4-aHRJp!m2kK_A}k)31NOTNC~$0zR%|+LDcMCNfVsI`uM!^1=+5>)@+e*5|TKlGSbH#}I2TD5-na zH#5XHR6a?!ND}YTI%qq0pPtxmMtD~fbK2>1>ovzwt|pCI#8gWh=%%{jFR*9jBiO&ID&bTt6Ip);~et*Le_x zy9O9yed->0~HP7J9^13(o{=(F)FO6pi(D`Uau?K zL+GO3LD=X@3+Md{h&NYpsdI-q&gwtdViQt6dGDWLmI>g%>xzEfV;NWUG4Tr7-0{W0 zo#PFN7bUTC3VMU=2(fifBvP`^S$I1HEl2DTim#omaRzakDlam}9Y`uh2P2Vk9iU3+ z#d(aoQD96Lw|1!H%^6&BXZ z3=F)Bb&Gh`-orMUSN+ybj33 z-lm`nj3d9qHu(I^LM#L0nEvGcW=e1qe{>VR1eTMqQalT96K8l@-wPOkBlcddit4CS z-RyNm8)SjfmMRk}9Av8L8(pj%;F~XBux)gfPhywMVRwJCU+Uf)ck%&TXnpPh^HThQ zw-rW#DW9_B>sQSSN^kXFZ%lsm+|i`<^QYP zw`l6P<1C?l#L;=$cp_5}xe;3kg{F+IIO0MQQb6dqohu%6G$3Qshc>tn%h66nWGW$N zqt8287h*^jJ}E;Zcv=vgPT7rXmn!f+P&Aw#@vt-^52e{Ks92v(&hfup{JONicEjY{B>>3>aX|0ci$WftFCspW>6Q~q6f^=j?d4E)mLEW2;*BilcA&^pPqKI0$Ku5a zjtrGINmO}N&g9&Q^MM7|96t@kL|0l=jOq(3&T6#EQ;Ee9LFj^it{Aj9DC*zrD0i@gcl6167Urtv}7ce<;-$SviCs?td*NNp+*7M# zTy?#{*K>_vYq9s)$1JTc?x&rWiXh^KB+9R(2RSjX0K0Q1^)%m!2CfFSHo>BwAy^g9 z>T?+ly!oJWHht8`V6DMLOLZv`M=1emaK&htJ4{B+)?UqvNjetr<@CR^fxQwK5WYvU zMpb>9WRbQIUmsthxVh;%&z(tnezUY-%f45weJdl&#SGH$T+@+z>(HbZTfjPV28G%Ara&ijfw9A*W+|&!wuJe)*t*uFv zBM?UfsxF384Q`olEb$j?b}8n;opR z-`&Hb_s&zPDWCQCcjA$dF>v~&R)ywn{-_ z&#LpypV4)D$`jO51Ma1)%r>Vt-BZG9K0Dad?lRn5fMnVxTIW66AA`r8N3l4g5Ev|> zJO{MW^bC1lz0$aS z5y)}KMPAHaXPmva7!SI><#40~rPhBldrWh&pz)Hep@`gbxL?W6=V4S- zdPdhUC#lR*aw^RRzm&WdW&I*EynaatWIfrwEQW9H39+8W&&QURiEQBpvo;yiLA-QD z0OB@Ky0{y%6ZvDi3G-NHQO_qI<^4|87-uj23o}{xD$BDfV6Iy&LZERWh@r_ACgkZ3 z^t2EJo&*c?T#TTFQ>#k?aG9ca*%OBzxMsbY9pw`;+U`5&VS;8$fKVgIgMNl%p(pev zFn7x__z^Wd+F;?{%yY+|hkY6s23pzbu>t+?W0ma28_XX-bebk`-|qkQn&DKp?acy`3Ac~IqMEb{EV%_5j3NM)JVZCltNRA+*V$d^$S=DuYV_hFjnl zX2By&Gj9ZucDHbQl4Xv7TvYhj2_qeTVSUAZ^?2zy-jFYFkA|awt2XBeT#;vh$MvDi zO>hoUpU4tz2)^<+P3b zfIp;R*%JGb-3Jh@`eyfKjIYS36I0`m4JD>t+SQ*Y^v;nK;QrJ*n)im~4Ol?UIX2Ye`p08Zg(C36_q9Yi?1)ob4-9odM=llJ)v-g0+s^o;*Y1y?rz2S zksXYil2vRdQQlbBZYC5%mBp`x6B_*@Xb18|i97Qzh4iSbkD$lnHQyPL{HZ=N+=MSn z;7CzjGWb8Y<+{JRRIEGn)`w3CQjXe9A%qi~^%Lb11c>~6{`tlWYF03_fE$rPGZchU z{nJFVN73J((xP6opS9K75<=(Ibzx;{zm;oDn4Sccna4NrJ8#?mTq~kk3Gsy?XIx{e zQdc!IY#qVuFMOuqs;UfC!og=is&Al(@WwdX`$596P!!xb;X_h7N#)2(ljYq z)pz0H9jtq{=j@k~)V#H?rCC_X^%dAHKE4nm>bBE+W>EPffX zLoL2#XJQCoSRr5)yO}_WVm8lOY%xg`!V$!}P423~#UuG%*!OHK#w>xS(@c_SAUm#a zlxNBp43J{KijbiQ$uy&&#Z(E+P3#LeyGGmBh`08PVN(mFKDK92Bn53vrXT>M?TbM< zPBR|^{m@KniOUF4Jv@&CCA$~LgrR1FGRV)YARf*YwGHB8Vqd5kiMt;T4gCUEN3up6 zco=)hR9R>SBE%<$yQ?dLA6lnu5h*N#)}dW!3QebB5h}d0I<(UQ?JG&dpivIZbWcq$ zhNeBmRx#BsUf6QcA2%DYx3cmj1ZDHAs(h$l8m{xJ&Vbx%r4rl{b+ddicNIlVo~Ea| z`6)k3nvdA(@>e__RI6DB7Oz#3YY8pf)mDCO1sq5XGcD)aZJc-(ehXqd6tk8vv2RCi zo{T1G5)l1+2}b8z(Q8tRlA5fAQ4)pZvF!7%A(_QZhaG`}25oi@UR$_}2oBGOLgQ1j z`h2H!D;1gSkrL73wTt9o-TK9D57Fvh_NpS$m9SrDXWC6ijK#ynbDJ{@i|62)#R_rQ z)}=RC*X4OwJEVBs=ILYIlH-NSlrUbf!m(Xda7g428n4Yly;#pn19mry=k(738CHu7 zUO}kFy;i7YzA@Udutex{qb0zxKks!d65tt=*tjLbF{iP8GwhU3)no07;~87oI&%4| z=0;B8`|qAo!&p01VDB9}WQse_@I(0v=e`%6B)br-&{#IIUE>QsivBFmc=?6x8->5i zS3Vcq04zWU!CNzj)X3^3`QRIduYQUY%P*6^aC+Y_U{D}V^e(pDjO`m4NA+MDi(fi| z`Whu3zkt%^%7!~g6dQIb727SXa4WxjYH3AMll^WP`PJ`EdigeN60-k+E4BC;48;;1 zxwz5%ST3L6C~kf6z)X_fUc%mto{2n9JYdLE<^!wP0diZ z+EgbW8CvE3@Hsj$>#WJa*vW1NgaFXUg<-ascRw=$NZm!xUO=S_g#SD`CI&8}W^XF^r${Qr{nr zE4|n1d2^bY9lB=~K3~z+N_x-MR%Qeoo#|DO<-4rOjIq1@#4Z$iLnY^QCfVx_ShYJV z%l(O}Hy)is9z|b0>T&JTa6jqsCTepldP`!LfvwZFxZ<{4SAA)5I$9JapLlX>*7PXX zcN4HycZ~yy>E)h{v^tY&5zm=YQ@cxLplc-hgn$WrUqErCr&CE$7`T+)@VX3qPyDJE zX-c%I{hM>-WVGmh*gD+oO?rGE)MZ>6QH;pZLuyz@68VMEA#loS7lL{Diy{>=ofs5e zRQtZ4dHVoAUvC-=u1t7otzKe}IqRTk&0m2vR~gi#*-LodH#0vhEbJ4 zdZOzHY=sIY#*pUf`7%e)j?qVyY{GmSR6S~0*_XA877fYgHdaODCBAH6Oqaq6m!lXv zNQ*GxFvRp?glZ_oC`-{*uFdG}Qof>wKB71EWs=h{Z4*kahhtqv5(slZ%)*-Cc5_v+ z#_SHh8~4)AvXj>g6jkFO5o0}r{(r>1Q*dulx9z)P+qP{xS;0THZQEF}ZQHhOd&Rbs z75ikLeeS9H>Q>$Tc;Dun594LjtkFm7t^KaeVxWKZb~nl5Cc z9K(yavUgXY6~+1DYoT%#u1h$(0T}Grx+#5orq+d}(el%}q*l>7wp;jqjGK)p2SXb% zV|;KLX^JJ#AUk?%;tPH6M6B3i~Zz^(fcf9kWu`zA(I8JWV*%> zku!0+$x^VhK|xNk5_i(F@bDL#^<_=Y;RM>`rqxcInO=HPCxx03m82-))Uc{G$nNfr z>t&r5hXKW+c3fNGacS%hue9WYY7S_(exu7B%j~r_xPU-2CP+s<3z-ARv^MRf`!A&z z2yq9KM<~vU!Zb%{uAow|jm{7CfgN`Hcf6A51lyzsZb=Twbz+OkqHO8japy+w9pbZ} z?}^TxMLeWiRONaLwvsNA&rEoR*zq^-WO&A&X=gyMzNtlS0MFQ>+=K@qNd$nW=t3x| zkLbcEnUC-S1=*)=l&k`&i;C zkK!vErAn9b%t`y=aJHqCnTssyD}!)k?UAzh2GgNx>6x8CNWdcHs-~hlvY1)g!7Gei z*OQVpZnYqgzzU=1-vy?n`NgtQlvP27f_U8#mj@QsMTWZY`Gs@)T~BfN7|wt8OCvfS zuv*A&11XGSUdXbC zYfsTWVz~553HCUqUbWt!w)0kQy*%;k7PQSw?4^L} z)Wvn3WguP28}pCyczxDGL3<|m*vo0FeK1YS1zHVpvt5BUY+uSQhexh$E87hhO@Y>U zWI}z1EP;t3g1WyB~;~36Bwku17 zRoCnFhf-V0E#yGKBhf(Qt!MLjnqhwnbA-w^@IQ3bPMA$xN;UC%@`&c`0U}h7NZCT` z%Fa_#%h@&u8tleJQv4tAV%tdjSG&KZ49Kt0?m|}o^AIN*^ zt&A;Ow3b#ehwfc-xrYWkJl*MPxPt|${&^4N%x*9T4)j(2F8}nF+g&hrB_UJ}=#l|W zC4;ClFvMkf9GAS`*FysKA-}ld#pw?dBbcd&w5bPxsYjqQB9fISJdDuU6L6|oaXQOj zyPdHDSRs(8NAXxOp69)BOK9wq(092i`dZ7O&C0QNO2yoxdEG-*^@D1N3algxmz8C# zUNr`LU915hF7kLbC7$bI7iaA2f^JGATL2kn0hi?{=!S}Lk3H*BWs>BV zvUvM@6!yn5bD{x$CgTmHcc_71La+6!6W1+mp}PBy^aGZzLZLHFeEvP$%G+>7rv z6`u%C{?^RfzQa$Ud4bWIZ2euQ+@J2|um*Wd<38`}%M%YbvD)ls0r7V8$B8Gf4~T)w zUSR0kFGp6~YSRz+Kb4WV!Z5|gANPaE4%}UCXs>$DEF-xyCu+GUemIm;1^5iEuf=Ss#?`Q zs`ytd1^fk#tbPpEY_{}KeZT*nBM8{(&WuB3w=zupFugw5bn3b9GW++ooT>*_2W*F< z9uwtl%y!7i2|f|OQokeks|E<62Nw?8d<2M#n{oyIjMI9TA9voN*T3*3&ELpfa5T_y zU^vv7*LoMpd$Z4hmzp3J%N>^j)*CpN^_D6tDOS$Pomj#)cm7TTE|+EQ4+UA_EY1sX zPC{u<*`5Vnsk66ajX>d^21xWv!{%EAu8+#-%HiP!_m4FK4z3i8TCLQ{!Y^9TqsXCD zFmGCN4VVq@ENhD`?W%bTPt`7j#syDyVdAoBvneZ<7W~7%iDGVbukF);#*Pf)@R0Z@T0tN2|t{XX>Oa%y3&Emw%4LsP4rdv|07`FZbA+5~oi(4##@Mm_!D~ zjok1P9SRGtoCD80c8bhq$MWm>`S_rd(4?B_+< z7_-SuQmiHz^Ksy0rXV5=Rbzs#{AR`mTN2T;P1pcJx7yH%ebV^E}9-k zz(>O@Q6gy3W?-i7Exq-n8?KGIE94vu_FBdr%-9w|j=dZtbxlL_RaaV6+*~#Gh-o7v zTPXsBjHP4s{0#IQ#rrWz3j(lI z{?KfwVQ3s-n=%;QP4gE${1)@KT_JXsFDW5*R-C1KVCWF0y?^EpU^^tWG!ra%Q#_e0 zkQ2!+*-H*LPOCSJbWr60)m zFvXMdxNkaUi~`&WHF+8`T@eKHEeUH@9!7Jg- z5^d=(Y`%jpTjuj9<#jhf>FIk+ex43w8!$ZWw=tbsJnWfh1VkmaIZrHwd`v~rfLj-r^Fk{Y*BKnh<1xXi&Fat0R)%-iIgFc&P)iiKiSEH!P zbSBRlfmp;JC|^?%Oi=hxBlmbKF*{c6Pk&e_jSRrP4EJkaGQ(SM-Bhn7WzbA%hw(V7 zh5%<5TD4k}Ljpxa>@stfJ{Qi)*tPQsRNE%Nd*jgpQi?ox;07gfwYV^}SyJstVX~!v zq~#k$4-+i&l#S#*xCew`Sz3FI34>aML ze9Dk|)uN1~1CY9I^;+da%iN#~xGex19~b`)cf=ijFf-qQ)aZqA@1wr#-?YC8S*02D zEf>0WSqrjVnLPZ@HUMg)Ir(g9@NsW-> z3I=qrA*`fjA5ul?tIt5ykxW5e>SUrVP8Gid2Q(G!S;q5b-Ui!DS;QC}TUCBM0uyFM ziQlEcVYJv_#c4_K`bwHcN$4;y^X<6*N^ylQLyQ->|6-3ql%~50^ke+#Ki#iDJu0#I5ROB`=4zcy)fq zOq&mS=B)O8wo~?!=bNvV86<{GBVw;x)#%5Bcu7oeH|fU+N=pX1n_?xOMWHX1Ulmxg z*yHpGpC`^mFZ?eie8w)aFvDqU9J=|*^_pm%-cpgIuR|~hpiAT`xliJOwF*b z10|!FJ%B=-uQ$|*;wf=5Ko_o0$r&U+0EyM~?uE=-lpTUq0gXju1L-t&#>(Y1F^_nh zk_!M$?GEyAxu?+;a#!PMGB_<&dowwh8ON&klE7sM<>9Cs>;4LLw ztHm4?23$L*BYaSE%9s-3EGdj4My3cB-$##u{4bT) z$jsp)jQ;K}iDNhMJ2GAm+gyL=1ATjhSb?Tvn4LAOi<30Qz8ldgRi&rO=ufL5_(6US z5>qTvJ!U2*K_-^T)#h4&O;jen^8v{!Jj~7Luj}AI1o;VD+#!72K?nz2f)PnY-tgFc zjfZ_M(mP=5g3}YXpH z5zUmi2*F7-dQ@kC2LiD65h-&J((P%$A}*>Ntr6u7S(nydv<+1d6bHo$O^3 z^e9tIapp{Q+99Q;as1;cSce2v4RHWRBrN}s!mmov1XaZ+^iGRPm#GpU2Oc_8A*J*e zaMLe{NVW%Ngj(W?28Ayby+V1VqZ7X1EeAhII4Zc0uZu-<>m_ZHIDIJ1^eqszxiPcK z5a+>OLeq%KFe>(h(EU8Q5h1+Z-yo@ROnG6)@zc60_RpXH7Z&{A5lBSO6`a?f>Id+H zC;Xuk{s$rSKhT8#8=xs|;OX&Sy&y#sBRfaq|1N>L)!zTB1p3}4Wy~Cp6QmTPOtJuL zSfkbOFb}?^gaZ{$EFlMqdYm7>HfAwqZk(AS2dzks_^WDD`8dD+M)|Z?XlXtPj!pMi z{rK6^^T_cHCO}YqvYozlX(XHx@Kwj-cFp(fHNzKs!*{>$m*a)shyGEB4waQMAMm`q z_t^#*fOwhO@;=VtUfK<0<_|&uw5bjFAZ;>;jEIwZE0gC#7}NHTzMRtcJRczQBlT05 zut{N<4&Kbfm_D0g% zuhom$Er;wG819+av1mX?kb1%POOM#C3Mw@jdTrs%iNB@Y@sRxuzqt8vk4(>~?DmYW zZ+_X5?L5F1+KLBo-Z*QzZepQLw{MdEE%ykIx0r~J!Cew<1*O!u0Oc1GRH3t>zsm4r zg+<3zK=HdN$;J$sWTY*}-i4yrPI6Uvvu+cX&zfTXhHI_pB+&WDj7w9KLX4(Bht8nG zlKV1gW^T?#{cU=!o@fheE<{=ESMam|}a^*}cHY?idYWPn-^!V{ia$|3Tcu<=rJLkn-kle@&e1WV8g!M4^u3F3l>0=njyu*;-c2M?`q|`+ty{fl)q7E}9lonZ#2(u;#Yk_$W6TiQfcKCeS)1%Ik2Vkm; z)-q+!s%gwy9BNR!aROBhxHr-a5O39k-4H?P_>0iSUXrS+mKP-CAZq)937d&gl@eoG z6a#qS9?YyAVj381Tb&=hbify{(DtmATu+w_k(u|njcmxZi|lrESZH6YgSF!r6l_{3 z1N-aEg%tj=-_>%$>>uV#<}EVtv)_t7UM)9nQqAT7^{&cIq7W4{8V*x9mSU$i3e58` zkQhbb_eUIxJTUj@;^Wx@pSQXcwr)ks^>kJcKc)BtuR~v+3(`l~K%Dxnq>1&mPw4Hi zD`?$5B0~f)9zn_tJvKBNcaR#jTbUmtU+Wzn)Q-w6pLgy|FQTofI>*I2{o>MMh9v0) z8lVbIYiS@VV*zS%PWk{;OxQn~V9wuc+^+V(JTVIh4YyyRhrePs6jR?kXW3=du`xaB z%H28B)LHzXPmJ3r<*GcxVjaa(>d|_6T$38ftVxXQv$*F7TGyCUgS!&D#!@}Uw%GfZ zuTy14@f$uIZY4TzrDaV;x!49rc}QuA^_EIIS;}=G>=|{bq04Ih=n`;$3)AME-@=SX zrv7VY$&908W)K0@3zIMLUMaM1NTDbj5&nZD)l@iaHPOGUrenufstN-zfzevh56ynN zTdfY6<1$5Rcd*{U8?vqhz~yfIQbY6iY#Gmo8FSRC)r z_6okLjsj5SSbA%Y8h8^v7h$!CB}jMm%PA-HDPa4^L{U=$-06;8d< z!d-?unzLHyuaI7v)sPhyQ9Y_qrlnMN{8z&wt`@U*7U5o>@x`Zxn!uPCOg z4*Kf}97>#6gz5{QHd6cJ+2GO{>w@<@shk`5)jx*T{Sfvh7!ApYVaH*hrxYD}nBHFu zXrT>b(A&kg$7`w>>`bTMf$w}%aP@Ux=B}ptL-iCd*%us9n$*Jr%G9m5%N`YF1=ar; z9<{7jZk2vwSFPq1kz;?qe`~2SNxHE)+#&VW9@iC#$!)+t8pW^WFPLa_EE-tWR4QHL zb!-aO&Wa?;WUGOPYj%D4_YC4WC*pTGj)=91ztorW0z4U+B!S^fl#Ic@p<5ISa=VSy zr83!e-anKk?QS`O5ATXy0Zkh_?F}CXT}lU|#0bj?Kauj@@qwz!mzvS$hpS#RK{i!~ z>tY2oyPBayO?J<}fp@;qUv3K`Mdw+%R z@e(WXvBW{(T zZc30SM#5i$44DgXL*4y~#t1AX!!2s1$0g>nJtgtaug9XriaL@Phz_|v*RguJg;i~; z39lYEI0K`t966(YI-LBr&L%#?Hc&%BpZ~_EEb0w+))Swlv^Z`l57LZXZVPWxgVOSe zMAZ(oeTakOh~z+@dH`pqE_>rRC)*tO607k|w6`ayel#MqzY-EYMxSIc$FX%V2AgJ78~+1PQ1|xH%^2g=S;f3~y~6c`jmN zGf?Vx^Wx)#cpUl#W$(Bya;~|gScCEsBTP7#wR1$7LZ>!bI>#c+WY1>#rAiTb7NKG9^ItRCjf#8q_N&%H|Z~MH`V5MWy=IvAb*n z8#_>;QM`ndVcpVNm1J$myy5opoHLxCWhMyFXZgFF8z75j$m(-Tfof3k1Ht5K5NhDe!X$g;?pO>>)9 z?rOV_nWy8=Y*U1?>^A4NWF#A!W{bVLS^K)cU9py*Uj=>asLGP=9M_ak)KY%Whhe zl1tVdyG}y21BWg9)G5YjZoXe1!)G3gqN#mIKLhgt!@ochelv~||D|~2ryQpozUQ2i zuFcPbo6y)_+I>>+9@-Ozu&E{CdUD)B3akKW)gSlSp~x|f#NH(}l!(Kr^g z)VSB&{*@azvU4szcAtM60zCB$tjC`_@}CCawywD(hXjh_x-P)C{MARsq!z` z3UZ&*6Lb*xJeAe*eB+z-_Hn^TCylF+eH&K6+(@6U4D>x_Kqn3_$lEeW-ADu6v>aT2 z*E>gw)dqLm4QE`7&aY9SPrX)@ZWO&N0D;l?4EVknOhRFDCk%-16$ZaGE8IMS)Sdge z^gq%xn=@|O1;MNxTlzAXbJEl-kx#?g@%Pij^^vlbOgi&`? z%wBQfz<2YeO>cV*`2uR;ICgBHj$1Qnjp6Rd7doS+=e&TQ(FC ziv?AJl3N0*1pN(x)>^bWoEc2qv?$T+JMcE>HnKZ9>D<5={%qDxOj}74#$(JcRRmF> zK>gbd3@80(P8`mgAwkUa_1*a(twfdoXZ-2CXJ;z?JKgDcYW!hBU`H325hC|EADhj! zP2Q^X(4b1GJ}N(Fc(Ng@m(g=}D9Xkn@`-Vjk7Y{CYh znVj@c4{X+U&dy$-iE_zeO<_-=1n@-8{G$dy04M-a#gQg_fUM$7vVOz5hcb5=G0nWc zSlfsIah^}j*hL9f&SreN^>Q8_BOg))y6WL*cr%`M7O~MzHZz~-l$yx1yC?e;l1xu$ zpyFj3M7;czzqXisYY9|*Yc;bD*fu(ztayMT&#|bYMkcI_nBsCFMyG-I9h9ZYZ39vc zs4Dke-4F~8t95j-gL+ZgZc)P0sxFN_MWd$0x{(({D89LkA3`{ZrdC!=)IF90-F6k3 z(UcoJ_1kZMWWChCrS1Eg1%qge1Ksm8Q>;Jq1`-W2c|=*O`A6r)bSrZ%sKemsZDZxy zNQdZwBPtT4zlqInv^63=UdZ!p@eyR)AD$hH=4$hmhJhfNb!-06Qx5ttE!=f*wJ_4y z?N$)myQUi&u614E)zsU^+&p+`lP<0XtsplImj-K-4Y4KylRznfw4v(ojkpF}6OK_w zoUWDbxquvJefHGNNG13zQ?9{(K$G+-pR^F_@C~?zTvLugN0gIHDF?I!4c2CB{z||{ z4{bVYqgVSUm)P2Q^tyTUT>dj_z=-zlC{TPFC@YFod}Q**GYYOmkV)HSLTCg~OK>(> z8rEx84UyAq*7fbz^{Z298>PcuugBOE6Iwp8B_NtsPa|%s($BgsaI$bKkQcRz3GRQuHb~-YD0iDnW$fQ3vn3O+SHY=pNz|Prp_L!dVyTa z!!%d9+#fxKt=vF~Z~A(&TUM8jT)4ecvUQ5EJzgW;*c(&xBdNRJv=VmSbYAz|Xf?DL zzf8qez(%*9=hvu`YzqQRGpr4;OBhfrgi}lcmhqt5Eez2GmLeI?;q;5l=;iMSjp#Cj z2|yX9a8_RDJJz}++EdIRZyf^#M1y+9Ds>rA~PPk?e zv-rfkk2^j{4{h?xTC@F!GUY)or$58dp0;N2PxKf#k%9?h8|&b33KKwodo%9K%f|V8 zpcJZ_T$X5x?GEc_q4J-FO8;x2D`WhH4~=7@t5TK?8G4f$mi8#FokK^{(D7L&j5OZB z_PN9KDI;m$oc6IpjVU9R53sB{BVO{p2t(;X=YNWt-uGiCgVCCf2-ryErL_uG6tPAi z?1^^$vTqZ03*sXVS%d=Nv-T+;xIF_ahafNb0&#jHvoFCP*uH}|ha9&QU&Os+w8FtE zt3&QWpOJ=psKtjS1;Vt1ZZNr!2^WD)Qt8Zr-X)6V z%JB?}=gHt5v$)C&lj;)i4q!-kiDjs->Jm$0jcU>-x}si|He?`spuEc)1Sti z6-%6$;l`XbO01Z-;?Dl+BeSO;eugurQ=3>OS!7E?rMHYX$(67$XQ!SONUQyzau#LM zZ4tnSxhuZ6Xjx}V;>8Biz9|_cf>NpxsfoVc{8X7gH&gbJG{Uc45-?Jw`mB) z{FJCV(;}=NX3YrL>9Zxi8G3@<##VOuvedcn^#1d(@Zea7RfG8T>*{Bu%a3*Xf6&MN zU)osOyW5f|>Zsjn%UTNER?8m*??1p#LK^)X)xj-EEp18S0FUsDk~4O)%p}rI#oK5* z@4|wBfHFdA==1&+u8BMGZ^~FV@->Q&>q7u?MXxdc_;uC>so7nhRxr*MqGtC*>sl zQ9eFP_+-M7*!F&!AWXx)&(eoNu(_? zjF$bfB7P?lEUuCaMSzjl+eel~DiLfoFpns5%D@sRu}VKx=g;QLI?#$Ea#BlAZNdd_ zWSWZx?ayvWre92AWO{4+b#w81P&Izf7eD0L1trg8sO$8Nu21&bDae$Gxj;o8bB~Wl z^iLcqK;>YJjnyEHokBwZvl+ljD;GGas75TbdJYmKX#;oKlAti9QAW4Yprw@qf3r?K zhB!7N=#h|VAu>B9Y>V7zYs(GY<&+3r*M3%!u(?@;wP1Zn-h6RfQrXYLfd|FT0W+`+ z**ZTqUk&mM_s1x$Or3?gi{QNm637~Ii5vivoDL>@b0-Z6L)`_XgAz0sJU-68Lp`}x zks?)zE)!0~RtRvN85+;Xf{Tv%yMQNuA`MH0I|`g@aF5nT;O=n8@99}lm#T}ja0k-3 zYh8yA0&VYGXLOTZwOa(0ae)GDC_410epQhC*g$j-VkFFdGd z(q7|4B|GAToY{e#F(N4LJ}cMHEF&0nnI31KXyBl5hY&XcYczpHfwNYXUus=7sx07& zC|R+CZV{x!7}HChn;uIp*J9GvHntS_q5XSGrC6ww=bX@N=bS4ib8+r9i%<(fOtmqA zeRSSQrBJMvaV$LN-3XLo1rC|*iP&h6i#!0S@C$7G!f9f9{;=Km9dr!KOMRL; zh0esGd}}gws!{q59sF~*OW?M>&!pJ~KBWSS*|DU`T5&w4$o3d++0wD~7r^z<8OM%b z?!KaUzoi(k0;lDu*Yf!%w#cRpN!8@RLvRHxvFBES7U-=eUWU0{0Z+}uGBW3W%0W`K z!_lasU~(iEg@j8w!x+23;j)7tIhr=3euLNuxTI}vl_+UUx}e>wtzW@{CRfuS z0Jn`1t}O#}FX{HSf3)`4|3LdI6Fkul_CR-S`}3(gNY6s6k3VdR1$X0r3>k{C?GLGj z?YaG`FLqf*OLGf%H~>2T9qAN#0B+Nq7YJ-q9l(w&q*)nHd7Ri!!B*v9g5TeRscWTB zf@G+cw@*E!UhhiXveZ70Qu#P!sl(Jeq}eu_j5)%;Z;DH{UL$xUEoxO3criZ=$Dh?! ziujDWk1vaR9sG`&_0EJF^EWRtv4~RzwG{eWy#Sv;VPO&A={f$Z0AepcPSi(Yi6Gn@ zPh<%m&mVGL=aPLBvcV%O(zOqu5CAyg(7N<{|;GR9{5LR>$OW&^b?e z7&(2d0cN+N5s*4V{wkxe$KRy1SfjB*26`f%NyPG_kq*{o1wg$zp#VNp zd8;Xr)r)vyNt(5Sm1Wd60<>+zQBWad!D zR96BL6cZ7T&|Csi0W|0jq@>^gatbISqJspvfj%>e8M(FPmfKZ<3lPnxUb6gWJBcRGp8yy= zgnfGLcPlRiIDUZ)?YAapPZ2PBXi+nJl>0aJDqoQ-dY*oyCZJ5 zv9{hAJ5GbhABlb2ZhH{)AC14h4R=Gqzn0-fcg4ZKrr`>26+i@B4|eVd28JlWea}OP zKc?V}ZgUqt+HrdO{7m*!U+h4>4+g$F3~+tPZuC0SpE$sMF9x~?_S_sd%yxY_vLyO2 zgXoE`Wg|k#Aqo>0Ww4Cf#>jfpfgL8N2c5_l#WbV?-Y}D{LfT)-A{}2+-#RGOeBay4 zdOwb#nt^D+vCZcQ&#seG`)#p>#oNrwGGmGg509M|8L{bGk(jl6>R1(DxHN4?jR#Vd z`-BS=Y^jTgE%4~zh8=MZN|3o#b7Gtp)wy+ZM%pcSW$YKBSQT_B+RKy3Cl^sUvL%#$ zBBYrm^UfT}3hkNH<{^l=7IWjas}hZ{70!dE88&mc99d$GI;?6TSrif~%>J+Rn)-9X z9NK*s6MbUB9hx~msLV)~^hq*&O{vm0^K?kd_S`Y)~JNZBy+dMXU=nnJhFsE!GqWm3mdG!op{I>KAP{)wS0{Vz?c?X4vT2aGk z{Fk7=ofaTySWOCqnY=$kqg)r$fw?9@o)_2X-*SXb+;@ez)j~LD^X~a>8FlpMg7MAs zBVjvML8_g@S#>SvOz>A_QYRLT%rWbZq9>@ZB?fX_;qO zPBjsD#aF_sUXMHxx&%2en}$JpOR9-Y!M9KSQ%@;~t$f?Z#Wpr?seMi6rt#ggZR0np z={u&yT5+61A+`&6j$BhWTN;zr)X!|+%0sX7bH}fA1F!G<-kp2P&OsbEh*}}QT9ibe z;)D>qcI_U*HZ>1hCi7G=nz0_rGF=yqY(2Ab7*8bwpD-VsKm{3T2KNM;cn@QTftf(r z+`h&k+nq?}LlV@>d^gt<+T%k)1f7`!O4*SuZDG}3aln|&t~I|s{Un`uF{#sWqK2`E z8o5^ypGGqafr36{r+wky{Q4s`b)^5;#@4DaeN$@@XBmI4JlD8P>smWzXY25vEWBr;N1IG0kkIy zeD>t`mGGi8z*FH`Auw+u{nMM@MFH@9pDytdD3KJ}dWtix`XWwC4775x8f9Sf5{c5J zh?%q>>oB^F<&x;Q>86s;Q%k?GaWpoG%dl{)Kx8u5XN7-@a(1|N)fMemia>%aung0R zk>CX7PT2c^?ggm2xdPf*uXaMxLbzqjoR0<19gq9EGVC3ute}jr^72#u`WV z_h?Ss=AxGK$1H;iJ@u2(7-ASn`BY`}3+-cMAO}75C27$LAzYc!{P-lR#lPU!K_a4w8GKEB7!fN_bWF?ta)@CrF_@?( z@45H=+g%fOC?Qnauyqr@fsi?)=;>+U!d}jcaW#sZ%R51AaZXXt)cF36&uk?^%pD#h z&N5#mpq6_B`MVl0p>U=Rq)Hrh=nF$!pGSE&WnNV8g{?yswf|*0_q%1T0i_n^!@)y; zM9~=i$Yn#_kOK-N`fUB=^?6auhF;a#7DT6G( zAvPeZG!T>{Cp^w#G+l`@i@HIp5gG$w{kK0Yr-@IFyet%ZCte@BS>}S3wDv7osF(dz z?5e`g=x5SFxAhzic=tp*N9mR}9p-~ppl&1U_&HB+5AVRiXa?~f_uoXYd3}0|DwMRz|XB|`01R!qIdIxgf?Q9Dz$U_f2`NGaYQlM~NLD3e8Sowk^omKS7Q7pvOA3P+usST7rgDfUj^y^<;{AF5xP_eIM z_3Sf$mg56nCG*d=lW~JT&>?2t^p(LCqM0ij0<+K>=Uo&%QV`CWbNyjg`l|quBn92Q zmgxA@@Cl*A2JpK&?SeJb5~q(B*0#Y3Xd+JBt7XCJ$TYMnl3 z(6}k2C0BBpv(LcJx;FYfkchOiZ~I(2%FITf6{D3wuw1;=%r#RJ&UIaE4B-@zcCpE3 zmc*aOTzS~t++@n(*2L;NxfgJ&;ih1v_Le9#Mq`j_q@4{zpx)OO2mJPEYTBq#is@*| z8N#jSaL7MnwhzlW99+=vhe`PPtjlkv8rzMT&z>Jr^t+<<)swZSiUj6(c#+;+qCi=} zzv8z|G@5Jo9?X||B~!M8YWHwdqvy%H#r>F1eG>We1aKZO68VdI8=mW+#^VZK6ZZJ` zu`D{E@@7SY;pumz-!frY%XFR=Q92=53g#u2@?vG)cs>KwPU9l90{NNFl-{O9MVU`? z$$ez?><0`MdQSm~UoziJdpxV7TnWihuR$Wdi)^COz3C5nhVd8#%GI+Ob1-ZY1^2*< zE8131KR6Bc6`LcoISz6kfLHKeH%M{P^Qi`zyFa?!6ZT3@S;Hu|92rfqPb>w%O8JKU zyhhZ;?n1hZ$tW<$G&}WDM<3~r?AXs|6RQ|8Lq3o3$M)HFASW9@disT!x_PoAev8Uj z>iYYlSwZ&F43$Pfd!ExiG&8(j#kJuZqU|PXlNcwj7Ppv6SPw`SNsdHGxSZWJkW25G zB6m;I)amvm<(r-X+z;;q+HOrwHM|q7DkZuSPoOxU3&Dp88Dy0jQIUlQ_EbuQ~K4bf&wmp zdsJY#{wn*N>7Zcj3Zu~2hm9b& zIjKBacU}R^HlZ9mVNgNLwyr!}F(ltbIeJ%6Rv%#B#T%`3&_030mabocvLssLc4(4N zE`>JQiE(5SkuRIHQ(nR?DRUq!p<9w!A)V|bNy>l*DNd2yR!ff@W^ob0#=$1Gx;;T% zS;C-sjDj%=mgJn)uafjJbEr6>OhIGXp*a1UGe1?fVTu4IsrF7B#dbm2X$UU%tfcd5 zugELhd;r<6Da9+2Z(5A<+=KxuE_v!7QM_las}$7;hg^~7FuumL8Vn~nDN^-V&X6=^ zMRs)13fQ^ntC1DwB;&I1S&u|hndUS*Q_OTpg+gTe1nV+#bCOcq@Se-wSV&z;$x*MA zPr>(PgRH`&XqnGpi?%Jzp=*UTzj!9dzb-}F(s3I!VFQmM4QbN+u^;ym$gtSoh;uzH@c1Wml)!HSNh!+}#f3Yp`FqFPCp$C+S z^4H6VL-w;;&v(n%$;wia#EA3%^g8X+%$WHbKYqzaIn{H8N@s)EVK11CIo1>58hdWY zwtB+x1hWQleP;QTvC?!d$qF<1Vm9XDKyefit`Te|LMZ7HDbqGP)iKe-g1s zXDyfJcS%y(@xxA}u({6fu`FwZaBnE@B!0Zd9nq3`VlSK!(sC*%jKle}I?sx!JC`|= zK`Mh<21b9$V^5smk(=;P7Uz5O4}1*fYv<%#fGrQk1zCi4-sx8Wf=-JaJEe7O%V?xsiaR2#T$FtD9E%9b z2wcmc6IFW4eT#g}D=NwU>5`-gsIFw3bNlNs7}Q?gK9fkh1isABSyqxO}%#La&xNR;9aaM%hU_pu?w2=hNwNARVz>jN&GH_aGUfOX$isQe~s0X<5#RX{u zT_zoh?;Am&hKu3JJ0qUeNu)fBkDb>&hsSNWWHvo}G+bRGE2EGv8Xc`4k>~`(!t=yV z`JFgXWi>B>sd8{Q-w)eB1psc$R3Y4-d{De}GpGZ}8wX0?yB16kGzmiE?M#&RyBI*?)_$xDa)|bdL=J;+Z&@3R0oi$MmM`Z2AYC!Wg=!;>ScH&QCx#B?~_pS(!8&eQip!um0gp~IeTt&#F&Kg zF$bO38%)nyiPOgh2_tVlS`d(E<6A5s*%%9%)L!mnR@_`Q(L#)COGy|w#snzP zOstBdRZ%9UWUZV3F%q`q%~GH9%_{YAb8(b|2oQ~D9@J*sn74NC2l6D9#Y$ghTf|td z%Ftu?`t1>o>t0bI(Zk99LY9Y)zgp{zYSv5tyLOg;sw#`A)lg;m!GlG;VgEKwMwSSA z!P7}@;x*Pb%YD6&BjgL6yBFU;aCDLEin2zZ)0SXk3@U4Ws@dH{gcBJq>D_1xk5Yp^ z_rlmoSUBJ-E>eE8vTx}txrKAu>BJ_Eph?qFz(4ddOmGzI7aS@s#uC)e_}+y>6m05`f9T=RMupQRLh|6nbhsZRMV$f`2^iUEUX~5Y=T(%X+=sRk6X8_;<#JM(14iW8gL!YFtOc~W>Q)%4_Pv%_93 zhoCFjW*G?JIr>Jsvi)#QUlq$x+Prd&Z_#q$xSg=wZ8S++gW#KzK4ZVn>r|0wjR0}M ziR;*H#GL3`6a9d|@j8Z}@ff@Fj=h1s;%f+y=I}P^D zcrA=y@n*-bZ!F;0Kn?D)yZ|G9fnhPK$M6IH8V3>$@u|#x{`ZJ?xj6fd zbf^~d26PCUKoati1421U@oIa!Av%J>>b6JUC+rP+Dzr69#%K$o4hygDpZ;O2nRfac z;hQ)kZ8%Lj7&O$2@Cr2f!W@LcpVuVH7N`(T8~XQ&Y0IC^B`)q2l3pxbGnh23e7il{ zG0hg3H(ubijo0amsc8%Up6a;CUv?oQXGVp}HlfmXziA#SkrzI@X9hxvI^q7|dX@og zM;X4G?5bi~HpA_6>+RTq&)VKHC-=)q0UT&x*FeORxS_jaR59Nmamxu}6zO)Y{0(CJ z?1{Jn6hf`8kLdKaHT8?YtwS#rjW}={r>rgm0w_Gt3)9^5Q}BgpKG66>3j6bm-LZcz zdVGLJAn<5FC9p>wsTbVAJ;OSqLY6Hum_uzc3h^BX>R8 zFt`5z(youBUlm^&Pi>U5wa|{}9m_)~4`24r^-&`JRV< zdf0&2CvF`8m5uP?_Enw+K06S?15@0;pnJYjfj~Gt14Y$}_Yp)#+3#b{HP>gfF-Am> z5fQ!PgiC0Tht@C-!P;5Utb>T%i}EB?L{~9-VxZpl^TRgc4AUBe(XN4wH1|5F4$(2X zQsDYgm|TWBOgN5Q2SUk1+h~?KQDgxqVDGqdLwqYC^~~!kAL-LteZF#=2XHOb+lur~ zPTA8co8Y>p+0*g`t^Ji*e{*nl>r^gT&3F zQka1w20#li{eb0g#dqgqU+O~&bVFgDmoi9+hSKaCM)U*BV*sp90GT^&%@g8k=tx}- zXCwJ9j9En72}I2=%iT=~Q*eCyT~56r!?(5;<(hMM@W4v0nv-BH5cyRE5~U1UZ@u}a z5OOY(iTT8k?nFc8#1e_?;6&te_9?xVvm!aoa&$^#`-EFuR*Z3J4;qyuvr(M^4nz9P`_9dB^fU&QlN0(l zimN6@Hza{3!6v|-OQT?P@e+6>)KhPQQbFyib*XM`2Xi#2UqgnTTiB@6?0);isa~~cjn7oP8Uo3YA>YT#1@nK6&Je{$n zWoM5j#Tr-k((au;6WdbL9br?)+hI*2$O)`EVDnWVGvpRXoN1z}S{n)GW)q9{nsZIq zTEv6~0JA(x8{W}ko$N94aQvx@$<>{j29v$BB1{%Fd6y#%M5b4y2br98i0U(Q===kO z#fNX$$ScXrp6)5cwGjRrJ66qXeiH7rWTlBEAitO1gnfE1US@QtjdSM79>4dRaHf|U zr729S3YzUq;PUYXj4#S`1~@W9hIB`~rBMi2h_Sg!qk_=icrxSHQwulrE)rs;Ud z97*zSf2o4Zb(H{P4B85wIp9fH0Hgv>YaNUCiZq2|Y)&sSHAgT4(mJ3w&C3Lr>|zx87qi-ZQ%nuaOM84x*o$Xm4Fl%T6b6s8D_*Y@ zP-|2T%Zm}Occ{7(q<)KMy6=as&DloZ)B)QsH16gFz~{WU0j zs}S*k5#?R+_*MSQ_l7vKGeHf+a>c8AZkO0b;l$n%x4_&vXvF}8OW~9N`b$%z^==;5 z%x!3g;a-G}{!FmX%<7v*V0*(p`&AGeZG#0+5-tN7}p8#4@Cc<;X>BrVep$!{B@H{ z`ivIYqiHU%cfdT}1<3W@)%iPR5W6l=+%2$9cbOkp=nVC_v32;vxzpMW*S&yzQ(6q) zw!%>u;N4?eipFN;f z7h}|>=nZmvVx6DK+WdZIk|fyE`im?gWmquYSIdo@xd`ADi5^U&9%;=8IB$yF7=UMp zETXuY{K;{@fc!+N>C6??SM+W&-({f1_!6hKN5y@%8M@(s6$TV^;>AvXLV+=316;lP zBlRbcJA(XB@@_F?XF!R=?%k|JEwZ|UKg*f4#;+2mSlXiBupgTRu}@LWiANL@W8;aS zeDm@ZrqM!7J57_3AmF8S=ri1i^##qtCxU61J@A{$kg(?@G1w!0i(PTX1_m{STuVFs zQZq8}-+BGYxmQqkjf49*9ITbj4#zI)s@7Khw3UO&xU^&t_9NfLOAHCeG#ld`8|Twc zRU!si{Q#1RYhSsiP41fgs6VjN&A`0e{?lCshrThGSi0Ip+Vp9b?G8PqM0UOmQA>$q zhi@yhCqII{$Q@dH<)Rnu8!3r%mrC7slvW|v*Si}ChfgAH9~KgdS4&xN&HZe5 ze~sI8I|?i$yl4#sb$(AdYWIkAT+O9an4NTq@SGmLt??)|jGog%IQe+(r%ghSw_-!^ zqM+}t)nd*B9J%9H_QU*!X9#4}17_-kW#31u#i|K}xrMl}^99`*vEIe*M7J9`-y_;g zND%8xm?YLS#9)$yIYVRbhjT@rL#Xcw0fYZcJp&Nx$6Q>Xu94Uc$~Ir@#Fa;DZ#PpY z4mow?EfQf}qdQdU7!#ydwinKyX_#Q}L1+%OX?qMyw}@17K8t%Op)lVu+iY$a{L~v} z2Vk7kn)ueMKv!RM!tz9Sm!UP&(M?k=(dkLVucX_ww}ILH^szObnRvao#W34_`fx)% z*|vk@DqJMe{J>KJ+O%9TfUf|j(Ne2a->b&swZu5t>quByAwHc^?#X7^=~&Hb5y9Vv zL%AtYhn9BpbvwT&Dg)3AhY`0@t(H6|-^Dv}xtM!AW~%GXZ={R`ezbPtf_n8dI!H zq%{`q20`a8yo%cyP6N>S!{%Q98cHY`x~#`|Da}6#Wf0BEr$)?Mio+6OluTgUF{bVv zr6^(5+{-k$doSl+>QL-N&jJfHEzHT=6r@!8fC_v@EQ}vuP6w5rYrI~SA!k!Jx=8VL zq4=t8fVoB^-w14z`9El#5W&p0QLwZl7}`y5WzR=ox!?6t@Hfq-!5AphT~{{BNg|^P|P8k zy`;OT45^RD$vFr;Y3nPS*w@#Bt1wB!XgyMAo)iw9L>aBYy2}^zHO`Yo-Ia#**7UE@ z=tb(-TPl=(qk|zO`V3jm((`iUNxtFk(i^(rh~uI*1MWeA@0?CbCUG;o%GB)=>V5(b zN#UTB(Q1`hufODgcJi~Y>SypsYB5P_d8D+xf%tS8x46Es@MIw5!u~9rz2k$Q?o2-< zh?l+89d`Sv`-$I=&MBYsIrw{$*BHj69BTN9lw^C7@5Z-mBgds|b8AhU;%R^gG+1 zc7M_`dt4Z~G%W5FDf2&9=dYfK)_1Q-GpoJ?Yd0>D_zm}e)@$aAw&g~&lX8gOSd|`p zqZZmv?BRn*Mkqg!qZj5){~AZR4{di){UEh>Jl-(-ewY{bcl@tj>reWVrR%^!*^BGR3&#Zx6j4ZM*9QomzTI`Go7tZBHc!GGtFSbv4i zE57ew`VvSCSTINbE^}s0USRr`N+`{MK2xD47Kko(i9#zvwkkG0~{FERs)~Pi2knE;<_*GaVN>qE0m{<;^GpS1!7nvq_eBTExsK z8CFg%|NUB?fL;pLD7mEEWL1dm0yoeytK*9$}qZBlTw+DH>Bkz zF;VX5hd0d%IF=Q$6JaOkl7U<{=_&P55RCJzOKFE8()RG4-cLy>Q6}H^0qFL3=Mqp| zJS!NKZz7Di-E7+p{Bt{e>4hUf8{2SS*pQK{5FHS|zjTWisauqINnCqfVTP1%c^Frz zwmC5d^EEn%CJ#$r;HL1{i!FX%1>HdF5U=2$G9>T~1D;yNER!RYr`Cq(6>lMQWKMHeI^mHb6EC~V1KQg9H&}d=VY?|6} z2rSVOoFonNnRD!L{nMz^vUzbYO8fn~V>QO6?O+Cy0xy+dpKWNcBd*I?VFrXCT6_0l zf-WCA%NQwEcmQBi*#ovAwdi^9uepfBV1_*we_V`CdFk8rC>iQtGOI~e23vo5l{mkd zU@sN-jOVW(|Jq9RUYIwb4E*CqH|YOGS>b=TQvIK9XL%zlQ)B0!c20orAA3h}!+*Rz z{-24NDwQ4i1qIa4EB0ku^z>hlzao@;%fBad@)1Wjg$KY#$@y=bE$x?$E_K`O#`O;y zGBAx{{eJi*-EL0XNC~_97xJBW+s?dp=Qv&SeSF-5_A=`oG6wKynlPmoomK}vvu>&z zNToxJ^DZ)08K&$-E54nIHESQqiiE-iakO1R4;;lL$qqbn_gjR|B29$y@*=w)f>n`o zVPW$yHG8&>lFY^JF^TN=8jW_=`!L+(vxOeM_w$RCQ{we8s@wwCZG8eL`?&=k&c*;@ z>zg|#c_lYHVNkc>hV_nvJev<5vy3gKRkj_b@-K2!u%<5whPCJpZ+uj-PNxJ|j%;i2 zwu(|z#FM_3>GS4hf)#)S)pKfu{RncY^U5gzkd%5 zDu;&Oqn&9e<i_wzJ2!GT;=8YyA~{?MuiL}y<@j}A=3&4&?9H~w!`gT6w<8-U>yvX;_j9v` z`D$l-2DZxxOe+C{npW4^7GH(cQq>1C-DZR;NwbNIajM~?Qqn8cdMej?39Ue}T1rG* zA7OPO0kLxgshHp^#OhehCI@MEIn66x+2XW$VY5*hIp=H`cAqnW8Kf1{dm!p<^e#I% zW2oh1Lawqo=qsAdF54^V-C$~rrn*DW1+i&#Ji)MRy;GM(5k|7-9OPEPXy*a|X6Xw~ z6~Cwj%1C`l36p~7yY)^IEzdhtF5iQ;I`-?|bi9dpyKcw7XD_S2i7fd3gC_ld+qVDg z2&_`mcG?g{{akjX8&aJ&nD^SEmi@8zlNvHr&j#3XzOh&?W0(vP6Q6y4SW%HQmNYD> zryu2uaFTI)c3N;JZI6Qq)|_la)@Gn_cGBeeV$JDwy2V`m^Pr}r2RLm=ohxe32aKTf zg)t13UkSh$hSJIw%%R914$ZH|m^y&SuXtOQb#esI9tzCDK-(NMM3r`It=P>8q!wx3 zcb8)wVKiLCzt^>ECLUq4mcvZ;S{ZT)oeZ>x2~}nrL7}P2mPO*-&!o4!W82xk;zq!2 zyP{;7Yer$-tb7$3IFz-sg>l+gE9%=va{CCiT(H^J-kr;+eedcynTulo}&1aBT=>^09j$gze}mQbmC#Wv}Q%BfetD=|m# zl#W*ww$Z&KGY&0g5UIHki^LfQIk{bKGMU%DaO1uR4LPjG@Cf42>56#Adk=@Sqz`D3 z9O>(gCV55axuY@TL2l5wh!w9g94WgCdRnAAYawsO{FK)NCxchfH8EKHGw}FJdx!qs z?BrpFpoTALXeKN{`%e@QD(c%?%quL1UVr1fJ#i+X`m=TbSqd)H$(?{+koCOgfMu3p z>tAApB^R-vj(eekCX&w6=^V0*_yaFkGR=tXLqkfNt=osul`=IxGm#zWcf1cC{6BgV zE+eL^GnjvBq*$J9@K>RH_y82Cxr4#TU%EdlJaZt?b)cvg0R(Jt6O@!1b2mTOLL0j9 z{5piXLYxY;}NhAbt~`QB~D)TbUeT-(;pEy5>WNSX2GDIA&o1te)$ zsO|DgdD$8F35p*rwrqDOXai|;_LAt9{j@aY@ZLY_8+oqVA3!mEFklWI#BhS%rAbI| za{1lNEy*mhi#4faWeNO0P;s&f6nzl_XhwxcJQ5xTVKD$q@CCcb&<1tR>4m6k#PFFH zXIotk^SWeQAyD(O`*hf*e@Z|}MehGBUSs-(E{gN}_)mU5IAvLo@Tk846@<*sDEJ19 z-;j_Oj=CkjqOxfqS-I-N0J5EC?7gRRu)rkvs$VaN_-A-xrW%70jfj{L)7Z6RDmr>dI(mi z@68W18XUgKK$cOr{4H>1|fA9ze)Ieve@{cCI+Q9hz?`*sjtm_L4q z{{MFp{}tQ*t9w|((ivq1?Q=MV?aqC5Qf=->AS5+TP{bNC08D9y3uLhJeBvy5U}i=PYih0T4edD01?)s_p5Q(PZw3pwwXXSq39^Vd_O36*ty z^=D<59vhurJ6$^;+dlT*j{>QBU<$F%sFkAXWZFz%6LWMV^xRMg3D8zIP>N})l0>@U zs-u*g5ynOrt(2Z7S*87at?GTES*TTSCYv=1pFndS3IQ+(W9}NmRA4M~M|~vuzKcO4 z!TBS6?!pfQ>Wm<{s1pKI%f3>>CA=?5o+caObUvy0a%u6pw%MMXTd4A6nq)Fk@^CIz z5={o=wu6BijVdGXbU*vX%}B6U=CPDh0XZN*yN zmI9`JNzg&tI?K=`j|flXAU8#YZlYu?XthRaPdNU1O~#TLZov$j8X53pwk6#kbR=pU z3BUbyOj??LIz8g^K#LpTSLD@qddk$L9b1CuwIAb9QGGS7uT_(D$U&ypwYZV}pCF z-6&?=uW3Z;An6zQveqa9X^ovfJjF%c4#Ei*&uf;`6ts7APe%a{XCq+2qLl{X*$h%t z)ShVOY2?}=bv0$Jfw`p?gIZA5BSX{iAgUC^OVoSKiu>&wtnJaPJBL7@RC=kmp%EOn zFiMf=pgREBGmM_>YX)43GB(OWnDS6-MHMspU?XJ(K>X z8Q)D^1~?q+eEqJzpaynvFk)c?x5}a*1fuTlmMZ;qYMV|Od*rX&dp5;Le^1>!vg17| z877AKuG44FbJ&7+2NU+*Eb>Vl_D;ez*wIh4ZQi_z>rOfvRg1J@w0smFVUAe!SeMqN z+>E;EYz7a^!K#bgp$Q)!I1x)@nuXXhDVW%6%w4gUrcfCLm#MV!%S)plpSs>aA*p>L5exaP+3*5~Y^emPKnER1RDU?PO&4zf z89sILD=T4NlBaPY?W5>S4!#`IA=ySQjN2>`fmEk`?!y{4CmC96yYA7J7yWGT z(S`z$Iox#aNcjp)U-4A!(3{lCbR_Nk)DBZ!$I-3Oxz=5NY(F_+CnO9^0^Q5FJiATRn5y|#u8ZRORmICDw^imu!W_eWxCMTsI%cKr3x!@zA97-H z--$_LU#c(lLb2^PBa_w_*0>haA%QUSC-GkIm(_EsH+f7U_Ps)E@{e&QX>a@5BcvVr zUA*YaZ+byrVe9=gf1K0wqJC^hz#YAfp>z|4Ax(2#wJ1un z6F8dWwPVa=o!fLlW%IY7^e!}=CyITRgjiE12s6Qqm`XbUKjQQ`-)c;{*0=qKbd7uQ z(pBksNPFT?@KQC9jLmI3y-`9izl@%)V)s1XRyLkH5WjuM3fq$|T}wW~f=Ef7(GM0aaK?rr1cLIXC!-X`FxAJfOZKTYF=z9J)1fT> zNgr9x`cjPUkI3nH`qYpSa2YcXo^r@LQefWLICkVsINBEEMLe4(szL0Qw`h z`s)1Vt`|(Cjgg$#d)u)Ot}zgX!^$IopT9U(ShXR&30BxzVta}f)&h68{^e8qTU}ZZ5Ip^v zm;&s$>svwo3SQM{4u2y zsJs~!N?(EIxU*xiY&y!Kwem4D(ekW#c#EZV8eKY#%x@JO7GMxb;WPBSZHl~gXh_rZ zt=t7K!V#MybxjryK^4R)7TkZDj*9ETcK~pSQXT zU~2H%tPbZPu%sic`)_7Uq&WFPAwfEqWkc$08j?a{Z5K!Nz_FSvvn7y3K2E6uIS$WT zmUk!t`-jheBLTz=y&gk^`SC*s?Z*$=|0^^S{gzHt@c{gnz;lrXjCZCg+Sd+4EW<|v ze~TRqdP|=_y{R(-)~u`X)OwxOy6^^E%NU8Y2##c8Lh(hOoG7Am#Hd#%`|+kbD0xk# zu}0|cT~W|QBIr?R(4##NG@$e^r^MnhHJA5`>xqxglkedvzUM5j=?}YSZ|5&Y|1~L1 zPN2-9*O+iR@@{LRif~(UJ0^#U@C}(wJ7Uv)coZDjO-rMW@ay!}iNP)Om=V7DP3yjp zH(FF8+rAh)bX?5(-9b4R@~aku0eTp8e8j!$j=kYpRs`s_4LF2QSzePp#&^Mtuq}5` z0p|k=ca@A0nfKLbtBX2?z+4T=p#${s8)cXy_kLk8U$gz6DDs+V;ZyXybl17k_b?c~ zRM%bTR<}{%`swa!ecR2Xhzbh6A^Rb>tAfnmGy!np5CiP(B2;OgPfvnQ#+h;O5pSt3 zY%s3%6-8AQuZ73(M08*hC;Lw^^hBFUFpVwQq^(JKk-(SH?Z%yf4P`#z0>4$=f4aE3 zG@5|Mk{sDvT3=|8tAMD|>TH&}%p9mlPi zyScJ8=g>xpgK1kAgl?s;rO+;>3=R&a$VIC!3?9eCJi%}9e=1fIAxDyOMOS(LRjMP| z##5t&6E)gv?(0t86IBP8CS^qk@}fr3!;upUo2NgS*qaFEaU;q+n-TJ`!|)JT`^lYH7X-#_zsZN3f5NP8$woq$wESbM1wYCPAv|AB6Nwu`{^oUq-kXVXHQK4;nHd|ww?PMjswD&mh!}`eO zDDXOSLa}_NQA6zS1k1ms9YPLpyt_Zz_Tkpd+Fki8CQd}ieFwC}YfRnJ{bZRp(f_AVV<3Oy1CA52u;(w}X-)J#KJwO7a!&xf8=BE5pK>#;wB1 z2Ke@Zg~uov5Ha%R6^HXJc++TS7f}foH4{t52x27$`ZNN_nYsm)&Fy*PagoMe{f{wP zrSiyo6hJ!;Za-zZek=douUjX@(CSEw3h5GB_nj1yV!0F{uohJ2L_JFcom}Mr*r`}0 zskGs4p`F3pd!L>^69zW$VfIoZ==Itcc_?P27J6uAs26&uX6*dD&w$|ACr9v&UEy;F z@r$b)LrOiqHVQu`;B!Z@Pj}x4a=W3hLq_T$nO@0{Rc8y0`uwRcx9e(hlWiaX}B z{G^(CgAR)VFEVCKIi$`|59|LJlguK@YW>Q&MG(*5s+Z-JHaZzpA1eM8S?qoT#GiDm z4}C=yznXNB^SDcLQLt3W!FjhU<=b~PUZ~hNJV$0XcnxMb-jGj`HTe`K2$K$OpM6|6l4 zjIP)62mD-ZwVIn{{^$i3l~BD_&pw)UJv08TJe(ZMDFqX!svj4uHHTmLGWs)7a%__9-H=2vQh@ySsAAb6TpdQw$-xB`2;PGHC32}F- zHgD_U%yX7_f2u6$b~u+^*l~ACR+xG`SCa5)gWnbPaL4Njb~bY~R}#<97t0^?knZ6e zw}MPILa|}zXhozNmHXO_r{HE8U+aBleXPhCSw%tOkNnHi5R6HMLtASp&q5O=pvx2%2W?aC4%^)pY}AUA zP-yvWBm@^I1v-VILT%Cgn^}kgni``~P$A^^a8i7JmuuJX1XHvvOPQFCJq&{}&~~Ag z;@A&0CO{%qD+yNZ8SJVIxD`Bz>d22NWB&}yG%;8i8W2Zj5Jzf|x~u?5<{w+IyRRK4D%SAY_&dcpE#3t5~Ccu>5)LdZ!Mayv2@hf6o)S5^) z?VHH2I0-PsezBuVaXl1AN~1eYj(CUHx^2)(&lE@Sqb>6Ee%>?5kpw%wZP&M39Pq zG~c__ruTd_kVq!|yxiGgK&@Q~Bg1N=+x)?i*TpEQWcsLRQ-zx0uARBJoVL7?eZa5~ z)F$3!Gh`seV+xP6bGj-|`uE7WobkFGZk~er#p59$yv3a&MrGBXUPyxKLfl2Stc5ge zqO*vHNw-hCSl`IVqsT`DefB2!mWs%DrPO_R7pyCl6@!j-f3HRK&5Gs& z{w5g9M@rYy62`)u5xGb3D|G^+)Vv#i!;xAhBm`{LRa=?ifnh`U!L-nksPt81)?Ei< zrX0rhn?eWj9QNswp@b|i#+;|t)_OKTnJ4blV^zS~G0I|}k~tcTyx@v<1dIdZKRAP~o@dC@bury<+)bzs z*&IfE@kP`sQJ-(*7gh;>hKfYt*;KQI{Ho)fIhJEl5%Ohj5uLGSh(rqPQP!M=cp}%C zoAq-VJ{;F`>RuwUhSU#fyP3^zler>m#x=ds=%zK?T&CMA+E5Brsz(kHFwNYaYTii} z(^^9j_)JSlw!;qiN_{F{?NCac^`|@7$QZlv{;zoS`&X$;mEd$pY*|u$Tz4 zH^l!42dNs`Pu1GHN1AlWviHc^f3ES86#BN`@X|cPP!2>^Rx;d*dVxvos^*;4e1FA2 zd}TU6X}?E#O4wIY;o1WN>azyG~Zg8#IpF;T)a4!6@e$yI?b;AAvID% z*2^#j=&mH0$Gy|3<#Ea%kBH)5naI<=6o9=CEzKG|8XPU@T*1?^2oq-qT7tnC9TQeb z(Ji#5uz=36)r~)IoQTnYRx*j5f4#V!Z(yAd`YuY2eHSGq{;%1Iv!kJ%lbNX_gOjtN zvnhk9shOdRjkEGU{wO%wyIPw3w{(k&w(^1?>SuF{AY@4X(J>riQ;`(vwZCd`UzzeO zlBxJk^H>7-Mq$Gm^C!?(ggn1{Iu%~}EyPO!uDmd)#I2CWs9U}H%+ZF^^i|8Xem8L1 zfC!$Z%o%6w86tf!KyiQ!?@S?ETu1_mC9l2kfVujizqb-xnS!fmuRPQmHvIQct;S=K z@f~{0pZ>}Y9#C}k&OM%D*?E-&ufCKxv)`@A$Kt=_i+T>$h3Oeh&~;~2O+!ucTF2tF zo^1OV6AD{g4)5HvorRNdRU!{xn=8gQrBDeHq9V8B%w`?(< zacMTB+>hiq8G+*+=@;z0*5 zIFPUwp?L%oYfKz}7Mg&XKzRu7!UPtYgPt1jEB^$>j~k69t#Bd*?6yE2!AsPQH97YtX5k?t{3mgN`0#eH)Mtq(PR9Yd&_DIpD#RZ}IqzgjT zb&}0njB>%1Gt__Gyh6 z8-L!n;BL3mogks~zXAoZ%LMRBCgGS0f#MAE@C}?Tepk3+>A#}L{r;`^zOCtV8UMY% ztG^$?|8$@Ke{68o7#{e3K|~QgB7)z-S-{xh`G-Fw%LK=vLZ@nfqMFW`jqOo#{s$mJ2Ex{2Z+ettY2-eM_jV-dw=pYw1F6R*n0ueqG$G=67m>+&|_q zafi4-$#Ka#AQ`uOJ`F@5qmplQ+MZoR$){l`JjQBP=hHxZ5E|yc|#Ef zaeg(Uqc7VZIB`fzk`9~RTA<_iiiB(OKpf*>nr-Kv!1Q7(M+=Ks%b=Q#0Aj_Tmc#q^ zQEd}i;rM^1s=>8X)K>!zLc!A(BzlB0V9VDvLh=lU-3TVICz@I3@QQwOln&9_evCDuI*I95w$M3P{sNK=n*#W+`z z2WpJOdT^8lDNzEhs_Y~l2-RYZj61ZDFm0b**;4Ap>Czs>QI}m5#TNI}h*jg`>)XMm z%^_O;$a?O_mr#SJlm|foXCvTHKbx7SnCc!|)jc^WcM=~kjw8@R7ot0J#W~RBFTR*5 zePL`Iwno`gGekwaGF8+~z^JRc&;p1$ni!)J8?~mV`=J8?cR|}=U@5&_Q`Y!5cF-}u_ft=*7Kt2G-t6eUvNE{rfH9=ITST)hC%!MQj=}K=K(Jd*>sMt%TXa!fxH%$(G+wj$0FJyCuiCYi z-~B_LUSG!OgAL9#BUmE6-jM2~Y|~##%IDIiwn%-W;wPfGEu{F{Dnl;v0l<3s;+;4kRMV=6wVo21}cdnjz2zS^M?aBYMZ&|57# ze@@IA>)IL99SIGLAwY=C`MNB-lYirBsOgbqIP$41i$dk(=L2Z?t+i>2(5cx7uki=v znyd!?bpP;8sz^{`S;fAEnUI0<&-U2LS_Pzgl0DSixbwN0K~@yqK2H0hX2KFzH$i21 z2kunB)<=DV`%%3B3 zwFH5u;G@-Vje;S5={52~xdav<2%?)jKD3u#6a)u|$Tf0+G#y}xmqV0?9`ZNKbEp*{ z+#=@`3Vcfl%bD3xmS1^$C6c^*%^*QKEgKTgv*B-&2SA~%Jd21UjN(5guv$WnV)i(f zjz&AqDJb0M0at}=-VyRqdpxoO>3Qa-jIN0yB=U2wgAv#VU8oSKE67Te3+Y90@FvT6 zqqCB^Y^im>$X*U+Er*{mig-J>(cO@Mk3B%X zgFhQNn)o@wZA_YV>@lajVe$BS-j+D9z7F+=$cQkcf;Mt2 zV6o?4o#3hMaINt-3U>QOa{qx76ti=-boLN7bowtioUAA-4Z?)zW4Y42y!6fA{)5kb z7GI|kGH@s;pU@YNx=X&rA+z30nw{>Yffow@9q5zNp-9(^gTi^$- zVMPS-^0)?raY2-9zL74JM{FnJ9jep<&OD*XFL%nypw7nicf8}7xPKTBj4`eZ*zlDg8dr655Wr#Z^95|DtflOtf8gu(sX&@lKs$wAl(fUW8s z>#=op`g8faR{-smA_`_QBZ{77$vwz@iuAFvvDYEf z7DvP6_=K3&kZYmR!~v~HW&&ZVgR184-`Xa~OuQ-jzt_z3n*{YgST`k8Lz925-9Lj@ zk&3N6iV&i2N{XJgyay5z8d9O8ev&m6p@yi3D(Ia5-dnToc>00VRki46GAS#Y@69*R zj$-K&E_e{3vSfa{xZ-8uVUfRn&du%lA!AGv!Ecg(I^ql(G%9Y9Y05fa5jsa(pegPY zb(|lp$DCol4xFiO<7k7<*13EmK2ZMy^g(t*ZMvLHSG^5e4YV;i!V)c6#0{eOh{}A% zNv4Bzw&z1NM{*#+b?j!w5jan5fxU|?K#;eN0a1qzBUx~!*|H*gTv3lp-(*`!h90A^ zi2*P1IeOr?n?*bD+GV{m^I;t=F7?4IFOhOwPUARPfn>!V*Ssr2lJNz#rcMjr0#DJT z^zHeAQS`dwyBeLJqQw<>eCcY!9sB3mq9E^Fx%$P6CbuZZ#M3gr>Ni=N#;wf63uWvW zjj*2eE1ria+&t8Z*OC3Jm3s0oy4~%?!c=Gxu0oJdsYH(4Tk?GN+QYKaGS5oh~sfUkDj@$frsw zCDn(#CI9sNScA_Xs-3+BrD|G06fRvwlpeIE78CfCFy~WOK~D||bllS5mIe86F6dFB zDQn@0Fq3BzC#dwj;V6rgkfD_MRVYX?&^R$Kl}qbPVSUO?G8-~tQqUMUDimcL9CyXD z2PJ;aPg*MOue@G&4Fex1lTD%^7PoRmE!=6P8tf6Rf+Sn)yQ_xg$ubAYT91-sflS*4 zq_oWl#Sn^`vk;qz@D~bu42!SG{IA zOe^7lj(G)8!snPz*7p6bCu?iLW45p>1NVsT1i8qTSt6X)s+aJIofrmN~%Z^!(xW6L1Q*F#8ukfP}n*ybV6!_iu^4xYS5a=r^YG{4SgQ2gl-{8NT6v zC!ycXhU&@^sGsDZ@IPb370cE$F(USeODvkI@-&J{6-8lFB2xbVn1cfVa-NO}WCBG3 zzbj_=H85POs`;PWdWsR6c4UY%q}1D5>zTb~CUbmVpSD>}Vte%8!FOSOC5;8Di#c_q z3n0J-bU6m#Ae1eg^WyqsWy}FEELfZ3bFdBYS0=mr+7M2mzUH9U+-YtAc|s(rfCgnl zl_wL`ztZE{^KkIpQKn?HUSvuPF(wMyD)H(n^{iNf+zhvp};3C#xi1y$Q2Y!GZa>;0d-Y5)kDl`+m%@6`L%gv5!}p6lB%YK zQFZ|gjvzcPZ&N3Q++<*7@E;ti>P^39z!CYz^d&Ms4}4ibadfBW;kZg#4a}m^a8R8 z=%n;@%TpVny_Z&0lH3Fvwi2Uj=YOXHueg<_0_aYmD{8Kjt2-kTUlxaea9KA4BSlVK z96%+&{x&O_iEf6(CV(6nA&va%ieygFv7m*)T_x?JSDRe!_A9u~4>ZN7fCLq9*}Aud zwx^?|5kTks$hFdvOuxrjp@k`*(tYaP5Igrrm^|bes{qMpeX3TXnvG{!+NgpZZHU;r zw)X;SO|pmi2XIp}0G>{$l)b;K%Xk4$fbej=NKN3(X5che?&CzP2y&!zX_hr;a$AWX zm2IkdPQ7q<^WcKfUmZqU!4+ozZ06_p?+tTR-6g5^ zjxv_))a!F_gv#2zApJY))88h)E|h_V>kH4s`2ZVUlI8k4_du`11uHvE5PN4A)qEsl zf7I|tZI*+qh8?1a3Wv=0mIl{Ua`7IL_3A+k6Ymal7M^zuyfFtC=R0h+d{rsawupm4 z#sOU+qlmd(&z_*`Kmcv@gO-zKqCCXJ5)*DRAR2S zN(8zFKj|R?$!)@T0(`eKpbyCYkO}2k66UGK!f?|Od3}$s-xrhtRs}h8YhoWMv=(X;rDdJLZgn{86_y680ZZXZ z2*V=yfM}kExuhD8Y^q^9xwjk5ri;#N3=h7hW0PEvmkWK!XKeJ(C3K)5MW~P!XR@Hj z@n>DlM|Z^5#yY+8(Uhr<&N1m3H9~sB84j_eW)e@f`Mhs z$r<6Mop!ppTdZc7CVB7J4g6>ArlJ)yhQqswH~1Mody zShO}*jG!7|C%*i$ij%SMWA&H@<^{l?_o?S3&64b#>_027_Ach)Ff|N+h-hGzz!VJ9 zSDmRB@=B2&pZbgo`dN-=V~ft`3uLNMLzK+}t*R>_r$uB7M)Szd|KT#g<~r@4+1J5+ zPwGTz#4ZZ4mrUx$4atmu1p-SZ5$!%fCutO4K+5*sb6*`qyddhs&;F4aZio>ZMiyou z583Phe4zx}5k=EjVrc)@unh8>l%@i)(i8ZC+6>C&@)W)4h}+UhQJ6!rBm9-~ggGHT z_7&s|fz0*Cf&F@yZ|B8%7*=jVZ{L?Z{P9n z=Rde#`Ui0R_xq(7S$H5J)Zp`1>j81vtbk~_LAc;SseY)Obo1gO7kMod2UZve0Oi~T z?Vk!MvgF0{4|m|Syqa0nQB=?1NnN>)^$d4%){#WEyq5daxQTHMBG{Jxkl+wEMLP%2 zLmHH0{-g}+X_~p@Hu6Ksere&&~aB^awsgtxoo(5TD)h4L<_&jW3orA;Nj>P7uOAx}{z6F*}0YNEoZP z#`4uoZx9H+OkegqZqhM5oZG}Rz9s*DpTupQ_hw`)%o)X<1LX7n;_MxxD{Ysp;qKVB zZQHhO+jcrmR&3k0osMmEY}@LjlQ(DY{hhO)=lgLm*7~!?J;wTRkE*(=<~3(c$p!Mt zXYU(Hp1fUUEWF8q(pyY;shc=JsA!OP5hslj-q?uwZCPmdjO9t{CNLgNFXF0?$_r0Q ztno`1i~j6Q8@T*V8;d?lAbIg?0*gKxqW;*By6E_F@RTyspGO{^XnTq4%=VWQ@aft_ir&Jb zXRJ2j#Fc7<#_*aoX`_N+!SpB_h$l!>;P_I8!v<5&A^U5!I5JwtaNbb{>aLNtpnkvv{oAUg{vG)Lhv@>#0*P-{neoot)Bi0gH-5hk5OP?vBomKgDw zm(Xk8wjqt=WH~L-!Ki0)qOhR6HtB`~Ot@Q6$4Rs~Wo(}LZTI2mQKLo6c_My`=5Y6# zcM2+o9oD4+Wozpjb<5^}O96_}W>3kdd!;f{(Sepzd9pb-09SV(E@8;wDa8)d_QBQe zlZ>5u8ox#5E__J8q_>3ZIO{&tw4j71r!X8SORzmU?ov25`SbheSOz>ZJhcbXh!NH3 z);?Bg8O_*8w~6Hn;c_w)?Lw&nA3zQ0MQs7S;eC(ih-q!o381&Gl$H{~c?#saHdI;X zqN1Y}JzCS;V*3QRBuTvq@$v@FB7Lh+Z0#&5i%pcibA+>n6NkzKnxQ&O0>HC2onUtnk`QdjTssWT^bOi(+H-k_`>CGB zEjD=Wf$4{!=C^nYrMGlT{;oNqf1@7KNEUYzys+)^j$3Q_5*p*Tr;qhfz0d!W8^eAR z2tei%mOG^qHa(+~HeG;qNe)^@k&}+c#?&UJB1D4+#f3$B$e4FYuDmM)+)+$o@>ZWw z>BnpWpjnI0D+FZc9eM2(G_uUNZfcg7Ojt#WWS3O)xn$KB7F{~pYRt-rbf;8?piECY zm<9w#=vbxIN&78ZxOCN>i`dmOJgH*CZGr^h?`PH*re=5~iM!54-+$<4=T9##0Rh#b zomR-u{!(qP*va0v>KQCqlg4PGp}@(ba#bx(p^%Z%OAXdwX-d&RhkbY<)kVd;s^{iP zxswk){IN2p+N?!2#iUDH3SQB;2o>U5s&_z1Bb^w25qqu4we=RibasPMRA=%dHBe}X zqb7jRmr;w(ES@_)Y52!4(>1#kDr}{-ni_YgU5c3tG%PbsucT5fB079O+{vy< z?HCR==1TCfIV}j`4ZQnoz!A zGNpLaSDXnNsb#vzg{DY@%n|AFcv`}ecRI7!Dm?Y7!wg*Foz$?yp`nB#B` zoa{We^yLoo2?_~Y`1%~Ipj;e8Gr>b$c z$m}cKZ400Gz%q)B4rZ~(i+)jle(5R32dg*LEMiUYTb*JD?K?8U^`&Xd?Po?O9-*OE~w-*HZ0|_o_rI;KVJ{;S@gsx^SEnk z<3ncIzFW2$Ry*?-n;VOc%647x()z~W^UF!=r$KJ!UdPDTMlFIT^L*Gwy>W4D5j6R& zGRQGYvr`mRlE~iu+$hk{*Pjc^JN_OP+K07FpuA$l#+^8 z8X87gtZP~5`&{0EI*5q@uy`s#q93QWo+9Tr*R)NUuH@?XSdcm(PL1&Biz(-^ouLwp z%=TV z5YXggafVX}GW>z0r|&Yh6%`lahD(T9ZYT>HijT%#U4AIcMU>+r{AN!1ZSbY+xWbZe z#A&*${tLMnEF1usyUg>O%&k~YRe0qQqqLJ@nq%ytER4lU#`H_7>6#&-weqYKExA!E zi(*!uRLQ)~a4~-hk?;XKAwO8cp7^?LApy|RXpn9&)V7s+3;!40h>4Z;%Lh7lg)Ulv z5o3)IcnflS2t=O%IcsP9XluVX7eA#L@jZyBnFRyuh5!8QQd$0+>j{?UHHj#id+Ng&b#EKG1VxWYU?80a zV4sC}b0jTx*>}_0A#nVOE+};_h+mE`nub){n^;G&RItli*KrEF((6gU(%*!@KBy%*#gN^u}RUp)B(4GZm?>V5j#DlBwtkr>FkJ8>p?M|Q_WvAu*>+OA0J&5AhQ)mpL+a<2Ec9OSHdoZQX`x@R_6`(=k_ zF%X8*6Oh&;7+q(cdyo<$_Z2AgtR+eExLhyeX9?_1c{`bEStXD*%B$>)pOK8GM~m!6 zniicVZBvY&^5Ycgl+0300bCKNOzlpKD9&kj4dJf9J~BPp$Y1>FHTXrAOs(nx*`l_X z3|;vM?6ffZqSDQamjNgx8;OI6lp>vaUc=kVwqIjyt4+6^r-_&`&1RAS3xSfL8E=*@ z_IQ#9!Pngm4l1c3h1+Z*s@RU3!L_^Ow zLw9!|411uo77~LWkpJ19ch=VJ;JaGSJd6D z@U1$!$nectj%_Bt9+!*}dS*m>tAlvvj_5>G%TQ=o3|Bfd@mO_7);n&E>lU6Q_?#LW zSg^{c;;nu8{OicH+TU2e@ueg0{E~^P{vU<}NxT1?5&k(wlKd0zHMVy$k#zZ&;cv65 zwlj(-%7+(yx(QP|#%N%k31ZR|9mDtA3uv$p}RhI zKmX4F@G2!GQq`SC`40kV-p$T2J0YkRDW5dH($1N~w`tDqw>SNtd|(cE)NI4I!wADS zCDE-nEqYt^H#H%e*uZIx4vP3w+8#6gZ$TP$j8W2TtEsd)Ep-;V!eArP2NgGJptX== z3RPkaS9fXNu)Ay?v=K(FW5$$L;p93@Q6|;4+!G8?@W|PIg%-k`5XEu50+!9oDvL;+5GpS$pHr@xD8BD|bu+BJ?%<_kyx&mZ4D-eU~ zEAS%@i`oefShI5s3$Q$SG>+$&-=+=4Sl`hGB-p)06?e0wFrYNyJSs^~e(xU}s3(M0 zV+(coGQ+_nN@0*&F(%03su^>)Qv)N)NRB72N1xhtMW1(mXnykJ&K5Uebv4smQ2<;F ze>?a^S03YjwHZ;sgl&G{KN_1@=X1N#KYf0o4h2)ub?hZ`(bTIP^piX%&?%Y(Lw9 znby)U(|C=|g&KI-Dw)&v?XdPv<2xUkefuTcLUJ|FLcZ8sg)Tyy5DLu;>{LxyxPG*G z5{^{~ttwZ9l7Ge_`-D|Xc^c|!+@2UJZbs>;XHie z6XqC-WM>=*2N}6pJUeZ1M96}Pg!8)LvvBC9O6JV0eI5SAM44{C;lLnS)H;mQ(E)YG zYX-&-VfN>f3+*5ddA4b|X&w0Ka=a`yC23s?Iy0_Wy0})$X<``tT!$8hyTK-Lb6f`Y z?M#|=S{B3+^stR~Igr$%o(VL3qlZ4cV|uTwMqG~5X^^aub1}W%2{IO5&>#M(2A_X3 z8b(hyS0ni{8s_=J9fbdv<>W6gl;jr*@vm!CRZV9URSaJD7CAF4AVQFSB0`ZYk~p=n z2m~R9NEkUJAvCtu%=GCZ!+G8(AV@p|Yq7^hbf1p!hKOm)soW04I z6GNCND=V)r-!|W}uYasxZhyQUs(&lJ5k-1qDBrb0LX=@EKvl*nNeoLxXQrIo%h1x3 z-vzU)IIKq58hEpM5f})?;*%gMkj40OAJVtlt-3&Wsxw4EtTD+3A1$uCxb5(kr)k#Y1guJ=C=X=}~2ANIA|$ zw#aILYIy)&M&anR>*POg0~Jm&vFl?&Gud*K8ZMw|H{%;k7aOJn{YVnxV;9KNn65tt zjj=B%bc0FAY=sJ-_HjCT75QnZ{oyabvdJp_NJuUZC965XHWD<%xsaXWEGm`JC>PTs z@2H0!~1juOLj8gMPOp2@B|phbfkU0E`jN$2@;l#rVrOylNN zqVF6W=RJRJT(r_$Lt3w+Me5}H=hk*m<~uvuWX0)`%kuF1gFS*BHFdOGU0cMAfH??@ z77mnhGT8%KVzFnXl0xl` zZj1j_gPqmJbL}yn5@h*vPaBT)Bt7gHm7}0DQ6Wwp=#yOpb(|3t`O$>rqC+T1X;$x4 zdSMDx?=M)@%}N88uBiR0BP|)ajv5`c$HIMxHBheB1Tox^+4B4{V?-Y=8xAZfEISHYsaydmIPT}&%s&jSNwpOUH4JRmx8_S$iC z=c3d+S7wTKpVO~1Q1?s7o9*fb#iSDjfXl zuAEA!K&YoRqJ{)l)g30LcCR@CHOR@8Z0irR5x?3gnb0n_E6P49uxXrfQnr{rDUuIr zToI6_U{LS#dEE38At|*4Q0+qCih!`Tys-l^7Lfg(jHqWGpOPyhy|`5{S0R1aqP9#W z(~lI8iQAWe#_`2GqWOD>XkOJ^wYaFfE-A>UY>fcA8`!6N&?OwNzsnKc zp~qNZcnwS8mJkd_BlZ+}lYYXWYtB2T$Gcfh1AQY70@x#q;Z#WfE8w2AO_ zt`cK*;HctM$YmLRYdBaHiDtYK6LL>0`dP0XW1SE)&u}nfK zDA-1acR;X)?^bd|G8Dy#$@8I2Py4k_#I#ecOwW-hRv3MAQfB@L|Cukn^fo&XZ6TU) zdlPiO%Hllto^F3$`FwdU^#gW*xW}3n&ASl+QN(D#wibok$C^aL=?#t}XO$U?L=}Xd zp12YM@kBAHH+3Xh2`YflkLndIN5vcO=h=*qZllq!9bG0jeeD3)TfCS-_k@%Zt%TTH zduRCu&cqU#6J!b97+Z7G5C07QdvO33b=RE0v^4hMK zWR7DOCul0IC{Z2J$`fWdZj1#hvN$Q58I5DOXBib3py)bxWicaLc4|BUf|<)_H>_7v z++_?qv_>O4)4&oj@6SkDRIyfD(ag@E9twHbP@P)C?mB|?PGIav;Z4_( ztUM8MQ6kJ0(J={Qf(7iu$xgJy(_`eGA9L*>pqOab>6Z?X&#j&tITOF zv^#ySt(Bmm*palA(8x=<>b<-Kq`FlinNKEKXAk92pC#f|_vtiE z9fDtNWqXCSun?P<$WLYW7wekvi2RN6u8;1>%xvZE@fXa>?$Sj$q216Gtq(O&%kr)@ zVt=Cz`*Y#eyQ^?tx2s^67x#J}`1T=|>m!)|`~_N1eMPZw8g*|OlZ+=+=F^GU{m;MOifgm|N4wkZ;?>RRcWXjz74)r#d{$7Z2fXk5q7 zK=4CP6G>{+HYf*d1i3EP9QWq-}1 z;()BOvl*aJ_!+ab{2o5+l|uXMfM{#EPaZ+{Cf`w5(P9c(Q}k&6z@T=wBwl{ElFFv; z*&4!*f#L#JdpoG4kkt%PG77UEhJna0 zTsGcv=6%gYbFE7#D{oiU^vfvQL1C?o%WA@r>PF^>zL~){m2Dot3PrS;T7_zXIx|n< z=rRSQy;HqZ2OZKe`-{7PI=x7zX}-T4Mf<$s)LnNZSu?|SJ9i4sYz7tvE>{}9D0E?j z`IPyO57g#z?MAZHiBZKgL3f0YW+se$`!uOvM?8cb+zM5GR)w;MhNvM+0NrS(%_ldcgQCAF_eB=oplPJ4N%ONicaPpc3LcSba!R{49FV7nQ)umytW~jIXJ&tnc;Pc z@%N*5n%BQwtJ3?@z!OH&zsxwd6(gCV?}PX(ou2!8dB*wX3_|jRy^Z-G2}>ltq2%-e z^FfV-g-&1pd3U5S;JYSQ4|DP!^Ni36efjg<Gd2`F%VM2iP zcbka*BU7wT;pD?d<%1V>W_2chwF_SeZijKQPAGNd(b)I#R}P@CzQnBQWxxm98wg`7W8< zOC277bU4cO6%4V@Vt$UMJ`d@oNJ>naPxw!Q$662t{8-vrKF_}g*UBPTO^YjENhav)Rmwq ze~7-bZ1OBw<%iI%7D+3SiicgWtun5OpJESWJBJ~chNp&m9OCRbZd4%R{}K3rZOUTlR8a&}MEkrlTg8?Ql>Q-* zuHitSl5qUe%h?*umZ=N0Lw(Bv!nr~Gov*M@*QG1AEx1f&HlLaHoaXkPzPRS+_x+wW zWQ8Qj!(?cr<=MidX=;{~&dOlk-`f=c3LJ}B#XABUya8LZpNt7`#((|Zs(Y1u$y4S$q$38*L@Y0CR*RZ?p6aE!1-is$O>Fx-U3%iy8IUv7ZuL z6j947KMI@r){}=m(OyBb-ZdPFYN(zC%p{8E(5lrU-Lw{Pkn@wgsGHd1v>lzJZ`Wjy z$nuDsyzC;ZLHQA5t0ZHow6mYwB)CN-+dN5OhYy!Zk;>-Q$0>mdFMAOJ$_@ipVa{C# zN!I+B$nde(@lsarTq#+R_#v~a)@ydu6vOhO&+xJh9y(fEJ%eA>*(BP3OtE#z9>tWlGfJ{wxPwS%&LCYaT|(p3|ML~}D? zVbL<#ARj*^+`&Wi$;*l*?<3fpyU(qWco=BbNpsV0Lc@#XRTX2uE-pa;%`cVd! zbo;6EeUfoH{+I$KLf;Ptpq_0SQj1^G;Z!bte9fCJ!IsSYYh;=gTdum+ISK`hk0k@n zMC+HWI}&5k=x?#*JuXnE+t?uAg%V~^im2sjM-8or_(GxA_k7xRF?)A!Hc7)Ldlybb z2l(3GaeyKipIaEW=Vg0xu&5i4PBRle^zaoLHGBC2rsCL_ZlGqF=ePj| z^q15DWyR5J__%J#f21~pnWNR#3X6o*-kv3y9DuVOW8H@>pLGe|4Wb&nAx+cw-YeCt zj3tH7V3Tl8h*R)5rRFuBJX59<#(F5Joj+hrtw`bZp%;w~d&)FQ#SH4(x480%6Cl$y zTC$hOjuPUehq+fvg_Du(J3Vn6-_vQnF2nJ;yV9-wyrQz?4#=^7v#xQm9f+lb9}it~ zFG$OoyR}PqXUEvccQ0=xob1tP_izpJSP|mwI)i7crvnW9q=|w81Dkj!SKZC!#QUL8hm9kd6Eb6$C79*F5F}ld?C1ncXq|?> z!&GxOxg0c%@*^UBAE-+ciax2oQPiUoMUN&Vi0Hivr6-28hN|CL^u-kNhp~=x;z+fl z3pV16Yt3zm9^{eV-+<#|0e%>YuV6^~5{wG`&jaEgi@E+KfB);6Qni0d-O-@(EfGaT zQRpG`@(2{|sH#Nfv{8_g-11iC%s8#Tv16-s?Qm4 zlw_GnG&Q@zA7>ag$#D)Qyk205Z4b3>4d@(SrfG1Y;o@p#+F(8GGKmV4WuA%Ca@nXF zcGlTuX_9S}h@J**&8ljV>n78&J7$Mb&n#ZGvSgZZ-EH%h1pKNZ)SGg+{k3xrn5bL8 zN0IlMT4pj0vYq08354g6)&zI*aKxG%I#Vj%RRDlWI-n zPO$HCvK-IV+L*SY?q|KUxoaYo^`Ap!wsuyGoAdqcT@Q2zNL;L-X)3S68pZS%cOJqwa79is$@DQ%r|qENHW8I}kYx^d#VLKbzqzA?`JmFO;H=gy zN)%_x?4@x0eF_DE*UD|5qPp88Y&Z1%;dBc4q(mJ;)Fh8hMJzzd_-AkU7(M)leKJ^c z1Tbz9xc;av>OTr8%!OD#e0X8%YIRt;3`Fjfj5t-I>^Tc1eawkG6B315tkps}{|1pn zX`S1T7oj}X^PY6=AYc&O&!2ysqfwU*z?akj@67q; z!|ZHEc09AK|MUB#F?Rq-4q04(BL{#1o7_ExHj_59nzQ9-WxO%j{DvnI6fKe_CsTu| z-o{Au8_+N9W!0?~Vryhw?L@o%$h(V~ zoTnx0<>e1|DkI_rV0v7&p7EuO!v*UqU2>Go){=Ml`9ft~_wD2pIpB1g+e)4Uw}s`H zKYDmbRB~n6cQrhDmGIdg?nT;-&p+flSpzjdATt)k821Y+6cqxFnrqgf+3{H@?`&g4 z^$f~e34p@8y%s9V*fTj3I#zcg*?Rt8f-9Az{-q9ma$(Uj9r z(tdi8l0gAY=-#a*1=^IktQX1hv!Z#Q2KGu_FBBf=wSfj@w+xp7v{9dD+()?H?Fkc8 ze=+(U>KO#$MWMVfBkqRJoihk+0a$9cVgy8v?Uzws09sbUKQd$_ka$+2PtHbkW&g|@n8lp$mZw9r9vXXwFUto zSG`0I!xqu#Rnyjb6yBoo&GVyo91ff|MM_LB1X?BLD z4rkrb2{S<)Lsy8ST_;Mh6*6ez){EAhqG@dGXA72Y6Q&c85JIwZ{FXSFFM&D=QE`_R z`X0!0%&7YQT|IzvlNs-N3axxR$jhta=g0KvwUgOnZ7ts$xWD#oehjvOHAg-%7pA{F{ZQV4o4tKcn-DA^+4@rplEixxh$-W~`{;nkM zx#7faX|{(HUvA0(Ggr=#Ggr!%RTEh z-Erwf1wT<-R=sr0=`xSBf%U?Z#f=_Ywvw;cRj}%f_^^k`QrYFoN16Z+w}S>j{RHjZ z%^Br-{nIkq5zRH8bIoM(WRf<+M5ydc%Jr^Me{Sp8K?>zE_2E>6W?Jjr{S#4W;Ay65 zYZCok`nD}DyS3?Ojc$qGC{>$Yd|UWM(I7J0C)ul3v;2bLa+Ez9-iEmv$~Su1z?|kC zIwyKNba|)~Z#lH82>VVs)d6~tUuKyDc{PehGI^!(59Q;edw-@%*TlrVB2sC%wB%c= z(PTy(*QwAkDej7PJ=!NFOu@C7os}o{Q11|HrMhx1OwL;h8ZD&4+;q|ez?7b88r#$x zDx}y0@-L6vnZLlHVzjpKRXE{gvqZb`l)~LqAY6FwmKhCO3l}EcdwS3CPF5$~WxGs- zrG==OF3oIsDXBIjH*1qxAjVVeoYT%68VJjZT~oZfP0a1aY$V&pM?-kEgp6IYF5wjQBYb+_1b<)G2FX(q5ZAph2k zPo*~AxSv#RkcYA9i+1wU7@!8z8>)rb04M?zJ7J7igS8mP2Iv>+57@)JqkZw1>~`E= zx&izc9>4=&TZ7tQTf^K{^uT_Dn81GhomgAU4hY^Fd+KE*z}NdUwOL7t@Gz5mVE}&Y z7wC_Q-K>D0k~@wv7#IV%K(C*QV4w}#EHRS{sIt&k{ekBc`w}?xLkYT^oVu+UmY2K% zE`?0>^;1TQ5qgy9k#w_G%G`x}=AU z9q|e4+iF{Dh+|DsD6V3VSp|8OmM3m`)jQ4hC34&mM)E;}%aoa7rt9|E56L3TzL4VG zELPeg&7GdnxoC|_^;rdHOO;?fieQ-+!l>;Tf1p4&3K)PWeIMc4RJqG)K2UEktSgJ+ zel^jIYjAxSmdn2PHKzN0OV)Ga&((fK8(^zC%2MiMoJfWobUWx8jJStIog1F}9! zm2r{}7ulTNmoQ)t>O`w*Z$%zlvjeXeOvrYZSsDSdBi#-2VaM!1G8%!fE6es6iyt|w zR`7yoY7ff}mBcgPCnV^SU79$_&&?m>0|1iwPMByBx@}c;@AdcsYRFaO6)^!YH;O!a z$DEz}^|PyONFRJipYyY;AVZXWSH-UAQa6IJ`_%69cz4`w)$|VDh&r8Bw8Ja!EX9Ta ziEYG!;{HsR;TuJlc-6C)OIL<+2(pmuGbAA?I|*J7aIT_8Gj?NF&TEKGkf@#+)(H1e ziwyUq)(fR5KuUa6E0{Yv?q$rKT#nh*_e^d^F#?z|s}1AWM&kSQvC08C7ZLD|?6!;r8;0 z_}$b|O57IbN^8^-W3Uc5>8dQ%X>LI$@3JP@ImEsrinU{ukwa3qtNtLX?g%Tkv($79f(9~Dbe>|A)Ya?5 z`mepEbo5{!*4Kn5`gIil_ZYadv4yFv;XiapVwR>hCd&W#P4Xp6GByA5S^C$Ztg5KI zpn>v1Cz;6tV){MaZBAF$L0hGw=A=Q?raEtvb}wOEW~5Xak~FQvCWDD<|N2(b!E94g ze8k)@X`eIo#Gi{#VIkeQC(--rwa3{mp~w646Z@O`jXD%pjb>ZSHf$93%RoN^l+BS+ zn?H6J&&&x~AyFYxp|;I9nHbGot87%dMCK@l27<(-btH{ZS%dPRk(zZ__Q_i)%ETbZqt73U6-ovr2V6#a4$Ue5eg`L zeq5@YE{`_(3YsK)Z9K?Hwl__KRu!Ly1869e*O|n@i8qcmb-9P2 zsOPw=H}VKn>}Nq~f_CxdT#2}|B&Qy8oFaRXlvWD*WOUtT1@EdGER7NtaX{&6;bU*- zPvK9mA{7)b1VV-#k%n6fNxPAI=3^{hp5mYIFqVWTEk|A|Jn}9n3s(SG_c`O5_!*_r z^qJw<_vx7RwxCWhDj;^Rf@ViBzcr3fecPyAq|8J%rYNSU`LB=Iib76r{XjJIB%bHs zgBbg}TkocsGdNgBSIZ$AQO4*}VG>#z=LC(DRxP$NkQ6?@Ao_7 zdifE)0bhoJ`+VL~i5>F>$>tW(S92A-(U}7y^nNzvwfTzTm z)Ue9LDVB;is2=?O6;J7HmT2WCylo(Gm7+39X*tCW{A%2yD?>0u;mN#&TiMhI_$Yb& z2x)O@Xvw2Cn#^zJjEL9V`GePr?m#_!u9EJ-TXYPCzPuVBWl(++6@y9xigKU{57~IJ z{;UqU{%o(nV6I0JN_Rbh3tz4v#R4!q!*6`q#*qMw9jYmbNb@^v(F=15zfw>1zwLnkE%pAHb*e6wHqIonh7NzxI9IFw^8yNh zmP(h*9pu3OS3kt7)= zR%@Fof+W+FBOo4%A|#%*eTJ6YxX6;NOJGJLZQpk;7-3pgQv3G;O#VtQl(5INWTNsE zq-7*}h>ld-1dnNa5$$f;hSqM4$~CFs%8XR3px^YpJk-1tbShzo#d?2sW7{$0T4wdH z(&e6}t}S7ayL!Ij6d2?B04<`88=BKs7}{{zo}p+UL=4D;Sv_@Am82CV6KHXl(rGy! z`jBM0dU@o~kB36ayB-%C`2pVXOYl>3_3`tgkqWrVGm)&gUt))&jX!-}DAl})Kj7^j zij*B>8R=DjbH|jyRa%M61^%Ka;J_C}&cIxQw`Mv2+U$TTHSmW5Zzmrht|;U0A5G<> z%De8Rci==tG+tRPxPmzRBimx^TIMiAnw2Nk`iX4{3&zycEpHviTOb!}FdulyWq@MX zFe#IF2t>jO#5Z7Mmx4?)KxCW98q~C~kHBhc?9Ce98FCO!k91lmM13vCGprV_+Vfl; zTU+QT93@~}LhV%h2~52XFU|!uK#BymEn=c+#Db1XAxWoj=}}3A{p7T7O38%Tq!yv+ zg#Z*~n=?!Te$akw!QE6uD_o*lr2ylPef~2^U5Bv&SwOM;cjZ-cl1peCyM3T3+opL& zp*6&cd5GbGc>cehV?(BA+_hiHB=D6?GXK|P61KOo`SK04w72_9PBkan{)cVg$Y(3e zc!wN@e1!aq{|ah8RM@yb2s#=}WRC#j%T-NvVUe}uI-=~4UGaGlQPNxBm;9(>H&<+B zWK9Oy>-KK=pKtunIo`kjj4a!KJ5o@H>r=skvKya8c`%R?#On~EUBKMvkBZU+{lj3g zk>VIC=ySSK(`HpQk(6Dea^YBV5)FfbJLgZEUgb)k zvYunPY1u{Q#^zA~aZlo~O4E7Vn$DcN?s^o?wzr16&IH_OzIcu~xM*r_0Wdd1|y9NF@Q;EDTWI!%kcVhYjnTQ^~eLQ2uG|uz);1yNIcbB zbzjmn_Ngb%Pw+P1)okgVtw!j;%70E;QRE3`i3El~G$k-n3sBY~DX9~Mxdh+m@`RY+ z8=R9ZTvYa(FNaCyP=UubeVnwYJ`k~Lo_hAy#A|veGJ2n93SkGe0co{~?+<2o?(B07 zCAh^4Y47@AD7nqewaDv~@`CHHrJhY#uF; zGkZ|BFonlhHRN3AiLeT>vpbh{z=|E%OV$_aOS8sPF&nl=-&?@1K3$kjB3h=1hs{FtM~CCXh@-(S*>2{-an? zgkLUOGC~?EH>P(9GHH{8>5>{770oSFXsI2GbS=&Pw)LPD0aCiw#Y?|B(6yJZmO88} ztCK&DyV<4tgGuvQx7po)e)7I?(#QHb&G>vHqc$;Sv6~UY?AhD`IPQneU&oo1f_L5!!yoO^aXPo=xp}T!OdqY`!TUyen;6&1 z`0e#~GquYzT&EVoOWfx{!?)im2Y9bU=eFO5VZUv>w1e;8K(phy?lZFcp7z)BPz>(d z(Ik8<+o`lG(mKeuIi}EDCXFTH@r0CadWhxt{)#@{c!~Zpkn`9I+TL@Y?;?2UrYJ<$ z^%x6D)%6hV)2$haI&v@x3RJ^{!i>Mm!;8jJMAuVwShTfZM*)aLO9;UK!j88fz|m6U zlozB`!O*~i6E*gtP^$&!h9R%w7Zqd&jWRH9-Wu-YjMRRV+eBX!p6XfCAVILoDc-pY zU0;h*h@u~yL^q>_6VLGR>b$aN{6QNa$L7mh0zNR+BlgrovApiG4TQw4UuQk_3PKp` zkmD)lJ-7o}7!UJ`3^R3!ZKXpbG9BZ_qEQJ|io==VDoSC!u#S$2O~V*daSAa6WGm)F zfMo%JWty+RQ8Gd?SJoef!bsYV2!a?#%5g`QIIf_qVQ9u~^)T$vwn-GxVZ}oBc3|R9 z?jubc&>|4U?fRq}2q%?bnr{kTaJbv@lvl4u*l4gvMODJHA&~)24;LJD7SjSotW=t4 zp=hxpuRUvtn6(}mqap3DBXgeED>>V{mUgADUEI3vq1gzWFHp5eQj+sDZ!<`PIZc_i zLa&S|P*gOO`iM$9BofRvvFIG0GQ7fjXR@&czLKS3l)JhAr%=UMxUcW}QJ4!nDExvK z_56=mr)|maT|FKuxOsk$0mh>Q&C@f1{M6Yi-s$@Vvn_8N2_;L9{hc;tv~Q<^t*yBW8+pC61&y zV%mR)|ZETqgw#Ho$lrie;r(YM36SHo7KNHGku&BglO&stH9tEKOajk3Qys4P#$M zjFELHi_+-K@(x>T*@k7U>!CZaeG34v-=+Yl9g_jAAsOyBHGt>3U{9few=c;DS5;~^ zDU|Y?QFQ-~7_2!o^k*FUmTH8>$>a=i@N7vAs6ITFJ;eZDJSd`MRa0 zSfRSd8@cS*`Yhso1WgxHv4nZ_1yx{Ae@Qf|bW zUM$^~J@q~rHZ6zBTxtChyU`7|Y~2RQe(zjbN<3daSuC)kZtV*R%BE#{mgPHy9)jru zXS-Pz1x~W9ctn8Cr&z{8wsqD2Jp^=Qyf57;^_Pm8>Mzw_>S4du)}s;lFQw*HwY!J) zLju1RE}7N~#Ff~Q5|HIuaq+SylBqo_O^XD{Dr@^h4opq`P*DOL9Ep5y?4AGB5!jQ; z_vdYItcproYS}g-c%4Rx?m}Md^7A>Nv8trRhV--`Nn|?Nhi;<5>f91 zShq-aP-WxJ1=Skspb$p+hohH%a_+lP6F*1MI}4}+nryWw*kFI;zB~<7#@0;U#dtC?u0y^OySfUUarK~dBWxT@twTH0EE7s=aiER8 z{pq9$xwi&V?hrH(8#E#h87xnB! zGQ85J5F6ba^8G`ssLNW>t_K5#h&Z)q$~%b8q&#RVmors}??wz$K1d?{VU+J#%=WQA z0L?AaXEIT+v{g1yrX=5m(~7N@pG+wF6C&JW2sg3Pe_D{ww#@F6X7(Gb;S}j#GPbOwO8=zEAhf4m9^ORM zV#!cDo4OrvwBi!OigViB)K)YiaoXr}!@brUqx7T70dQNA*s+oKj7WU4w|j>FywExB znk&2yq>QLUM~9GPp(I&IX$U4`qcYDVj*l3MQH*wjhB$Q%(069Q;?ub}4dj-LFdcJ` zxD_eQg3(Ty@KwskcYY9_zm1nzgt$@(Dq^r@phZ5hYw?wCUp05EIUO&~q32VbpK5nA zOg_}?Bwz;H{3Si)J+KvW(n4_z>5^HdI`zd-nW5eAH*zKsk6 z>5qn_XCiR0LtouM^CS+0hJ(dH&dxDbmy~jY^VO64Kp&9fhuuthqe|Qmf;qtilK(u)nIqf;vk1oS#z8O)w`hvR9cqd^s&?vooV;QpTzm!!2mDlC2dEf-DwW z5~tjyTycWm{feP;Qf+s_Td^bQpq8Rs<7q84DMwPNdwf8gNh6-ahdybADMg8%qQkE{jfkVHWG@2BBXlu~bgY7tzT( zrmu>eS_9zUnBMV)Upo`6)4JipwoY99V+zsc48HFNWIHxqQpvB9imb-xH-txHEs>G0 zjBnbmLgN_EdfvK-JLL#dHZRN2YFv{&>*6Jy_=KAByI?XeteiR{zgX) zOmrP!`;4glA({{vq$0#L+bTNT-dW3+7lq}ZU8>K=^Z{#ESiZQQ_K)hbdmf|FOn&Qm zjvo}$#`=Tbmv6*RL+t|IzaNx5L_hEKjQzw2lq{ z|2s_g!N3z(2#PV15L40sm_0#6Hi*K=f+2wNH%9F$WY55;DW*FnD$k*6eU{d(*ONkV zq`PqoEHbN__U-ECXIpD{@9pYp^WOB9?-kuh1MY>q*~`q=nZN%J=TqtHFMk+;R*uO~ zva4l>c3AAD{xNWJx2^`fZj#bWiD0LS+}ekI0vk%|&; zzQ^jL2`6^)&=`(@ZEldC<)u3lJGn!}$#XHJ%S)Q^llY+Gn{cxo1^tR=+s{8y5Zmp?njlKZ|R}?Q23YMy%2)C`3`oYKQUB)@>|{Mxt=tCPa)OVkeP&mnD4+D zbNBFW0KZhxK|S8Cf_=zV*XOj~upVda=;dpeH8@_WDL%;Thd?}K!rJk3Hf``VD#oxd(jo0Q?O!s28so;NTiw-(BZ*7hJ<(%skdWQUtz3;QG_-l7M{wmSLQ<9wVM(Ub#|*w1EL!<>$BhmOCcvK3 zK@j!lmyH!*Z7F09Y+fLNz|sQ!*la=v0`tcuYpCrOx-gR?^r++O(*r8ybPB({o4;%r zKUA07vCgmv@(*sG85H#V>7k$Kh)voM;WJ{tXMh8%K%t>C2ML-tQOQcvQ}WhPnjC8P zR&5c+8A4gUWcPa(?}L=gvfJt(K9Yn}g~Z4?=~>p%jg?ZZh#ddAoP05!T`bC=Wb_8^ zCxrP!nE?90*ziU~1M6r}V87>qx&`K)lRoTXeA7obBs*;li$7^j8-&=v8)2MTvPK!B zXLC#-%JX{KK+8glny8dfNCT})=YKE!Npz5x_L)6>V;R3oYw#Jlw}s0V3~nkd$`+CUR{Kayc6}UftFVJ>Yp6|o zKlBxuS85HPSS*u!^?Sq|g(-IFk%D6@oy=0v3~Dk*(QQ1Arj`|a-(49sbr3D`2R7HbN!zIx!JL1Tp-<4#zgUpW*>C#|GmoSoY@x?8`E^LhJxI-UYzJAvQd0s8k1k0Pkz zWD{~fXtP_ZZ^2~lT_ZS>bI#R`cIKg_rl>n}B!SbHBtLS8$H>pOITWCrDIaS5owx<$ z08$eDXSmq+>V1Kb$z=?ueIJC^X#QG`o`tm)K0aY+qpeEh8wOyIlHuLnfrQJ&RJ z9pp{ac57F#5N(rFG3SniLM-Kd(F22CpwN&sYq6mgMfFNmvz+?={Y9!dIqoqF!!6`B%K$2db@v z_l)O7l%2%*Nz4z)DrJOAF$t}Ww6wgoJ|yH4X_-g%YSTkGwYv`bXzd`FG9RnF1Yyee zKsn^Cimjw+uLo#3{uX}n3L}ztFQ`1b^B>=VcL^@90|Dped}?wX$-njo$_cPQA;pkCm6$B z;s%IpL{N|UJCPoWCfaiKsW%>h;$&VKMu@)>1CuuE>Q2lcx|LzM3rUp5}X zkWCvD6hSnL$N%7k86*0`0*te?l*KNV$YgaTt@6Idl({8yGi7u|FPQJ3l_W3ZUJzcG zcaUYQjf!J`S9kUpfo*u(3_GDuY_O0 z_qxE<52Oh+#y)1|boTOqs6Xl*t6T{kP&~=Gzb|Fbg(Yq^;bPv15{yN2rtk-0Tjxd@ z(8HW^NHQj)mEVdIQeq?;sey?SXV{|Ro1x2W5br>pCe!(& z0dkkkf!Aoa^(o=qDTb{R$K0sV35;0g06sb4C^_=+`T;mL;-GXUS~?<_Ix)=-(3JE5 z)ebmr2TnUc>ZBs*_@K_iqY>U&?7 ztH&n(kfap7vZfLA(PST6@Wozk1$JWg*gK$EXC*?BQ(Wikos&A!uC%5gdc zChM?_F?B==Onkm%&6teyP^K%i#ts0oJ-WUZhyIphyBI`afv3_j!e+@z{7qS^r@BnC z1;lGMkW_-CgAcw2l}l{eCl;$K3#R@~H(id5f<1%}z;rFmn(4m(ttb{$Y6ojd%P4eM zqeL`L){|mDYuA{9W`n5oA3a)B0&lYrHSVI!dMU;p~ zwZn!&X}j_-)mXk2W(-!{`7jO@XB8*3h#zbM$WX@3*c`xTd8F&IC7-(9ak=7k*`iy$ z*m1e(rOA0khI&21;SqS{m#7No_+WT3M$8)|_CSx{oYxWRJTubgwI)}7OIcN9J%t6#1}~sD9D*P z%AqGMVFH|-nkg48cFr*LevjTt*vNE*`PH?7jvHzP1BiHq?cfgO742anJ{p92Tr-AjkxT^CPs{Xn8}J-5&+ddJ^OT zrTl;NxEPN831jQ{jO_e9nHq!jSyky&3F$UM+SGdenLg+bA5#qcL>fP?xhEmtl*1JM z3x+BEWv>Zhv$p zUwdU#_E@nHT>py5yBj(pdt)E3R#Mbe=iHXFe#DqtUKl-<4Sh^n3HO_O`G+f}R?3LX zqX|dA z5^xNZ{A|QwO;mkEs6INg9y+UA{;(%{edSwL&;%v1<;DGs7?bJdD!wf>I)ycQh?Xt& z$)B`l&G>S7{%!v#klHPl!^Sv^`owu`i5=?rQPZ%_i70a%s5jTa-(M#kfy= z;K@!k&B4hSC$BWF*^WCGkuDsPOBb3+OG{>)V)2P7tHFyOV!Vb(Dl;7QW0}a-QfXOf z=QPR;rF99RoZ0AaTGttbIV`ls##{@gB=&3FhgX#v_Tm#Aa=RO|E@@`?NT{RNBZ5pC541B0DPVjLbt3?d{nvVCwK9v3!K=Y*OBE z?l_m|uFj*Ee%Bqd)C%6%?zk&mL6?`K)JAg1q)U!0&0*4bTRS;a_1VsuNK$1I@lB}<3IB?3 zS?yM%TCn32B~R_amm;=Xcau<@2w3%vwsTjHN{qDV=CZrmuoc7jV!f~HY~KxYuuBlz zmaIw6ZR_(cT$x20hULhMN3E1cCQBOky98;}6@0Zfou)$;%$~C;lYy6wG~0@$HLE;= zZ(7Q`56x0KJ7=0=OssX}OGgBEvc&rcny>j@;fmad47C$0GA!J(6)mNsC^l-vW7Axw zS!ya2k2Xm$_saUKKdwFcL;Br<+%mztraR8zR%14`iVWd*O+fIMl%<&}5n&V~3vVqt zz^Bl4SlRl*S{RcpEC5pQ)r~owBE# zbDmFzym>57>jvvszkw;?B^%zr|G5L{FR9`B(3#G3@U%jGgGom>!{{TdNIvqbbwhSm zDfeOU`QbA3*~C1NVuOu0K?3c{04#qw^5-8@WyBbG>!fT1}Ks zb9mo%N*KEumWQPr!WVjj1d-j-*mA5&BOdQM8EY+{rxh8k208_0~U(HnS(Du!koC*!9+r@xPaJ5W# z*W(_36|>>k(E#FW#=sAnjq$mo4Lq*e_2k``Q_ z;{Grki!t=2^gHZWaJC{L9e$g(i8uIGk9E_vs!jbYc7G~=6PBWdu`jJZ(RC33B@VCp zSD2DNM&4|gp@;n7S#Kt-{~3dnoM0;GFjS6^n1gC_|2KMt>Nm2A2S=|MTs{B|CtHir7AjTM4+DKk6YjvOaWa?h zk)=wEVK|!BCOzt8g}`RxNHI=l(>v+rqap4DM#gMoWL0Pt1NkDhBTQr^u|jRJB}`;T zEQ+fUj<>{;UUGd-bNy#6;7euwUN z4Nn?jPmssWfEh|>XfKZUb|ZSt&cuh!8{I;Pyl8n;ZB*1NFeZaG;FI2fjX;{Z#1UhWi^3d zt^h7oiGg;=~EGN@q$zs<-5t;RsFjE zyKV-Z(z52boKZjDnYjoJ_1lciI+^mvL`tUaOgApFAmd!S^GiLR)0IlQ6rWP`Xef*g+XOR zF&UyGB|1Oy9|RzO^mV3x*mZb@YJZi^#Ut@1IXLh9s@q09T; za~y1U3X2}jp@|W_z|9NYbI*BSzvbP%!@AzLEn;5Laiss$*_-kxeFMc* zn1~Jy?rtXJNs|a~uz+I6fVj&fid>G4AJ_Z^7&FoErVv&LDj*r8o)wPli{ouAO*w6E zXm5Mc|DS@85Gol`i#B1L7hx7s3?x5JJxiRuCXs@jAtIxF)qL&k9F_G$U}j%%R-0GtlG)j+PBXx=brP}d!F|k@9RY-W?m0O0rF4FU=-rUyT&V)a`KKEPu{ZglR>d@;^BL*-60Ex)-KEuP9_I&SRV6P*ti&L^j zxlsu1p7|Xpx;XY%!eBhwmxjQ`+3P3Qb}z{4{&@-E{@v)2*d*%Gk|i6lI;5?#wC`9F1b@p2Wo;%Dc{chaT}XoJka3^my#d4dIUdHoFsq zADk? %*<3BEL!W$81o>)Ext(+AfISk120w15cUO>miBe>WfO5ixwCfweWW~jk2N7Rt|=>XxHcYKdL!(Z0^hS$HXfmcCl{-(0Vw*IyjT>a3zG`9_C1ztsH zbc<*eT!qktZNxCLjc6rY4b#LmyJoNs*Zjn-_rf5@;}~)XIe@Yj*W`xjAE-m_lH>n~}sy$Ptp z<;!jB3uqt1eL|T3tSfp^OW!tMyIzg^lKs7;w)_4VoYnb!iAl&lhJq(gD)#`OnW)i2 z;s;%q3#`3LA#sO9VPx%ukF&hMI%P9~vV_mhfm zxqZ|+1o*kCwx~WVfJ(e%J`_-bmMzO*vzYc%LdtlKrv#V5Wjc+g443iPOcIyjF`g)< zQ5;2)lcMtubOuIKO+4%TjN29aQ>h?Dpl%8!{akTIOi^?IPk9P~K!0FNv}s-Bk4Vpa zb1VF=1)Maj>I)vgYj_F__5f3Ua6RlKh>5i+(-zRn-;-%rEChsogjG?Zt=RX1mFnpP zWd&c;tgi3_lR|!yNYxc_sGMd9O+3Rc?-1$Mh!Xnp9#`d(-3FlQR*1K|bf)3Ww{TFd z1-2@;pkU;X_2M0pxwcSwWCo?f{u97Q)TGVol5U8ZS1if8ofB~t4E#iudKu6yGH&-M z(SIol|I!!t=9?<;4Hw|mm+>}a#OVo3KWO#L>PTWr>J%$UKg3@3VqU&l5^V?u|3g{^}Z%IS6kknNeOvM0hslG z+mm*&;XJY}(lAx-#eIU%x|Qf^b6<2fjWmRcgO&ac^ ztm%)Cie%St?hrntie$$)btoUnCaI&FG;AKxCh3bVs!M#ci_{k(KPQN^CarT`4DhnF zNC@-JWihH|u}OwVPY&BAhu97v;m+msC^hVFexhVrSX8kh z)JmYD6`5*~gjJ-`Zb1TfXvZL0ZPkhxw*D>HW=k^OkqZVsFA`*NOCo6Un(#ZyTN0^} z4Mn6p7cB7)kzn1cH7221yeGXa?OL;vO=-_g*tN^ec1JwSr<+uc7**8kQsBRy-rYd) zJ0Lnk^1M+xp6Hz?pv_?aV$te8{YC)^w1AR88hIt@l!PF2Udn{V+Ee{puBu(Z zuxiyxtx~-glqga4QQ4{65pUOB>)P7fvF-NN(7t{V6n*iZ?PboCfdCnL5qG-f+~Yjs z-h1!+dF>$gc^x*-2W61gdl?+vGM*2}_qxv)eWU+s$K-!JIC~4d_=?BuSIO1)ytn@4 z7wiuAwF91t{a|O^xbM#04jzWaPyBmye_kBZx zga;NE@kt_ouR6Jx3Y3Qd{06BztHh&H_DmNEpfDVl5G{{SSgqQKQmXo# zR@_ilv92XBtx&4x!U$8TmQN-Ws8^)|5!UBug`7BLc}+8$^dZW6XuD#dO`NOb)1)v* z9m{Bz5U_?VV-$*FRF7%q{FaVVAUCfbQxd+2YLHB$q`Vj#chWpKDjp=KHcjd>BwBxA9Ow~K@DOh&w*piGi@9KuMO zIIF?2f!hvNsyQ5;>&Jq33D-(;l&E2TokW;-`RRo=_kkRpf_0GRpq-9>0n@(aCSO6_ zv;sp~AR`j1>!>9w6;As3X))>`T-b8TKpvxy<~ zdTTlg5*_BCQq#3PFyXLr58Hf4sI5XIU(>X8(`T6FSUdg;||?NTy%@bH&F}y4UN-i z!l3TkF5>A!S8vhe)xvX20ZVjXaU#{s(@hh0v~{Z?R+$qoX=W!-o3dh-X=Af{t!UPa z)ba&m)pnRnygwLBu2mdN*@pAHBYf+8;wz0sZumF6TZWg#*+zKR6wWqEEJp6OkN0S&X?ua>! zV@kWjXk6{FA)!{x5msw1sRr;-FddqR?^>4r4k2hZI$JN=nt_iWAa4XDOc#z%+11+` zQkjYnPc0URydxs4cRV?2JhdRrMw1-IdV|nMHe6#yYQ0~oo#A(s*nI^mbB9em5|oI^ zjO!UKSh=h(58dcV>G<6(Cat24lY3RVW!fhlHt(?==yq#3k7p>Zj-hRz=fh)8`;3@E znZeDgly)`SaGfIw((s*E9U*ZG*T;EAHaJAGo0Z)0zV?7W5$?U zsAE{ zSxa=O8ABzGoM1Xbsj`M=tZVm5m79D+3Uj8trlkx{la($H!HsemYhr*@s%n`{Vg=z& z*~kc6BfO-rm&|*H!n`A$a4W0-a-B+YK5~3@!=Ck| z+J>05Z^pdKOyyWm&hok9McJE zPT*xpD&cyH{A4-&dkkfMYBu;vV4DMt!B~-RV2e8hik303ZC;gtwLMeA42;X>L{gnUX@bR}tIu2te$>{Y@r(ig_`39)7_S>GIQT+M~a4Q;mmZmFHaZ zi@TIkuT?yk2@wz}Zl~FQ^Y!r@q&gR6h$JSjx%g=-hCZ!dhkf$gN65VMx9fx~vk%cH9wt*yX@&$XHs#=pKX9 z?52BkG5Q>ftL9K%Hi~hN%I>%osK-AOcBHk;9T2>OeEtT#uifCyOD}G^9PP; zF1`riEXd-jfkh%iY%$Y)k^$6Y+W=elK&07KRLQ&v)^j%F2JZEug(_8mC-lfm%dGD5 z{3M9kc}EgYP13S#QW?K`&PAKLNR1u!3(8k!VPdlub&jxH1>a?o0z&duHOZPNUe|Nn zW8KW%eTxwvMpK5Y)vYIXhW7F@>oPKWzcBc2JoU!#@>xH?Oc{`aa?x@Gi8H@mHMmyDOQ>vZ7h3iG zJMG_(aB|=Dgm$|v)${_%6kEMrJ5plXO5XpYx;jbE>u33@i9Q!+j3T%@bw5YOF?TBW zA<^FfAZZZ*FNg@ksJl}EG_JgcUGjYKqV{b!^+$hjk58v^OZ~vgV2H)lcJP~Hwi973 z`P=d3w3*xoQ{^gpSL9C;Z^yqWmu|9l6rsK zTAr|;!yhS+r$S9ol^+Ju@VK~YeJh@EMis!w0eiuyb@05D^*N1(*-+C=RQ`!#OmcYy+%dt?n1HapTs zy$s3wE#=oM6k8bpF~1yy>||6@Fhe>V+D$8Mj<0QMghBh%m8hv1;>-eQSPbb_3vjF( zu`cH$;5iU+9kJO8VbkZ4(v|4b50FFdX6kGX%=d5P>!m=GFPdWRz}NCH(y5ka6gQz%ta{Z|UXDtD zb8;R-;(DVeVE%|86WtjzXNt9sDC3z~MxN~>_$1<*+4+&*II+hkmz_dV>knMvQ|v@9 z%6c<~SMik={c`?PT#bJ)Vw?I@?1WUEQ@TLJ;QC-uq9 z-mO!Y++dJwGY+So$St8XgI5iPf+q74wlzRqcBC3z%j- zc&S;mC-vI_7efhCY9ydpS!)Es2i)Ddnr*+jBOIl3h;ABbwcIX3LM5&_@Rkv8Lx`JJ zM^|!V)&ij~3S^_yLBCY68`6B$qe^dZ@+%aa6dd8&kEV)>5^JQ0=Sq0yH&(M1OfX&P z=pF9&5HL;M{Rb~8xZmyX*~u|k@TSN8Xuml++6b-%p_r3LKB7lItsmf+v0#xcc8@A? ziqqnZ@Uccb`2y_jh&%GgQ}M`q^QQ^=vh^uPBK5uDf#sNqV@LLx_QagN*|c1q9nh47etOYb_tOcehM<=$#3@|1w_v~|I+Zjz)Fu~sCVSz6>X7ye>-(Mi8|aroP2la#*bwsmU+=v8f4W4`bGI9> zxCceOYbPUoMOCYL`3rp6#{p_#s!3?P;m)yX4|O|Sw(>Vq1nkOD5s%HXMw9^dBMVk>>K0k4q zpB(!;`29UK*Pr_Hg8sou4@l`dZLuNuKRNxz+aJ6OuLQ|kpVROD27mnX8Smm_%>0Xp z|7Lo;!}i7heIwnwMFGD(0OA`7XTpFq4XZG4RhUiTsYH(#yGz}6cIBocz31Hkq2w?$+P^yp`XrTymUKprj>phVmiBeZ@`@!SP-E0{jbGV9Hq?5 zfnSS#_`wFW{5MdIZq8C8J~>$QzcUWNd!aS*k?R9_BY4KjN4oBw_HV?+eGQgeD%u3c zbizGFa-;?csYMcHhe#PQ6|+O^^bjqRcD8zDJKS{_j|)tkN=n~K?4oFnyFZr{X7z(Z zDFOw*4%K_}OLGDm#4!7|6GckhL;hfm?s-+c0x~I!ROpX=Oc_G9ReG#>l5(4=lV-vo z;K<2Q=9zeLcc`DcOel4VSiXsK*~DSonbQNix87ZyZi#pR?z(n|TSu8Uw(?&mzzfSm z6TbmQQwpcDbp?>8aXL&S|EjXz5J&lPS~vzLK4|F0&y`$~Jni$(umAaY6gVwUh=KNA!V)(zFeEze&_MfSN|3GND>aafAC$2xgG#^%uHp${nL?kIj=l3ZZWkDQ+Y(OOO zP=k|OI1rPN1lmlrBWbWTnIvIY6jp1!(YoT?Y-mecC~as-^|Uwa0_Fwu2eb<2l?qIE zV08C();~&0ew=Sp6Y!XG-p@XHXWeICbG`c>fA)&U^nMP7fz-xl{O|_ufd~eRH+?j- z>#^RdF!djYgX?iUrV;7mtRFkr>f2{U#{ZuMDqyVaKWxskolxp)Ij3M(@7uvtp zL(N^DYJPd&+49#7y1RVj{QCP=7kKtpBLeelJsuPE&(JHJ&t5$9oe+@qeNv$RW3ghq z#WFF@Jt*kU&6rUtjoO6KS9svB4`ek-4N}7%n|*YCEtSQhB#f`|0soIiAyR6UCJ=tM zJ5*5qXJhm@?|bHJ|GF9ap10DM{)b}tpOZ2FqZOcU$RF$dwTpRsmj`Y9zjD0ay@C7{ zOZ^L{tw7%qYoHv2^I~w0;Pb@jn;r1?3}#A|3rAC40EUq%hsgLq)PA-UFf2Ak&oC8i zrI~TbSUEXn6NF&$R<2BI3H)FR!Q5qBrg+DccP{ArCX>K83>=Sq!e}f8PF-BPzigvY zv-4re5Mh$D%VEi4ViL29Vad(dc%I#CIW%m?c!Azqt92_un|ZwpjTY8SSvWS=GiE52 zgKmN=A1+obxU#10;>n#I1x>_>QYLFqR9R^@<>wzRF^gGwB2;~x?Aw@gHnA_HM_rm- z{8`%sGbAoIP&*riBNYak@?6a8IQWq-hyd+y6l0r@;tdNX-gO+~JugoloM>}56O;{) zFuFIU9CxwiPQyV?*|y|J7i!Xln`H^Qt-g|xds<)bqraM3l0ow9_%~DVa3jnhYd5Y> z1~iG2Ftb#Tzj36?#a;fPu$46g)w@oxa~e#!IA}S4$u@a)Xcg-vMwNNW=qHWEP$A#@ zm}&|ebEZsDCst(_kS{vL5S55Cu}X={Q)^xninidM^~pKx!-9-(YzCiOVm; zi!zlk!$vK(*toUz*?pkq;NxbHhnp9$;p)@lUi@G5>M9KLI|r?U(ASozB(FT zatLRQlO1CR`VW79ze%tO+BiDpnsp`bOHZQuNYho@J3qX@@X7O}&fppJ58lk&s8Ho; z=Euavjdt2Me4Wg;wfs@ljqJ%riZ)wSo z$Ffw3f6y>$eDl zC#AG$I>pSAu~c=I^iSC%p0EUrjh-y5CLPBk?ZjXTx>FwI*<~7asqFmQDqP`3wTf>m zC8mUk_;s1fcqo^a@ua9zW8A7$^K8oVVr4PBPcu76BO@2r2vTLfE}7k6ex_5TjS1AA zf@b!F2%?39Y|~z8&`J3;ahVPougLLFxQNUKNw&E)cdVey2GK1NOFoLfg~}eKhU3MR z;pEApHWWU3S38>E#8q}4(WO(Qi3~sSpl#*bH}a_$&uW2wHAHD9gmT*?n=RB7Cp_r6 z4%KRc1Z#}h`!$YP(p3WW)1n(r(^Sfi6KV5$3hyFm1zmVZ}_!( zO5q5~HHqZ}KIIQmF_H|FwQcN#!6q4sqPY}8Y?gOHJQ|YZXQSz}Hv=ByNj+Z2_$z7- z<65hWtR$1kMf<;Ty^YJMGUAzfl(L5{F)6@R%(=}*-nFN$Vw|PNBd+HzOa!u-Nvi(N zOlodLDDjkQlxnZmCQ@=&_)_K`hroalLHL>OV=vcC1HBJ2_Pu-p0=aUnXSXQqx_YPJ z0@E}yIlJnW*yeVpCav0C-Hy%$>twansyn2}W(pd!(z{!oK-KWNzU zR7DR#jnTRi|Js%(oSKKYUvk3WJ6ep6yVog3x}7FU{amu$Yv<|~1fR*&;=1?Rn~@$& zqbwEqjZVdC0E3)QTw^VLJ8T8O&0^5(#G>02_`}9_nY45|t1K*QQEGY`L34RsWwpwj zmbJT2Q}XiS6-iMxmo)3OXca__#X6<+`Hj+ga zmFCqJjRR?GnwoCS9hPh^lhLYnqJO)1xHAQEx1eyYz_2yTS!q3*3#_Of9p0_Rw?M%y z+@G{~*-5&11v!g!aq#Y$OUGaZ0ko4cH##CIdTNnUigLY^f5s(v00OM+GYOsKk$5-d z2SpFtIek|%Ww79)x=RyL*GqO~U{ujR_S@7(ZCWQa?!ObGHioh7GYg(Io=H?|cZKX5 zwr`EPJZfQ@)PXf($^p=5t?9mQH2C&K66nvS7o?^ojA-~fczg1;?vodj#ELv!N$#$i zIXs6a1MiOKvS|M1v)PRZs|xQ=;+e%KWY`o7a1>qmnYwb^I{Rd>3WP|_0@oH`S3z5Z z!0ZR|=*@GZO~uAqT%u3ik>0%B@f~g~K5b!uv~NX#V(upg|D4k?wW&qVo%IcguZ`*&_W{N*Fgj)cFE>-B~| zlrNZ#{FC2CFY-}t}k{M`fiW>Ni;oE3yA={ogY>}Me9rCpXD9)}b<;4YY7p^R~5%__U(`$+n z;;?b!$e&ki^7D@Y>`K@Os#*jgRIsq4iZt8UEMyH7hqV@u;J$)Kb|FH$cO-Ia<}miv z1993uPIHm!Lc2k!w-mGM*iW(rgopJOuwbBqdpl5JUu}R)IxR4Z`{aat`la!7PHGe= z@HZy&a1hU)+#vv39x9_SS*R=_b37?p7^MzQC5)j?0uDguCW`a*ZdS9Fy5xKDed4!qG^5AOTB+fNJY9rvm{E4-^WRI^l7e9HgV zCK>bT=kpEV8|{zhKThwtjdy|@QaIbNJUsltusZ{gvs=C|-Wz{OyS798ySVbgIxEb@ zxWdd_x}lu6bfU@S@UA-tPpfvc16%LF@-Gsz=PC8i>LH#~?i)Z|pa6MH2eo19j(hM^(lss?-I3ZK+4;0z@&-N7s8A32 zSsXCz;JnIb&Iv$lTL+3bv5SS&2asVDeIokjm+nQenls{5DHs|myMhdk&KY|KPa@Iw zW9frr$BD2j_D!tar^C#LQaW4C(s;M&{oE0QlcOoE6c^J*mmsZ*%|1L+@3B0!>4S3U z)$6r9uJZd6P+qO%aaE=EElsYnc=MD`oK4?6hRWeie&c+r#~*WH^QBsLq^;Gwh17?on}Br zJLcl5*Y5MM2QbdmVdu#)N9{zL=9;+%^gkjvlNq?72e_sqbj3mbfDT?_6g+c$mO)gN zK^C84e5FAasYB-T9~2!7XY|0? zB?!+;M!!q$4!0YZzmNWc*-ho&<1Ws+?F4e)F*y(&eD$gobssOx4f3v%472BXYQ@VN z;gEc;_g*WEg!*u;62-UW5rlYSb(;V6B$rnXbS43G|Q9IWg0$MuLgY;^LK&I8`4JJkdhu}xnZSPxn zCI7FvF7!-rMi=-u)BWNH>Hkl;&d$!$!h~MRz|}zTe}V@8GuEZ3?kjDIA$-HZFbPqj zAZaZmvrz(Kv-hSpOJXgs{7v5&UR!d^C(F6VIbfI=OQfdEhzB3{D(06doq7WNP{_F% z&)|?biax~eYV+#)yuS7!|M}eA{Q=Mz!RO)=)*6b8MJ!>K^v)WF#nrUOh4e1_+Lc#0 zl7;0zdqZqXK5c#2Qmo2j*;;0dVa0vDYYP43GEGoBK!=5AIl}mRyIzaCV}>DH;)aRj zajU>2lVYQ;mfH$ntcMN`=V3aLOPZ~_zd7N1C=)#D+8={AAPCg4ROtg@&3FvtBAY=@f^N zIzt#KJbU@%zsyFzKqU_yxe7f?yRJAE5bDw>ds|^QF00+bzkEwwRS)AG>k&F)V97ky zhwqAVhl_~EOcDpNM|YaNuf!5qeKt^zwSHpHP>p7_8a~rqLHZ9`kR10izotsAU{QmrNS2&4 zBPw_Py+CO4bUmWm0sO#7oP%$d!feK~WzC$oFAYmV7C1 z+Aazp@MaY|7~)_}&?W;3vQC3)K_lGti&h3w6D6uydJ0|uZ&+^`M;*pklivwS@BkwT zK|BM$Q4CiXL<9$`ko+Qc?`JZ$bME8vc4Gmzf{YHJ&>qKr{|nD0maf9+tR983nEm>wrfi&33jFlqNzVL%EbVrKk**=;+mD?M>TRmv zGGbhd&)szzQV5B|gvv>eVsCCE34`m`cU|H@*$G|33PFd(yZyVMx5%Wj0UKOLg!#1j zhev@IxDBRSqdeboU>Fmcb%}QCFK+0FpL%P3Wm-+aj6#{VgYRx6b38Ir;OLgRk9hgM z0_e;*+Eo{feto9kUdZo8H36Lw#Z0xeQ#>~uv$9O}%gSN_>6F3Il-9wN_7{x6zk;e> z)e^K~@pu+nW8?Pk-Rd=Eb8XwvL1KMn&Q)!m&YSF}>{E zcqGv_L=2@tfxs_y&8rv^Fs?Tc*N>f__*s(woPNx>m5<<&sX(*PiP;iin(T?V3Eg>Wi5h3 zMgS6N_X8L-Hc&$b5`sVkgGd4pQ^=4d8QU`Fzi^$_1v6B0Uk^U0;S`dAW(&HSd%QvZBWC{@ z7nKiZcfL2%{Srm@bD5_+P%V4I{`uY(1wFvm`7Dcq8tHT2_q#lh&Am^JDvkU(J7|wG z4R7mkAEn$G(U8O-3!y8CGL@bt#bu&aH0^$6V2FlZ-kXF`CK^EmcB|=AbkCf!0-#{ntqDYI7q^#py>b@+R;z)JNN2@NE z#_TQw8J9b5)8y=-6-$%Sl;aqyJYym;neCXX+^M@nFzF!%`l`-lf=aJ)o%nbRl%@lr zH6be3fuY(lOqq59Ft_M*jOmC8UpJo6;>d|^b~1PA7y{jr^5ubs?xItDQ1~aU%4I!& zl9ldZqJ{1v{?d8*g+K>H&ZK-q|-9NdG zO;Po6=xincF<^pq>L5DIM4#^S5IM}5lGpL@sMvAL&gchmesnLLC&rGnC?B1xQ=lUH zgLic9W701#%Gu5-N_w-o-O;Anx#JYMcVF~vsDwmtV}&$zL4^|KH1*Sk?FV2T?|3h| zlLMu)F9i8dhNd&(&hAh@{!Hq+6uDd%Ty%V=iA&w5_U-{u;D^Jh0&JIeNL}B=KK_vr z*-oCbWcDVSeQvVvv*bKeGGDQ@wzG#O9*-L3or%6S#^Sci2e{C$qAkZ9x?AV@#Rt0Y zcI~v+_A^b7aDKkB$=R+m4-a}jZ^5+Jjx$jYen0PdN__i?p{Z-|j`kT+{6u)}Y6^nN6kEEukj}I_y4xrhH!mSppvTW|uWuK?zIqH=xA!k^AhZfYJU(Bl(>=?) zf8>{tAzakGgnN1l^q)JAC^bmEI{44xYWw9(9$TRtm%_Ljl>Q{~pIP19Uf+dm^56pI z0S#P2T0aN>`?6u>D&+MAIOn_Xea<~gE$8=#l^-Iw1-%r;eJ;>nYNOv+Yw6&}cmnTQ z4HoXH;@N@r9*`3}Hj%XQYBtTbJWjOVFUL@D~{*#wx}zFD91{ zIwm_D-W>D`z2JtcG}Pyhp578ZRG~cdy({u! z5F|AHmVOcC0v&B^k?_$%nl&R!{_ioa6vOhkBIA5Dkk(c2s_;`nh!|%!_B1cVb0BSd+~~iPXf0KYB~Z5ut;8&L36Crhg?j;1lXj zOfc{TdhU+Y0~0+4_mu>r{sMrJfqM(er23+)>u!O18i)BLd;{1|;=`Q`8}sh&qOU%? zP#UGhRX^pt)x=UDR4SWmbpgQ)o@?@>a^ z#whBo0hiBpd#W%n4T6Xy=pj4XP@x!UUGS1ja;#0(k7BcCf#jskj8&4-!%(Cz1=^$z zOu1`oYc&;?R(CM(ip7%?=d*bXD)u$z8v)Xk(evplERWlOQ$s8=O;yw2Vyo+Kd2C@x zC&?2qk`G+Xmyik1BOig(rVg}wiuA(J4t$S^dAB^Iou@CswJo2=Rwp6ETi0oEq?}*^ z2cvRvXY%%3QBnXNchco89k05)!)A**WXp0{KZ<*?i3U9v0OV=udV`3Ed5S+Th{f}R z$Ol3-MA52hboZsnyM?LSFk8USxJ|vR6CZRv=LS#Yvglhx8>ey^6O=-ovYv#kV4sJw zHl&2drKkvz2!Q#OEVk8iMJj<Cwenu@@>~oB4C{*U_V{7o@`W{=2(fOTZ08fPH}? zb8`&r?Qu}uNaS(!z+%Kn(&Z05Q=hy3pmpkA zS;63Vvm$c&V{*fUKOW0nOggo~kTZa9fW;kPf5Rb>jtt&zkT<)D|6KDR&OXo*Fhf*+ z=fR~txHVD@uve8SXgq@J*VE}yYj13pEbw=8H$u<1s2Xp{h%Ebkc zZ?nrGU&Fi$NhwRo7>+UT4l5StL^p6LG&vfpS5Ab1{|rF%LKR#l@2JVQ+7b3%JAb2Z zz_)DsN*YmM-Cg~?S${v1jafv0q0wNVD)Pm(E>cpciO9POaPXDcaokoiylf;f3DT;AJZvR(sj8$5gs|nQHcz!{0GHmq&MjiY;J7cytpHE7BpfGg-Wcs)6Ed_?DOj91B1IyftXO(l>c zl}Z@1+fsXoSh1y9&dPt7oZ-K3iioXY_U2@kWm39EWsexz2qLGC^pLGP@+%I@F=n9n ztDpcbMDF>Deq5sT>#@2Q(e5I0n2&y-#io-68ibzDu!VZ{G`BOR zN@b)OCNBbR5ZE_$MtI@@e}ni?}+Z`K+yiLY74g-)l0-hT@8%-hRA zF5>I~YrcEg(cdaih z@if)>e%c-#CXb4OJ0DDc@p#AIrQ+HW_BpHv%25EJFQoQh9qeDk&fIYu?0dF1UT>Xb zE|vK_sOq^t?ll|bu8uOA-R zz-8!H%$E%gIq89=u$R})>I=~y4c-AP^dz*my2 zE0GT0I7}7c5801xxUwjx@H4#bH?k*hY80CAxAeERf@W!lzVOlnyT@!B%sak6LEPkO zy1gB7tjyFq82z0~=BpHPXy&R15pO+4sCQV$SM65?T^uHs#o!!C-E&|mW~T6@P;<`s zC#?2{GwtS_^K`^C*2{CiWA6-7U1vOnCY-32;z2~}QwR}NrWu?=x`~c*0m6?F(oQuCbJZ=S=@D-_GzJ@ae4{@?t7uI1Q7|?QtspAgijhKdSFCtz z=QD}sqyUgR@F+#nhQy-V$ zqVI$B$sU$%$|Z~6n}vtw?E)vV8HHOq1Uc=!o(XORl|V`XXbX|=EgGqhsdrK-xr zQ?<*hdOMlTX+Por+K%_9av4bgVgP4)!4Nd=mSd)WG=;y~f))~03O2~h&&S^G8rE=w zr^5&hTr4CfR8ULr#bB>--mQeKPLqr`>{X@^=0(J~)i*88H`npUIm^rfx-R-e5X5B8(-|kxfYiA zy^q4&7~h%zJS13j6Pug({5pCuv$1LK53=KgBtwCf6%8vIFP7)<7v4@h z%dKE#l$Yk{mg*fvqGG|?8$VSM`3=9$p&+yRg>_F@5v2Q(bT8SPd8?NWU7e?P6eC^^O#RALaWz%s~S6!BE zAh)5JwX#%+4Hnu-fLStEN@pL=H%crTCRpH$t+n3uaGaml zN;O{{tAjgq$aMjMzfp}2#RUnu=7%`7%J9lezdSuxxXq3P1QH{I zRnhn;;K7bO7T(ND8Z&DMX5-$0c5x=JPMq8i>G*a;os0_?OLzlUKR_G!h8Q3cVPbJx z=V__&V+pPA;K{esO(a`ntJ{5B^Jbh)!gR0Mi%gZdQxaz6Z)1`yhxYN}9zOC;FG99* z{cBCg)JRS5ly_IJe03+9mNLT;WTq}r5id|oJvmn0zhkc^P#GELQ42PZ^{wKF=D;+{ z6#Q^`vpSwO^2LrfJ%4-byMS5WG==u`Jy7O$PyCvpE6kmx$EId&&Aeh>{Kb@)&i&DA z-x&fE2%0Izs6t#Uv|yzmpJ6ts!&;0TXNS!^*U9)qD`BQ(uz8eoMxchD@eX8fF?c*l zo#xiQTxGSc-qhS(Xf`o=oQa94!M@zMScRm*&(Nx{qLiNF$Ry>kLfL@VriHcCR(?n= zN_bi#>1JEDv9+?cvbNaiC@9!AE{^z?nE8-^$DpNFv8L^p&0ak5+#+Jnz60WF3QJrj z=N#$A-=kL+dsIfQn+xd4wYLZ9SkIJdYJHJ~g-nbm1C#yvhWk)hT?(Gg)Xu~Pp{26f zzKrwaj2kiYtLPB!-t7Yu@^~L?M3nyKz~>v>LSQe*wj%M7Ki*W$ed*EDC}|G{w2O2Q z^_o}FcaPqJYL~@zF0afh1)Sp{Ww>dKGj=}r2j5<6X|7eaX|mOCjy$odXHe;RBz^}G(%bP|Vg zPifGO%b^JxENca7qC8TR{5c@eqb`X02L*CSmS|Qgc)o>r0;^reRn~qX z2&!j0*Ve{}5#bp~6NEhxT9b0FYmB9wgL*C;3}jjIPyWyO*KS7!Zp)rOqTjk*m| zM+L7z9n=TX=y1W@vIE=1+=QI*q86+ws#^9B22lo{WgbT}CO(AP|trODC0lh%ZxbC_lEAmj^B9#TBK5$ufd0E9us3*!y zK}(OlxSoAh+d9g)Tt_xtX%1F&SKDyoALhDdQ{V@9pu3QDK^r=;ir)dkKLs;o@e8Fs z7?$&|X*$UQ?wChN7%`chEFxk?sG_zwn`QQs{6%Z&A_GPYKjJx5Wd?Zu9&En-qBC)d z*ROBH!Oi88yo^SQsGCb-iAI~b&80*Wj0|MeJRDXKGrf{0NNl+G02=&C@?9y)$&s2K zJry3s!l%grwqx?2WT6&NHGH@*4J*?_6~R&>&kex{rh%VG{WXL7Wzm@xjkM@cn&u`; zOJuUi^h1WNrPrrZInnr%BzmJFG&|F)qL`8q*fatyT6I{%J*>;tNba~%px4V9810>t4@Dsh=KeS%Nm=K_aXuV>V@^W^JGJPZpoKqRh0mII^2 zHS7&OKyT%>>?1TX8~4ujLvWy}(J0ku@G?;0_b&|u?ja;FfY-tMBZ>7#=26tL8KJ0C zOAJIyw%N@{)Y4(f0XjE3q<8iG){_*)B%D zF+(13$!-qCD{MB`rtg(o;^{e4dWw030xQ*QMh#o5@KQE_LOk_bd>G!6+8b;3$*hfj zi85YFn(^}Ę-3`U8T#PHoKbSe&DAueEQ65xwm-~^T6@j{ZIiI3QrEu3s|Wz4Of z8m8gxi6943M9sRH(5|+bAWs!DtoyH`B8;86N%RbrD^3YlOvsnXiCI$#CM%qcg_*#$cAE)ZnCPy%R;SGS5>1J{YrzoL+ggY^R)9Z0c$hAh}FS& zzZM6OC4?{ZN&&^h!D3~DKChdxHZHb|n}L5OUR6ETESD7aHEnFdrcN7Yuk9qnV$^dP zMcGb`x7V@gqAw105AinndkcE#1_i{U4 zq0LBbk#j5wa{Y@Tx}6_^G>#{^+7(74p)Q9kBwihIbyiU7A|VE=y^;E0gf2LF6K*to zzr!YU*);&mqgXaO{lKyZ?!`Me2Sdmm2HB8!dJJ+0R@oJkMy4mDtPTvaFKYD|>=V61 z5{2@CD=mmQoGb!1p)DMap^4NM3ztBa2=j)s1EVYwt;RZN)eUOa@H)bb$r2q~QK~Q? zqxkf6>FPQ33CxB++6zFV#vV<uA<(5A4Yl)!-;>PyNR!LPi_wE)Ux z$P8c+1Y#}AAsrQ>Zu4DAR7oRbOvx28>u3chfVO5kgH;T-CTrK#70(4e#x9_XO$}C& zHRPX*@t#EuUv||39G(|OY@KEU(GYotv<)?{StyPx62dY0&yXE~5|Ef40T4&jp>N9i zTvR%H9~5mU$e)qqg9hVO>wY3xa06DiS3p!R0OZV*I@F(9UmzHts3Yrd1hl`e-Jm3= zs7|?#PHhOPzg+%+z9cqH6KSHo>q?YrD~%YSR^OTtE_aLq_KAH0HX3VqAHw&?o!dU* zuXb+zpb#X1qLEUSKfqdk@J8_zhO` zA{sgwzv$~hXal>*hpIJ~Tf%FTItmvRxVew4iUXRZubku3UKUL}2s91F%;8M^hC?U~ zuV8Y?pV;~R@QKe#Wb5Y7zzQq1n0jXDa&~ABd3pG~p%xJFdbr+Vdgp$1QadQwjaqg| z+t6`e&`Bh>MQ%HEz(fxN{VVZ;4}jQZ{h$kY^oH}EQ$;(B`;>6Kz=7kE+8{u$yiULd z7>Bhfwtce&5`wyxuaZD_-aismZvV_uOZ9@2deT9iYs+c_Fp*eDZRh&{hZ>5+@(;m# zLTO`us|K5zuDySZTyrYl<%evpXSJ5}?HjxT_#8Ul<|{TV0oW5QA;+;HTd5MQ@PRrv zZxycvqlvgkh(RUz{_gk%;|CNF&n9m`dXoCzTE|}wM0+&%2KR`Qdj#Ela!D=@vil6s zpNay0HSIHr5mf-Z+<)DKX83T5;)c&MV^BfyLxOy22uSf)ZLo{uT8j2P1u5(hKp5ZM zolK%_y(9nrK8$oUUqGt-3P&U^XkH`3P$K)ipx4|K@s{-gMv=T-6WOxyJr!skCBINx z`b~6KBgXfb87r3vvSlOq`N~V+W@Hoi*~=(FoPZ&`t~k3tU@64%FSPKDi^ugX?*L;O zeYbdVWHH<}ZzSW%<^pwg2&PX!J?eJe<-nx>TR3~1zi@MyIx_Im|06~!p05<;P%eDp zS<3tdg*`mZs86R;a*Kiw&~Ume{H^#6CC{}Px9o`=UC{Vbp61nb;~+J9@D~Z zkxJKF3q8*xO^ZSsz3uFv$ux*+Q}XO!#R1LA7TC%*42xoAOwRyS4H17ZkXvSjMLl*b zQ>e2E$=d*oV&({yJ5&SsVt2&T$|Lz$N15(;n|0li=TqYq;vi2r@Sye>dh1m(*ZdKs z-kWYld5NDtyX+(o^i>UpartS#H$UNmDW|{FuxD?l$T<&$M|!IB0r57>?vzKYDYB{N zCd-0o^_uV`vwII6a{(OW9tN73eU)a)aU#kIMNK2wDolk|EDXNqlPD_9HX?22;F%!( z6<{m9rqv#FJqn>O!2TcF;4^n6lApidk2u6?-kJA3*fpwOY1FXZIn+J7^#lEsZxSB5 z-6Hec1bT7)t+ZsFZpRDAoZCr<`@-p=cEig0@L6hxkd9oTfRq&*>pf zm+}^DfQ~qH%3>O*0|n}1Yum;49cminzw_&Xsk`4 z0?Rz)yfF9UNT(DHtNsgT(iOPQK;MdCt|+|xbfimb(iPj3Hskag(Ws7}9tMV!T6QCP zr>|-M^pM`jZGP#qIHwy>xxGNQvwX_iH@ED+1^2R_g6f+o2TPw94hs?^Tf#uG(3GE||s;Ih3U{Y+Ds<6uTw}hUM zYa;6RBxZ)?$|kYyfUL~DL*Mq zGVV~nngup#IJj-XQu(=m;1-^oewu6(j3m9gJR+~_hq-(b%!@HzTw=4sXWC(BP+n(E z#4tW=V!N^PPGRO%yif0lp?vs7x5H;4!Ya1#J`NxPdeQT4Mo$X`Ze{Jzz2HxAghJGB zy_(=~kn3%xV^HI0^~I=2>tapEoC<^2&5}3yk4GNO^11Tr<2lY{Pjd$Ag0tgto|+$L zvg@RcCT;1IZc$3>)Kn%4G)V>{3nTioWS)XNkRvX0jvq0og!ILIzvyme?5U@5aAZPs8br%k4BT%)Q&K@j%e9k z_#~M;sKg_{Ov1Xr@vh97c>Mb59hmfn)0A~)q^TVylP}3-6m!?wMI67nMvKYh@iIbS z^K`H|N0VM)?>`9iS}GeBqHSe(1^71h=wBRS^t&>lU3=u^I7bS6gez=--w-VCSeX?3 zQc1@xt=skh{24DYU z5%{(05zva2zZ;3J8m6={62GLKzXFU$u03XR9QYaTmd_{|5eU`uiJz&%FS6~(lN5)} zf>r1#g8WeaQ=7J*yPVUQa9h}R>};pz7T+T8qrAyWBF2lK4IgrH{D%Lpg}BMHz^=`| zWLNXw@;>(eH-%Rgt|r3P7ACgN!Ujg>CM3-NtItw-Qx2OEftSppl_m?xyd6bql5{R! zipItr_?eW7N@+u?`FNXgW7?WRwdy&ktHj!{`eEgxYRGgLba*P#tw!yvu%b+T>r! zNfU8m=D)0V?^KUfJaCN|1Vh?E!n6e6Tn=fcMyX8Xb?gbRU8c^uK4n$gZ*Jpw56MkG4@z5)L0 zM7YH>;}rb$UfUA$jYThd(`+e*maQE@-MpY8>A}acq-@*pKbIaBkv< z1kvI4>jR1DjQ8b0#dXL2`Xi6(U<^B;6W%xyf(t|4sM*IsceJmDAai7deA@Tv@DhZ9 zAK%3XhUrrD_h5#9U;b{kc`{>0@7e>&bu>sE`a}%l08b4hKcu)oOE-7-lE;K|MYSb$W{T)ag|AO*$T} zKq6bJHj-k84&_v+}D_US>wuNGP90`|V`EEGLvgQRcUo6ITmk1v7^bbjhEma*TP*^@bLmjYuEV%^HS>9i|2a6n zw@5`}=sN5#GN!_!HkvNB$d+tiwW1L8blMbMDwE~FS<~#8T(ov|yClS;Or5XFLd4Qn zQ%S;wbz)$frkw zx|fSH&FBf8RW_5C6_k$KVVy>wYtb*RQ-uBiRykQXTDwmEiw}AB{ft=QOoUbjqM=P> z;-bY%sQV~=a{_bNGqhvF=A-2=FqxRQ1h*O6L~CK{cBv*FXx-VB9(#P3={vztv<`7m zldwa_yF@6fUzJhZYC$OMJIaQu7KezY!Kk_U@ZkMt#}e#X{^#5%U~-$d+`@P9DhjD2 z)>VwsK~b3pPlH~Ulty9ar7;YWl3V__BsmtCszg2FQ_!m;yU%Dt5Ni_>DVAt{SF@V zr+B#6A9LjrD1gG=-D!BPrvDGOi1{FBFSF^_O4DS8K&E1HeRB?Vrn%|~(K&(~Z~m(k z9%y8u8M2+Bq+|r4dB^(DRkpo#E^Mj50jz}#=#SDJm7bsJcY4~Q))d21JpKLHN*?UW zd*M^`vEYD}_m*bp8tZIdQAZ2ykqL~Z%A4=&dVA-awO4Ar*awi&&KB|{;USM4?7vQW zqV416-FkDwz$I@!UK~2DUhfWp3@@B3hd*;14^RLb}DCAz#iyvuur8qy}7NG`qv7qF;r}4}}wyI>S)?nwQYSJM8LRn2_&J&!IA} z9!36>r1Df;^@^h8bczk2*Hl{brH0h46>G6ks@=c3rMI|9_eZn;{Wh&VnE- zpHOisJB&o_#EaKp59P7?7tiMK7eM`K6Mjj}_fOG>%a-Z#Okj7=V3C zh1BJ@lHTpx$UJF`5|Q-`6@}dCLoEk=qDZS-(bTFlohb*tCKjIGQ{<*8z? zp^$S*C(cdW$KNz2nV7ncl=e3wP{dc9W@G%O;VyoBvhn30g*Q#(Ea$w`>ke`X9K++y zO|mTB51oyYgm*9YSLR&?&(-W7E`B|Da~Kh69ZAon*krBJktM&V`HP2$GIk~`B>X0E z5P55Dw$aR8d^>M!*+(fz&eq!$#`?SH)&k6y`W%GRSoNFOn;s^&$?}rvli2>d%m4sl z?v0>h(#^e#B<}?9|$j3rbpH6K4mhd|s_^a#G0$ z@r4-yDzXhDo07AOnfP@tuAb-8w;+kdBkhdh8jbE9UDC_#=_I?xnKgV#N#+>)-vhRP z?a>4=W^dQ7w!uTg3yr}u^?n=(XL;T;ywv~p*}A;2hBi26DnNl0qsV-WFr@y_5|v~y z6UB$VhC7LMEqYFigEDtZ*n0q9Jgw8g; zLm`)E9Xb(CUz%sn(9-(L5!@q!U13>417+&zgA$xACO~)tuMGuFu}958WA>SquRw z_`0@YmP*z`AX8b8nt3&(Z4t?}gCdmv3Q}PMjd*Yqd(IEl7k2k=YdnZ!vfv)N@6-2+9FN=s`)~eLUdgb47j^5HKDB-scpg!sG0Pi z2$)>%@-(-^H1Chct~=X@+d-vn@LSStDdR(|t0ud4hSGxeZpHFf2h!sl(9z2=^*dis ze0NmPOB9`~=^}Wt;T{4sxZ3h|6QClWTi42`Sc99ZNL`AxU|pC+N$JQ1wIJ* zD09+*=o!n=f25PK1|C-H1ZpT8cpUy$7ceuRnB?WHDXT`JK+&(tWvCW=g2x!eG7-rh zg;xZq%u(Y;kapTYZM^EtgkvK1j6;8B&e>j<(`OvQw{~S+atKx=OEW*u5MGN=;PpxI zNT!6#M~BRZiDkIrnC5x@irfd&+C*G~=|X~EMT%;nEl!8sBT8jYgFGp2O8k5AiB&ch z?@SGhqkL-*l&5$jIWx=igq@(I!G4{RgOyvTAWYFxGIhe9WRYJ;A;q6o;x7>{zo4Py z4OJa&;L_#V9Y71c$7RiN%Rc=H%ukbYu;w%{d=_e)FCpMPIvO48svS5!{@YJ6(nn z+UbWu40#&PW*6p~KSR-BoHAp!FFJ}G(;H4pB;ZyO{$}cBN*#SN=J8E&;CsnzUaIYg z7&oA4#P}ycDp)E8Pa@<+yI_NdK0UZ0sN)WDE?j;o!yoD*9?uCvuNU$`tN=&4mS;;` zT^hE^r0D6fKK+u?rr&eD3w3Bii(gHnGu)Bjgj)E6HgXcUmW`d&^K$m}-{?Q+j4xSD z|IvH)+a>z{C!DhWXDX>t^^;!|!0@%jUP`$jBftU`&9SAHc&8*no{vZfHw+}apDn`C zmRu#a5xR-Fn>VdEHsX&+7$9A2vkO zOjb3zm}4w7P#Lw@T2LHLkANW0A2;H)L?_(5~;$MVBA3f?b5^ket~Z9j0AYwxdDm*VL}ewe(3$*9~yf$2aUMV`%yf!*&r!lu*oR4<<1cDdoc=rfrr zx5w;fNYUtr5zQu)N;6<+q>uvjBopdIlj;jGbyuV!H|38=AX`a$v2H--T7YJmVt86a z+4N4+rhBzqx%k^!KjZ=1hS_K(H+^WZSZ2SAGm4{`S#X$$G;908C#$bYNyF+<($tBb z{J^^YWOKi!c)5#@%~!fKcSGyZN)0?(g($3RYnje!5AFmuQ@H9M&>g_HWpY-|1N>Ny z(H;gm;=9{V3NtS#k&O=X{>H=%lhZx}1OLPSgGsh2`Bs|DEv^Imi((2$Tx?KT>st&f zxQ$z5t4dLT7nYX|uJT<*=PB-Mi;(WulXotgmF+B!3{J#?P2NEhrgY`mM2l^;ocst; z8K1P#1`#xH&6YZHexOqSW6B`?!?x+@fji{Pex{P+89#BVx&yqlNSNP6Dr<-TG42)H zdlDZ>!|7y8r--_M?1wi4d!)b1apFcOE9qY!DDU`uk0{T4k% z{RpfCepVRkBl1ev?8ex7Yhw*e+dBC~6-)gNKFqy!&@xu2pnC!+H)63N>M~+ptK>yH zs3%T%VYg>`L|rIC=s#!CsSw2h>@tbRJHZ`(CI4Tu`H}}Ui0!YBu?^|}(k=bJvbm_E zqn+dbBXCJkTld3ON99AN?`%c&GPO37=W1yhYeXg|v{*E?K=Ht-nn#m#r7=_dyP%P) zlBrmdfSXpLkgDVgm0WX`+%hL(kvX@3v?4SK{3*AO=zfU5H_EAQr0FUm`DNe7yvuvK zb(`yTlcM)N`$!Mi|L4sEn`H=mV?XGQ5FJJLZr_yyGH&q>G4q`Q7Qbj85{qw!;O`#t zwYhI%@BiTJD}yTQwj?PEx5C}s-Q67uxVXE!JB3}`-QC@t!rk57t#B=1=y}}}@pVk} zOuS#`od0L-l{<5-Oo2hd-ko`4{!#tRUbB(Ew1LnNIV5*?aPmygplN!>jH5d;#=dDn z@x1_J-{4-gR+Gn%6{^lJ*qOrP~P4B;YJOfEU9 zx^mha-X(KYqDQDQ)hW#UfSCbAsZzX`bH~HHkQa1glAo5(Ik5R_6zq>UtECCL_hGvdjWe}9jbc1S`3(fu8O4pg^1WQ`CG)Z ztk5JbJ`kgk%Z9TgZFRR*hjoJXF}V^ArakS(>M&mS4z(EUgi)FaudDGCm$|wsq6mlE zN2_a{yX0`pQ|gyaw*PvBb+v6Fjd`kPYi7<;1hfR~PpmYg9scP_E$r4Rv4h;CB}WuZ zmz*ik$>L-Qb3Wr#ppWP;e~57~Q26BR9(Av!p3UaLsLHa{L*gTi{TG7nEv*S2u_-GY zlUN}N5 z$B!AC%xNUqb^MmG!LJFxnNjDZBUeRL4qnOMLSh_$QGJ2O&f{sDj6Z5xccZS$ab#;Z z6moY#d3S)g2jC~i*)TQ(Gmb+dts-QBZN*LT)rqx;Koh0sj19!^ov2UQwC981MWrcY zTw1dX#V&jWW{I7yvm+^Y;iKznb*6~{s*Ow~CdsIDDGf&tkvPbSyU}OiWSh+vO#QTp zHn?V$hPn+emiq_I!7Zgq)F&23Lp@~7$+}9i5$J5v%{b&EG8)+v)ax83JYv32q8Nv{ z>G{q>dlh0BP+vvdOaV+y z%al(b(_)!-Unrp(;8LVZAUZChGkEN@wL*=mHgkXc6_&xuOevr?wf|A2w76VaBL>|; z8}nt#MD$iLZkf0wyM{WZOG^P==LO+(b)yw z9dMy`y&SkV!HQW;4*+u;>-jX0P`QGLlY{X`Ex=F6u2D zH%}XE`1ESafEK*=S`x71WofX|rr)*|NOYF}*h&Isjvyp3*y^RH=9sy44<^pJZUDj* zT~njfCt_Rk-eyIPdHQHJ)V*+AQi@%v9M3)~%HR3fxDK=X4+Im4Zw_kmqt=>t#X6cDc?Bq=KC0Ddpn4)L-k33~Uc%K`_P3beSOx zQn{ye8gNS^;a|pu4cdcv1@HOeY95HZLMVv!`xWuDrVF&ssrC$lRi}^P+mXHXq0{+{ zA3iPhrkr^0ScLXdmd}fqQjQPexw)2#!dNcIr!POP zC_8mph!umJ-#hWdS$+sDIETJ3QsMnnOKE9AJ0)}R>9LnxE*sC~##@}sZKu8HaU}|V zbHC(c3(ig#{wbF?tSE`j_Oqhk;nKbM&o4LbD#l1jvyD9;^tEi=Suj-{lt#`7=xciS zMj`vRF@$tGOzmH898jh-YQC)owX2LPP0Y`;5WR5tfltbTYF!f@5UPWEzm5twW=YxJ zMO^g)7u9kTSac?p<^@dRSdh&Q(92Hs968K9Ip^1@+kA;?b|*BaDxIXN^~TP*8vI#s z(hgpeI>T5pxEjO;YW2q5?4@@XvWSP?#jp021xlL3F`uMW?T80961`$RdAU!ZjfLu*l&N9^kV_uw1DF8b}wXgTlH_oOUfLmBr{Q30l zW+rz}nb!dQ=mW-!t~}ns0+C{(wqMc@r!%OMD-^QPhji~m0KYsQipl5$$Le$$l6;wA z%fV|0?^xM>CN_#!Vf4AC;@H zayLk^#{>pbUZBs_N~8w*^cmT=T$AD=X7~QB$fSAKbtf2p`o#Drbi37|5kCCV*J}Sw zeeM6aXpDckXrS`V7dlu)z7oVoMGozIV^~WM z27JkDh|9K!WRn4!w_W=MPSZXU57T7D1T7B3!#W#xvpqgPe(>@d zn_F8HY_f|~ZE_iwKGo+Ls-IXD@S~p!u8J^8LTOx?DUnL_o!xrDE$LK#aU)=POk83zG!rByb6YUz(Cid-ENsVy&22${{nns>? zBPL{CB=WsH2Wq8c^GUz!_ujhr=$)7Ecv!nIGI==1P$6|HhU!Vb*~IKJgx>=zYn8g@ zO#qW&bGnI4tnn@45j3F-|bi9w(q&TLb+VtAVmZNUePi`YwpXBrA#D}gEjI49z ziRO#U!EiDd#gjWxW8<5j1GQ@iC7TTGjoJ8PCE^TnsY~M}C=@?CfU6I7`g*5~fi%WA zXsr>ytnI~nRA}RCQc@TZeEAR_e=<1*tWllmk)v+q0IsJ|(cPqW*MjdUzH%J0v%59w zRBJU~&RZ4ptc)?rN$;Oje5T$Z{`u$BNuBe!f`Ndf!hnEC{u|1y#r#!f=G8E7)S!iSsa*8sr`mHeD;Jo zq!l#A4-nH@MNfCa<}Yg7?`~~w(ob*cG2L{r7bMuK+Xz;DfZI}vrBa~4+!SAT#Hmtc zihfpoXk#hEHlO8?<*CxLjMrl5o@|t0T4~J3&Jx{4Z=V;z{l^k~ZOJsysd7I_dtQBh z87zn%vCCpLnS&~q+)HEzFb*0Q*&$O3B z>$){esa1SwvNGc11Q49HS$AeJDNtoOo!ImtUGXf_cNbr!LXMIitZPDutvePPmEw9U z)K}$_W`mAEP9B+QKUX%@x+%ZExcclb{cY~)wUjWbT=Np1uD1}y9Bj(Ds>*v;fn(a6 zA<0)r%l&pvy>i=c>#iK=F-^&v&QdPKZb(+3YI($vDXR!Q;!$#iK4B5z_S3oc_e2kj zcpdHTcW@ea%3Tt0f_)V55_`?Q5*YseW8{+=Y8aTQoyq_wOvP(t-DE)%_J{57BVbcB zj6!uC3lPg-rEGY5Ra%E^^NaPqJ;muiVZtfvYEi9s;(ocNzFH$@v;Sg_2nWSk29I_D zYVNfDHs(s_-=^l3X0G91?HCZ>q_H??HnHMLwvnJh8h_*CsU;m!p8+oj551IwJgSdy zyN$Sm)v#n;aBs{Ws|XfbWL}B@O(I!jvVb!7>=5)KSK9R49{kQ|NmijWe@p5VEu?{n zc>sHmv!mkZ@pp%3VZoM1E;+s{yGTF2tO(0z%^`!o#@T7{DgLR-aMOnUz}6eVYIhBZ z0n#LE<>fv8jD8hg*)NUa`|c0%RN(ftV9?ga!lHTdj`&f`!ua2~HLNHfCAm4=V8#<{Ej~*9EZJQ1qnpI4HG8(1n|0O@)VCyEg2M=>30vX~{Hho?f-CZb+DoBwO5yDv0w4{jHl^GYSPA^A1W>cA~+kU0jB6A`?qf;N z-UhC2QI9$C4F$wy^U>f{vwM(n(Q_eimqRlZ8Cq~f(O8neGTXw#JL?vPeYq{QgEH5s z9m|N}xTcsaGA-)wm>cj_CL4;pQluWLKsjAIz|A1%JSk2yVu@M3`pCn`QCXG{!^O3a zybAxUa2n}cFH0KGR6%@xkW`bNd`Ht}X^{7d`71x`Al+WK`FPclH!i`Wuihi>(bFKA zpWUtr1N*Kx(u*E_r$c=}32U5Qb;h+WT!T9?k7G!o?(jAIyk{HEXcQ9eBbIgHd`Nr? zgFZ+$-qS0Nyl~0N(#=w~HO{0f#^CTN}UdHHbbj@N-Mds~H z!1OqjU4}#qk*=PzICZ;Vg%IFan_Z+~4cYj?z?p8;+Y~!y>($XV$!PRLlV9nK(&Cml z!nk~^x1i#1hR!G{oX>d<=3ageck%$$!lu4z*kaP}yVbkku1s`UEG`+`HU6RHka$17xqsCqL}Ln>r!Hy~c8W|Fh}#iY@kTgM)xPz=D7%{Tog9e-A|&7+RbB zNB5O8aW()r+d2N{z|?=WUmF058mccn^eQ5{hr~HfZ@zFTtysBgBi#_lEQlp&chg>d z8!2pt&W1I}Z1YmuN1$)n`#0a42o@D)sa#KivAC zGrx5!*vA}e;9uQEPgQUYv7s81#HRirQH$Z7fI_b=?NL%dETuy$(n39vp*tCGSG|>$ zVZmLeY^!eMlY7bxurMe2NNnUwOSq~POwiV0)=9&2$r9G)xK$F|K!VB`Y&}FZQ6=2V zaZhD}?Q|2%WpOyyydAn+m31U$%4kiJk}XaEBs7eUu7(*?C&|v26$*-M)J0_5da<;t z#DD9q6Fdp9$k?I)t;le6^TJLirzj2_H>NkxsceJ;oEd{JW$BnS;3)y8RNHqTHgsw)VXCL{u?8i>qD|iyE$c%@hh>! z*>bDVWn7ztGvwpMu-#z9u+Ga~UvS5By2B(htxN-NrO^>sB|0ZR%63pX8aCSuUz?+& z=74^zti6|qAp_dwRgJIep1ViMts**Gw1?S_8rur>bMWm%Rk5!=9E}zhH^=#6pj_h~ z>XLcWDVms5&N`a=s&#>mfnZw3NO6!FU+JowPMezF)NJ~$cIeqbH);o$Y=FL3;aYOg z$eYTZdbd3q=dd;1m3}N!0#?*0&VrN)Wu8Fp$oe(?Qtdh^yyixpcrXR{ung zx?nH9xnga{ul!V|c*qhBrWr9|v}0;ayyRrl2<6sGA!a)~o!x)#2UY5VTgEt7AG~8< zBdsmJa^M0bj@-JviT|Vc#YO79H&@fpP~`@>np1r|TH)3vKtNww(#^$98SQ>>Q771Cgh@yTTna+_x$Sd*>2uuy^G%6(iXg$Sbx0$h zbEpAdp=(!tyrvq&aBy9rT_Rks!1n%0e}UWWGaKh~H3hv_b%N&;BEd}2s?dB5=R6VI zej=oPs@V`gBf+L%$@Y(T$c~(?m-&NlHNcS?Y;VzXLaC{l^mM4Im6`_DyR!C`7^p7s z{zE?c22@yNqIIOHnaOEY zq9AIhVMcB7nK9dic${4fv|os@gD=#FYxRzZ7}R4P+;YGfOe+iTsN#%iBx!J6Fui9O z7>`+5I2LWlG7-|;PzTWg%CkXCY|*)pKZK#4ui66?-MU7rGbQ}ulW5>MH)ImjDc$EG zV4R3r65}{If)5=8lmxm3Nuq4BB2B9}P~oiDVh~k^*l&Zfi&Y59Sg?YP@dPQ*i1YCEolyIEVRPJ!4Wu3ox(nWp|4zY=j6V+7?Pk z8LZSdi9<+<7F4%G3CF_E=?N9Wo_fS+A^0N3&r&e*yaPWBsrOfaztn4P4;+(q?0eQ} z_M|Dpw&~M_@9XOs+b{l&@O-%9Xi87#`~06GzoWq*oJH*w<7BYuty=SUsYS|i>YQx& z;JT#WeU+%-<8Ep%*FK>71!)^yhIvIu zsW8Ar#Kv3)CoEls+?~98iKJAdH8;|Zv#YXU`$7Z8Kp>39T!K@a-RUVL=Lp!{wFvxU zq@S`!#@MbH?+26$Exz_Yc?^d~VE&>xn-`hXI<{Il)hL-tw42B!q?`qECfgtZkc<;2Ol(M1%3z;TEq z-R`|~ytHr8t@IW9)`z$mw<8V*(3@DZNO`NkZ`ue^@Npo$Q|Wr;YMNMXJ+5X>^%xlugI|VAG#cn( z!VJeL>cja$k8mO3Gd-B`5VnR3J)Gp`&A{xRa7i69Lo**KY(|)GpZQD!B8~j;jXCbe z%-P=YIjg3Ugsg`~fO3-vqwwUSV1el@6_o?^;Vt@N@ttHc$F}Az{56C9&syN4lV%pm z$MZbeI!Xje_5^zDWzZQQ?+S?EVh?beC`em!O>E9UP}L~&Oll`ugQbXnT8%KXQaBw{@@_G}2)8&dE=Vbd>9 zb<(eR>ts;MNlMjS7zI=}Y!|Sy$iE8(ao#yE-_*OT$g6{&Ay9otl0tz_*qUsO7Mwqd zcl$8L8u>=J#Tjq4A%g*CxH{ij!P*r?wE4gpL_SIi;qQGiPIx0u-2Pc-GNBX)(O)$N zME`#zL;u(C@>e#rh@GvIvw^L%)4ysgDb+0#Wf*OEd(O5Eig6f=GcXDs^ZSt3kh}w& zLnlSm6?&mg9gPkr&!{Qc%1)_;7x(B%9z%@A+z zN8~4HeDnD;nUTg}w>7cF_t|x4OWY3pV38%cKO*uRXY5MaYK~u;TMs$GyLg#Oa>nV3 zc<;>E^;%i?T3vqwp1c!hp`Ip6mZa@G620~yZW7I$Byv}&i*r{Tp~_5Zo@QF7gGL=h zuMA(_v~N~MW%#b!EImRuQeU$1bZ_35TzW1eolXck@mbQ(LGJb{BP5~HRRv`?371jM zwDafbQbWu=04&(<#I86u-+7LJv$wriMveR$_!>rg^oSc@ebXzZ(gJN!z&y)L8A4Ef^aC$fo zdX6FsfL0Z{1yjysgr>rN<^|Sn>@OzHb_hKoK-b?TsTZ1s@!ewM7$eTxpR)$_P@i0N zQ~?|tp`--UgZ47TFq`#WNvM!vSSpqswXIZrFiacucMYM9hny*~lDA53)c1E(I2D-+ za4rl;mV(ssJmbm$Yt%*k-6o|iW>~JKP@xSRw=psL$kS!?+1i;wfYBZ!b9-{saBq;< zdv3sm5(QhEIx}i(m?nz|isg7>36ZYbogoizrukN;=m45VLp{(@vap5%C74*VNOm-o zc@O92R)j~^`e?djifW6kRN;C^a4Fv;$NJ|3sNZ@%XUcAG)5@XGV!NnFY%~C?h$)JYF#4HM2-Y zeU^8rYNYv>|n4awyXWHX?ydg+HjeIAOO_;RB9 zJBdI0eUWACd))1?!zhgQQ<*REjF?Xttw@x0#DhT8FfJ?J!UyftNT*?XrT9-8AQov4 zoz#|SdZq1w32U25E{(>T2A7BMsKz_HBQ}rekkVbSHu7MO(R@ym(7`($D? zIV4Uo>QR~O6JDw(h&nsFp zU|QVwuWzl7be3~>CG|cnqPb{qo5knbgfnwl9mU|9$&cj-(WjZ~&W*MO7mX3FZCAB4 zVQO{f^RDYAxX>vtPPP)cH<`;{4it19%*T<_ZNoQ!I-icK#UUEqf?c-ZlDTFAJw5JH z=e-YUzB9L#`}WK51eZYFS2!|N62QH*XOf`Woz9BNv{e+xh3a})bjDO&^*kJ3>7?rN z^q1qy<&RkcrF)52K=_cka&Fi0qVAv9us~ED(E^eC6x8>ukGOolIrH$ji(@@Fr-@VW zKr!5N6wgal+_I}=GIS^gd+N4HjUUi%s(i4S$~Wl^PbLRm5?RFy4pvmuN>aHMA2lOS zQR)l?4%FR($l8*5uvC$++7ZcG31d1fR~>R?J4>?L zR_koZj@AeM%~z``B~ijGguUNEm;3+XapK`AuU~40Jk9%C9ho@`B4gP~4n!7}1Ke<` zr?H{Do~H`L-?L)KEL&ECmi{!KMK3hsq>kDX!jQtko9UG3Ub4WEhR*7TOhLmHTP!DJ z*YTkr1q-Ioqe?3_J0o5LqxxNt5=eBpq(|`_rbMaK-Li@qR00+q&_` z3-#;WIJ8m?>#^Kb?c>be`>T)g7mc%J1}D**iKgpQOstZHv&I#huBd+=>dH4R_0O9& zq*-I>BB&KW3X|2`XzntXv%!IE=h^DcY$K%Rbm(LPd4Ys4ga z$I@?U(@Ib-+!Ki^KFdV@s!w{q2v2_BLM84oaH+R#H9_Fm>=1wd^ijjuAdmO9;LJjB zZDX}ki`}@3UpHSYGKYP{5k=eg>>jE$#;G-K;GOH8BVcr)BVeBbPOb8tS7O9;U-mVF zC{>2jx?XxpPG;QOHqvO?kdgFR&PfSPXKvDOb6e@5N1C!DY~6X2ZQ1zdC$E33u_bmQ)nAMC?y+aK*n`N=PH zo`yuPU)oPwU-^{cDMvyN&d7a4*u1^C7k4!#i`zRX+LzZ9D0*{hk6bY|;&=M*M|}l8 zxf2)-yI*e%oN@#*96cjnqenA<>kZwgUb9EH1Tq}ELq5Yqy9Ky8bmDlE4SV&dIneqK zb9G~PrtDkwEIWGpU4Q5FC43gb)r0h999|B5a_A)Zix_POB9OR`)5|=Jvg3?83`b#h z9((AH%sW^I!{2iwWKVyaaD0HRe%$(%L3u-6+k0bKJ9>l7NH6NYZ6W``cx@1E*2Cf8 z&GNj9D**aeGMetXfU~*RB${T2nhSb5z$()h;lj}y%sX+PyS3K=<-%bDlwiLV{>uI$ zumhI>QXqXFWyd3$;CpxQ{&0ZZ;j_ceu+v|#YZNYD=)bJE{z8Wnx5%jf!2q;37a(yJ&{PSq!tkDDEH<00~p?gWlhx4!loS>{OG)%EQJ-SF29bFYi;!U2nsBzuysd zXl2o0Ob6qYZ(xpuNsEMA11eyS&GtXd4;ehHx^)lQwH+#)_b{TFKRE{AjRq?uw)MypuC6AHQEcd#M6~#_*2F#97 zzr)agf^%!6?Y)s2Z1quNKm&|t@ibNvtP06Gr?FpKd?r1s?B{T=uu?|}8&aDL*lZ!p zB_IhQVKjH(zsDb#Ym)E#nr${wHkenkiR~7sBAN7HfoXoD@ov@)n(?UV4%>Av`HD4n z@s+|MX#KZ-1s=AZMo6EjiDSB)iUbkIeeiay6Su{Yg-O#(pGfU`QJ~Sd`$l+P4N+Be z{p1lJo&6DR=pJ%Y=vl%*{)TK8T(>W>iUftDjEJmXs%ct`nAxE;Vb;;@IFoT6PH5OM z(0~*923~Jc3%$*#x=Qx zyCkw!D??;J#Zi4tPtP(mu>Jv=<&4*?vBP9Xt5a*Dq;cJ;$|74*z%tBf5v>l><#mFI zZi$s>-j=b)pbad%HABr}ZkLiy;=a5A$T3olp&`wJ=9~DM8<3ZH!Wr@A2Fumo-Nz+C5EKjjP32cp#4SP z-P}@3mFFdD1OE|LF(_uLV@&6`;SsR4s7NUX=7uyzo-L@eDYhNDB86*0tT&Bq>2h~ zf;Q2cKja6}F z)RwAd&dlU+5ac6L5JiHZ`D}ycVOyGc5{I>AC{p~k^SH+AgRk#%_S-S!Tc(%hlylx| z!Vg^1&#jcq>Rp6e@@PEL&)F1jaZR0%|Hk(4-zbf9e~AZDUu(2X|DVUy*Dci9#NGM7 zv;ix%FKuAch&Ulgh%yw7fUSv&R0yl7NHdRe5NZ~YRJx+mif^kXCRHu#jayrY`&b+A%6@KT*dD@b~Gx@fQm9v4= zqbROsGwmlJX^_iscEU`DxLk%yrkKh_X<{aeOm~Gxu@OL5ej@US-WUcMqS|^+Sw^M7 zyMTE2OWD`hyiR2q7hHELwmEoroVA~}5{sa51mPY;6i;XauPQG~HceRwI8Jx9plp(2 z$>Ne&uEd&T1NYn%U0E6fu4q9{rvl~H@5M+Ij~qCYs{M}}CedkYrN%bv_QSMQN=Ozb)gBjK#YOU#5n+PimRb`*L6`6u z8q>Lu(KK^*ILnsuf7E3bZ}+HmL?-n^mKz#;O@biSV6uz#igbUG2PPlPQ7hJv0l37= zlW|Lou_}Zxn!@ws>0rWHBM{AU=-4`$`hrE`3^vQZlIsJn(xl0SUJ?sa5*&3pveIY zH%XB-Xx@Q#=UZ2Ow$STKOVD1!ul=O^JkV(v@uFDUGO6y09!9@s=!Q-GZI6XKaYvOM zdS}G6u&k|Jgg7i5lL0?7(ay-S9Vw^VAf~q}v6Q(wP)TNLj_??!Wu-Zjwg2?LCKX4M zvpbGxyqofo3|1pe(1i6L$+bRi6rsy66!>-j|1-SxHM}mZIu~$E4uL@N9;3MzE;;un#UIL%$)wa` z)Xr8|u@N%m=WX;cnZ;}2DQ1EbPKG)-gP42|vxT8c>V7$}AA}K>vOXZ}Qhsjd-gz;N z>O%(hq@4MOx3sj9D3*&0%Y9%TZXZeW^lkcyBs=_NF#d##XBLIDgd^CMtwl%b-4=OD zz)H8(XRimYx^WT^{cbWQfOiLX{S3b@n!?NLaDPE?1^VsNI^*!bx)E-AD$_L$;O(Zg zOE+O}K;IG$DLN`9jM!&F9rV1yDZcvP+J?oKAj?09k0ylOY@Pf!ya?^4$uIatyVHIp zhl~7A2eH$aI%{BNVs2n-Y;EGmAnIac<01J49)(<-%@rK&TrG@E9RGDR532Smp$H*f zN`cZqAtNd(qM|X0njngap~3{BYf_2TwYodhV{wSsxb_>P4^qE_`Sl0l-y0`E3iNzl zlXR*-%CZv){*o(_M(9Yp%zo1Sy)`=V@%pw;0V3PyOfj(A4$eWFTE9c?|Jc!oVwo(+ z%hoJR7{jZe5gu_d~p?2iGiBCowjV#PNE(A(B;TYIaeYZ0u4O2&tg;3@G zq-l6tx5)Td%ET`C(V-6gsjvdAEv;#4`VVCTsn>Jj=|i}0RV4T z3q!bzC_yNa#X)~86xv|0WyJb1Lz?{~#Z{zEJ`hj5ua}O<@MyH`BLtCYmdIO6z|tSc z{9k}Y8NQWmjT4uGFik=jObQ{QL+#?E<2>enIGC5=DqJ@wo8zaNv@JqR?~B|+&Ubvy zOi;V2`xIu$sFs4q@%ZPF)jMRQr*5ZTGzm$k&r$uo=;Dx1d;(G2`AWcBpuJZgOiTGz^{xaK?G2S9sA8{36R zkxh`>*P^44(FXOnJF%fWZw|;UAz44#s^aaKa5|}v?{hLWeV}il#f>QwBtDq_v^4M6L}(t(8uul%`A|67f@kL) z8g`l0nSr=$eQUlK`eoOoHEwI^$*>j_F~G{|bcYJVT& zz;y-e=jb!ED<*?A62As#{d>ZFPVj%>lf)r4YPg=MJ>0BgyWee5qv5+%-AME3C-v!R>@+*-e zFJa%_P=F#$XN(g%Q2YWl6CrD27tK!eWgdA!=Y@C0KI(w{`_=ijD+heK%7c9vsj$OFK(fmj`vsVNB2c>)M_doq_CcK6D->W%!Ds~LO2^8wS z5uip8C|w)OPQ&06C|-vKNDLlHm}H{74};AbG4ogNpcsC9?UhnLCr19PkOZR@DA;MB z<`0>UcxRw|&R=;%_c!6?w9XPoZ&KEJ@U7-m~$Q;G=Q#S zqOe1a!K@}bdKe7lFj67Cz;@cO*rddH!2;yenXVqdPg8Qwzeet!dBJl49>!GcR;ZF_ zr&%@wsQqok0&~VRu9nnA_MjlAL12VCxzMU z&Yrju`lBz?oVFB3xCpdH?<2#HTgf@--g9?7lg+qXS6<3sp1RVA2&DLTXbc9S%r9?= zy`VF;X$hgj3BTNzty>{@(`}9E?UKP&#D<=rlU)g#f+Fr28{xZ#S`~iED~b1=O4uzdh6)xklp8LLm8p((J(nt3gR8cj8P>=DmT2wMH%#N zHVrdMq*%yy9}$=CR}<-*t4_+9)y#e-`!$~21}0iWhTHb^$X~sy>JbxOL|ENUup2|Sfhed_LN($#L69-K_*m> zt(mPeKnC8*j8!FREbCw7-|_bQTbo|YX@MyF zX+9*7>34DsXdQq5sPRT-JqK#DcdG51y_Uuy0cAs@z?JPNaLN7 z#%poZC8k0r@zX4wFgTU2vR3lMH*g?j=7=jqC!#*#0n{QR+xJ-^eYJ>fx~%q?R$N() zxiKA~yDfkEFZ>xzfBcYsa*ghR)uzd*{g_K zqb$QyhdB!Yt(#qzoFUA7wy^}xwUNd5w#MDL>X|;c$Fv3SV z;SFFOkV(PAr%M&X210vOQH(L&Fkf^vf9$hVJ85pXD#PNuNN*3N@cuz^P^nYlJenMI z4Q;?Pzj<2N@<4xEI-Yg(Z$Q65wYu=vLh7|?#n~gi{b`7?b>*}X?RBQ@{AgHH^~BZ4 zyRMJ;fRP|KU<1Fm+1euC8Br3PYyJm_c=KL}bcee{+Par+KY`U)THJyST`Q2+`crh2BN|S z`oc7q=HA!dcBQg*r3d=vHgJ^h3**wum=AM=^0lK7)}+HJdAd9pm&jm!m*|;Dw2kO`TSK zo4Fo4#c+EA7a4wDV0~FLIBDtJkTNTNn1q$PjCAy-CsKxzw6+JDjzbY}Jv#_JX2=#} zp9ej5VzA)uM5<|ceIKr2%jQh;0os`%m!ZWRe@weewaVYW)f7K4Wv?D&Rl*y!oWKV= z127A2ZJX&wGYy-*lI5?XHrx#Mh!Wqd+b)O`dE0i2EMTkUqZMt)wk{HTTH(^TM0r&* z4c7_lA)EV)Sk~7bp<=apk&5L1%SDHcbt?wa_ix<9?5E>bKE{Q2i;`|gc zemj{}8G>cYuJFx<0@ElOx`QZs*d600Bw~QB)rY8TOvz(1qRvp|7pZE|ZRU1@S%Q&ywqHvNksfOvoM2W8-66Yo?v$z>4lm;IWuY2Sj)MkIuq(D!Gib8LMmw)~o!*mP=)BI{Oefb8(Xb#*%R%Z zq-l?I1ifskDx||3?ubL!!*qIh=FFN|7G*nRGd`2qai^L4nL;RrY;4p!+g1}i^@yFLdC~|RA)_3x+B8gs-N$3nciHcnTN56>_*A|a7 z9$H$R9nl&>?abSeW60?&+OhF>IiNOrEvznbW+}_U!cTB$)KC#v8X;^ zf`F>o2%1BaD1a_H23tbBQW*~`g&3xR#Y6uh59UgXvI(g!fcEAfI0z^O)O~NF&OWr| z%csV!{W++6p;(s@;bsr$?V#T3r$e)G7aFwbtl2}bg)yhOM?oXl5own|BiJt-sIobM7q7c%O@+;+c5ZAc~2Od6s8N?k~(JBxM{F<%v<;3F7^xFhA zKzKekXs<@3bd+a_x%RcWGXgR8S0FS0LjdPV6`VpFEKwhqC>>izJpQCasV}$P0aiJ9 ziO5=iD}!Rpv2?+2Le7X8!vzurD1&cJ&d`=+F|WAq!mN&ozKP=Ae^m(tWS@6)j(cS~ zdkeUFcFB5narKk`P^Oz_&MAL8i65L9{)BXf&3?+O`La&{tD32Ce$@&Z|5K(BH|8$7 z<-tnvC-pb6<1`OHN+}_6kJR;3A?PKSD0VWU#dTQb^+`mySfnO2`S{48MY&ahqN*dp z;Q{CZjCbIdR)swg01a_9#Zs1!eL}8Pq8NqmIM~2tZJAPAHfh#C7bEveAcj992P(Cz zAnh`awYOt}dWA89jlb$5mjdGsQ2e_;-FzmeGvlWd!&oV=L^iu%l~Cm{xOFe$=Xe2B zt7suTN`gVmE3i^0hr5N-Qgn(a-3b9nyw-Yih~|FiQ*0hp5T)heo9h z+~phmPbgMk!4CyCb(C`G>HPExuF4FeIEz`&SDP8LT>JH>kW+&Clpc`=cjmJn(+bLU zf$qnj_|hj5LA=oT`ujV$mZm+f?21lzaMD-P=MJgzpqK-`aEF*B6*99_6qJ^9`8nbY zxJtuCOQg0t)V6WI>)Gj6_b05Ja2PyqCadHb0)tMw=EV?@NH>;8s60mu*@&`;ChVhu zEl#Wph~A;P;AUv|RH5)xC!Zmk0hZv$CsRiyh`CI0=d4y*r`%HvVapoU8n`tbs@OGP zK9z*`ODZDsS1G+=?y_kVHrPQju@ zOPA(3XWO=I+qP}nwr$(CZQHhO+r6u9b#(pR5f#<-x}MjZGjrrOawv(q>2$&jT*p#c zIxnxO1@i3w)QZ=gb8Eh>;TWwogoPSC@)w=UC#`Ltot5NVpJlJT;C#QK^iqD062$0C z+E)1fvqsT9h(^0sZ#B`Y=1ngT`anKLtJK0qG|H#qG#vdZ@}xox=)w%o^b-u>kqfQ_BDhm12QC+Fr0p zRd1G2eQ&De8b+}Uk45AzR!+PETzVeTO`Od4`_LWC=D3c}sw8CVqN?8s+?mX8BAN%- z05dYw<7;7a9*wiyT#a!fPy?W1KlazN3c$Ve&{Co{OJK1gs0hrvR*e}CCdGB5X;LPV z7-Ojbi!mIIyEWDbby7DYC+>gb*v{KFaOmggMv^O-j{ZjYeHO7${liQ7CCpC`3 z3&VO<6Aj_kR%J| zxo<~J1JC-cuWiQ3WsiyC_*9()5XbJJq0E)kk846E zab8A}cIm^IRDjI1;ssgb0>5J^S{3LeSTu~(o(~YlykL?(5eEp`|%z~I)LAgS~ ztqb4zB;yj<&IA3;fcACSy{2bN62R_=y@$gb1Lza|K@2Upi{^Jk{_6q$6cvmD`K>?S zVE&Kl&;R)V|3`7|Qd#>Q2BQ9K!J>i&=*r^l3l|~7kF_qY6N?(fF{AhW@0btN@g(|;>Y8o8ezeacIAx)7Mb$V@jlH{04KTE4trKCZtWzO#1z zU=Fb0HR=^Y!)eCe@`Ea&mJOML8lb|=?FxY*;4SLcW7Fu92THh-1x~c~XQWuvh19f- zXQ&XCesW;>2S);G@lyw8rN-$7&iY%x8w=3JrRdW6@=2>DUqhyrUz}45IDp)+X6tEO z=kAp+6ld${rl6X;Lr&bfqm{FF5Tp77I18JYN@65)Yfj2sOtb1t!b)$Xn@g8Q_x+>* z2W&_3&C3*7k(87IDP_zhp&bA-rJt7-XDMJ(S){hpv(pOZJecb#&LA>*6`mr6Z^vF8 zD$$6oYi?oGPU&yoE71#btb{5uG;7H+Zj6%yETXpM7uf5>lj$+!=(0ApQ9(Mb$5Vsz zh3Dx_Dm1ClS05@FNWTa>O*zviMDN)-P^U1E)F5N-MbTHcH+W^2oIo4~ADua9*H`QZ zFU^nT#tQST0IEW6EUYYIlVVlhN+8S0UgX%oE4IpSgDsEb%FN9Os%CRbQ z3j#=Vv?YCC0xf0enNko?T#$RlCj-|Xih8^eX}+#@N;un(uyUZ53;h}AFkKDQ&#rXT z;-9~Pjm|7V7c0fD2py6T#iAzQqrGn*BYNY!RE_?uZmP_tp``N|2_StG^hj`2Ct8IE zcl<+!ZF0=mg&Un1cmXf!v6ULvpjdM{`-t;{XEKf_wtXjN%?*=qev94WUJxqFJ$gEV z_(NM7JGAejP_BRljj8V+Fg|CHf(3<-;#v^$8y=4v$WrhWhX&c#X7?TuZe~oPH(+SEU>=^ zlNrf2DdDO#Y~h-vN^$_zDYlq!hrFG92W_)GxelZl3gvrhnq zGw}+TLqFK!yie$?3}#OBhNZsTNlp+DaGdPq>2zeUC%zg4Tx^~ zVjd}<(dlftJD5JNTrh)HE#hRW!dkDxS`8z2R(c=%HfC`FTSdZqlxu@R z_c+rAgDX&3zufTrNRKxfu)MShZ*N)Rm?dVTKHKggqmq^}oA=Zoo~ZnQJ{$Ry7vjJYZJZP42A_qsy0p6xdYN5xn?@W9{T zgcUL$+xkgMU;2a%?H{@Pb*aZrI%pR&(qFu#th~o%;{fuf+n1*tB zalC^0$^Y>*4`2Gq;87-zrYk@eOAGKLjQuZWVY4u#bAEvOnvU;?U1+zHtN-xVl*(Be za_-G_4qR){aPy6<4~mjP1n4(5lu8R#Zm>ipJbcR>iYvTcCT>DDe7wtB$6OF8t~(tj z@!FO?3WNfSG_tat5hYX@wr}s0spa2!uElJgPT%&u>(v@iI?7ybxJbYb6xOe;n zDuubw#?WOPp;PGak#b2Z9HIQ2PISVG5Ytb#`#s8N0;%q-5i-byQ&3`B@t7&Qy(^=2 zy5F3-J~{g!OfsRwm*_d^cBFxP(5&22%K$!m>G+EdJ0O_@w0W;gMcf1{v1>B~M7lf@b6hQx5eWYr zaW^AJ|2gmYBYytvE7yTlm5I0Ji{TwG+t(CRP1o=38CabNlC1&ywjrrM724YQudmD) zQOf4jZ<%QGOTzK~|5rK0tbfNS|EUuH^T~^CQZS z6EXi~K{+36)5S=6wxX)R0@9eTT**4l!U3d#@|$<6*&q>gEY%1yzX-*z=5IogYYcv5^BSwl?_6%B+iT$ z3r+U-Zkosnuab#a8n=2QyPS=W{&g4Z1$~F14eg0|=LIu}iq3Yv4d$1L{J939xhD_v zfds5-dq@V7JeaSZ-IZ@3>+bfXie-C>z?}M`UGc=At1)4<+u&Hp9V2Pqhv1Vp<&Y2h zKlT3#!*`q&uNAwA$n2-ux>mww26 z-~fY%+p3@FEFOybQ znj<)py5#HB$+*>7WSQ=mOhI+P=eLioC(23BTB!hFBs~K{h^_aSmoDB$@CUXWjw1HN zdBg1$H?byRgDqytkjqSqF5ko%@xkGAoG(E7i3`P!f~@HzP2QQX?UL`CZJYP_w>FEc zl^F+1Gir>r0~c$WH42=t5rgB?3oPWVC#K;{B`zs z^rgIN`1{SV7slP){kv)E9|W0ZNl9Jkl;X~Dl&?V~YIg*q*2pb3SBdfUgYDUel!n+0 zmVy7C2Alh%=60+;D5a;n$!xc*mHg_WMI{eRa$Y5p7fz^~_E^#3hJ zr_{ha6^GD%Wa6hbEU`iNLiutdK&aV5@k0sWAjM*JMH#_nclmUjb>c%hK9;#;FUy2H zHkPxMJ=Ox0t&l7nL@8DQG(1|Go1SZ{x-=|q0`$*(j<2kpS!i1GMruCm86J+k9=vuw zdn}IcPI+)VL3_#ThC-@Z8{mJZsa~i3AchZoIMM^&INX6S6C)jjV_Wu)F_|BsFgQb9 z_jlY-AdH(jVKIAuak!)9pbUP^O0B_QILCUM5g=Zyp)#hghCtZ;csS*Ds?hy1sh|g+km~m4CNve@EV&^j!u8ScIwk0S!v0o@g4@5NGmqRLtw-xalD}-+=6n9XZ z#40`t^gqeLHWI)w68<-${v?c^Du~P}?-lQqtqS8OA0}9-YyCn5* zWtrbTD!Qqs40nGWQ#E8&q>m>Hc*`U-%nS znyt+R2J<|$Z4qJ_rQ!MgWf4xKUlauvShgj%9!(-CvPwH7sO@A0+uXADsAXU`gs3DR zCX6O)X0>SbgboW(;b`!pV-Q_fMzV>zLjUHU$>Va6qMY@K2c{yVbj%c-kH#1{HYPv^ z&#?1rqOMq)t^s2ao-E9h-JT5V?hGN1{<4oP2;2q}VVHq_h;{}JSsx?$|z zQUm_S@$;pdsx1#h^h9)^?5J8?ur-WbKxZi!f|1o`TrBQK)Yq8ln0-cJ8-JYNXA4*u zuiRQUAujrZ42$fO<#<24T?$v{pMrjONOY_{JeJ9C4?pKACSmQ;0VkaXmZn4!c4}34 z7o4U<1ok#Rg?xBPp5c)Ga1@88Ml~rj;$e`M=xOYSPo>}&k6ll!gM>of;)MP^oL;(j zgC1QU2=@xUq1%tXE7l$L!3WuYD0*i-eX&WJcPbt`fc$ zd7@V*TNDRink#LedQBFa!F^Kt5pv9{WzBxUVbS4Z`}9yTX+SR|3A0WC9A7%y=^x8f zDaF7pncv3G3+r_6CCj%?VjedK5=?pqWm>BFCbTa_(K~zl+`wY~0c!SA9hAyxC~omV z4r?{tY#=NVJ3p-xDr(|Qemg0WQTpv^zNRUAJgVUrO+RMFCE!3?EomcSLwv^Ql4fvy zpMZU)36PenP!O?adnqD&mMQz`gY8N65TzfK(5N*_NTed`8F@AljYKQGB~*3dg_Tg- zeEtlb+J*jaOk1v=x)>h^d|kn~`&xNH7+ZTPMR{TRgCm^|B`m8+G4#x10*=hQBAFH? z;!-=S(&DLp9IF&c#1Om|C6dy=rJ(ChI-?)~`;;&%xwF9U!mF8NS08q_kA|dVicbOO z%^YrwcM(j4l@9`C|56~4&j90l^@Ygyf^++WmJni;kC#&Rs}>NX#=HrqJI=k7DSE$k zimyYbd?UMu^h6;O@Mg1yj zYS;_LxxxtJIeazk(ugPh0BY*^aiDNk+X{m)YV{%Ev05pVt9${J48n^DqXxC?l5jAs zmdCwve~3^JcNDlhOViOcVl{yMB zq>cdCpAgGH|EyKuS1t;&+afsXI@o0w>9o*^JIdQyDdSp(-#}-j@s*Di3CGzzx5ETw zG=5Ctq8J0vTvoF+Hi3y#7*SXk9o6hH!df=-awNgj53Wa`ik#G)E?3XAX#~WyerKf3Qwtv-rmmM*4pCw>R|8S_~6j! zV0UqDgjoEI1?4ABY`q5@86-Qj7fHfn(7RVhX|86;@i0+aEDz3d@ZJt`upS;~Q<)Zr zcQ&%gdiaHXsN(;!xiO3{DxXwqB#kh6ZBR*&dPx9Kq|T(U&pJ8VkeFU|P+MJHk@#|L zb-Z>~6~v5q5+E}a;9-l0*K58b^3p{BWRbXENv)^R0HRC*x}p%7h6BCz*Q=(?g8)&3 zN_w$44R5J}p%^FuXyOv`aQkc4JMy`6Gj$BCV4k6UVZdBL7Q@0st@Ox9e7)PVhGe~PJ6&g$s zbdIGYI|LTmM=cEdV{|PRri(a4oTsY^(f>@ql95+qN;7bWwHA)Bff;RW+Q2{1E4bwl zNsAetI|(l>G0+MvgBwlE235uWkmDT!Wn2|`nj6xrdYF&kLfgvvPpl?aj3J#GRd4{F z3h;b@Ymc@RC{k%>XbPC%4Rx(~VL2}DxZCCSf4Xu1AXlMIKw$KXqU6MiNV*P>s3 zeZVH&-4ruwKD2!tW%-L5!`>=94C|y-Axh+*W{}`5An^NoA z)0=FEl7QBK<{bcZ1*nlIU_PNej?O#p4oE$DH|_?^RV-@@?dMD49JItkQWp$LmP<4K zhr3kmlsH=6MOD=tA}($!-f9jIOe^V45Y50DVGNZVyv0_kRb>l?5CxjQfYwntyGr0T z++0HE3|QC5)qhASr>(Bhq>BjAagN=08BmJdic=DucMMCX;fPuyW!)`qMGXc;M&2Ew zv6f*_QDjClD;s42TO!%UM>9xAxfnY-H%P$)TjZoVp?$6)M*(ec$C`}TKFA%vlh{xf zPKJ?iDGx15G!f1ZNNpz<{;kyi$(RMO4_%S;<$%O#ZFaKc+yJb_I)^Yp0_(!`q2Oy5?@t z6rvNMl!i~MJ%{&q-TUQ3_}2)IO=+Gog~j+e#_(J#PDIC9^-@dUhArw+2qx6D73x5_ z07$DD6wIDprobOU*{;-?XQ<4sqyiQ_wrzj+&ZNE{=gb(FT%Z<)k4?XngRa6>1eXRL zUk}<~k1>{=tSN+FrsN)BzHOeHfT8xia+L@VJpsHEKY?GSaov86b{mQ z)y16JF|}}QQ7_WR&F0hZ!5GCHSWV-RleLX~bkZTBP=>==VP2Ab+;p;KC_+2GL z*>WdU(}T9u+yvZ@r~~Stk=bj?m=4{vLa%%Pel^B(;37SrMT6ZG+-|t>V%1xaLkw>!j)+>7637Te1gE zv6XkFXHf;F8$O?N>vD}d!4s=*SRRt~=4ru8^DjM z^b(Hqfd#Ab6amliVgl@Ouc!q_8L1ICv?SbN%AcW0KO>auGIhw`iA)zPfEQ-yS7^dB zH>eRkCW7C7cCRG*w<_uHH?Qo``*xoJn9*bfIx?y)v^CveHpDl|=~q;YJ;SS-C6hYS zCq`#2HAZmu+t+*=?^IH`n19V5l{=-{1``r#wE81#5D3rU9ngMmWThJu)%Kj{-RiCP zXLnMIdI%(ZK+cLX77IREmF5*?yA^p-Y;tdo21ZtiUlj!fpUk)H}KUvcUU@BDA7s>(yx^`bvk*Y`g-gqeGPbg^JdBUi)---zA2)9PWcH>kDY@Xazs^O)LfeLH-oaFSmIze zro|Y}jFtxSmWof0D!_N9}+?5R=$v7yro6%uSIn%>7*y1JTsPn zv<%M+Lp-Ra&{AiCU`m_l$WKfOqC|^~>kGfKL}Scu!P@BPjSV#^%#e7%Z@I_MI#DBw zFq2@9IFo~SuD-ONNuIXdXBQYKodgf|Lfe2-8K;2g&pLnrnjfT}-w(RjY+7p1x4gY) zb~Dd%3^$s2;W)9kP-}%8E<6BrY3!9C;v-JU2J6!%(zA;}F{JM5kEk9^ri$z49Ph&2 zLF;SizuE}mkmOc6gcSK*c8)*iTxkV=aX>ysEN@lU$dj$MU%{!hD@`qBInI=nJ|i@` ze*|x|Cb!^^CyA80hpi@P!Ub-VjzZCS+E9O~uHUa55q+{CPR0r_LOW>EKjHTh-bN&J)O8nvO)$z#ll@BTD{sqv;SK$&119|O_Vs89l) zesW>ks1}3WG@~toHR|Eo0KA-)dbQ$~sd^GJoSH+%>7P2u5mG%}B_eLbLcu7)=kW^v zc7SlbPaiYq;ACQqQ&b=~htR|2@T$etv9;IZb>P zq@-Xtm>qo#{oRL4gyaVR!!fwkC``W0#@04Ak1P9j#-ps()toFVt}XM2%Jgt3-!}C2 zplN@zsJs!iYZW19-9qERFc+Nj#=m%+69>vGb-DF|^t7%kSHN4PuaKfJr#czf{j~Lc zMY5BAUhHM$uaB0@3>_wjz%M`s$DdU%X6;J^qVH7q?!@T%42!4E*i6tFT^ur-K!`7n z`m*#D#5iWn82=fh`5mtO8La&oJ@6W^`5nf$eg^4#M=q_@?+R5iMr8{ufb?QA)-^8} z{{RC!$L3bXKI`RM!uWj!fbMt3ysMT8p}Pv%S31dp6q%p%^(<9tqGs|4-_e`>rdqBs2!ZM4r(Gv{-Zj7~6m*VdxIOSp5 z9i0M#o!5drzT3RsMnyuw{luZ8$6nogvpt}SRExaar;Du~g2sWk%iT}mbW6Z~DdM<8 zla7DL_=WJNB4GM_Zj7HcEtq>b5Rt()SctA)eYc)@Q(x=V|7rxL5`(7!JBM?D=JS}@ zEdzDfE&m4Dde_N3mR%Fq3ITo6bVKL0GaC`*OT5!B!JFz-f$xV;n)d_bzp7zfx_*gq zzZ*wEe@C?cOTOa&QNyI1EFI1MXDN*EcZagFo~4t~f3))=1u2DJOORKB)F?;_ zkdWBF`fyjd!FpLza#D~|*IP!*;{M(GjP=R<%ugzBMYmi0? zJ(G5wm(FW%hL3}jmYYAuwj3dd#0{eQ;xsD@weziU!&1NpZL3D8nrr(UoA#|54azLd zfmPS^hAW0sN94CaMO7(aVpp~#pk3f@Tn_U|=Gl7rvu>RTzdU+8gm%*OOO2=sCs;E} zV9zuQW*)|^<+WlfO%qOlSQ8lrWLo`dm0-}6(Qjg_{{RLQU)%>pwoZd<)<#DEdLl>T zs;1pYh1y2vb%ePt#D(21X&ZW}doD7UPe#QZG0s%1O-+Y}&9XK-_7mor5dM?}8bn;x z$xRuU(y~7cO86F`3{rUBEZTU4$a1VQ$5Vew_-CA|<(V2xSzf8TsI{vf{;{=Q@Mb|u zb1B7HTNI=5=nGH{wfLT**oY^`N&QBlq-cs>sYLAFn&zAI31|}as%OfIovfzzQ)hdz zk2r$)lT(L<_U?wh+qopqGnu5ATmfzMUap;>uD z=lXmqbwTb^##fFrt?-X2_@jDkZ*F$XCFR#B*x~36*KUTMHjZ@cmdq%{I@J1ygifz` z!iQ*L@w46~oT0AoGG9KhM--t)0HsF+!X=h8JU22{!7CYVV}AYmJP^WYi&0c#y?qsB zLVbR-G=ej*$w4u8Y>9(>(Tu34Wz&2C-!nI*jWC$qnL7HBbq1aH+meD!PsRS3f3Izk40oM@I-{n2G;J$A z@wXOeCVDx6JEE^t0X+JHAZW^#%<%$xOnJh=O}Dk^Op$`AH-Q^xc`!u@aE1J4?+*Tmo;$S?AVFmsDxg%T*bxm zkVeqZe)z%r*j}X7>1Tz*vxnO@YDapTq3NRLbG9a&am*~JV{l8B)0M7O|*4FO-p;&(^P;+O{aESmPB}YWpU#m4fQe5Wm}>0hT<$2 zc2yKy^tnVHOMa=t1vY|VkhzW8T>k~ z=uVqyg>0AOl-Ne9^fGJ4NG(W`gi143kx)`0w9+7!dHMl<_+!D#;{q#s;B}##b7(1& zqkL6XKIdQ|U?*E)2r}oxU5pFNEr9c4>k|)L_`f{~R8#;Kpk-82wnY7LkFg&sgT#kbP z;DK{bb>n5h>4BQ?!1TraW921GVzoK0yC`$>)Rx&&AlmxcdHEWQz3FSq2f}S5>IP1I zV(^9$MS>2}V*KU(^Y4H_p7~mShloHF!q2d3Pm^dceihV;EvF(ybG#YmeYP^uGROsSnj216SMqz@rlKi5f-)WQTExCB z_e78~i+5zvqI}@unKAGxp~ebCW%)DBAJ*tY2ijF_MRty?G`(N8Bj19@>huO za$St+lb9Z|(?Mb&jn@K<3{pJAPb?Zb%E@d214$gy!x4j^F1ZrVqDr}2n9l_A{wySS&;1tKkQw|=T)Kw#cWA7gv z&W>b~#zcj+JtbZq8a5xoL1eB{`VEN88A;@p&@0NpDTvC`O8(W)W9wED#7kVAzDUd| zN}`qWdPe6v(eDORt(gM&{nciBa>zqL3@t}nI>q{soUt>o`PbM!aW&H zwK+bww2Q64dDH=$G9v}ir@{AKq;FnVon&Xh6J5ZjwK?u-ZRhsrEUurJ%t`H&#m%gy zgjICjZ6hdQE4Avd`%!k#Ou{Qc^2up1SPsH}_Sd%Y-H|=a|Ql8ZsH8PzeY+ zeVQwd*9fzZ$4Z46imQhoC74oxVGbJ55$Qz*x7+tu0`J@VTh}R79K5OMLPo$-Ri&q? zkk|m84o~Rijx`tA?6NlZ5_(XwrS;{{*#|<+$jK50Jr|^2bgt<1jT&xsp;UEqm~`gG za!kCe8cWRhan5SjUz`VW8K}d}?(n!DV3kT*;BRWye9crT;`4~%)ibUAgAGzs7iu%O zEbE7BY_mrOJyMgDjKrvn2AwUEX(hyUOphIMD`2M|I8DFB7}KDRK3A34`MNI~LgakJ z3tu!(v^(G=Vg}6~5?REx<-W=KV@mr+u>Ugm$jNr9RR(V6TGWx+AQrZ8lHgsvvE`az zC@4*L8@wk6FDa%thmDCi-HliITE-!l%8C1n*HrU?sd7l|G-wDO(~ij+32qhLwW?uR zOO0AJdtA9m#%HVZ#M8da`W0n@D4!)2Ce8vSf;f)3s^*xWOp;i6z^ezQ>vW{=lgx|$ zbxJsWtp=;|1B0U6AV_|GKw<{Fetxh1nA_G8)7Jwpgnv{QPZLw3L-d_WR}B7g+=1zt@sm(>0X zsXe6&(h4L5Ys0mm7%2obaI=_a&yqW+DN&uocQjohExhl=L;Sd9J+w>Ygf6PkLu?SH z>lmYPmSE_R3ex8y>7WKjmfq#(PfkjS(XWP;r3)X|sPz|Al+va1iPGQ3D&_P)$KvDB ziNRwHM@aufJ_bF$K^^eF4=PYaq%0gEMK&RQHgKbsgCZdxC)#JOysP)4(4qG;|AuFt zwATH$qqK^DHsDl!MO5~%qawUVAdwx?@t4?xJ!$iQu>GtTP@S$tv3~=Pj=|!iQbREZ zH}ut~@TCxua{+yXTOkBF63C6I*04nnK~DwM0ZT!m91B(e1AZ-^3vuut;v4dR#>v8U zykqT`l9x^Am6Amt67v|NqG04310N)n-i~Qwwuh{sQe0lI6e_tK$b7$N+AI@bZTfSv z$a1@mF=F;~^w->`L}T@f^ore|!6lJ}DKhV^4uG@3bE*4{H6R^gX1t;Y#nYh@ z7Bbotv3fEpkKS4YBx!HUdN{ zm{;M&?hGNM)UVm^8r%~h@c4T}Pzqah$)+hS3%AxjG0%sb6x*Y6Oa2r@uOuF;RRk!d zOtw^C^)z0&Od(bwG(_mw0a-UIe7=dNu0&Hp5!6SH!MsOLHY2)BD$` z!#5GFWT1L+Jskt)q=YP|tVO3)z!DnmO~7p(VXuu)WlbmJu89Dd(<{a7Kq2RVB=jk4 z?X>eFgD}GYEf8KGj2PAzNYR~Ez5~8xKKJnV9K}Sg$2DRj*#xT@&{8Z-O^@cw8tJj+ z3+KOf($dPM7oopgN$W3H!uEeAF8&YpOx?{>amDF7nsGkCI9m-6RM3*Zk~bib(Q=qs z7n2gtgBR4_5EC-&3zcgybgpx6D#0p~**a^hs^R8iKDilBI*bZ0ZJ2m!e^9KUMXa63 zEVEc_VR0*CndxoXqhK!5I97Pk2l4B;yCc)#dhMF+WE)3@=jRpS53_hP_Q#znDhO?P z7oF9#8AJ7yAZpBkXKEME@?J^Piy)xPK0g*W!f?)RgPtdJ4$Ey2ZFWCO(_6e>$$)8Z z3ep}$`)3LJ*Ai`aaLvUn2An6tZ>aIcD}&an!`40roF~f8LQoIr0nMEX@VENhkFZI1 zO)AgqaR~T(3$VB0E*bz2G(q?r`)8K75B%`XLCN}u8Y`+E3?1P zaL?(k4%BCc_s>?Z59y%K`EF0*k&f_eMp7TFPXiU`BA{;e|9fo0JQ7 z>mCat>DB7q$dZNTjkNlDH4ci>>ou2(+`JJx5QDYFx+#G=Y!lmA=N7nmYW~ipkUDzh z{Z=fF?9LFW{`<;A22_k@`f4uP_*#P&C-v>W9DA(*AeI)r&MHZ^y#0HFp%?3BHZfk} zwauZGOuMXUs`Fu6d&FE9rsLO+!3@b2Tt}oKuCwA%tNus~1@L-$(0buid>2JWsJkt^ zfbgoei{tA-|G?D8FVRVHl|e-h-@6ZW*YAZj@7?R^E3c3&43O7cY=7L;X6&l?KA0)E zV&#`>XaWu`b`8myEj^TRc}U*FVU->LoR^&wl*!$C;MHBo_yN&tS%%xebHln$v6)nx zSMK)DsITYF3Qyf1Jc27dDpx~OB!`~W(fYMiI2*W7iz^>>D`y-M&ibs17|2V{B3&ID zxa}D%!I&rY_bOpdFxYwJau$=s{xxkRoE5eRz{Y474;P0uKj-2uofp7^(CU|v(Y#Gs3BPiXwHNf+l1E;0up5$mG8|*MWO)SqX)gSe>tnC3 z7fp1v$Xml?iGnaTgwLmBpQLNYhZ^D0Y%!;+iSQmcTTu?YxwasZ;x&mb<|ec+5#Pfa zK_if^I#-vKB|c%Jf-P0#A2K{_LZzPyFcR&+@Xzta=Lw#ro||^g7O49pJlXJ z&fRh%a#(u3a=yP7ji{GISkclTmTMWARsD3L^BFCn@h`EWJ1)FtIBKj=oj?_=X)_LG zUqGf|ltH#}L@eN(YTwYgnwZt@89|mcKjhzZuRakNWl($>Z>-RMq^w}AaJFBcapFde z7eu6yRzRW3{-wD^A20c~W@^+Tq=RZYP2xe#jDj3>Xd3f~TECL0y=F4*&8)a2A&J?a zMXVj1f??k~sBx>@xAaCOR^ujqGF4sUDDW3R=TS)1!}!d8=Yz& zGV2*~+WuI{Q9@R;*x+9IAtPmHHm=8(U*BaZA2rlBo|rPB)!=R+mPmJGU6*(%rvOi{ z3krsnz&DJh5Pgt!s8*ywyTwl|umD>SpPhtITxhr&+X#(`fV0x@M%=kJIN(~KyI_R* zM}WRKbMY0eijjD#aGHKB<(igAAv0{=NrXO|OLjMTPTT-RLBQ-YXACh&iJ)&m`P@05 zUWd4{a^(h*_H8~qVjAt<{EfWMz}g5fi?@NTqWEwZln7E$u^CeomQ)#V>`Gbb-$>xF zlxm2Ogu~Uk{n^VrvpDO(fb@C_tE_b)uJvTM0C(6lfq*v_ms`{ZqXn44HF0T>qDr5U ztrgoqvRr2I3(??|Qsq`PLne#p%;Yp&?xZreOj5u3&R6mUk<)?eJL*O8s${+NRiSD) zFTRz-ADCuB0LnXhW|rlWf}n&Z$v!A=xBjtAa{mh#XPM-;jDm$-lN6;S_bnr&oyac3 zwq5F79yhz@(ti&mV=u$tF@~MD(DkVpvXKDKymOC`E}Na2M3j@dFZ&;nV^UA_sSq4g zMl()$$A*Y$D4K!Lt7s!niaaedmx&Ht8$gB=BQf$~{uCm1>aH7*;T4BQy5b!T4Qg*p zN(X*ASVx$qQe^ozvmloSWWe~@qEY&Qk%dN8W2a6L@cTaAg@iC%D+dt$84%eD9Y|R$5Wli*G98>(c zUHOAa6*f=7oKx<8Q2rmrcU9Cm_vf4>rwze04H|0^mG`af_$!8z+1?U}f!9r%$H(ux zl^-yF<8pI4%tGe9`OU+U-IdNFwZ{j>9PxiM;?bOvO-|~k%cqebT`}oDO{ba~EO*Hr z@kd0&r1J-h<8b%B?d3mrLd(A|$m)HH3I-?gQy3LUFIGr8S4Y@Fhn=JYgkBB>0Ov5m z?G0t1L!M_ka?EdnOu2W3o>D*-yoA1 zY@L)P`i4zABO%J!?~F}I2qusS#73FgW!$^yi4bhOabCOC6*(LU_{@B&Ue z8TogScNGcz+<~9_xV=2|bVj{j{N5>`ls!I(Q19-0FK!dhJpruQIx_bxfe79_M?N*X z>$LNj9GGA1Jxia=P1yoU6_NSs8L}1|q~_`?QTfl5C>3#f9hgm)FSvo!iv}pic_0zB zC=;_037TfmpnXEiCM{zolAqc2ia;@53_JyuK$6k6*VvmG24@(hd)SB_R+z#z78V{C z2Bm5)&`t%VGcf&C7sRl66d4As`j`*Yy50BRsO35Qg-u?N5RRia{7J{0=+Hp7C$*-C z5ha1@0&I;ekMJP-`pw35t)!=(Eo?T^sptu##;@QT!BajyB={pE;GfkWWW>R0d znk15T6q+l*)`C?Imd+}C!??CrzFUG2eX9boPQt_9T;u~vn&@rgv$}GUiI835-G;2@s%UowUug!$v_#hIbt(uw8rE?1 z{35lU&GWx^8344dEN+Mqi@7{?ClImM-7^f1!8!$9_oIi*i!3UFj#PZ$Q><8M(+TV~*293oM3N*=u8 z)PLZ=E`JFl`n(?hnuCg0ybTl7Z*PB_y^~jF@4f@)d6!l9d5hkTTWfn-L61(58>1Ua z<~=W0o~kGA;C-0gfW}#=%rDr7z}N(t`TZNBM*+vrnB9Uw*VMl+2CrB<1ld%zFUX56 znueVf%(orD-ZQ;spc&@0-Mkd88ajRz7*GNtArauZpuRX9aYw>b?9?yRt1mo!S?+6TWDm!?8jnM};1f7Osyn=2oY5{jq*4%kF4F|m*oBUX}6q*QEVd_Y&&pOY;yGrcoa z=&%+P5a$PbsmSxTqF~Y+Qq*^6WGp3wWQqr`$YUx?Evu4B3rlR1X8}fPIB!H|q#`ij zl4xo*rZ!N;q?D}@bmWRFX^{|vX3QvNPo)j|=hU&L*p=P4R4$_U_aE7%DAaSgaLPPJ zL0r8vSLNU`MKJMpLx51b(W()8mUr@$v)9-x`_Lsx1EyEt=x)r9P?>q zZDihBbbC5Bd@zKTlu0VGpfhx-0HnhVu9%0ll0Mn`LerSK?ufqnY*JpbiGuQahdP#e z&KWfihtDBm>@qwBgwu&pyweFggY5nU(K^QIv|PF}DV2 zF}rf%Drh`?GGs7DyQLF_eeI#fvY->MXxIf9ca3H`m;sDmm_56%MUf+SfsrIa4~W^u zr>rfNyRO2+HMbUZXAPJ$W3N!&78Y8}nP}8rQhpP(SBjHHpU^WCu77F3DYW7LV3B}i zB3eWYEtbMtFz1)?g?H6Kj@o*EWBXa9ujSZD`&vix@^Brrqn#=(x$=eyE{3w6PtbeE zs2G0oF_Ldn9mKpMjN};oUyQv|aAyIx?c1?!chIqI+qRQ`)Uj>bwr$(CZQJPR=BsmW z?Q{0Sx%++9dRn#Ss4>U@tz8B{fud@e|*^$Ch<3qkV zG2cPXh%D#+)|nHB1&;^|0o6F2Xw;FlG3*(^Vz+x z5H*Zz#*wHXKNA+MSQZ?u`6@Fd8Qotx`Q4f`RS6)yz-<)^{c1-%G>Uw?+ z33qE=QI(7(-;N2fRuhbs(0*gBo7v1#{uk{eHkhi3jZTe9qjNYt#V#I5Px!{ z8qP!?6ZGH%w1zB(Wk{0JY3HDc+NOIPV_fza>ys&aY*Y_oFS0hQ?=`4G{ai!bHf}2_ z?K$0m{drGXv0e}a;}mWs7I+0#P=P7tbbu`lJLpS#5MEPAN^7Rn=FzqN3QcQs^QO+u zDF^5bIs2H~MEHE{2foGpZ4|$A?@eCNP}Js(*e&P!!o^E+QXxZC-Kd@;>QVvu)g1<0 zL8ZdIl(!A`3Cq6^UOt+fZiqEqx0W1&{z%cX<6qj|>*e_&#%|ar?{D-U#HnY(FS2YK z9z@MKur7IN>q+EA1Gs(>TVcq~7{v`83Io@DlFoFTs|t<{?{#TBzmo8T{CPqNpAnsF zL+s!@Ks(k7R>kfiR*lREb+{BFK~3_|QUv-Dfm;0^-)H@%TwzrG3NpUdZ8( z38ltzdC5Fc?$5BCYYm5OC)78Uhd4fzBVQJ$Zzw7~o<#I-M+bAtrg|E_#pqFlwMs%^~ z7J^so)p35tRdNkkh1=L(Ys1fv1Zi%+8PwBl=49ffIAo#NV><7`3mv zuLzRt+fO$8Hu&HR=^{-zL;5UGTxS~ELdxd9i`q| z170*`sfOQ6Pso3*r%kEcj6hVLK-4V~g~Fh%ia3YwD7YB#49eAYfZi#DWYkV6J@WmZ ziY(1)pVIFqwwCcjwqyODDYE|@aw`9)hFHSd#O8l#uPW8|{}iBnSvQOopZ!pa{q9x8 ztQL}QDWSnYwPFRO`UIOq0)}yPBtB;Kd09|uWksuL2I#3K-9x|`Tj{M1cBJQO2R`N z-8SOFkL}t1A_WJAG?EWNlNl2!lDbDMe1b>l!`q{O~$7C4J3Nq>O` z4^D|OtRzj=+CBaVEWs6{7@D}rg{$fa6yD64Yc@WtmaGT-)$uYLDp9sTpI5?Wh4=-) zts&TA2&FdIYnVCH2=+M(oVpCR zl+&25a5WdM1HBpSfAy)RAuFk$xnVnlWWZS`2m_PAr}wPTWBs;HO=!m6xxn3PV=^b? z2rDE6khm5Lw!z|1uKQGv)i5(m;+BYKwUuRJWg1RssF)61#axnQ&kxtSI_r5Mz*r}n z!wWHvYnQD541|r6V;nBGcPt$Kjp>j>cPeeQznnTylwI$4rZyQ-iGJaVBgctz+LTD$ zvm8d4Ze4ry@62#kzit+CRu3V716E1X&CKT_WW8JFx--6CwGS(3cE6zU_Gw=Rd9mL;3!Ybmc!D!|1k<^yGO|=Rbb6mZ~*B4p4rU$#n$w^F$#l zP9>kf|6sMGtL>rFdvo;r{B(`vrc@Xwu>YemXAc7KBB_~MbHzK-5oqtP*ylXHB$aOy z=hT4x4uQB)|DM_}v}TUQc^m`hx0^$kbdTBP9EAQn=i}E1Xv^xWNTJ0+6?vo+XmcC>eT3@0i-9&?*PVZxUNjV{3lmu+Bw9D{N$D4?>7o{Z;879jJjh^gL8MU zfM=Lct8YSUs8Z`7nckkMXVlJJl;pQPn}NpurlHu{gd=TV%_Rkmbm=8%ghGW{Xisgb z$Q8saQ`?$|2Y&RUU2lw+ zlLKB1(|h!j-S!Bk3n^{SIpgx=a&R{=me=UW_iPn+Gzj`=5>i|A6=lieHlRW!#R2Gz z7~NnplsSX~i-=Yk{PC~h*&s{5>rUa&?eE${c)8MWxBLILMV>&sC0PObnPxwIIlTX~ zq4vM~3ID0hF)+9O|BQtua#IO!ZMO#i2Xj+)W_%5 ztX_3})&c&)a50H<7^zJnps6o}Txq<0-2Al1dE;gH!lON38FXy{fP=pH+ z<2r@j0bqP6Y8Nw-opJe$(<}v5Y(Fv@t9GCv)Mw+_0`UuWOnFOV)JgDV&~S)yE)p(Q zMg98KvVRLw`-$+Iv`+u(%ODuAN|ruFuMqm0z!OGSmgLuhdlil(p%3K2Lzp?Y8&u&9 zHx`Ac_*U2bgH$(wb(O@lj})cgjxz3YiKp_6e%rPb37xl$k(=tV^K2*)`=vPivgnr* zPyW}=HSiaG9En)ec8VsDLdx%CrAkWQDSK(eb^P+d$6~&xTW#ebsk*Vo8*9d*sQuR0 z`=3>v!$*zK_xax_b?|qYdh=^zeklldOTjnIwl5oP_pO=jg}`xg!-h!QdtkaRXIoIp zh-B*6D&>uBbUp}tJ_PP@_zZww6|D8`NDsRtwM~7=0OLgcp-;fAOE;0jxGFk8*M^LX z_hjuDg{yUFSrvQALyAA2<^bN*F@0c+9{=@m@!(eR5bb$uz)Kg;x_5r)@PlqNlLQE! zF^Am+rIcCX8rUM;1nqB_r!D;dse>>m!qcOEG|&gl|4(Vae?|EJgZTa56a4@Ev(bR@ z)CMGd=Qz%DWo>rPV^Dx8of6N;?(k!86iY}h6ibNDi%Ax3PHvL4WK9OTHG+E1QikV_ z4nUwHAvHmkqS8y?wN)J30Y0G+b6xfG)l}Tue93P{B zxCMdZd(4CUJ{{oO%UPQ2LEY24v*G=K{I1>`(fufdY`7!f{Xngu=>J~4Z4dswH{kUy z-h-q2+6Y1Z&>r}Ht%dx)82Wy@h56`7|0q)2Q?q#d`{cS1(sJvJydMK%}pHNgU?WI#Kk%q8Q zsEwpts!*g>z9_|_x5&ym5l(bB!7b7?PgT)buNhZ$H?AqO{8p}3^ocME-qfs`;8Jmf(^Wr4m4W7#rPgewcoRSYcRp7-rYcj7CSBFkuj*Nln_tO+ zeD#C_BAPq%_=43ZZrrB;dw=a@P!FqTU&QEf}fS4tH z+2AQS%9Tj+5BNjl3s+T;{m!H=)3!A@(Om97eNnJ?hrb1#^X7x_OG5`H-@aFf4<^BV ztuTyjR5Kzm^!=puhS+R?-~%}Hq>e>^f!ySd$>2LGAa@2DY$<{thJ77MRo0Qu{|th7 zOJ*-yT#VZy$U}vifEga=D;DYciOYdxK3>>==`FC~VC-rdyU3CnKP(*{#D^D_6V;2k z%{^Gt>M?yI(w4q((>ZB^)=HKw=;PC56EZ%>2ZxNa9z9;gU^22CH9M|ZQW1cl9z zzkgAi`UZ^JJJ+OHfW*F$n@5u~kzeKp>vZ;clriW=nA6m%&Z6pZQMV>n7qK-c{1r2} zrDLs~9S%@plT-q!O!MFRfF{WV6Th0fV_y!$7NWgWQZBHWPyB}C7qco25*S>2!3LvV zSi}sfksT-y_AZe95Vf&g&^=cTS}rCOkIPvlM^sAK=BBn#mCE(BzL;bmvYq+lP2!d4-ek&D`@#fwK4)T@8E?!U%d2bsxK>C6lCc=keu@3s$rFe2I!J z)6?`uBxT61=%cefZz}788$TEJ=wZlQ^p!f?=e4n&SWHNbNcaYtIlOv9&?$`j=8-`p z8++9Wh^0$Qr*bS5LhCa5RjWW`(gL?+5Qwl=w0kx!h9hyF*wAwSc~MP8>4t<+3T!Z* zO!Z4$BW1362F!x1u>p*_m*ZR)gH3bN&2mk60GQM;xvGaj*qXRa6U;G#?0Wrk(J2_$ zW@eI*ZZLg6j?po21uDL3Ce|^7{RP`FdQ&D=`x2v2AG}9YImR+_-zMy}3@m5B;Ky;( zANkI6^oFOr)d7i+s)?^dLv$A*C$ssnXEodR!m0R@%DQ2PC)^41$;g{9;e_4u!Iktd z)0b<;O*9du~1NX%U$(CG|CDj}d9H$HZEoIAWDm&9Ymxt8O7xk#@L<^nkmNXHu4 z-_|O-Bw>;$>nFTYlo>?k0F*`C=q@oGoXnoJ_z!Y`yvTjFUXAfvicWP0Rmb0y7A!m+ zcljuEv+@|+sZGRs>8R_TS7*Cs^X=zt2?go63wFZ1rDGxLm5yyXypa1!#bSZ(IrlAsh*kgjCY!JsYa$~#IucZ-v56Z;77i@&yy7hd?^`KV`1jBQWjUBpyz0mG#CIvpkT!;DT5s+M%1tq)o~3AJ<;eV5lge^v@k2r;d+tc1$+P^bQZe7#JO_zNrE#WiHnDiZl){cs zRb?F~ETx#+;6O)**#eewcHszzQFwF-puPqWmW>*_@q`YhElJ9M2P6i$Uq(B+>6V?Z z8Ne$-=$RLmEP@kSy!&^Gdrrxo<#OULM}|~`lJ^K*9ysojt4FC?Ic+jK z^v}Y>l0xqGC$Dx#?>Wnqh^U@Zm6E7PTM6{ki>K+V^S9^B41ZgLGueA-c&8}fY;oALc%H+b_VCeu)mNIJ*=0x`EUcL@6`JG+YfwlJe!|H)~Cno*1!5y@& zYU9uJ++WYwa=gte3s5L`Ku~MJ{M=mHthaC32>)oft7L`Y3))=56Hr_JS)%G`XT-s71#M#Pp)#Dltb}}JzGd9`YvgRp}f|Zs})iOSE(KEJDZc>r>otl7VTb4nblGqH=y>$sS zr?jZvmNpU5e;ikAOKlwpQtVZVH9^&L)d4P4S5S^3+*jDGc67EuBT5akrx7@6?QuST zL-bOZ_Ck`-hEH8*_=A)y*nuvb@s51cVPG09q@vm(TVD%UZ%C=NW!U2V2wO=<5QLvc`Tuanfl?zpjsZ)%(UJgT<6{K9}}%6VV| z_b>LmN**~1V9V}KeLg|ba2|uhA%svh19?LQ8Fz8g0Z&o3g z*vpQjkt0RbUS54&K#zJ9ol>7{V;W})aJ8R7VXl8s9V8dm@r2g&4#BkT63@yu%&~N^ z9(8+)>tZbNy}Q}RbSv8*mtA7P(oWA`3`*fEvA2GtggJo_j7xR%4p{j^&4iDQs+|Bq z)dELeoMH^D)sEH3XXW*C=-iEpAPf$Um(wC!>>d+Qu7V+!_ps81A#lHSO# zlx0`@raEU+5`tuD=Ohax^gG{jUL(u$x7TH2r5P~{YM#MDp*{N&TxqjK_Cas*axzF1 z*9Kn^ zI_F+|k(!_tKkT$tG>z`e>g34Wq@7V0gcqg5+=B!gZ7!uW`FuoPb%t+){8Ev;v zJ*az}j!E%FE5XGR-3@$3X!8u~% zVW11-g0pH=uM4^e&63b;2z}S^fhYA{q`s((fy5rgBnhN6P1^ZnjQ+C5WVJ*cIqjUR zLGLgFh_WD5p3g5^DBSg5h|>MJx-8+E6(W@p{2_8h1y; zt}FDCe?7y6h7i$Hq#bzS_rj^ik#`WQ3w!^R)HePAc#&@wHXBBoPedmV9k_Nh>0?y& zBApWpP%7$RGHM| zCZ5vlj3Lgcw_O*q+5LR|)m4kRGi1dsnUi<`z#V`XGhopj3>lpeS|htN@C@|c4j!R+ zDtJ85<@V*W!|-}`*G#{6pxD8EBRGf@`nH?z$C=FQu~hkLPTl^giP z<}KS(!sdnR49X3jkq?DV2d*3p7|Z35bt2dwzMM_edUc{aiJ7aMaF&Fz>e6=WtQ;=r z&+9CzFLy^p(@}9Qojr`qIzLCMWcT~SlC4mANqGlrw3Jq9mq$cS!95`I5M981HgMF8 z@RxghKyjZVh(!t0p2F|1+`K$?c`dm2y9)|jske)K2g|DYH>Y_6TEko9LMD_Vt^kA5 zE;4pWc(9(tLQP)5@*G3PX*y>+0OEwpeDUHeFu-xZfOD5zreR*mu{CK+N`MYASUh`!+cC|T=AMVm$YSH>Xp$+8NMAj=%h4s#CaBK zgDXae*%J^FO!oSnuw72Q1DV~gmSAhQnGHs%LKI`DqXys{W9IPV(BNPQd>UvyKVwL> zM_B4&15u(}2v;K|R;6f@6qy4?WmgEK&~@#UJo1=kh*NCHeyj|x0i0bb`K(I}&wVxq z4%^5fD^>Ht&q#dfIzY1sc64{lM+dx@zjYXZHnC(=MlW6tBgdFRX5dcNnVUu$I;=32 z;tVGuqx+&v+x$*2;e}GA26V?$wZj!EP0?+JTQG%v_?6wl+}@$>K2?G@ zb7t>+!pUxVHm2XhkfiD&)4pKas>!~%ClmqZ6{zSn>2QHXHl`J4HZf-Sy^K7%#746Cdzk^J3WgV3aDqLT=b?z#| zv;|NlecaH;udk97Z$3eu1y85&@Nk zQ%Q6N68Bzv_7d|-(9t^Bm+dsmQ6&`3RX86P=s-yWEOlR56K-L+c(=TC(}FMA75m?@E?amqM7%GkON6({pJLs zw!;x(w#yL?u8R>5uF{s;h(tdT8_-sa0(lbQ!Fo#JkDHoLT8D70F)e~G8xUlp~4enjF{t+@#>OI>Kvk+ zEkGMC>LJvCMDSoe5xc!K#ISsbcym9GxRE}w0MmqAs{B|W$-sdSrjbB_OHfEK@~{ZX zsF9A>gx>Mc406scX!BH*8TH_2<6nr%!kk)Y+wCVy=Gkh9Y zuCB?QmK$Oa41VRdv!4l@>|$Xzb#BOhGGp`?YLs41)>$hVTS}crbW_o^(pKOt7UVYa z2S%+h0V!l2gG3l_@vo_sGd4V`{Ft6#$-Jm~N4}M}Uq)tfAtKV-%KDX?xu6tEifP|5 zR`IX46r*E@!BZ~huu^*xoDyz03TIzfAv>9|8C^0;@VY{_asv$$bBA$w9$`wfRG5fl z@n+iTX1bUxlc1p*<~f5Qly$g*X)cu~%$_Xmq63kBRR@S(o=fBCZM4Epg=ID-omSrV=pv;5g59x40mflTsx!cl4Xq zt6XEnuEE*;bxwiwefRL+V4GZqW;DU-!41_hREH5^=5>h})#WI;iiJfC zE$(XhBF06$#F&V>+Tc%KZ?$B3$KR;QiZNt=_qZ(&HM@bpX;wsPfS6atErqLR9XwVxv z$S9_iB}`5ii9AjbYUACEpPGs!4uE11efozdbC+ik_Xt)t>6|;6Rmfaar~H?Eg{;%O zOEMPA=LAl2E&niY_&;V~tJvu&^XEhSwNIStzu6IHs3n2>z!Kv(ZOGGv)^Nv92&eM> zovAJaMXiJ4NYd z{t^rHO){hsFbjzJA`s+o&cp72`oIV7{K0n8OYln9ze}{kP(Bm-S;s@*6(%0@2F}Vt z+qUq9`p(@@ABG`V;v1P!pxgNtzn)Nn@8Z6Kh) zWustzgfnu{3mD`;v1*;z67c-RsBH#?QCIVaEqXB}2V2f#5HbhB0*YQ(htkCB1&2ij zkTES!0+fwSlFIqq*h(8zR+&@Wfq8`{p$-(ufkJjn9Bx#Uai}qBh^MA^~a!z)U0? z!xp#~gv=xK~N7fL8`V%!Mac%#{Wo*lCDi_UmRUdClYo`uje?`FJYh#;dEV#pAjPLzg69<%2j0e(nPLvq%!~=+5%#|5cv; zmF}$}<0IrlMe9>CyAyHdgZRs)^fBg9j`EY~Ga~g}E$FK%-DI{WV#0^~F=zkDCv3t8 zCiVMn{=1wx<4Z12&+Y76C0)*j^6_sH`uqiDm&i#)+Gz@OgJeXj^pX8@r~+7Vn_`iH zs`9hF3R+Zc`MgE7s>T6!kZCD*f++>JRuO46S_&r+yVzbEWz#+P9lhQj7-iL=dk{wW zJ3zNv`-sXlu9X^n)NAPjURN<_E(w~gbWyg(!;Rv)6P9zYLiyRDASGjP(%#Oz`WW51 z>i8F(f*V-P<2_}P+5B5u9%K`h=CPwOU(1jUVu)j()}$;ur`H1bUVM%pI~4* z^KS8zUha8=UZ^~YAIA!v#E62Iu$B+(h^Lje8y?(?*RxAmP(wo?R&b#og+qd2E3(S! z2O?9Sn|$xsu8r;;MTNi@Ch?gJAETn+F_-bZep`+X3Gj5vwMgG(rarC?kj~a5hCZd@ zB`Sm|*4AJ%J4x%g*3%OdQ#SwN=GU22iw~{*wA?nBj%0thRdBfkAL#=@NHl;16On-Q~%tE0p6JU%pVDZ^D7<7!2uGGoQ01w`2F8ilAE(e;~>M3oV z$M$q@Ere+OphI8^@jZ}O=Zm>qq=qRo(0t{m3?ZZ~{D$7cgOpK$@WUEMwsvYWtahtk z1M0Ke9JZs7>sxWIuq~xY2Rm4hLah+{sRTSPv#k+MA%>9HRWW`QH5qp{t%9@Uo(NSv>+!&)cV2eQ3 z;9$dvX11{!v>VTfY!kxS#9T4_provt_XXvFxLi#@KCeSzO)erv~H*| zi_uXhh6?>a)JEGf+T@B5%Cc*o`2h>VWDbjU^&9RMw22wom-fI53P>i1sdU4+M@}ka zNao+R=B5r|`M?_0job8-`?gKZ>ecSZQplD|kf^lXoKlBUxf1|sC)_OqE3dlBrxNr( zsY*)AkucMe4tLrw$lL9RS(xDkr_9nkmfPl+j`zL>RzMXlI+Lby`P1ooi}Rg_r*K{ld$vQ!9w zsoG3JPt$$Fb%y{w!9U@Z)D_K#b&vhvRe$qfCDNske!8wA)XYd5PTdc}kd+CsD_Rqz z-@&k4Fq}SspZ?8L%V0Zl+Fn%-5bz z@8V>dts;jlT|=X?d2HHj(Lv+Qf(9V)Y>Ux~kokw;1p|2$$_mw2LE~}gCQV#~jZuI! zwP%w1sWo3)h4RgyS7F&THvwYKm?O_;dki2!{^o`HB-=>I&T;s^b^hq#9y9FF;!Xc1 zu4gDY(Nk%M;8))`d(H?yLG>s|#T&C3fvJbR;h}Z}xmhwaSJ{h`@oY!y4m$JO$(|11 zPITq+-y1Aezlw?05Ga^HmCU%>?r#n+!X7$>e--M640grd8HYgYmO0qa7?~D50D`x` zHT<2CB%^sCFbqhQ2|_UE?l%Y$V~Ms^KENa}E}v)+Sa^kJWo`x?PL5}#!k&+)x=U@v zHPLDwvtMxBsKtj0f`h5Qiz88C!am*p31~9VTu1jdn~a_?Kk|}Qy$RD95&?n+_Y1Io zmetgssiWG^0T*0vt(OKX(x$|DcU)Lz+eLRBZ#>vLe}rA>dgKboP8Xni^bR`S@T2&8 z2^L>3qOuhZLI(eZ zyrHH~o3l7$^Y!N#LMN_vn}3w5tM;%lB8TCJy&CgKrfcz2eWVV@JfL9vr1KTbwVZK5 z`=}nw+$CU(Kt8#7Hy7C8GYt?bcv!S2(7>5^Vf$p~%)c`G2<*{GRuJubib;E{rtp5m z%FF%z__Oy*Z~62OZ~5&{Rc-n3%q^CNbH7mm66j6?Hs8q1-w&2APVzs+T3{n~Q6q0| zqluTK#xo1pSTG@45CzMn@cn?6N5V=bp!gftqJC4B>|Kc{ySGEYJy`R1%SaR)=ugB!>J<=0@R5zHMC zMq2}HhUlRTof%-*G1_Di{9>7sU@Lb|b@W-ZB30(79x{KQSiPzHOQPxDXkXpKe7g=< zWvgq-3{})f8UvDzv2A35^r|Ks$$(2>99Xsdsxk3@FhaqB8(N!Gi!UN`tJ{*7(?l-L}HV zJJ1X&0>RaT(Iu=xZeamDc?JsUxRe^7LP$r_gOs)_WV4}Vv{anuqHQcW%nR(SD*Qs3 z{6x@?^5jxd5;Bb#x-W*$Jspn8dCdXQ_CyDvoA zV24iFe9#VNyc!~24(P>VCNR%w>nxVP5!Fujkud$mUO}toALH2paJn!YX zEMr)3dXrASd2G=ys#!E*Aoe$0j&Bi1Up;H$!i)Z30nTSE$csx5q|s2)Ah~}G87bJu&w_OcCMuKt(aa+S_|{ zxz-D;q^2EZ><&oLX}v)8@$1<|^7zw_cqYJ{vDWi@jSRTqu;I>Y=o(w`-LB+JCOgu7 zpHcA>%=R!dtbZkoCnNw?gy~vRS@zwnt><0PPRUBlBVP=*LMi44J$NwE_4YS@#Hv;EjA3H^SK9*^jbt0ZVY5~{$$)6fyZggR37V7gU}IqQ!%dH`ei z0i}1B=%Oq>rvR&HI*725Bc=--US5GRE#fV{BE6iWMff;;X3_ojQNp1?dLeXicd|se zF{cPGENXGe*v7GW^$T^)i?pH&R#Za!_v712`f!@I5r z%3mK|kp={PFB)Da_-u}ezbBHHxyzo($5}uQ_!NuC^s0I#jNM~8)xpF zJK%H9;sz}OBz#kaMvi2v(|G{L%xx$;n-WfUj4lWVp2ft}fL%w7t`za;ZLyn>{LaT; zTQV>FK?J>*+?!cR1cQFMe*2&i!$ES$`+(B^0Ed(@CRV%pAk55iCRmOlbdp;WOu&ywH6=_Ms` zJLAxV=ApqUsnEO5t-FoqGSro}UA&{*ZLOt=EcKs$0oON06wp#{3}4Ys3NxOUe&kQyNN<~dP$K}{oj z<6?T9A?FlE3pQ9xCCKYx3nQ(>;&6p@`h$eXd< z@`;q>&2RO-gZ(3N<%E=!TS(ErC2So{IitE=h1~W*BURyd%V2K6VE$WZR}sii7$HAC z3ID-F@hS%7BgmPzU!zI?!d+QI=_m@GBREq$DI2KIiSTWFVzmwi+I515k=vDuQmwYq zh-;!9GV>2=onAsMMah}S^VhhSeEHA`bzY5t0*TmxqpG_!p3HBLq@vKSUxrGLQJ{5V zwX%g{(Z}eB<`3LrR!WQTdZ(pbCYjYiFuH8AI1J@mff2WPQwNJ25?bS>62%i$*3}k8 zKG}h@2z|2%#0oj_n^ksZvutM~V>)8t9q{BY017FHF8`z>^^WB3AnGGKyUK97KghUp zK%(+%>5`Y?WGIVEE7~4B#;dHdTkjcvTa12;`RB!F&VcUqEc;)1&QS~3cI|l@>Uo<< zfdm6jcIiRBov`@Lu)3(e5LUh?j{T;AmL{{+fclcWI3G78^7sGU5fir7_&blG-j9h976e2Fgvb1^8K9Mp07^sHTi z9&&LQbzN%UM)S$<`kkEX8T3bAuoL~+VWCdU3_|V5hWOeRwu#JV&j0ZK>1@i0t1mO+ zzLGt9gw6lem+i|SGf2bn$RIoWmHuMaE&-p77%au)+%Pas<6otfRR_W*3-Q-xK-l5@ z?3uYnsnF<&U+;}Grh&hZnxsj1+yb3At=``%iH1llXn4#l8^#5XI>1=ysPWqxA&2)H zMGef{KLx3?5*R!S!2!>=ma`CApxWaU0qKwr?7=6Htg6i?TRuNScHB9{V zE#VQumyF{dg(|&CPA*?Ei%4>8I3$SFVJ8e>r1VpUh zc77Z5C&dJCJ!J{{H&*1WX1Sjx5lyhLg4h6xF|VA}%s6tqb2)tFn?u_@of+y&&~G+> zX_< zUmY$a>!z#4ru$|p!;Q~dR~Klte}dAJyCT>31kg!tq|>`V23+H@4l5{0jy-NGUbig4 zUOYI;zA0xFzs_SdY^(*G1j3RUD-(s9dc3jZ?IDyszZa*F94%H~F#?5U+7uvr5q+o{ z5<%XyB)k^QKRc@7+CaDK;xws3_p&JLEj}_bkIO0Qmy{iCwn$cx$%pXIxpW%T869*n zF(s5b>QTQsLDXKLyn-xQz0Wr5No4=wF#Utubn9h{b0v7mud&0C^m`G>j5WP&r}spQ z>DyZLU2nXnYk|S&wOi`BAhI#6flZ!gwk+&0+x}C#=MH>3Yd+DURKFmYND_GC+am=y zs#6S|oH^a=8*L_ahG_)kxDjjO(oKCorUX)D7y|cg@RUI{;0Cepn9WshjVgEef>`vo zezwf^9XP^T#Z3L%4r8na)ANYs3Z`jX(T=;^s%9T$jVz6^BgfD#@OBDyA5>EPt|}=# zF~u}kC&v_awu$=pf1?1$54fbUK>YfZi}pW09{OKjgrtq(e;p4sscP7ws-W^@0Z64e zNzwuXLD96+q+WTgSAN z7)Lt+4W|=Cl+nnTM%013I?KOUYolRuBQ`B^S!*V{iNe}$3|{u%b48zk3>a0Y>NX9} zPLjEzhrUdh8$0Y^`IVU$Zs_aFoSBr^NCpVYZ zg*;@ZG2j6$k>-wuLu*#0wx6Pb@NbKh%_)*;eKSl?)g!$*ZDu9?Vzb1c#u%cbE8 zQrTQnACt<68Do}k0Yshqa}$f5YS+-VGbs{vJMEDIF;by00IJ3XNiCf+*wr26aVj%UUxao|Qk#vGM~8?E<4e$ejPwUE)K$@v8@(IIRa|IKdeZhfFznM-1BIsC2QjxQ&au zkX0sRqatP$FG_dHHjI4~HG_XBBK?Dc6~8zCsas2cMP1)g+djjIfhGv*| zg{i*eBRxG@^1@iz)cY)9EoS8dc&zgn1owMESr=OS`Oq;(m{{ddm`F84w9KnhTvBUm z{;~d3poM_;A!tpE9MCsfS#a*VgX6SVbj?DtK zXMcEmKO5bBGdl>j*nOq+V{yp91`1;$Hta(5QxUaU7>uf&42QKZDq2ZFSKK!x9)K3J zMhuyS>sZ1K?t0*`tEQR4X05Y&prtou%f)Jm=@~?NVYgC0(5FfU{leKwz|{0yjCz+K z)+BxxNFwTh*PVc24d?)k0%_K*-F-Eftx4hQec)*(99|qR9Q~$4_w!1fjn2c1fjXqs zE596fjrBllxtm_McfKOxIRI1{JsQ73{+*{vH#B^~GW@ViN*33UA{jjmN8c`6&NpCt zo+alhm2sY(fQB$n9CM)x+yLygB=*1P+x+e*yZ?( zYm?r$#gBd_enS)4rH!hQJUGD-c?ke6OHl9X*j@Acu_^+j>KcGgPe$25UBi3jLTCe# z%^F&JKJIT2#;4@h*9bSTfxUY_XNfEhER*UWvy(lm{ffKDDx56Kj`f)L+Glr1YzwKAW5nM`eM@!G2PVUuYAlXf-)|Sz4duBn6eHqtU>MpSM$B z=p6ZvJ0agjC=dVpzezLRLaGj|A6XU#`|B6m|7`d5pDSJ&16xIa@qcZ$-ibAW03C1S8<86!mQ5r4wkztf#NRKd~+{2 zPtb+v<2H>Ti>BiB*^?IF>Xu5;0-s_r%9$NFP0`()Vzk;lt!08U!>ySrkEJUr?>OmH z16Aj86qOF)YtjK#!xzcqEa8G1;tnUgH8-L2)dgxRnA7MqeR%oxIh%I43Kd-D3$}wZ z@B;_J5X5w8CS6hGE5Pq(+EjjpjhBiAd~sOuDG( z7=uyLcDBmpjeLXFeJZoyvux#Q6U_U8hE!mZocFcLn+6x5Om-43&anyJQK+g7$1}`T zULB{9(n5-f|A(<}2of#ql6+;`wr$(C?W$L{ZQHhO+qPfXwyU}p6VuT>F%kdf?!L{9 z69;)R^Ds)g6L?a(vl)0Ack$Y2Vf{_J*9n5|?UAU!xp|dotRgBA|7WExVy~9kPQR_u zmjd^uRAUHg%QO`eq1_)7!Hp~VSrftIRlSRa5?fchtA(<&=ZV6%TOR?XE`M7H850mt z5P>)WN$KDdV_=h)iKDmg*D545x3MvO6tW!jqZHJq<|$xfxQ5bgYU%#hk(3T1 za*vpzbC2+ly8A5k{d~i8J0#_0hc1PHQbz%ELhSuJZZO7~#S(o$Qmx$@O6&ZHFYb+& z5%O~6VneY7W52Y<2;1#nwn~g*DC^k@g-3JLn0#%PK@9KO4F5NUe#3^mXG#w9ImQ!G zxtYH29KN4xS+*0I*&S_j1y~#oK&@gLk9?MODEE`amW7;hgPmnwsB0gwlpDWY!zJ9v36$#E1A6t$h-WwE*aDdtt@1VRXx~NZ5@W$NVvhun z*)6g^`PHelc9}KYOne_sC%il^7SXsvEiMxXMB^Hs*e_OR5{o^LSI|=9eHov0@aKQg z$3#osXsUj*jrZ>XkpIx6SKh(a%)!LbQPITC*5UtJHWjOGDq@KtUurK^Pj+ z!lELI_Kd{zqYL)!85%){u*WDhqiP8XXj)gR(!K_M3ikN5Dk~#U4FKAWync zdQvHH4@xotshKpI?ze~k-Yf?+Z8>~>j-j*jEI&MAOtCa+y_`yE)}U-pFF5jeHYKq% zTdBH#?mAHYJc^igDe3sDjeqTEy76*V;7Lzu_oY`4TZ=DiOn7tnCE>3%{y>YXfd)RC z#A@xoQru<4SB|W=`7h(Cqc8qdk-`#Xg@%r9=ShEUXCf8k8ryK9*C4Ukt5SXf1U%QH zILj<`a@MN#@U2Cxnx?WRdE3XT9a6UaEtp?v{h4D=Eo>`Y`+da2$OJMaakA=N_un{0 zdVB+H7+x-#`OLa>#XuGJwRAOFrYrVlCWZP@!*nrRz*?hhF+1k(Y?;7@|uR zAy1ndvPj-Vom%I$zuuLYA(>+0K^5b1SH>*I)R@uSl7bH7xz~cKY~2+w(QNQ~+=C>F z8h(argYNWVj1ejA*cXKLX6F`3CHVpHPyqjnghl!8Vfs?V@0DMftpOfYFpFiHLkW8p zcS+7Cehr*PE{)Bn2a-~&eGnwV1p#|ScL!tHym=VdIJY}rPD(lcZ7HLZ4l5h*Egmrh z+!1qK8kMdpp48OfW%oy?Xo)3s&O7=L)*&NvvCzKh2R#`EZBbunp)`iO;=ZwTDbbpA z#yiH4+FQyH3GTzQlt7~fjAM`|JX#q4B?}7W+Gvm4KF{S9qmda`sfLAftmiEXHEocl z_REbNlE?~Gozi){G!lBJknQ6ye7J0#rXvGLGI!*;SvYwrx39ikywnk*B-{9Wl^;8FSb z&l9u}Xn}1&*ggJY^R!KPX?$LeDraF!Sg0b2vO)34d*+a4(CEqb_%geR$9PYy|2kudssBy#aSVOhDcV`)m+$WG9}@&5>jg>0>@Eu4Py;{VbRS1VatF7dFHR~gM?!w)Qg*Ubb-=t}4GaA-`d*R+PQ(sCRr&?UKsECRO89e4kvnO{zh&EwIC8~a)M{y?*Qz0JA ze%PDE=n^**^SjgIC3FwzF}(f=1Ew(<2Sk2 zNrpA4Ckuga-vGi1H`YU*_*W{s8l^H`_pF(857;R~Ne%%+<(3$KO;$inB65>cDTNas z7K9KUGn@3%A{muYw6!8aIaN5xZZ0lW&&+x_l`wGqo@IGYA@Q?X(((i6ez%-?(=7J&BaqdIz(6c6YQlk*zv$)OkQ3kzQuZ4ycl;Z${*H z51M{-TJzrkqebpN?J5I^Z@K>&LH%)59TvaQ^Yz=ZGyG=}ByZs0^xv@Aw5pUNmI(a! z%F1BED&5{%{dnf?m?f7KE@=q5P@NI+L1-9usF^;s%%O9&%kbZ1%H1&p8*Gkyusq0M zNHCv2X5SEOu*f`(R9c$RN-c7y^o!ezdUCxb&A#6sA9#Ok4C$hwR}|6vKMXa{CPUO& zhZ{&_p)9=ozyfHVjj-TAnu*H90^a52@gUBP>b(+%N+rBfhKyhaov$^f{_W{*J7kBx zp}=fpg!r4RmVZp$xu$LJ+5_{+49Z8l_Z%hK)HjV*N)JXmuU4j2jfAniRFyU*rTvyY z!xRg)3&q(f8YVu!tz~-M7oAuYw-^qSOwUQy%2hUvWXrMNrnq#fDP4&nvd4@AMG>z# zOLI`qxtsD~(`mL1MvnxqC~cxk2cMhL%_chWrmMz)x~+ImtHqtXH1pjmRhgpWS=n|)&*>VMuQLw=;PX^)0QIlYJ>ZR+n!~Mo3m9#^}(%O zVV|*alLi_?np!jFh|c1b%5#6g>5uxlvn9Z_)Mk}Ku7q+7@NmpK1)&lddqQ8L7+ZHC zLCa-Ids)cQ?X;+Ks^`7-BP`17KZB*Yq0oV}UNZ&NlNd)ee@a|Z`?^LUmMspGzru~q zZBHuIM;m-U5AWEG!E$QD(!n3Wr_}Aq*}TKd>BLIIg`gAU>B><__5+8>N=MY$R^?!m zVQu7$TC~m3&K`5$1+y*Mr#Wn2LAreC9O)ifvYJX{iP2q{@!i3f<&w+EuPY=(IJBUH zSF*}3m+`F9YE{m%EZM@Fp@CP=k&xbx=2B?eA!_@TDQ>(#)h8Z0?D>OzC!D186 zXKg`~oHctDG1Mi_UH~aWH8F^&PF+!s5=qW>@Yh{)K7?hF0Ttlq0eQOpDlklBX;{c_ z2D~jUFYe=|ph<+2ERpxQEAO0cpWPY?QM{6oR-`a=&de3aibFyO@gc+l0V|n&%k+7f z$lY>h&>5b=p`T^vIJ=DAe8&8&bFmUp`_S`7j}~)*_6bVQ;WJ^M6JK6!df+dSd$P^( zA1LE5VbY1&Y9u@&7tP>uC`Bq{>HZiA(}*u@R}veMBeru<>7*V}?W7*j;p#{XMuh)_ zx6;I!<9~N61aHvZGDT7AC*qcI(!KhRaTxGZ`eV>vf~KAfVi(SkNNus`s__G!=)y4p zGfHWe$U~eQjs7532_vYudHnDAFlVUJU8KDk5sWk$MK*IM+Ob8#r`-vN<6#n#K{%;N z;Ozcd26O%E$k`wi?J^DSU9y$pK@y#F;aI%|CuZgN+H&H1L}!Zn#JC;8a+z7zprr-7 zqms8+l6JnxQ^~t*$-DXFYpMY^l>I;ob~+HcLt2n_)R5nz5W0&0wy;2aD}j5XT?cY# z=4dB!;lDV&)=VuZzGCQiq@axaR|KF)2GAVV-~}0-Z$+;Wo@i4BwKjCHV>(^~2ZsB` zAjhw%qK67aeZ2DHRXoVI$0dcMKZ~SQW&a-i{IALf1n`#g@0VXY=KmjSNB-L=tzcIwwM6MOb^% z)HYuceJY~qjwZ{Lg<>^56vT=bPwTTdSCz{BM_5bx0eT3d+3u1pyioLHVi%ozL{BWO z<%eeT#2TY>$FZEQ7v4CrrgmvKZAmV-Sks0#!BbtZZgz14x^i%_PJ>hXTs7U*FkkL$ z{&Gx?d~x&R1gQQqI>34(4d_abTbG{JI<;qg;vAjm6%(2%o^WUV_V(#X*5gl{eDmSI zp)DF!&HkAKb?zFQ@D=Vw(|8*OenYIVD#qnoc`IgBACC+RNmV#=pwtm4&OQBmv{ z*+}C8zs%Y zj~qj5xaZZJv!U8CI_j;v?^gDV(&dpw;ueS{v;L&`mh#}10m`kvGkbI7+%XHxos-Nt zp<8>6{48evW~4buW^POA{d>^6uT}nNxgWT*_G0R*3qxua{{Xk9i6PNOYVDM6({4+ekSAXAo z{<*09&Gzj|wPSnqQ`ay1El2e;HNjVEqo@9ss`^bw`JFlPJv#AYdGynGUt9HQsd)o{ z^%H_e{3o>PQ!Arphm@bba#xk0tZZ*#cudnr5!(zB!VWA# zMN&u`Cf<@_z#x&8t`9HYSEoeDJB+j#gDZfSpolnCkX}YS5AfKdP>_UEgf%4PG2NRO zwtq~M$-GUVjVayPQTpK!{>hB{-UmLnBjk~7fFxTGof^hLcrX_r(;D^~aPwEm67-%7 z;~{xK+KCKas{ggS3(X>+gh-{#R1d-;L)8*#rcHs5DHD^0mpZGD>R8>C4cDwJD@>Oy zOgU0jkn4r%FmH-LS^8HLnXFnbiB@U7?^G{}W{FY6rc7IDSyQI_Pg|*_DnVdo;i)FN zMzOZCvb%}*^u?)pc3AePr-pxS8AW^V{<*Ebr^!XZdn&IgLWKor*LXRVj?R)?58I+4 zg2_%_Lr2F%M#RzENJZsI5)=?%K8x4a)1q&a)zkY-QO~n;GuAQ2W_#i2{IO}N{Zl}0 z?%+aD858JQ71W9)ue6L{Lb3x<1D$mm<;~WS+y)*%A1DvydWf6hS_*8g;T9A zP@qK9<@Y6x&hh!ko6-f(l}?R*EZ%A-Yz)4$FjI>nLo1U~qH%$(ZaoV~TfIN-1FbD!+|!1+MoH}~%w2vxmK z^^LM)GN&h8XGre-Xk#VfLBY z^)u>uPyep_anoVR>%>nIWz1h0>Q`=U?O&%TMlqgio;-kIY5T)K$6)zLER{pYpI^d; zq>5@W7!?J=2cov~vRIm?Zf;}dHE2E|?D@Y>VcLOtcEn#pMrb>wRHw-K-Z20}VzQCu5+CIzpsgitG3S<3S4?qhQB za+D`gfn=Dx?XNMKnmi>S!fh)(daC|&3qM7<=w)JX7`1DwXuJu%iFr$ukr_|hOeHe4 zt&jtDq8lz^(K`*m7TLz#6T^c#U_=!O5n(K?osy=V-k1Vf^=aV+p#Nb@XP~4xCHJ=p z?GLE(1#}6?1|jlxi{enh^y0sRR7^$27||=}5>ht6Hxc{VLuN~?8pZj6<3#nch<_!j za$VQXogLC6S$<u{7w= z6`TZa$Njy3K`=%S)V0*)%yjB5l-c4*OM|@#edVN_NJo6Uj0cjjixX}sK;0OG#2ccpd)JT^4AGcMOP6>&?pO7rK&Y?|Js}GUQj$;6QrxYUmBiJ{MN^=*8z{8rpyzWAmS^>zJ2&J#zB(NoyMo}lqvyGX%Nb8T~zBI2Vb=c=J*nT;6zfOPfz zM>+J_R5avam!+0;b@JD7=#f3-#ljd(JM8%WV;(Hi&4baq`Fp zJ_k9_XjhtYBl??hzi&6G$cvVR5!FMwWH3NB36CbVSn=z4^d*ftQ-Vk;o*$JQXDrGUKt|Km zjTIvucy7A|4zRs$^uRjVz*(?e0yUY~f`NNrooAaIW8yl5C&n@~skltASbKr7YP_2A z#Ut(|W{H^Cs6d^^js>M?VX>pT2Gun*HVsOr zcg1h~R+R&=Q{rPE4tNp;I64aE49GVUySBAJHnliO^(WA1tvL(RGWgyf=7oD$Mgx`$9SpEtnyO= zPL#E0K32duB%y}ZwXVtDouydz%Z3iKFPiarC0GpYV!qLwg{cogN`lr~D^~4q-n)&f zlrTw}&EHi}*9$|nJRu8v9TJL-xe`2Dqy^SgWoW|;3?vu)kNqI`?3!sH8dg@VZeF!Y z450B-qP2W_!EThxKFWjwDNXPre`<8d`J$)WBmrFaCM_1l99W1k)+o`5x9zJnq8tan zLKl=yswRwIk<~P=1MkKKI`P@w>g&0uCT9{&gp8Sq%S+XR2O!Smw%L>0;e24SXewNK zRD7L@sZss$G)*)l>PV+EyAp23iVsTP5zMSkvJpnhV zY$k}Eq2>feXyeMx)11*>e6W^+Se75PjBv8MSXd%0>*+vcq^yq9*+6NeO}1WXBnfm3 zp-Rny?%o`qY&}@4xH}2nJ@G_HzG`bijY1Vq#qjd-{J_XQhfM21Ubf-QA|0LXc!aX3W|-0v_a>kcw%LTd^s-_TBcbA8ld0w=R~bYE#oP48;x?gL1_j_eyKmL{dTDA=wwx)-bL?t`tR9!q~%DGOkg2 zRGF4y?Q7a+_pj77gK`CK^yDcrSu(7JU%CldA+28Qg9=%paXHN!hGAJ0WDZT2g(gf2 z(A#0p=+AA~Fbq}imv)$&#%1D<;@LdCN>*ADRWskRGHVNMXbNq`F!$@aqVLPvhtpm& z4s8qPMPjh(jm3F&@8br_m=#0Eh&f?f(;64AoW+3tpoU7z6+&})bl^CZ>_2qYp&TeH zukuo7`^>akmR{`Z3T=^Hvkd|53p6!{yr#dl_s5!AVoZg>8*^%SB@0+WFs(kyruquy zl?Jy#(J*ML8yCWaxQTOH)jil@vCrvdP0uh4$=-q6_%ynAiBowj!FLLaqj`s{O_wsV zx91OD@sFI{HgguZJBO0(nmZFAj-6bqhE`TCBb?qoxoP3(^yC?3Z_e-EFL=Pziy@q& zg{)E08qj-$-TwKCJO1tMLfoXeW4PgU@>UydI3*G_51kgD#F)b&>0WJF0e- ztPV?sxM_41KsB{B#uB$yMykQwQQcVzeg2B@!d8baLSC!g2H|agq5S<9^rIx!%op^P zrkeB~eHlvxAo?M^A+3)^A+ z{BGtUw*5R>Sd4wG`C3c(RucrYfw{Ccjn%e%2>Zr-iwu!fMQ9l`v;toj#)dpolQ@>R zY8V=oMhs0%N|0&zT6{h8vyha0lY)NMCTg0@7@&he8tuYs^M?;ki)K`@Ik8>}SZLX$ zd!l{_73*n)Y5(lZcnd(#k1%g5a}MH#XVCPzzpic@*z7RL;!5l|r9@-R;F*wxtDLM7 z>DKm7z+14Xg_}1*x8_b#A&1fnj6JmDNeIs6mhB-arAI_VkeA1H?q4=)7pS35+CZ>! zX#1Yh^@Yu)n->J$d-U7#`wONlvLJDtRemLDvIHV&()`_ z_<41fm-wN$og0NbVQsL*%w>PgU97A6|LNNjQe++oj$%8GgaMMi|78Y$bNa&{oSYwx z`Kq`bVA$F>1vybo1r~Aramu#HH!D z&u(HaW}dX*n99{zqC>y#47}hRMoh;cToz2n(k?41t(em|E7D~DMGD$FfpvYZ#9Zmj zp1Ol+y2dmKvD(kSk-H$Bp-ja3>h$bBYx%&XNwhH|BEXurcWAy`HUF5xMSrYnV%Q;! z+Due%4^RSmi|q4&2pYIV8#PPnCM(A5fB2Iey0k+COY z7?&u|l$f={CX?f|RET-PiJkt~#LgiHS&le3HGc}aDN?jgvLZ>}d991{s~1$UHqH$B z0^fi`(?AR;EO{B-HZV6zf+y!<@WF_d+QNIznvpflBmNP9{Dp_tf^S<(d!kzkJVRYV z^QTTAIGNf-rDDIZg+Q$59{2t0*4&n6W`1*`J4vkY#T<7L(rTfU9<32c?UG`=-#7sf z2}NTOd7OSzsyRY4%%O!0_CjdFK?xYQc~RHl(-fwAP%y8sHjacSWTA;UcRU>@3pSfi zTN^Zu+@QEVqxvn*P3l?3>|954V;oTe?>#!S$ot&HktDhwNsmQ|C3~GYFq#9pwB&vV zyk=9sS`sL`_c-?1kcc!4j!!WohVf9Tz#J{!F+Hq`q$0NJcJutvQ`3q|r^*C=2u;+Q z@nId)#34xF`1m0)Xxr{M>0)A`iS`I3fnZ>x<)FK6IBXO;a_k{?O254XVGVk|WCg1e z>pVs{{cy&i?K{pmhAgDY_G!eG_3U2ouL4utP4>DMhM-pQ+;Y)t@wSe`g}_;FWwOo{rQGC9{5rVNsk#C)Cjhq zRg)~rv@O&k=wS;)gvwjX7f5Zb{D+&48XB}SwdKpPzF&DHFx+a{XX&?KJjZ+T|?ME=_xs4uaY7$PZZKx zvLtoUDe!H->X)o5)eWu{jWXF3S_$pWnF88+J&?T;x1wnAB5lwc#A^0oZ`IgHSk?n^ z#Ot~Y*(#H7Ae{WY8s(~z8sko)3>#%YWaC=o`W(1S5GC>RW;^ zEjbR7VJ+~*D?&oNOk=eUzTGDfL<)0fswa9+#71z-xzfLFxF}==jVVdX;kteZVZedP z!$}MA+?4tJB65%Aw!@sxrOCV*;HgBqQ5rBqDi+Ft6o9A>IPhv_4W#b!E}1rhBo;Yf zvB|C+fRMop2>${VpvoC-myU-l_#Wsv-jF*0B~alEOoH}8$Taz;WUBXQHOV9vUp*HG zM8u9f`mA;J#%R#n=nFJ`s;(1kLLypZdG0quA;86@n+KR>&j2_IB|Rk~;IQSuihNq9 za#Gyqo!J>U?j#O@Okk9!$#r_^DQ@>Z^P$5bBAiIg zekA-w5-dzgO$2gl8`}@V2h1qgpN*@ zb6jF*;sR1q0HP8Q!|h0DXuj}?RWTQV ztfHmJeVmyd=;pf!(($e07JQY%Yz~U1Lsq!z$pau5K%o$4c~$K3R<%1$ z3V`P6*R3d6p1`l`(@eKsE|(yce{?luf!R20=$XeOB!?!s$XW6}5{WEvTFUxXBjJE( zjQrNH?&0kDP<%;{%u1jJ4vEo1M3?6&txrj`*&gV!F;fhhC3|y5l~`^@O^(DZW<#yI zU|M7}_+QT5JXVxf^`sP}UI{j&87daZA(ugv)YC^I2P;;(_f>KeKGDo9@db~xBU+)8 zx5^6w`hi__={oIR?pOE6Ce2ionUxb2sj8~_cHGP`V1Q{{pT+M2+o88h^Hj-Q4hv(; z3PX~@1(kO9FXCO_>{mpyQedZ9K_BoEOBsy%i5`^=jhPO@E+TsoBOS%dM;e1=z zD)QN{!>__X1)wBK9-KZM3umHOF{qOvB_w}Kpbnk?z8kmBpp{uy%Iob>SYSq1Pa^7O zH#d6@iorpTPz?0 zV<=&ZB;|Ks{2iV?pQJ1h zbq2mAxja+=OxzQuL~v2sp%G@1hH|OgVy6oE5O-uuU%36xPu0*E(;~1#mc2r|vT+nbpU`Ml+l8?S#~g(0?kbKVOfdQwab zlLWKPjS;AKpqG`T?9XIB%JiSHbaagZk&Q#HP)$;5AFS{|5&UJ!wAa}0Eu6m(FSA4% zZf5|@hz`;>)Ai@n} zzW^tlMnk?>@Zh@6b((;yu|n(>>kv37$)?+EQ;gzwqZkdLJ7n7#9}2Rye}Zp6y*eTf z)Dc_L;!t{}~IZx1!`iMW^b+4E?C%r78LT6J-$0@N}e37qOzj@Jtu({zFwIXI88}lV3|= zoU3uq7RqZC_q2ZLcF8A}BG=c8WOPnuz(N zR;0+_40ftXXTG+ss@?>)V5esSMg)A;NnnI#d>ve+65n#kJ64`=ziD79A!`GE?;_*_ zE@~YbUNL*>O+P{%tUpB)r&@Le=t-vzWe@o=7*?#W@|p#7_Ua@ASvB;f0aK_C6BWps z27E=2!%dI3B1B}>mw{FTg0}^1)^823A8DtYuSugdRS_Pdt~nqJLMkP-&R8VRHD$t5 zg?dskxPp=Q6o?cwBfGfnki85vXvnvRXuYp)47Mko&0=1KurtqcjUj%5QYA5u!Ep47 z$6rIJEouNhLZ~qstncMyfYFBWCzC&PIu6K8DoCu{03pZ`VJ{gxBMzz0VgFbQCH+R) zv1k5o4|di?*=ZJ1f#b1wKvt+@4xEJ@z>KTGR97LovCXZhU0;$_TWke6Kf_@OmX>hFc~b;8bW{(DB>&S zXE)~2ZbzEMsAvgJB|9N^HIK~(Q^ML@IMB&v%;XS_U#4hJ{&bbd^k^<1R8c`6Dg{IX zD(X`|v8H~s=1UoRKq_*mz5`vAqE70;Y!VNBaz8G6ioa5$rDvWWfhL)v1!iDr-kilc zlt|?zD#eJm-yg|KR4>_^U}>`n-L(3NoCPqZ+{-K9lqLY@+~DnEeY{n$#%zke4Z*KQ zI{=BN&IH|j|1Oho0yw2t{X zYStBGvbunvJ8f2*;HPW}1E^1fol1$P0Xv_@L*sL(6u!Uw-czCtV7)lkgx^_SN}--W z^K9n!TgS0=~pclj>A2`c16{87;b^!SJvC+}h>? zI<-icWs1EVFh0J{WG++r`4H<1^F05dURY0ALhosrchwu)CKhW9JG)!eOmmC?7gz^c zNSZ$`8hd6Vo`+rz^^VP!y2z&+lLm4@7?AF7>5llBDAFEs{A@5x+#y2?UGENQ>n09a zmlCY2YyGLtcM|9Dug0IzD`9tFQ??0(++Q&qu3{lkP!;JBaZ}k$Mo&7VJ`Wt0g}SF^ zbXkTKS1Ue;r;e|}Tt<_Um-_RG;Cf@Necx$5{T=0X$E!RLDi4R(k8ump)K@wO;46gt z&Q)>wWryJfy4rs-Hmpb3`VGeMfd2^yE73p$Ml~>&h$(|#izW$su*MeajrTSj03Ty^ zVkCi|WpN33h|LJ*Nwi^6JI;Y4Pv4GC1!l2BZrhaZk#6_7ZqtX?cWY#Z(!tPOSp`^S^6dbnrZ zgvA<6Ckt?;kX;95HAKH=XFb}Cx=8aeeAYTx4Sdxm6b*aca=L89PB+-bT!P6Olk_-m zhNNQCW_J3sljpI`=^79&tW5XU&00L$%I2a_oqtK4mP;TTaJKr2&|EQi{3T1gB)gtz3|%P~uo*`qaT+y}bmbwd27)7_1&2?tYYNZg zP-m1R;*B2@7oVdg=_ahvW%+XyJa8stxxQTErZC51=bdN|G4>4^=P!%;I~5vU-iTtWoqp&%o| z3%nN}@JXSLit8hrSvP$`GaMB5(r+PnhJSpHNQ#Xy zTfD~L!t`3nB%%n3ss9Z>V zC#>TN&ZUiGO=f*j=9hh8G5Rd%fmYeVIuB>3+W8v=Q?Gx8tpPLyD=T)82509D)fg2! zI8^bzBBi zb$n;-+y^4B!@}t<@a0F`wlpCHhUYfood%qWNV~b%Y>MY3vg1?37?z!k+8x-@b*202 z#;o$^C3x))oO{T~GA9iP&Pa0DHp#{q{uWIIvmQj2f#8HZwB6&MKNIV*d6?c&m@W)f z{MFd{d?D4BjBS7ytAt1hQq2ZGF!bSHQD4|>#R%JNp=*O}OUvR5*<+}jRl!JcDzwGZ zgLy<%e6@+2W5#C)Te!4RvHPr$8XLuHfKmjkNGg2RkvP;l{BN!Ls~UKNagG%;E<|U& z1FQ8Z4sajA5)T|<;0|!w)G#56Wdv>79;*{7xq8U+QmFSo#kv{7BuaF}$dtC3%5n@A zxG*>*79@C-@y{+^5^MjC$yqcBVXh5JO;sa@ZQ1~_%*BP3$Q_+%{)S)BA9nzdfT197 z0yK}kMPI54pR%3`#1JFWjj&9KI0`4E0xC)Kg+Lz+BRJ?%9|evhjw7U!B(V7tn8qgk zk&*O6ICFMgGRFy^SO_FZ9dT$H@AbU9`TgH#lU>%B2j|_lNQY}wsRTzgkp0V^yO^LX zB&25%QqmJT7fTTo+U+q}1J=qNx+%|nFaehfmct5$Q3(ZZa2{elnRqR7 z=`FpHUbVGK((Bv|`~LD_lFmI;l*fOPA55{_;rn&OpG|N!L_MXet%aTXJ@`LfKolv< zudSji|L*LJJCoR6xjPi;2-SLk6YcNMF4f3ydqYSrS=!s)*uRXM=z0Zlq$jyq8m5#K zPQFEHca_H*wNrG3deFdI1G-~~n#MpW(YwKgx-W)ZoBUPz-}C~`H6(jrQn>=ZQe^Z; z@d18eYDz}+f1?T*ZE$E}&Ne1dGu=#8jyI4OTb(l5o(8hE>LU0Zo`hQS-*`?n{jGRI+@Ts&@@@c+OpU^c6L`W?=%4MDsBFh6~!)+CU?=C zKrFmkh`3-WT5+L^JTs&0i(2Tx3w@9cHu*-J8Yn%2KcH%oRBne*+pcDV{w|{>2PF3D zd!w){%j##Gf^6h=faDzMC2ZtUfm=!;Cg7d|d=H^Q-p2Cn62`6QfuK6~(sAsnGWORE z+H2769daBsWvm=>3R}uD??|SEaPDzCFm47JpV1x(`20zq+1NwBaC%0+!*lLV>`*=c zTk7fdM4$26GQZG#275wU>1#NL-qVwTGKX1MW3|*#Je*J*P`aZlbZd%d8!hOV$**>eWSZnb@&>GqJ`ygqe?O7Dw5)pch}AC|p9 zZBLlo*>a~_Z#lfE`22S!PwsiVaD78^r}+91KHdPoS?G|W=#VjMk&)bz$%l84-V_xk zErtEq!lJ!`*)!R~BVQ@Y(Rzaz%mHPMdWAyAMx^9@zj1L&op>>U4D3|(txZmwh2kLG0#{ufZUogKqYV{d0 zPtq6g*0nQKLuLc{XKVYOX^KSNyAukl0wQv&g(cQcqy@HnVzb&# zvO9Q&Yj$9QfgTs241jJ6D0E1XkkopGXK*Iv|7BU>YqTcNx(Ta}&(37v2Be-89hGi_ zyLkTPmHeS1w=NHmtWAJiaO}u#vASF32w?@=wyjJeH}m0S-G*&MwnM^NkbmOnC2R~d ze6w7DlPt4asXil5pB*#c3tk6Oz00io>qgU+wL1806-N;k&EtcG`y%SLm5IP5J_Boj zH81aSA>=}HzFif_Ab#T_zK0Vx%|jnj8w%2elwyZhb-X<{H5q^dUiiELmX$%SAqCDsyzSaf91ca0HU@;OTF5$< z6sf1tc;OnBl&{)lbTJ&UXKCG$72gL~gDb1n(04e=@ISPGCwS_)b+cDrK577=V_!Zs z?y7*1*~1_8_ORoNQHYSMpdX`FA@E$9(@-P==dqbTJADH{W3TlEw<7Fd6cg}QR@wuk z%rq@HQ*?`MNr!znKRVz(v3$N|{i9lrO=&^2?!og(HjwT?A7RVRojDJqS*;Y?g;FKU z<&7xCknV?aGs?(Su5qRRwn2PjP)pa8a@PPTx^LZkMr?zF)u&glfyuJ%Hjv?Z(T;~- z!mr{Ve6k+^Qd|Z?V+X{DVe!OhB6*zi2s-MQiH zn=rZhv3#MGNvs8B!ZfcG2r2qk>0Z9CvF1VKXJ?UUj{5b0n`Wa zfZ2XHoE}1VvA@Bsjym`LDnG71==DW@V!X!jL4CYA-qQ8N0{Hry+@`Jze?wC3)i!8; zb6W2Anqa*$T~hzfLt{IMh>w37`h8w`KN7y67}X)B%zi`%Gs0m>;swcZb`R_rob@co=hptxV5IgLt#7p!mIqfNgj%e$uaaMAK$r;lkSrN zRjfd{{1ht&FR&aa$S@pa=RuNaIx9dNdt2e1!=qkau~OV(LI#mN>j zjIo>cr5nG-6@|?FL=uJm?0NBPqXHIJ2}7ijKEVGTH_P*5q73yQ(4hXDB`lleo98Nw zaQ{p#;QsX%Ojjx`AKOt(-&cnIBNfV$CWV)Liaj8aFg7|2szObD;z7~&rco8!+j{f5S~bn?`ANxZ|gg>r&p zt<&*Zqq&l}+I-5Iqsw4FJhmWsuS(`?thnK;P{1RN^{AF)I(1TLOKd=}3rVI6$Vr_R zdEv8L5owud@YuJp<;0h4DR0L13fNb$(_g8JGb2A$_Ed(A6c%x!5thExbOO?h%k)#$ zTgZY`O~e9A+TTodoZukSgzS10qTg*;4WIb-$(_X__l_APog-M{+sgAXE}*WyUpJOC zLVZ%Z|fcCANhQco8V7*33ETeN;5y4Zv&cV1Dfy%}#&ose@hdLxT^Q=rRKA6w=tH-*Ix zkx`?yShSkE3YE7geJF3j6)U$!3DS7Bnm~V1CXNfvg_AxAF{k6@kAFgOrsd`ZzNwM# z<`%SnVkzH#nbY$9a^z3UDSQ1I-%{-ee%LZ5fBHo}soJmugRDQX+QRXIxaiB)hClWo z9xro{pmh-d{~=?3N~o(>h?^A?24>TV)yVB#do&7M!itJ{f+uriJ<&Pjr#B)%+jktz1O|3 z{2ix0f``DuO^o@vSP|LC%qsLL>=;4S>|o!lN>Vd_KIFV1SudkSWV0}Xz-Gx7q1B`%EZ!;f zXyF>U+2lENwIOTY`kZmP{7rbh%!|Ue_#FB1+roi0*$xSz65zHJMTk)s3Xko>caCrC zs|f*Q0VowSRW~X|xl;sZPG8pPW@FcX%!3$Sx{MjD_Cv3yOn*mT14WOCl2N2OQXd)Q z(MLjErbZHslsD+X+J=z2hZ1X9`NL=rw!ZBvIl*YQgGgE27GuZLjP#8n>c*sj+KRjK zsx|e&<^)`O<)onH$Z&ySI|#=dha@7?+9i&(&s3iRUm@mBAUk6I7h~_(oLShd?S>uO zwr$($*tTtUY}>Zev2EM9W83Jwd1~#dz1MncSFJB|{(x&%U1N?h&f{>J_2xy>B?a|7 zrStrk9Gm^v!TV=I(9bJja}IAp6n+9=x#drL#Wt1}54SY8NOs72m4`WAM)GXA!8b>? zYDgkMdziNk}vR?%q6nHck=5Qp-^Seu$l%Z3cs`f*UaUYw6S zx2!R=em3}S6xb=fMObYgpI85+=#(%fZsiU0EJ(N5nWtqgz@&7l&l_H-WoC53>5QX3 zwam-4Mq^V8)pr{-7G#8J|_#} zWe6|Z6(Jp~BvP8K2jjgFLScHD0xu95OhaM40zcP#zZ~P&kljaJ5y_R-aAh8gNM6c25HWMo9wp_+dctDFBm|t+&l<5^N(J=Q zaE%TuaGv`ypUBf<5QGl+CF#iX0OdxO7Urvxm=bB1hVa0Z(*+ykBQ03JVbCto|HhSz z9Sd<~5&p?qW4d<-C02oVy>ErmI=cDQ6plkYo`yX(yk8Yh?gxIh&%vnG1%EJ1U{L4* za~v+v&wEFdP0SzYbVsieE+B%pf>Xakx)YSL?`tva1+l`hXRqLc%VmoTr(AqISP%a) zP84TE!+2?ZG&=UP#GXls0PrIISTa7Zb~%m5mc(m13OXX35}B!6cq{hGk*}3lX&t!? zOpymDPk->E9I+c)q)d`unI#bbQ#9a2wIhUW&K|ki>-DXSBVqF>XOHWYtpT(+`*LNa zjs#8+A`z68hCOUU*)LE({%3#;pd2cH{Nrh8rUwGz{@(>(|F7uuf5{q}wP1Wyeq;?j zQ>5<4Gi1R)0RU5fAR>9l@Juk$U_udm$Or?$fUJy369ZJ!g3p+ACyznRkN90H_=t0^y)ERj3;s{cwD%9T((#zu_-CPNCbZUDr& zgbJuZl=5I&xyAP;^okyl(v6;#19~aP_pA-fx<^>N$^cDh{d|WCX&BFzs1tv-T$)T~ zgxQ0)Le2bJVFbE;=n$IWR%u%GDozKunTck6`V+B2Oxn z`JV%GGH9)X|2WO94eRH;hk-M1a*RpfR!aNWp%vS+{lef^=)kYVHrIK4n>8%x=h0y+ zq0RS6dh7Kul`Zrb2x}BTj*RC(n3IvR@+KNx1(p26`}1?@ zl0~3CbJ}TS?XL0AK>?5hdPeX^Qh|f5He~{srn@;M=6n(-sIuma(}5(vx`B)#wsR*E zg3<14wC|M{EFp&3>>A-;3Eer&$lyepblf8t4koUwBH#!D>9fGTDs8B4r@9Q$T5gfjBe zQA8d)LmDpCoM;hr-hvhsMmFCNkUo}Ts)-QJ<0j^vZkCsX3rl7}I=wArte;4wU`Mj; zoZrBH!uSVWu+)bScY#Q6IFH!p_d2Y-+vN0q-|ei*?hrH9=-C2FXVzYtZ-a>!zh}=V zcElm>XybZc)9GjH;SCyaV8v|g0U+avuM_sqg59wWsxQ)-9Ce^3MUXqE8_q{#P(%nK z?G;nd*h0fzC=Lh3r((=cTWP+@ucg&>`+E)uPEHV%Ecjob@?UCiE~!Xd?JrArOnt0xYv!V1jPanbj4H zdDS$Zx9zp_n>%lqnUrD2CYTx3-Q0yhr*U8l@au$8Ynrm^|G?YoqyLhgoXQa|hf_Pe zomy^mM5QvynwY_}@C(oI5xl%SMnKcVQ$s0|Xy}&d-zV%MON}Iek(;k-n>u0J$WNrd z^m9L&sAz;Q6^p1`MwGq1lRt$2cxQG$oN~;8(Sua z_yw~87i zaYCSZ4iFQ2c7;pM{dyB5lj-!DAjb{!Y|x`h?L2CJSC{;f88T3M6W)x918-F6Ys3CB ztm;#D3mp(ILo(iA^P>1ZCDOc#ejfcA$&9BIk`_-!WfKm?#w!wvF>;9!r#SPB`8^MV zy)DRLwI$KohyNdWLy`X61*1xg@xFJ=h@<#)!ctq_i&arSRZhR{lY0eFy|4hT_lZuBYvp#k7_(b`fSFc zOGWMcsRa8^qCm-A*_&C2{roG7k6_W}sas5Le$bC!qLs?0bHx2NN7j18rS_-T>Hjv* z@G%}uO+p7Bn6>=$?2!R5Som=427OFgT*X#A>Ws&n1sWpibJ~DCF+vv?Cxjy6`XpFt zWs1t>?V&vB#@LztC=rHUtb7P|9*n*^#?(@zZ=;+#Mk81~qP$m(=pWTtJ~F+x4d}1FO8Mdb<1aVz$^_OIOvfR=+J#rsfAqrIe;LSyGuw!Y zI%P1b@E;%B^l^V}eB}zh`T8jW)LH)l3xr3bEGBicolOGwVBz)+jAet8+#hb*A5R$>lyK(lPEU`NO*Y z_6a#xdV26&Lj1!O9xLStmFNOLD)Mwk3Ar)pw8D9CJ^9`Xg4T)~j+PXJbYW%tywx2~ ziOy6u!?IPr#5hV_8Os4KDig5iaA4tNAs&y%54BZWt7I{J{XhzSSvBipU;9PJBk9;C zm`{bvKM0yi=SDO}cJjRypj&XMTjV!H08=ByW&i7Mp9sjfQgC(?Mq2a%PE3yB8mjCE zG^|+D;I`u{prqLctX5HP=2FkS%*^QGJVxSW;vd*m-@Z-wj8-j? z`z2f=NRMxx=xR%`#{njZg&1P}SJttkj#@wT1F&(;ez(?;<;64oG5fjqJX7sinmGIV zh$A>%+-@~4GV-tw4s4b#FHdpv3mN~2S&QwJb0WT39{_-#Nr1dET*AI*ZVB( z%iGNHrZdaqrGzP33@I}(o%6W-&;Su)mx)W)26Y2v*=@@2EJg(8I-d(H)6_TT{8nD$ zS}yFCXf5h~&1)24CyIA(WVI&h20q@~=vgNBbcAHv9tYX+7*lH(0a@5cS5R`aoy%Pz z8$C>1NLg7r>?5WXv8!hcC z;UkUsO{<{*ps+-9myG$Dj331XtF4XR#U`oW$a6Z1wU4clir@DR@z7OamS{4#vx-DH#ZKH}u8)Dm+J^h3 zJcI;^-8^TF`8tvhXU^%9piSa=quHQ3+hY3l0IM!K^#fCF(zZ+kN4A~ZDs`bNwz)T? zw)Z=%_KwZas!lTXmpP0(G#x&F-?!HpPNCuL6xC4+qwdCcFuC*4V{-2t5tV(y0>Q9i zy>NzkoWp`Y1!s?SiT6s*e=_f3-;uXzTv>vyMHu9tuZuq#Rz*+*?}X-jAR9}Nv;6;X z!hM_eWhv+bFOevncs^3{nQ3ku`1=IH7eZWnk~6k3?rL*M#y(dj(cn&IC*`mZ6kJ!1 z&vd+hk|etambwrWRSfR{XC6;Wum~=OS%~9%J~UXG`3AHa#JB0)QSSr-XM%(4_(SRw z3V^CWQv&aGjgLdf6QCwKNJ~wHp+@`*DxJ%!f7Vw7{i3KT)K7}@li7YmKhuC&xG4^1 z4;=k!T`hb+TfE6Mss5P-%UVbi3d^(WL|Q zZw}E>G70D6=;42NO3H%^?B=;bH}Toyo5gbxFXJ_Ht+Z0S5V7(L52>>ZgVB4UES_|= zxcQa1n)R7T)2vZ{sdE?Qa6Q>L10bC3vmC1inNPjUQ*aR!Tzz{u{)Ag$&balSE;F|< zAgA()Mv3*pAIJPdZoH{{rUq?l+PQM=l|AUPW7wYyd6{39u*guyDvY}KFO!wkM!`6V zunlsN>W<9ObM-$*P`X00LX3(5*&sO%DA~C{EiXf3z?eBk?nbj5Q}JT` zabSXh=#7#d-OR&J8DYu8*?_WzsH+cLOM{S;ugaKn9D5#FDWt_GNhi`o9>=b(ZUz?; zpeo41xRSzi?~YqE8IK%lL8lyAFfEOvg$FxLWpvU+t-M<2odaL7L|w20e2*u6@fTzZ zNniALPdap4E_8c+kSp5#}szX#QgIkbmQ-fKHWP@PB}7GX8Da zyw!1}h^+v^(FSU@r)3i_rs%L{E}LaV(FANYA?-mQeq#+pZvHCo-fl25TOOCNa57z< z>Zr_^!EbdOT(sPkQ%;D@V-_ireJ1s0=m&Ew ziI31KMyE0HB^+2b^h+y6Z9m;rG7SB}p;xLvT07v+@MBi9qfr41_cn!7KHj8W(Tus* z`{w(w;C(SiaZ1=hw~^^Gwu1}`!DD!HIWrLci5U|}gpCV??fT%T zkWOg@h#Fm;pi~(9uOKWONr+9?18?kjBxCR-+Xvc(EAG}$C$-Ymh>lnE-{mAT%htAUt*hE42MC4^If zMPaP|SC3aCq$x(3%C;s6C9F!7XiFyDzYOVzv$wFhXV$4$dL>fW4FmtcDqU}mbRSCs zmnS^`L_JZPNAy*q^uo0FRl^#rTrb#2F?3VV^>X_9k<2iVGB^=sPsAO0bED9Yc#h9; zC2E2&z;(Qj6A14H=)^I0OoKYmj>9OByCGE*Gp`zgBPk+t$bMVU8vS+Emf-ak%gnlH?=HVU1-i~RjF!U zXv!e{+kG)}Z|s~3m3#dDNZ`2X_Ppt=dolBLdl88P>XW6C4*>d94S3nJ-|DLCvwH0W zA?OR+`V+cUP~2LJ@nUNL;MAiqOrp#jdXfmsZuuaRZFYI_2=F|gls{yDb-7?(;0 zU_MoSc56k1gw&J;gCW2Dj*%dz+zX~a(_mn9$|*I%D+9$ess)+g4*(83oPqVmQbe&s zK7r96?GDM&MQFE-%LD*Szs5gA9RL=rQFLMO*n_Wwc8-8uxkIzW=n`n$0d2Pf6D$M; z1NeXbyRwE#rA{p4A&Z#%0rZb;4jqA}oP!}p^woqb4TWKCe5p(e zL50p{hblj56%LY)xKcIZ!&#O5vy9r46FE;k=fk^Y%VZmZ~83yp)Hm60mhZ5%k9gq=$6ff5Lk&|{7; z%3xlS)5TSicBfIH;4^M*2G{) zjiIr!pUAP9K2_n6axkGA@Q#tj0eW65+Vagrm`j7Me$%Y3%$bT*ovBuQ%0tw@)-BB< z`Dt>ZV7G6qi`al;++y!cHTdQ|ns#LNLd*87SY-{pbHvx85!->mGT7i_!!Svp`sCj21L{c_4%zPDQM`9$!}1z(>bMC2<7P`kW5XBiPLVjG{t~v#k#yoWegp0 z;$Y^Oq4gHKmQ%nfVR|e?oaHQLsW!6v$=*LcchUsrd~9;FTMk`tSOhbg?`oOar7cK8 z{{4MP0$H>ep*z6F`eOc7u9Cn+X)HN{KT*binp~m5iB7jAzbau>iqJ-c#{Chox)I_q zXK*(rd5G48r7J{-43|`6`|l5_Ywol(iOMj;y4$%ha3aymamw^=8n*@lx{YB7nQnwp za6VXcIO0d;O-33lSauWSmC|OQZ3muRmJV|cbZ3_x%~w5uR;udGDO9*Q-x^=zQ}r)_vaD^4(k=8UTz_+@W&DzLP``p`6o!C_>>J%Ye6P^a`TnIS z7)XIZN?H|2(c*37w921&2X3}HJ)9>+i-&V%T0s*V>xilMW}NXI7w=%vW`_tARg~^%d9{a~+uoRRWrU=M zA8vgj6{+?7F_iL^?uffIhGlM*sCk(|b0gCLN6i03hx7J_A<;DDSZ9D__%otSx`c;m z?x=&wkzQnC3`kWfQ=m5)bj?LOWSR{s&`Y30U!PJVtqPi0aG{f1Fm-X0=vZjkrj(Ra zDzL83phCuF!`-_wiEAQP_nMji$qrxKdO^2Q@ARpxlo^stK*FOr_AHp}&p<)#gmYB9 zI{#1%u!sB@1H)p=-DB{np3ZV?qPawRvFwF8?FjoyHKbr)ivsot0ZGji+ieT>E-HUZ zt3gFd07bv(XAC8JpY7;d2Z9F?fQ6ByI^kA?$kXHO6bvUOs}S;J3K4O8K#vQ zC0-IUkR5Xh7>VH3NTtf!Jt{a01-hM%#G5KT2>xjef7}6MaS%s0N!?T14alf&O+CL8 zz8JNM4fA*L*m~M>RzEpesXNj{-R1YE?o#s)VYUDUssH^V5T`83&wq)487*wdNN3pD1+)8mPg_dUoex-0{=_5Y8yWhzCLtuOLM*H@kt$5}Arx=n3 zKtmfasC)(KDIdt&o27p2qoV!?^AL9+OV0q#Y}zBt#sk1(<92(wkd59TljY zwB>hmx`EV8rsJcbVyv%MyxN>Mi-E@kv`RJPFsO;lQt_3S*t~Julj;^)=bKFtW z5nqkO8DT|ysM#MZ#fSMWPu)w{e-A}9?tN1*Ad{XQSqqE;B3oe>pFA$lr8bStnQneJ zA2V-uYtH26>Od!$7W;AUI>ltL&X?lOin}$f=|!WG^^NKt73;vP72s#Ko9RhTxhX6) z)OI|RjIZmsRWvDhWHnEuU=nAirHlk5oHz;3Or&jtD>mf$^b$E*N{BPACc(tEoO$cV zw?~^SVP0j}{TZfkw>C@SWasAf@pt$&OsMNOjaTJb=PwCjU4KpF%ayuX)U?Tc##MJ zJ^6tV{=VA*LwmQDUTTy6@?6P765ktAXxt3ZZfblX*hvpvqW=MAm10Kwl zbZ(-U&7mBRbEIhj({IQ;0P4(awleJ}YVh~@r0R_QKt(s-x}Kgeks&S085qkH`+KGP zL3DBsJ2Q2aot;)^S!r`)XKrV1t0{71@_Z9A(Gs*`COhIn<*Pk1#O+o-`h-9d_bmWqdvf3ft;U@Jo zWM*KhBKsI8CKoFK){;h|*zu9{eP^Gfe5qrD!rxW)GgE5)SPiO%@c1RN9^7~$x|YUp zBHuQuLVxU)u5s$~q3R5?8y803xtE00?}Tj0Db%CsEfb=N-2CHK6DDFIM`^lN{;*FA z0zzh6w>bg~afjfs*Ib8n_nz5iu~g;KmM__a$^M{a(nqU^$3tAP4teFVzyOiPiZY{Y zK0&*QU%_%78uv8z1L5XyK$iYlPQQ}<+Rq-V4>1m89mA}^3W2k{Rm#c&FywAWTyX@n z%&>>ixI9y1-g^YJH#Wbxdha;!E?5~S=o{xNoi=5{@cD2VPd$X$28kQVxC9m2_Lg~? z%kyqNY?IudukJAW8|rFSQS;<+dIpjF5ubu88~Dgd=%o7b=ZO`SZhUwyBN)6v3s65G z#IYoZ0*xwxnsf30-25%cY$u<5GXZ*o)7ql{d(ByE9QMyRFo2R6B{M;$Jc=;18C%hcLi|Yx9{q4~czYo4J%Km*p=`1LV+AqJnhQ!lxOiUg&)Cw( zY{RH19N#`gEj-S?VnP6XD|Bzm`?KowDc zKl}Q*;ZKeBAJ!Gw(0^a9nr+pXb`oU_OgqI?ajXAnHIW+pK>)6B`z>)S(_0*hSgW$* zG(%_@Yt98Kg#YpOXJl!m$@=%DQyVw_NwrB#O)T;`SOAg$<_T?RWd)7eOBM;kb|B&K z(7TC8IDoW!cZ@(jXax!64?1#x#qostPlGVmkZ~C0EsLbt+EPVKzSvxE9lZ~toTQ)I z>S5RfuDVRE8C{SeTY46R6V6Yv%EOtjAmLO~1sycvA1=##-#UUQ5(5U3|1ij$g*8() zyg5+#PR1<>h#&e>&LgijkiaNN?l(0OQ#2+QaMr{-Vco`=+q00LvY02V$-DEDkdUjM z6YKsIACp4m!}gnY0b+rc+oZOTSR4GYQynBgk+Rl}Y_(ID2~$mXk3PKr_QLR$9kaoR zBsn>H3*^IgxBi66T@O}#*no8~qCfrOUHx*F#$%xylsx=I4Kpep#8}5@`1LjNE`x^r zHu7&`1XFrb;l>jy>a&m~fo!eYJT8ixAnq4JZCFsm8WoQLuRzPd5-Q`OG`tDp(Pm$x z5sYk{(E6{(A&6!0l@_v<7A#2Zg6!hLwv;v%>UN4^B}Ls+dIt=<5NV|2EGIO`gla&WtTlk!p`=*8|?m8?TG%azlnn(h|7LNm8>UfQsJg# z4`+CWbQ{3IZCwbll;*x^DJi&`N;Jh=5HMrE_B)QvbqQ#z)sS`88(FDhN`(6b1K&9& zx11NN~}Diagiapi&ncEMYUZtgw*Qy5O|bNKYgB znY8npJHo4br>E&xX3A5C4Lo6EuYwNV4a8v85-pX?#XZF`{Jnj|2(s$~X#Ka2m@pmj z9It&)#cf(m+!Ht3!PRW*=#N#e_pcD@L3yb08UdnrHe`RYt}IR{JW1aee+T_KXCTSb z0jAf08g}4~4ok`h`5h`gw+D6#MBqU+aEhwRJ>J*};APyCz8%a#jleHHeW3034ax78 zP~07~s-?$Jn!JLp8z;b}f1y4JOaUk0PRtMxbu<^sJ z?%_Dl`GKqg${d1JfHmm)&}*bQ zuvsOHHVMCj_|M4KsJGz|q+SLF7lP__yF3W>k@mK(OiAz8z-MZD#n0ACTKs{6D3mX z;0n@+2PrFO1AZMKj6m6~n6RilCFXMRbh=~I{Y#)WSbB=64O!r{6P5kJd?GP4P2qj;$p~}5 z|94IDZ++Xg=$6)}4f&a-i(TL+=RdijhoGhhN6gMJu=AgI!5});yU7~$pa`=~62K74 z4^*KWrC66yF;s$Feg~YybdA69R6Ct?u;|f?VhXzxXJk&RcwR?~+ZLnEa+-tDjpFc% zs0P&(_zSWo5x|6nFK@5`rTl{EJhKFsd}HEagaKRRhHyssMMmVikdk_n8;EyjqBW_s z4>mcTjC?kp5%CDhT!NsCY$?Q?fF6HtKP_1--@oYCcak_K#m5*6648=e;|SDP>!5-jVT`B;oXbSbT`9*RP|{=)x^h4k%S{PF!i z`gR8(oF`^K!VDNtARwXt-PYHC_3Zxtkl`hXv%d!f5l4DMgiF-!7|#2HgRsko$n_Kv z5W<9sY{E({?-+@O$*z)fNUMqRtmN28bE#xnLC?2idpoxJR&G%q zyKBz*xiUyFx9>RI`S)vhL4(6KP8o$Sm(_OuA^6O18+)64Y$(J7+$i_^Dv@>WP0@xoHq89{_36f zfS1tq1!{(TSQ?}$ED%XybE)#k{l7O~fi~zEHo<{_$bZ)N#Qt}OC~9hE=xX!d`mg`P zanh{rt&Spr`VChXGf9jJCZ=80Vno=tp;d)GbibqllA1Pb<+x63$Tn`rfs@>GIoU(- z{Po(ZpWngpR_@)cgJRzyKtFV2%}c&=rhf zqGG@rj7c!+W*{(Ps?!(c!>t%4B|Dxz5*;1oAyvAq36=Kx`Zsve=OZ%OtBcqeUfsID zP){7s15Ye7+7oqWuI+#0e zy^&bjF(RjqH(7cNXFlYll({*muxMQ>*BVzq!fV9^rP|2mJFkGz+-ACNCEbWeQX6Zo z%X9H=zG048`KzX_UqQgC6i$9GnLV#KWkgYCHIXIL_`bbYJEA$^RkaGlo$FY?Z z4dB4URRoV2y~$>QP87Fkp;lZFzCzoOju98CoJvHdfqU}VH)YF!*h{89Bh_o3NtfPy zPI^6auwo~+C-=UK&$oCsR*EHIWvg0*dJFRO_G&pmqK@A16&`<4saVRHbK}}nQb~ql zNotj85vrCB%1v&yX#tj28c^zt5^BfgNI&O*NpEihig28q+P-tsAwGdFRQz~h$>5!@ zz)eMXwnG(Kq&-|vEdz6aEu%?}u)748rJb5!Amfn&i2I&T{>C3D)J85-H5&C=IeqohwrydM}hd5?O`NU9%hE4jXUkFARy00(ZrPY2bC+7;uD zTFyD)K<>duKlS6rg5DYShJ|2{3JiR4%)0mp2u48j363N4Dcr;4KZ?p@g0*R8I#g~D zOj~@-j`|QDNQ0i)iw&+r^@*`Z^@+Mc^@$@u{S41*F>xC|Wz2>pGz=<|BhF5ynZNUa zGg|v598;9`9v$DPMPe{0pSRhR9L4GwKv-=Ii)zgy=>ko?{v>tmr^p;Ukx~1OxAtTm ztaXrzy{D{QhF&QW*~+~yDGSeG+yrG&{=NrrZleQ90O4+AN7>;?#f}UQm-eiQ zHUiTbM`J8kJTr%1>{4wpeU_$2R2O+7A*AvChI166+dt%|Qk9i)xc^DjOX?;BTGDY{ zWlDwSbQoLIG8wL=l5|Wyvb$kC^=L33I1t)1eJcG(P>b#dxZAUd4wYu}U1IXVFz9u2#q1H>;yc-}P?hupHtD0V?dn z3~B=op;N(U3sDu3JKLmHxtKHDYAdC**)y1qJ&qN=p&f62@7hrDK0E6Xl|-5N;E(qA zQklB>4S{uTqH2BD$f^;8k150FOe#NsWTfSNM*!r2fBwe2P$ID?4XYPTj;Je=Zh+l> z(F^c2*92oF*lZ7*%2aLs-v;Wi^Ie6;7dLq11g>B|v{CpEXb=jpY6>rc+(Ln|t#ISw z-zA8iphSiK2MOM5{qaHBB3IPOYuM*zBzRnxCf4P7CELG3(RdfIG{p}1SXCEIRzVqO zXNXwiV{vc&DOl}K9N+n5`rw9qK`0_Vk`)?2W(!7Jo70}YQaENjNVc zzLV==E%b(ab`?bo5sq#t;ZnGG-PD6qXjUluNJtBDT$6b^MW3@#W+MN={P){qIA{1k z?xzG01OWo#{omal%BIGyPL?j7q_T!yo|5)9CZ$~0 zdla%26^jQW5*si=h@3lUgXjKxJ<&$9)|-eQ9Ox$~K;6-)$+ro+#qzS{JPkXLsD zG^4;4FQhq`8UQw-S=>-#C?OIeWY4O7wQ2Sh3s`$r-Q737h7D$mSB8d1XuEi+K_Z}; zhru&!4o0EInySTOW;+|-3ngm*GiNBd{FQ4!xwe2Z?XZMaIo(OmM7HVPwh53He@m1F z2jwzkwE4&DeY>5j0Bbh*5M0^m!;GZT`O|*f$wSm!w2tj$HE;S<;M&&{*M1PVq7dOf zBLB;X8LOsYEF3iB@FrjmJ8V&R=))+ZjP@$eLMN z?SGE(-;YU+F~=$z5(sGY=aT>Nya3rdnKKwV7#drcGFVu;{XC2fT?}pP%^7SB9h^)} zT#Zc`{xf0ykL2b5HObbfTe_pHV)>0|i6=sUnxPSi+6awcf=g01P(d+~Q#Qo(!vJTs z8C%jKB$y3NjTyjFX>PD;w>HTY$ZQnXbe6zL(Fj{4)i%jx`Pm3OKW6(~Ay_Eq&*mhU zLebz#ddK>@p3HRe-#qlFd%a9(+k>t}3rN%>wL`+D8#WOV^ebvGUqXWHG>{Y_3Q6)t zAEaxy#@DCq}4|j*M|j65+AcL z&rG~V2ESDVUZuC=FJm3_7HNhCTbv-d>d2wu@@3UGMZ57TdM%R^wf%T(&BMU-yDS9* zd$(n+k&oTZLsxXiVl341xD$`0jLR~V8bK>dqD~eHcCi+1p|ZH;Xp$5jtgK{ZqR$R= zlq!qNre%rCj_K%hB`qWPt|}X~XRW@X1N~04Gi}XAo1dN9kbmuS7+Ki&k7Hy*OeGXE zS+%GZuaq2b!n1ioXfBVDdf~}%FuR~jWvmO&oT@q7k+YGW^F;u&Y3GOg3^mdlK1@r9 zx}QN6Ni=6eK-d7Ax(4$~X=a{Ve29^C4V8!xk3f?iW}nX$saCn^?$98)?_;JAYx!c5Gs5z0a&O%vFfBAFyt7s}O zmS)IDZx}JgHe?&h7yJ=;+uaj0viKc1TUbK7=(43=u!NZvrVVd{rGL`@tkS|NPttAH zFKuab*Z`ZdXkS!cW`M#z=^@q6aVrN7wMBGb3f_4OU1I(e7HALHL{9Y)eE3U;>I0z+ z-4k^~^~&#;zZd4A+8=jo3|5g$6LKuHejVldoq0gqviCF8K}f!8MjfJEML!2D~(@-Ej#d#p7gGsea0#D zG>*lR$rvfmHS8SW^)MzQ9{F2>zm6DnIhqcta$E5v+MAD9PK%?>z$-WlP6_6yK=GV>!;M*Yti zTy~?Y}OSO6b#E^O^3Bv>@BCBGviET^=FNV zjA>MBT!k8gOaT^EP0bm%yPmP-T!-qTI2{i84(mngQ{1`u#+TOMB%1-xl4D-F%Nl+3 z8rcp*EJYF{k|6yA)DghYD3^$EqL;Rn!PLQOQP;Gv4tF+zRH?W(=jE}8G=4qABUP)Un5S=k**#8%gwBjc;DKJZVCdoad%-IXYouW+IstX z>?j9nW%~gBBq>;%%A*^H(8yAArq~gqAMMVnsL{7A75rG{%DTh&_@BP|#qTPp)NE#F z&zJo+_%|z`oKfk2q3dG=?6e(iN^LL>vss}R==@}_R1u3$Cqy|1rSR@fa2;VZ4jAgg zD1B-^!Fh%@gWJi4PTPc^;-ZrB1^m<;^yRFmv^dm8fR(wW zW59Alby#q;Bgm=7bn}~Ity$qW+oDgvMjJ6|xC8JOZ4g{rVNyGvW%B{%jskl5Ls6$g zG^YU79)!BWIK2W`RfE)%s3G9_GZVxyMR%wudiQTc(_hawh+ixH=z%P>JVPPOsFqdE;DaeSxNb+z|^ax>LsWmYlDg5z16Y?opjcRS% zW(!@b8rLcqRhezA)?m@|1*>JNZEb6`nij1W-zxv<@9dWyS+m46M&RC??>onnosXIJ zldZ;GZ$QNN_6v|d)dVBL%!nPw%mBV+&(q$Y(_{R88PP}x#fe)5a#K&rES0yt`@}3> z{QJaGrPwK0q;egEqj?3aV;TE&=|NR&Kp`kE=5Gk-A<4JXq#dD!=tO|USxmxCKQ-Py zaW4eKVgWNg%`~SabTJmgG?*oMrOIi)C^a(k@P80idYQ5%delXO#auciRC>t^DsOJV zX|cshDY5rP&~ix`MXKE5zmBL?Js4w3Wt^+S)N>0K9Pw2Y64K0|f0sb8oO&S4jEOxQxd*T=(^on!3cu&FM(L=~BsrUyyW(rf$^@?{ zA<&7mVn{4FS)yXpAhy@4oS7ih6e@!`G>82r+m?AkmknZ#x$k$F$4<#qu|mHC=PfU5 z#y@TPotY1N;!z$V97vF=R~5C=Nx5G@Ieh4x#rnf}7RNEA$x_*D$GLp)zd- z8M$o-yf=%BR8ync9_2AE^4l?_qYD$pDySuO0M&UK9302*AX{jZOBe-c5aL1yXQE5Q zh=bG+9W~&RMQ*M#j$`sYiNPAKdxM=TdXlu zKwwK*9wF}X3IUy^^RgbYpw%{E4JL^}FE~rW+XDeGzg4b6#ApIi)%b|J&xTp3pFwn( zg1!;&6I+z6$NhGC>NnN7{=SD7FK2Y73Qi@i27bj$%4t_O)1_)m7bk6EmLWxAWf{pD z0{j%p7mF^aJ+D$%(!2AeNbNAXX+tBHZ|a7g&60)vJhHpmzflICR14S)2lo5t(F>>H zF8}m9Bz5n${3@a~RZP->&qB5rz^s9pWe>EWlyW(SC~SasTv)0w*-)izE!Er5w5S{N zww&jfzCJSLoxVdo-*}xVh0S#ox36av=d9KThsh0!g<4+(EfBQ%WhiFog}>g5^dwSb z0Y&7j);1rXjs%IdY)khfUi=$3QDpMdAqZcmPy&Rxg(BiRwe-NjnVk)@hFEe((yW@R z?>;*r7VP9#p!j`muxgHGlENv{cXKHe%3?AUz<(*m#xBO*Z17t?-`Kh~<_zv)O+Z$B zYKTp&3C<*znBPdn`_z-0CZL-gni}!&s60dX`JAG3N@dg(g>r4POC{n> zm6a`EOyWs&8-D6-11QT8*%{&2RJA8U>Hj+^Mj z(kdWWV9cwNTys7*H32qcIKfprPIwA0m+<6oqA`k$f`zRwWI`<_jCXU~66s;FQ7Rqj zvnD?2hoF##o*U1!7qBzM2*>vZa`_RH_a4)mG#+ZOxUY%PlVvg4TBr?x`t_wt z8cPXRNFEfJq#g}+wRv}O!W6l%lFX_vxgOk^V+|a;C{|1*>U!9Z8cb<${Xw-3(ZRR| zvB9Ud9~Y*T3)PqGU~^lna;iXQ zJ~8}r_v4!ebfcZ#91%ESdP8+-u69fJN%RW#{j3=hPgJZKl22Tmq}f`=_D;^{EoRMv zJlZ9m|Mbwb+Aaxis$VQsMBf~4mCPd~4&0sx#En}otKjgQ|BNLl30Hd))dR28RxK7> z_=m$U7mq{%Uu5y;OmR@Mj|u@iNFsj06o)^KA@);FvE)t_JMt`hpwIuV@ z3h3fyYo>`*nx3Nh>c=dJuVLC87>!p@+Ye$2W8A4rnx4Y>zO!8%fq`Qy$!0c25n>xU>)Odd?}fhn0!|_~z5xeWob0cz>KnW~LZNy)0M+X`_ zHq?I?9r<*JQrbIEK2_klEGw1Ic9aiS@6&hN>rOUkiBg(+XHKe%tg5ScjWTqoT0yge zPAwMlbL^f0TS&={GyZ{Ax7D}9psEta^Du3@FEVhe&1->tUH!m~oW?4xID}~!!w>h! zt(1(5+X?z;$X`S0E3ujiGUp28J4U-vF>=Tdw-Zl1-WyI?lmL*(5>Fu*mj&~pjT*`7 znV4*BmS16*JUlxgKN;QaZ^V@t9xNJG_DaLbkd7;tt}~g|YtSF+D2t`TQ1KMD;ywUa zev$61$+RQxOP!+0EN=M@hU<2G@D!bXt#)TG0`2PJaoT@gQFhUxM!4~mwT=Oi-8-%^ z=&)na32O3WH+?LRqoW2ZIG9T?53}=hoke9zti^cf8P`4Jt=x%HTqL(}c&8_HI%>^U z?)vgG7vQPOQ<`%I2KpTZHY*pQ?#r!yFjs;|j|MV3!EWz2u1(2~e>0on9b>q7)Qu|h z)yex-^@UXISWVN(xFZe07wG1XBj=YF@vR-LjgN~ut2(Q)zW+H@GPg;4$!NyINc-6= ze2OmBo$-873}KoSH776F5;BX7EoxjB0Ew#M+w0&+lNSP?MWB zi3|EIBAl9ZrtMuJB7Exyn~r%eaN5e2sh6;(H+5r~gFx@f)827mZTQgWCOB*b_ zhsWu)kHLRr&oJo%Y0dT|^5`TezJ8{nrORy~(9Bw$b|iV2p{2W# zG~;WJkn4}1w74@i@f3LUFGG9s@{REAp9iHc_PzF?ZitRsk0qGEvb3XmUt>)M6Z6Bm zk^I7SsrEE_x&J^;Za@4DX@3T~$-Hp$7V@*&SBY0*eS7{IYp5qNK1-HH)0pck#s`j@ z*Xi#5L$qz^t6e|8>s zd#(V8CI^7YYegH=b!BUnSsoxcTxzwU-eLKEU*0UY`b94 zuE`naqVXne)aG*&Dv2`skLvp^_+%dCmVTA{&=1x&*pXqSWPn)4Rw#;h6hAQ=kAJw2ua!kBm1(Fxm@C7nAz$I3=(9@q4;VE}0)?sh00e z{&o(MhT!qbycW98{+kJ^TY)aov-;D;rH{o4L)REPGq2tCuwL(&COEv_+`P{;9yd_D ze>1O^bk{N&jUjt;a-V!WloN+0oLIYIpBqhBn4@MaAp_=YqK)8g#z4DIGO?l{Gt2D^ zY+-OmN#Qg{e$jEIsp|D?p_ZZ?>IDn#E9>?^-qnlmBaF7Jc*Xb@oy{xtjZmS-SHChUp;W)~{>WEK=P!rpK1^6Zvf+$8q)nS?p;%~)f?xfX z{*^0iqqu-4rOgby*8bG#jfg&9K&r1FL8jL9!@5Naj?NAH&@xK?YK$hu#)4Xl`LTiT zTHx_S)cN29#s|&iDVnhE3`rx@aegQ0+dY1T;my>I(2xh)+t#MB zn~^+Z7SEsQ%Z5D=3L(5=P+ROezX)#b;nP{ebiymKDUXmzSAT;{=AXPsZP*XT_a=i{ z%9uLMN_m#jTj{Hzf3dYDi$%SUq+>>bQnE2HUPlTsqwjkB%mU2313C0a^jQJu`;Unq z^LcSuMTFT{=^DB%*TxFoY2pI}W0S1y`$WWJURS{&{KFyGHarhfQ)g=VHn`hV=j#%u zG{6qIWZTV^f1^hVc6a8fZbrp`_cFMruNiD4@anQ4#fQqGy{@%8&jWtH2xFP9`;vP% zl2O6f$}I=>+<_qlz8KPLVskL5udC*l=sCydu#%IQf28Hf+W}&|Z-uUZERzH2S|+&v?2O;`i)*2yXvedZh~t z`nEvc(!Rb3Q4ftxI=_fjj+ReSzsN!FyPEU&Dsddh9=rLK<^wIt70Q9LgnRY8l^AnQ zrH6&aY&dfV+)sI;RPmk0gV)dhT&K1>X|!nwK+@KxdOw+WPqZp~kQnJ~^_bVc^L(0G zjW^s12c=?&w+_?G8DoHS)t|#<)9b?m;!!7iahDYj?pxvph>vycv?5%4y?`2IxfGGJ9JqCHq9u5(FN3K|PUV;-7Lz=t#asmXJ3Lv_=)e)LcH z3D%OmZR34Vv-*@@XTVEy6`c!nF_Anh5`P`OL&p6;@pCMb2HPw4%1=)mxRIc?8u@1@ za$yaV(M;p9Po|{;`O!WvJOGKD<;nqOIhqH9fJZCj)Ci<4Sb7Pc&Ny@bfEygL86Z~6 z{x)Vz_2J79wr&h*S`ja!coeC$sod1sK4ZEZ0b)~_xsch}YF5sJ)L(JqwR!h@xV;E%TRm8YdPt1|L!WU2XUu{lNi`gPAiX=LE}HTSqFf zk=CxFP+I@ha_L?PhIkXYFGy?cn6@ z_TPA&(T4WIQ^)`M=h`!4wm>Ss$dHnhw&0LNp(`^dPhDcYQ6it*at_}j$FXR(F>}+@ zMGYTCWb{*5OdOmW>}NFO28e@d(Qi$4G)1uyT*YoiG~|k}{|ZlM+a*a0yga@)Iqr17 z?ykP~rI#D{UOE4WwyqDe*s9(a1t^N5-mKpJ5TFYub@5U{AWR&R)PKnUdjW)n-$BFY z4=Z!?QrtE4lj2b9E5CkHa}R^Z*=A5oJROFFjWH2P`|0!sltINGLc=Q_G&A@yV7k2! zM1FV&GNK(sb5Y-s!yf}yLm8(|HIMF6@OrC^IeIA?^BD$Lf$-dZs(ZR1yDLhew^E~X zRu#jZ$9FX;e|>R!;|Z;-`dv|O!%>bTaRTUs-+(p!ifnZ0DgwXrCmjWozk}Oe217I* zA4mbj;f5IdnlJ9i0m0?q0knn&2|xN!UwXm>_u^5$6qBa%!9R=k>tA$P5X9}g#IdRt7&HCE8mzK>XY@orbYYNbHXPEM$u0y)tl**+uw(P)X(suF?Seus5 zpfX_A9l_dgk!5=+F&<2|HBuVHkU0j!`ZgaubRoU(aG6PwF7BVO_9NEUFFAYUr$<>p zR*o%HnYM6ESL!OTRb2-;6d|CCSCNt4E-NokiH507f@rOQx{WAktahTwe*SN9`uwPTHSB?71-H?I zJ#`<0gM?Xk#)h(c6J;+Z3pWlNI=X31g7`UunG^?BT<+6~nvd4_4$qdLa7m+=(*nF( zo;g__D=9Ob@p%6#GwG>gJ!*0qiGP$>I!s6&4y(Dj)`jdd9U=uv#x*r_VOiRqTzatN zM|nM(O)k0-vY7zl1Jvd!VZ$_0Fe9$+vP8i4u;cZ8t--!Dscgj?aoV( zQEmu2>T=8!O?dNwkp&rzXStmGYx7{-8pShDVgr9@|5-AbkS-Y-2e->~1>A%%LhvOD ziw*3m6&NolOMDuyv<&vuHFg9`jhCa$X$PBXwfYdM+sovIPM{{Z0V)?cz;4f5n2ai1 zY0beMJXXtbm@MnL4@Nkg<|7}h$`)NyunTixL$n9EZ=yA+TIoDtwp603;Cm%gkc<0d zu^;|sWB62{0yvqQ%S53P0uK6li7NB{BidBf?`36D@=D{cRjp}VNyv{NigUKiGAHAC zYns$EwpbbHhnyv)K@-R-nP(dCl~59Xi5T{RbDq?BxLk{*Y$p3Zp6-be^OVu#nw+@` z#f#uG3+HfIwTqc*b&GgeQ{sx{%~k~IqPeqWuOZqpm1-eNcLW#;8V3~$u;W9xw-GLH zqWt`Y1t=`b!q8QJDwC?9Si{0*vJ7bdP(L_&&<%r423Kj)BVN-I9%(y>Yp;Z=(TC&Y zc3+`r_#`Tv{xPXrfVf?_}1)P{n^FY$-bYtpYL6b$Sd_3i`7^Ci>b4<9&(o@c zs|lXVv4|mjy=4K%UZ|jJovpBG4OwV|S2=hrQJ%}v?K$sIxgI)u;3$Oy-6>TbZ5eQAH8HOHiO~7%vFuW{jKG(7PyqatB==DSafEbp6k9&lZEcXivmp4WDRFn{z$A}&PnfB{6^0?wdz{<|(&)I; zVJc7`Tak*an%Cq~Ke9wjZaX2gZheYjVbVXgZB5yX&PGAP7I7l{!Rc~46`8ZX=T!7S zDKWExah2qaWzH=BoJ(@-jy$gZQ*rgGBOjWV4+D(75i|`qdBY|;!nA98iL@6qg_}v^zc-^)ZhxN^d=H5c%>m7M_-pz|PyJHSxvugNKVsVy1 zW7hH0-U~Vs_^ixwD&%d6meBa8 z$>Of{9^t=99m;F=u%d>1CR~V$UBDfy3Y%1bnUuq07-@ht8-xCqZoT??zHOCX zoG~fl0;0(v`cSm@dwrEbw|`piv~(jTI4o%5I_dDkt}?N9_N3|;e(gBOO~P+Gj%?KD z1M7c1`1Z1Vs5WeQ-v1ajNAKf?iU+J#2jg(_u|>b!0ddK!Xy?tF626g69%)ZLFM{~W zaTIKzL^%|QUwpKChO`@A=B#twA7lY1ngo#ehtddbR74#BtZTjtjvs9%xPq{a_`Fy$ zB&$GH$vXAXMAhZ^En$$Xx4mH4)+nh=_=wy!@QQT@z3~vn18)LBMvahXGN@ZikB*NV zO>{|uulrZce!+Rf)w0Q?!O74i#9Bge=d)lwO6r^$Dsvf2X2OZ~t+L=AfEX}0eNQ)v zn#n6InIe#a5z-#o#lQ@jqH&!=754R>Mp2~xO0tph7K#GVXrazJ!w*l@&-m`ilJ~SuVCEyKdNZcu$)=GpPc(1d^)*x8VY#juOYnkM~_L`&N+?K z#~75(b-Gl0=XLLmMn9T8`D0iim1f;eQ>J4NYlcs`PDY%coTq<@FwpJg75>rp0wI_% zFV0a%$`dP9#1&d*@a)y)jSKx(@r2$Hy`_W4?sBKmbc*rn*3R(r%~=62dK=u~AK2{7 zKpbg#9R2_cr)^tED4I+$SK8z$3-0`}2fPutY}=rO+j@lhO%Ruc#;cgSpr3>Dl4cFX zh+;$FLuDM!Iv}=1SYZ8Q!5?}00EqJG&;b00hm=b-ATTY#*)?-u1?R*B-bg?kHzGS9 zI)!zLR3}Urw+E9mtb0dSHwX*BVZ6Vf)|2Jtl6wW@xoru#I+HH%QRoC_TeCY9@^tHl zj@xHUd!bLyh#{M%GtV)^y(Mc+i`&6HcLTp{U}GA43ytWITC z%U{dOUC(^f!q;gklQ@O;>qaT;`=G(-Q9Gsf>t2foR)g2=3!Xf}=z*;EhRkqBD>jb4 ziV6vXI8I504FTdx_sxis*=Y_K3u)r=bg=iU*LF`|VXDlHYBT}|BAvd|)^}hGL^(qG z5>&JDp$9nmqr^q@Znl(4`9r|2G}5b%pKMoEZm_dq?kvBzyq#+V<3@`s|71*o{Ej7Z z!aTj+&WQA7hI}@rzd2R%`VG4fJeg{Y;Yu&6H%GCoG^EhFJz7MIDLmuQzo{+_`@PA& z-uBFcrSD))nn6zgTW9Uf-9{(HPHNC^=M zUVES~RKr&r)-o3YL?q@G9v-u^na!7meZM}y8e-I8O&i7%$GTZyC+TRKOJb78hWH4M z8>g9&ql8@AiG!5hcEW4nA)e9-0XM$J!#x!P+j--}pUe=|xG(z1Vk#kU$Je3()42v@ z`j*dRUqQsTxAk3oB}o&2;bVibO3A=S9vfYRHhP;~j0qE(n2+C{!?-uC`p9J352cT0 zEu;O_mUD0U(^S_w!2?>)n?Z#w^!&hH&n2`00>@5~nf-Pt3I&sS9%-EUK(eseHbOVN zo#R^2eY|l~7J35b)mtvfZrliudh&`^!U&_O&1dN!)uctWfe?$!I$2U=GYu9L<{l%O z_${ndGz&!mGILEj<&=XL*Ac3l3F*0ihx+J3ejA83q6 zQP)2R9tHbY!b-J}k9C$2VJ(>Yay~p~Rexu$N5LywJ(t2cF}J=uaP=y{@+3m2?c0br zREOn%bR|=V_)slSDDnj1{6oXYDAkK^RGhC+LDBja+u?v42(og(>WsenNXc2=!Gurk zfsIdXpp8#$5f--u$6eK};BOL!;2k=>zMJssA~Lizq1mZV%pkSFT*RFId$mG`o%OHP z5HVfuHrceTzX*3753wCZ2)1;d1R1Z)kv>i0W~GrST6a23Z!*kmY>;4C(CvW%%o-jz zr|Q5nkVsz8RZUPjW61SU6|)jYlfCQ-@_$d;cUe?yD&NSC`DTYW|1)GuJAAv#9E|P$ z%SZn|i48A5%ihDWBq|Yo6~McROWY03N8;Kr#r0Gk-i^*?owEmCs^7 zy?j|~N9_zgC@X{o_Kw!yXW^?i_MN5@8=dYEeM2pQ=Mp>Z04K7VHl;kR#-qHVgSVbXcJcQe^Dxeyf_?afR<~7db z-PRdt{%<{sBfep{L2jhEYoytI*>3lk8PR-E;E0~QJC62hajG#5yWfaMnM{rP=(^2{ z741E0`<`X(D|4;iG;H#`pd@vB&4Kp%myUJ*_M{%6MZDMAe6(J^;aasp%nx?vzSird zoUmqH3on0XYxv6U-!e&Zgt4ZbZEfPLTJ9I{8!|nOJ*p&kA9_9`);#9mJnl7Ib(WH( zM@$h@N8u5&mOGH`Mi+IYUh53oZJ<57R zKSr5@weMZuhLa6Pa10U^i=}+l#g@mI*2SZ6C2$eQ%ktL@TO1+K`H%H@9leUO^;%A>+Z8h^A&}x)S2MFyNT)S=YYQ5_wBL!^{=gO z-y`gi4(g^e08jHEBG7jbu+rZ{Ap=APC1hs~Du#&5q#COTe4m_;Jv8>K0eL?B35pm7 zObCq6&WJ^Y5HQ|LHujqw*2`f7G6~db0&OJrUGscfbf#Q69A50j#XHDF2UYl!DF^PsqCnD^wPAfQ=b>t9sGAg(95D0A z4uXwBcd_9}*kTIJ%Z>Dq>kmyyFxz={VjNOB?< zt^7&;NIjm{@0kobBeN0t$eRpVNf5ee`<}v(ujWqRXGyXTw-y--1SHEGgU(z*U%dp{ zKQ*k-{mZLU_$sR@S-He_O5(o}T0mG30Jo~oj2dh!58iicSiZcJyLrg6jgC~gAOrnE z+SO&b^~Nr6+oLc!G7sIM*n;y6HFDT7w%mL1*rSHbIzQee1sJ_`&%^ptrYe91je~J; zmg*lb!c_FtZxR%J90#hXSlA&0W)ed%;74ThEb+*T9W@kTTUl~Kci~O{>L<9hyr$*0=S%FeJnMnUEVhxGqC^I7~Hn z&Fnm)h`kTTcaj}lhr0@cuIJ?uW>lpfg_*<2p4@NGc{IajmPVW!mNNt~n6MtE#*BW1Vl$b*9X5@=r8_NAZW1yN z{KAqYfA0}cFPk0ibdgdQMb&<8iI(>Ds6p2g!vH-EE6@hEu@Q1%J#PC8*KqbFVFTgZ zfG<=s={s4G`~8lVY&b0O^zlSngB25Olos*nPlpO!)hf@^jVbFDE;XUKYvvPvSH`r= zGYOj3wCQE;?Pe1o+Ef|Ml#IU|Z#U<>3J%5Tw8<2Wv$)*bCLU1Ya$C%p^BA-|!bS&0 z5PiTeN5LXGE3Fn_iz*-Q8ctc2iv7cjs>-7X+&<8di*|agBoZV<)3ln@(x5NO=6^N( zvudww{j}N>*M>%&hwf38xBq##juHA#YVU}-S9KeYUzL(n%PH%P{x=DV`JPUbUKKsJ zriNCNN>yeK4n4P`eJ&$=Oh{)l?#V#zUW`N8@&6pBy?2*jmj$JanVOZW>6MGAky|rn63_4{OnN>2SkZu%6K9 z2x=^9(suiWv`j{So@76rm*zI*Q5D9Zo7>gPy#K^bb0;UdTEUE?dh?^XhFo(}J?yHX zMACr|HPB(4KsJxU7n^%AgB2!+IvGT(w_Mmr{8Nadx!;jnTkfhQVQ)K{xw2%)Ob?+BJY{+YV}2d?PoVhf9Gkr3&$ex?Mlr9CtFw5 zZI-S%JUo{y)Q(ERIw^5ZW%0Jsm=Vv*)Ve2%Qq$$pHPwFXq;K7H^;G`1G5_Hc*T9RZ z13M;ISr~Lzhf|K>k37zMoM4YCU0)fhO2bf-(;8(7f_kS6=|Md+d6wv@wwBvu+T}#1 z;@J0BQ=4R&*R?s26d&WC`Su1Puj&vrB&DPbxU#-c?q=&SD|$J@CEe(=#uu9SBle?e zlLCiT7^F@zCzQ2WixdC=C);pPh0BM+{bclbx1*|rzHJkxyW-7GM|C;f)@!-6Xr-_x z75e3V^Hj5YnI^=|A0(UflS)RLJ-#cxGhmS-MD{#DCFISFt;&4Bdy37$fEgh>N6)Vf zaFsH?jzC$O{bcX4U~#d&wtL&Gj%LSj5-KiHSYt|tzr4dE9FHlu2c}IkLZ8&Utpz>n zHGU1r{~j|lVYTPbdIo2U%cw?*1S3=6 zs9sOZH)f$NXx;q`&HsTP+FEqg>ZFNg=J0ArXs)19f1m~!jM=!E^8OegVSjT*KOuzF zxY3(g*6HQOE_7Q~9!F-d*sJu*@&jcKX+Y>Iq@aWkv26&yb>V~eZ3e@*L?v=Vw7^@z zr2b&(@rC~4dNd<4C$CJc$mOQ0gh}0sUqDWAUykGhGX>iZR}uhuxT*6sQlF{89jQUt z3T@R_Keanl@V7YRvbe3Q=Z&g@RBShNY40^%9EUp34cy z3-4$aKPg&IRCG{1amWqfd7p|$1F~h4Tq2jp(y>NjA4H$6N|^8qp&d2}GXhUPt_)o{(3JWU@+o;Xu{=M~OO7`g;nr2w@swni>((@(kc@K{%<@! zH^QSM@K(dHjRcJ4&;}D!-srJ!HZ-5?d>!!ZdoyOY3VwLEOBP+o7no%j-M_gg3Na4F zy_hR&$7t_RvWjyqk#XM5yUiH{q^cN8ux+g0G?5fK-Xe3n74LdA&%L+{;rT`D9rmxu zecWliq0X0Wft@wp#e7p+zi_9o;MS z8MH^?+J+T4FNHH1ntOenv&o!4soaIMt|FQip?TxeUBEm#R>A$=q8%B74YgT7MF3j% zdV>SjKw5G|tto_cAM8}Nh&sM}>2z75%CU1UO}@i+tw7vC6l8otHaOc6%fcMlp9d?6 zJ%hhr$9#pV$N|`KCfxeQ5bW73=DM-EhlCU%lDB6J(#zo z&!u$ux2w)I)9A`4I$;TI;I(%@%?WTWZ^(5%SV*XVLu%(*+SRif}u}%e8{(pzrMfxsr+AGK$L#y zkBTBfOf~2J!R(5YJNaiP0;23PD>P?L&XagD-8k>AqS^upsq1% z^`t5(nnlBt+dhTB21U2Bd&haq0x$4oc|Jo6(o{*sxipf zrUb$g`~cNq0BtR?70nyyMLW@mK#6yo7s-1yE3bO>ax5*>%Ik0!~m-060A+and@MWK;y~GZQk8X zvu!9)6ljyv>rB{s5QVA7EQ(QOJq^OOy@A!rAH126-zh{3=2aC9}Q5|r%poY z3rz|$dFp0t!&-GOp$IP|jpWewJd&a8K1+A1YM^BMkX? zIBjv(!+m{yCFx!(&$&{EH;F)*PJdPQcl^ zs9r=psO7KxSDEI0x&IGIY6@7NIc4mjYj(-dEUO9(8?gl?MOrAWWIVHoZahL`^8MDpt9 zT>Ofx4{GAP-Zxq|E&^^ou+(esJI3gLfDmE55vZ?HZZqcSOFz)EedkXkR%NC91)z3V zOFEpp^2Lu<1l4!Y4KG4!9sDt>S~%vrXs%#VJaQ}Gw>J=r+w6~&Ylv0Jp_kYKOW14m zO}}*9vq}|>U=xL)N;2UnwM_2G3)R89VWNuo05-Vgxeyx`I-BPQ`hck|Zpy3}G*0o@ z=?V)#Py}L0|I9zGcemkEn|`<28n0~DtHhfiTtvE#fCvP;0GDv| z-&4TQ9P!nN7?iIQcL;3;oH5LzOGe^Qjh~15Y~92@Ik#T=DCg0YIrmn@2rU465YJ&E zFdQiuS&R=X(AzsXAs&-0gT2LpIigfYOUgzn)QGfpUp;JcK2jELUzs;Kf@{TZ&ctj< zflRC{z3?q2lVw{VER*AJR(VGw5eY1x!{)hEQaLM09=n*?aGdyc3hhm25BW6-sy1h? zd$9+Lbtl6BVWv5h*fqeYf{Xr{4 z(05pm1d#?c$+M(yk5H>R8Ya(Ogi~Y@U7uxM8Uc{rJHcHazfU1`daWW^uePYhLFniW znR<-op3I^Q0cXUeiebT$<+`d(%s%2A+>s-hQb^)k1<1v=5?vKfP0sTEKyR?K}F zXZZ2Gwz5n|377`xQ3=M-d6OgLRJ5skr`QOuMpZmzQ91E&$5&8tp4a4wSuU)>9u(A^ zBP4w@q!^0iB#(dO$j8GH$8*6M@7XDy>y0x>#CLM&Ml#SCJoR`<(XO$!X`fExFw~NjAVW{fAk^F&I%A=XZbEK?im;uhX_QD zTmmu+7;_~ks zGwvL0&_gI<586b+%8~)ttd$FJdgp=ky{2cs?iot=eenhNB*S)C`eh1he9Sfy_anQ% zeQZTDouLO7XWls^I+%0kp)vkrwkKUEgV*I2Iwe`e3m#^R{OyhP1hk}FqRKZa*osjv zF}6BzU9mAlDCdtB8f8TpLv#qsk=2C-HAVR~ zAgSOJxlGXV923x0Ll=cA{P8_sn>~{u1kDPyQpjggGZNf}FHZXWQQK4efY|WPR@?v6 zTtoO!S?v%juT(qQGE4&xZ0QOK0W~G3bstK*eUVQ{k>@O5Ae=ksXQvEVk3Tj#K5d1b zL_x+en2oba#`ZhZ*ZtmC4m&$hIyAhkv*0+9{`r=2zBOokm_hEpB=o z2sxYW6cmOn0nrSmtO+5*;LSQ60sF|>Pzb+Dn*$HvsGXZUD?g|o!OS7}V9g)~59FibULmW5EE zrs8ZrmWR&wmpALC+M&5kdIQQu?${2G@#t+%n#=3c@8Roz-!ISCE|FmnKYkp4-!K2+ zTmApD$^Kt|_kX|DYc(!4(af>F;OXFMf-%L=*u+^!z_-M%Ny4aYOrXf15YEJ*tbCyzl{}dIoZcm6I2vdyZ3%P)CX!o`!g47MIqWM}n9fX~$b4gpO#U;{Cg1Olh*GSyg;litd*;C+$q!2=0UiN7jlw_tJ=A?38!D22?#|z>bjGfUSTQ$@AWX;3KVM}$$0EFNK}Hr z@@6fN+e{PK2-3RAJZApBv_U`Rw1MJ7L#~qNYwxqwmOP)q#TGuD0Y-$0{gyz$`c!^c zW;_xzBcK%>rHGz_kJFBNXqFkY0{%O3KG%mU{xvJ9tY=7#Xf7GvOWMWByd4!zF$MF*=cvz-CWZd%;p4uzYDl&Npt31+t`!lh#({l!jpTFq8{R44v} zj(N^^*dTA0Zykpv=~{pNUQ>yama4jP(F1YQnl=wz*;#ZTZzMfE>Q*cNtF&kw;4{!A z(?|0b%ulAa%P{K!CWU*2@W2jUsLC22$EAzz=jhXR6d+A!Tn^^%Zx{!OGnS>@{gIaV z&hr3#@o&Ftp;_yQ&@&+w3|@U`|4Cz`Ou3r<5~vd*`ARI8=4?P6LGdg_{u(p>mwbR` zl5?Wx`XkXXNApfLJLh;pq0UD+3MqdBE+tS()O3+8Dj*G~iz`h9=4{X*4q0)woyKCY zj>K}%BryjXcVuA@)ei+Cmugn>mv+&rH;eXT6P-_H{AbD5ugO(kouY=H zug)Lx{=nBAyi5tI1t&!FRB}#an+!4Q5@!^f66+*_rI4FK6NqEbMW|gdX?ZIYSqJ56 ztv6L!Ga`jw@D}b^)yzTeOxRNukIw0qKYf8QkEPl^)#K|Lf}k@CnM(%rtCw9~>*ao?LuhHUM(h3e_ttcR9KqWgSqmh~f~6ZRx>O-1EvSvL(YzY3c@LG6o}akWSl3gUa|%t^cI>%6f3QM+jBUD`+Qmhn}ep0&+EmqwI%<%7;`3Pxcb#n{tdgUT;| z#ZBXgTKhkoy;H1cQIu_awr%@t+qP}nwr$(CZQHhO+t%54*Q?6St9z4|%1hSQ`dI5{ zW{x>WAH8+d8Ur&)cFh4&^lgQIwEw-ZL17|Gh6hZ-dLNM|kQ7Mx zC9^;kCAB1pYED!(^^G(%dj`dDh%f%JZqwQ#-ZHq0K3#Nrj!)i+HF@&Al7U- z_PXpjj=bf5&3g3Zot>{V?R^3C8-7T_8Q?kDc^ajV=jq_8!Z;H9|T!{Hoyx;p>HoWOmH82b02KR&1XZ|I?J!9sk ztIvKp3Jr}U*B2i~g5oo!@;CW_Agk^BAYS6R-26b1h8B(xW)nY7Z z2^2n#L;B*|EZMw>62>Q;hH>dss(D^6Ry`x@bQCeY`fO(Y$cvW)yTg&$TxKcXNrB{m}va3NYk=Un-P(__0^`lsj zMzPR1D{*o;r%silyFtsF_Hh#Bt(=6Csa=7N8dDb5WJSnuB}2%fZ^~(S%1J|RvvHTp ztYvh{&9{?#nf%T=`ek%_N^b&Di{8WAT~d}{dDn(ydW@&iJ9j-*jA#>V%4p&OD@yHp zn*}9P;1u$pLJK!ql#vp+Ms5+-zmsxav}&1qzHOP4t!`Apn(=jrO>s3bEUg~LgrcIS zG4W=qhs<3|MTp)bE;cNk-Z1s%JWtQ&A`4GuRba-_7G$g*zdJH8N_bO(m(ZXlW4T#ZE@gi; z2BU*sV0sfX|7|{_uCwHj1cex&a;fNKb9a&!;e~qY+9JDgc+#-eCz2;Hnp8vIrBx9TOp1g4~4TO1L}0#>0J?zKnvpvH3ox4BTriR$cATuc`=zj z9}BWrlB;Nq`-@1mT5evUS7og{y!)c6(COko5TQ;+)Ip+ng|Wp5?|$J#ve^^`;rw(7 zgw25Mj4-!^O6RMTuF#BJxtkZ%5z3#%U#YdG> z(&Cg4d*jnNICc}+3lPS_gy##fqH*N;GL45!fCXi% z?6`a_uh|G_wUceYiOzJkjKTZAA0X-+LkP^w`{XuQgH3CbdQABV9Fr=si++IRB_aVu z8_W`98EM0Y_SqWE?vp={fjMLsrCsAx=#9AqIhn4DCP&ZvKbks|Gb+kTb=h~n>ggPJ z^f;KJ?Rt|KjmuXh^#PIJFi+na1Ny8GSQXB^Q&Lw|)F=`L<7S5pt)3UK$aAa_w=Ts$ zw2dcw$%xqHDS3?Q8IcX09V3&MdNrBq%1H{H=W!H-07DEfZwG}~g z(2|QsTTCWHc@OxF72G?%U+HRYq-a{#AO$v#;W~@2OUL{#8slTx2?WLF+6ElSlR3KTZnV>WabmmjqvTJd|UQhTyp%d!rJ+@*no!}msk6Hp52>Ht2phX(qNtO*67J%Wy>?FA?BxRyP7>}Th;|RzWg3QN zxoBuI%cs=h3L5c}1+M$-wT*Er{X+xQS&i&29p+02a9$!Jbz`Z)<@&_$cp zTH#%ovka*;ZlmaSBKQnm`gw_6`I26Ov;hByL%n*Qi2!kgefek8T$@S+5hy64{ie_+ zIXKi`B){QBLn@C-X7=D{&lgRE<;mIXA+aaC&R6ONAu^>3Vp@0AGfKS*_DST%*OjX( z?N^D28wpBjZ7qSAg=i@V+eYfH;SH?9vLlgmac#e$>del=^C*nixYFl*`c~~@7dzr6 z_D1;47Vq+*=@`EwS}|suiDMIIUrY*n_2cN1Z#R#5kN)n2mRYIS4f{h#W_sCCPG?!Qyly$w{EmV@0+YlpQXL z;3YH;rJQ#4c%cWm_=HyChj3eSv!n)6*6ISK${PdGRZ z$S=BAPWj?))$R6VF`8}ZZ5M?!CHigYW1XcPM*nz8vV5wl)0fP!bb`XYzt&nx-z#5|^> z3dDWF#ZW29YdRzWAqrNvb(e5~#S9IzQq@Etu4SF&lFDEG+~-%!!%uRquHy=M#)vN% z($9#Ul6CS1GIX|Ch_`yvDX!c3-PHSAQdE%1W;K!_H)GO4^fv}GeeIQT_FNVVP`L}O z6vQ6&1=K}6dbo8gr#CuQg)y{aY($Q$)agPu4Ceak*Joqg;WovDCJPQF+h}^6E_^tO z6m>|ecioY?C~c?!F|LOkJhi7%oV#4peT!bu4_V#Epgfe<k=N;&V& z0*6~eG!CvvyS=|21ZMI?1d%`Ak58SOd;V{e8k>U86ir&k5Szi9F8oRS<|kpDg-m~V(&2#&10fGddJ z24dq4O~X2@VhzzpkDObx18CRfKeGjNi^}1+LmS+Xz;H%=-Ef4j z53!x1&%l9&Io-fxhT8cD%CI_t;XpGt%uhpsMf)LO4+M*b_+I^)qQ&dG!`iUdDq>@g z0oj)*d;q3t0bmy7jgezp(^x&i6mS$xMqp|RsX#Kb$Jeq}3bl;NK;y8`#&L#mhH0LK z7nK&=KCtWNzizr=@Dk}pU<(N*ofFyR(IWl*zTMH)Y#G*tS|LKExkjsXIVat!i(>Sz ziF0eYNs)Tm7Z-**x5TDodBZ$&_zj-rkVRzT+Ub__%mJ?-e=E4ujx}bFt|!Rzj)Nzp zcc9x1DYtL44eoZ2zbi8IjwdHhf{XmwO#=16Tz&XegNyPKqMYjp8hYXB5Y@m5sW3!S zoLY=3y2y?5-GiIgm4Q`M3g+l=uq+irO~~d(D0)hS(!B$#nE_3D%awJIBBFI{S2X+w z{$G|B35L$CLFRQZhr6b1&8=`8cFn*jguE~hE%7yHdF1pPJCC@ySJJVF7v$?Ar7PkG zfQ7JN>C#%4TXL>RpzbgKZfdDf%NooY#9cdB_K@-E>U|+<_CQ}^o$B$PJ%-CIh%Z{p z@y#pAi=z{F^424(Q>o1t$zPx0c;PN6WY*FRv0xv{6n^e*_?2FJQ1Q!GrM__gT>pt6 zZFd}Jh9Na)zam6;@$M2c=NO$Wy4?lWJtI*6B=+NsgbrCkf_@R#Ck-dewu~q^xCP9-sIl+mFKzjBv-hH?2Zzt@%YtYF`=YuhQ*Q~EVbw#<>6I$|5s??Y zSZBf`m_~@*c}U`!9!qCc>bh@wV!~DkXJ*1md3<&P4VH7wad*NnQDjvg6Y*?kZN1r#gumZr~4>r(}1cO?kV z(!8F}a}v0FZiP0ThWD?)s~g+3;cKOumr19a2QoARcjy=7)KnPi;pJji{S_KU)Dz0! zPUUwNs9{-+2+GN!Oij_O?fNluhkGd0#nR|gk!GhU<0Qdi zbyme6>`eliP3P?P>34P%R2klb{A48yq=ViWnHm(rlr-B@Khx_jhEM2+BmjqU}Gx_8CFfqYG7>xU1u2E%A5VYmo8v+18*lM z0Bq{o0WIOuxcH3$9^YI!e`wgRQbHz1?bAC8AHY#@T;N#S@I(~_j}6mq6%x z(i?WwRQ=Bl$lng+phQtgP6ssVF`n%|J}JYb#>_Dj%5FLVR!6T~S}SC& zP}TzyG3epN;SQ6)s$4$Mx@?MSh_2{4z7MIZ&Yj6 zp>FAh?I%bhjg{VFhZU7vo5TW~OX(x+xuWPQL~ei4raoqGSg=YsK5*H>us23Mt!duN zpQ63M|GnEpm2OEm|0|>5{$jWPhn$@M=r;Xdou+10Ygepc6d#h5?u4$Ea(eNp$_0*a zDlAGi|E@7+7#9KH7^FIToxDB^<`h=->xktE{GzU=<8ELL$fTScA8T3J-Z{sW$3Dc4@>{sH;fhN1!Zg%4ILwVs;sAl*sj^&YcomYOsPtx&#+kUBXBFG7``t8YLCzVIsoh+ z74-CG9FDw-(-LBTv)Uo8ha2ve4Vc}b0k@3UrEIVmzr)y!U zsngQTweDe^950HHx1nOHFl*`e=xOLl(T&N5tnegnvH0Dqj|)poboJdWqKvjiIE+(h z0^4obYwdyy+L|U#jNsaGTtyi}+lDfm^n~`!yCh&C zU=QC$%H*=5zO82iqw3y1qvFtF%)n`iBkIA}B2Pm}_^XF2M6NU#gMOxGVvtUTkjJV% zq&P{jD{xa5?qfb4hbyn@7|(%A*&a}Vn$9#(|6GV!`-+Uh-4nv_F5Xk3VwDfO*&vTgJ;O%L1w8PpB zv^XF^z5+)M1r4|Df3Mi%;90zhjn>{{`9G0N*7PYj?$?~-$}T|~7!2qY(OXaFntA*#74Eai_O?~xFv@IqfrES<6TFGrarQ;06&d@+ z&3TEUUH!esC1n+DOs8chOG=$uzWiP%n@NSa=E)|Rv=8MbUD;B%R|qDE$72IkcZIZV zZ9Xomv&M3hBiXt=bM5T`IT%Z)=-ys6leeXP&X(6oUIq0Ly;sV1xEwUzG_Cd-fngWzb4}l`Bd#EVAsv%jEL+ zamj?aZi%!E#Q;LU-oNTzyXVE7tKKgQF>Kz24hkvXA)y5KFv~DB4(655Y|u5i(q&7K zh~LG$DRbdPuY3u_|D;8bTz-4aR(Tpl`Aj0{4V95u9p9DR@RLL#uBmdkrckYgADCo| zuyk(4$rD7@7bZsnZN!uY4MHJ?;e2~bk6_OCWrF_@M+X^QhvEuJpR4qRc>mRRcgnZc z1!fy_uW*pya>;ZSZue;aOj8P`)nIuqx9RNyzSjE2dBF zJ{+XLU?C}T6U?Rzk0Ol}Zq^L+uPfplTcs}Fh^^*W!^OUg&7ATqQ9xvTOeQ^@{1T8> zeUKT_g@@5F37e5MlbO-9jBu#D*L;91vow0)+r*|ysZI;T6O#&^eUs+BB0b;0d?ndn zDXMcss2{s#?hhAjvGZI46+@dldW9O)=$x#c(Rs})SLZ6C`L_q zl3PM5q1zOt49I212z5|p9y>idoi5uRAsLv`5_twQhB)V2dO5pc0`jF6gb<@V_X29% zIlqJC&uVn4n5*NC(<~EWj`&MWqC>VwOuTdMZ4L<#C1a0&#He}u#Jm5pR_)k_hz3S2 zZMKRHbmnjuC$TzrF(1CFW@u?Nl*mW}qhR5y&Qu0*d;_?_6mvstqd_r@OGRp2o~Bsb zPWAZCV~#^V*$JazjUme8QWSehJXX_`K#V~4c!qa;!jz%_Hf5w07^FzLg67557`6N> z)p*0@l!^AHd|jo?06i&`$;vb|6$Ew-F#+)f5`^YJX~2sgU)$!~st#;H1<->`(@^{foI<@}P>$?5VgI z3=<+^jw4lSyp3g%t2*BeZMNurKT92A~6Z;!bu93Cs#};ZbpD`#(&NwfnOiI-^*aJ4^RyUB!R9W6bjoYuz#r8^c|2erIuEZqMJac5Ll9 zIOwGfoWF2>vZm|RE8`f~RWeiN(wwf+X>fjqz&Ln@lmYHh zV!EASadGhW4RIdZN)C~TJu^!!CvB-WwciT@y9sWrE$#ZP|Z za%=ORRR%gZKEBT)SRT<6WZ&!{y`dOk({Hv>ijmqiM$6$&~ zpfpHtHZ{2_09I(*!=6qzP9!YT;}(a10X34l?Wg4Om?NUcG3G-rTC92#@592F5|h^t!R_)?Taj(zXc=?5ksIYzG0xP{3Y2Ixl&CTqW& zU3oErSH}e5@@QpN{mjGv({t2I_Xaa32doU4+D&T6s(b1=H;{TM6J%eB-u{I@M?li7 z#a?wh;ZD8E`9=s3CWru(y=r@Z{zo;~lCAmAaR%}yOD%oPUUr1w*Pjm3pxZr~ngPCE z+dMw;WXn*xIEh_8NsC>WAO&yMJeqYRfPm59h(ojCzbps98|>e~*GZ+fo2ae?0Kz-r z$Qno4Vgy6mgKP($GfM(L$kdO_stE1unCLIA_Dp>2OI2Tbc@M94t+igIwb&>*l;EtF zKY22|8jHQ9CAzm-)+#e#9H-)%KwL!l?ZQI;FbE)PV0{D3G%Y_EhXl;JDVR0!ed2

    + zb^G6qZ@8pYXXnlx{byQ#|HCfH&Y^l@CW@4@5V>Z@(QJ!=)jL3vlIezft~qq_&yHXU zy+vtm1y5XP333Ao!Ny6c?wQi*r4l09-^P-kBELakSJc(_4a)4ahwoqu(8hXGr=RuX zO}9Tr$PIw=hnng&R585nsaX=0N!L4k%zH>8`5fo?)^bmn;Te(T(Aeh_Qqdj7^Byzu z4RZ6y+2Iw`d8c3{lWqDn-MmYhWv5@!O;z9?9AI-s0q>+$-@?T(=oxE&5q7{)q-Ew< z!&7VI)t>hd{E%RTPR6ap021zwM#vV3Z8AmP+!tulElUCJUC$GDW|@mi3J&lp+w2`r zP?0%n^)4(#^c_ri(D0{UCc5}baH8y~^gg#16&J@XwleH8&dgufDO)Bne%o9g1I03Vshrhu2{25&99c!!A3pdLTwOZ>j+JJnysy zbP(St-cU^5B@s*zt)+j6NWU$+KjLV)W3SMv>SFFP1!a_cHHJHK%@b$dvSBUzmLG(K=_?wAsVs7gE%eDEhp+VJm zSFB}}e>qgJ3ZaTe=nFBh52RsLcsE4@HfY& zUf@t)U9s@b@ghx!^WeCHhh3fmgNgWD#rx;rQe1-s zG9q=^sbo1Y`|aR4qfnSS$YyEHr64WQ8}CvBV;JQa>1p%}E!mP&80}qBXo?K<2C{JA zRgIA{HD}P9>z&h5{Cb0DRrUj7$T*7(mU~w0WeB>gjkbQ3Cv^YsnVVClDG6A%1@uGxQBERlYPBzPg_hJHT>I$dhe}b9EEnQR^)>-= zq+#WJ6IF?%bwFSfv(nAZTjgU!Q>HE*ZRM%rq9-t;Cryh=40h*D%jz6TOw~;iGS)3w z)qtGbQKo~4l;c)*&}S22sr0-x&2pv7cMTz`WK$v(#8AOAJv}@RmN_Y7LE2NLNBoJ6 zmCZFX9_slq09T9VfY=CYT^6X!Ifo41V`Nit{v00hbMt}Pm^vuvAG?tmC_2P-&xo)W zY{9E(ug)-~&Rzbj&RxN+&Rv6(on2j_uOKb80;x?@>GW0K?^5(!D7P1eWOnM! z`8cl`Gg6_D-k~wOhd}<(IE+qX8#1F!N$0Q@d<@lxf?h)<-c^T4UKRUr96W<~n zEnmEH*gcYoiKek`un1y;CrU!V4ROyvU~*tjmxj7}xN81sFAv7{BnHtmc#ayXAD1L} zM^uP*4VbplX)pI9Uf_dSh#)oUP+!m()Y%R#|A18tsoe3wnx3!XSi(H{=*npfNr7t>ZIDhhq9Fg0iJmx~Fh8Bo;wK&|Fj@I;m zIc;|KV}2yRPMI>v!U#4`h%q-8|5srdOPq-IyEy=cUQa~&3beiZSwHpS%ly!JJq^n| z3SOI~@#V>0yAp~~!T=RKAswrHobDH%mfQ%xb)?SIHG}`K19u#r!gE#DXh~Zl`|x20 z>uyEB9H46qA^V6ttvnciVS55J8x{l8%eF^E>_GI(4K7DH@Lats&&6y5sZ!J#Y8K*e}1KMWDiO2pj1J*Z@1Q_9s4rVcS4GBiGl~ z)Ki7H+yu1#hsaXL3nOujiu#v|DV)72lF9`fOPe5+$3B+9u&%1_*^;$p)Q2I!Rq!ZY z_S#0S=lx6sM#%PIjeF2qr@jYcFmqk!VbM_ zuY)CoG>=#0o!YGHT7A?>xv>zr3RWRCu~oz(?S_OTL?)X3IxDxSmh5D_C;a8GU|2+G zy(m`!7nSDft;d?~y&bm;1)di=sRj6f?pon~>eGn{Hn}EA2qgfUa=m^NVi0|c6NGg@ z$pNbWU}wr5lK)~_-t=)mmF?~&ZR5YD>E57fDaghAAXpyT!^f9yY=IY3FluDso;?TG zoKckAug^v{_O8TV^yFhr(As>C8Jaa~>YTSq{zT8K0s()PkQkR_yiZJzpH5$x8$#j% zu~ek0JaQ==t3uYc5grOpq*b~8rp4ZG&47KpKmWbhZX}dQDgP!@v!ng_Bl|zQAphUR z_W$+ff7`$Qv*tFdzWXALqWsL9ttEGjfT6?Fhu`yC8xqoupkp|$`Rm=q55@Fj2RAp3 zSx#yQYsjA^H%V_6nJy5LpJ$giZXYPEr`||%pcrBFNWF|N1j1P>Fc=6CR zdOV&?fA+lg*pB>s?QXULIPKx_H0iCxstggM`?SHMB?RHg5|D=^0WZQ~*khQ}>!J>d zPS}B{4nK$v-_i5(#K?-h72zqnRY1!gDjK{wVekZ+PTZ+`$w23(-c?1N4o#p19iR>0 z5!*kd8hZ&(>7ooiGU&iRt{Qo1PwmFtOVjs~9TL;~N)9kX&@w7vnlr|;d`7RUZ!?bCE|%az!Ip>Pg6@{#Mpv3}l+!Q0l#^Q%`6~7zuqt!c5Mw0yvrT`~Gbb;M ztsHx4f-&$KT5w@vGbeg*Z7`Y<74kFZhm)LB&g;+eCQk;JJJU@iOeou-x; zO>r3$8Y|Y@5|FsT#*J&)i%T(G>7A<2|LE`g;jT^xLVG-JNF7}(5QfCtmBp5qrxPB% zSPdY@SgQU~O6);hdlBY^Ai>A%JX{ncW6Bfh6O`aE^_f{}NSyCEDV3JeYDyYDF6~hgjY6_3FX5fO)znv__xi%NN6676 z+wuMm8!=?lHHQi2d$@ll-Mx1DQT-(gtW>YttPL5|pjck3kHjVFF4jd8rk%;E=N?!K zIYFJ&pl~U7*yP^Y3dLHJgF{aNO9ceuYqzQ>~E#vtyBx@t9aEbmY*J`%x&XC#puXZ$4k#z8F133No=A*lUJTKQx#=ykWZ$Ro@5L6c#vsy zGUA@p=s4xP7UI;`5vqF`gBuOTHDqc`YE_bF4{@`>40UaWs&w_ zFhD9fq}hy2@l+K5hTJ^6T~_%pOexgh0^zG3{AjLhP~gHl(b^ATjWO&WGO*a;0dB>@>V)Xm=w+oECaCUFjOQFIuZez~?hUeG&P9I*StDk` z3&4$Y+=0X9p&?qF*Kmgk+9^8mL=Cb5!eO3M+y?T=uFc05`aL^GRmFXVE!+qm&D$r~ zfan&%Bm@CKkP>Aj(TV+_1MLiBK*vXNjuDHPzJ^*G5U-agGQfok%Xh)mJViZCJ!lcu+sBfh zWzA2u9r+8V7hv1Z{?8AkRK~#s{@4w>5INinB?e(96}Ww!V1SgM|9l^(hBInMjJ_$DcPlKzXWYB z)r#*T)r85^a%QYc8+T+&DR(H#QA@>+b>PENPt-F&?4Of*zh&*fqnyCaWzN<&nd?gs z_{@HHs0=sCmj@j5ea?F}I1PBXg%6ztM0jp9f(`I68zp&ps4$xSd>DHwsQAQx?gRj> zeKsMd-b(ZWpFA((ZGZO*F6lf|UpV3`+o>4ws>PZNw(uvBB-LqBCgMgVOvbS8fV)s1 zNPfG$lw=`M3g24?m6GneGF_mzMP0YUHao#KJ#e9Jte|f`LT=!UUw;m|Rw3u?9h_O8 z1EgS_6lamV`vt3P3Tf(d2UKGLI%#Ouob}iE8L^$~VQZwczraj)3As)E242MiLl(^E z)Pg`o$*{eJ0(}bl2ZaI)g#ry4YzfGFChjz66#`ND1X3AUG`3WGnlw6%Xxu^}q7?_kCb@+*(Aq6eihz3gD>( zM&>3iT&G#<7@yz$o34LW?3nprCSlyzY^Oo+QoXcHz?;RAaJl9ez^CD#lW<44PRGTD z2zlfvVn?}F4dh4&b~4Jlr}s!H7!QK=+uk+TSZH&08AYNT7qkC@YxNQ`n%^*RU4C_+ z$Q!G;pm)_^f`^#Mniv#fzXtD|d?ztf=SaA93^q5NnlCUCs?$;Jo(88Sw?t*3n~CfP zMtQLw3*(nP&F7aK;id;+ZvtJ*iSU- zcdhTDKxmI$uXXQypbEZ3w$C}m2s_Es{^S!?Si^xKi5l{|Rj6t#^=MJ2gOG)kYR0`MbcYVp| zw2HqJ6F`%KC<=rys$~GV_yXfi0dTTut~;`c^W?eW(ikB0tN! zwxIa0^?OBA@Vo$Z7-q&QOB@M4Er37!X^4%ZV(U4JeWv5g`ZPUjc8UQjYpEgL&1Cy@ zdFH3fZf54D6YTL1=PR)_B*fCsTifl9<1F`U&aIa(k$AY zL3XN}J2@?q0kh7-okKS5(}L3aEkO;lTd>z9v<~^M@`4uCVkJxsYg_-GNZ=~^Q&7L9 zu}yaW<#A{K9!WMWy<1`bG-#_^d4#6(&QbBOfIDNa_$jr{2UDwidv>ctaG(^bYYZ4K1_7oSr1;HpeUa)@5-t_LD=H zE7~0{ESepjXi@k@SF2x(otCL)4@6)h=O=EF*YsX1)Lk7kY~)=H(zy<;(~kZfCyY<3lkMF5whm;VE%XlIbTmWP_$2G2 zCyY<-%x!9OWS7w+^}<%sW3|o(E!4{+SkZgl4Bo>6KA1!1)4;o$0)98)LM6kSOMLP+t4LbO-wOzzXdG(o@T(I#!5EXJ2P z`$0FT?}()6TUmeZ@;gr;@9FXE#WMA)SXWsX9gXr@~4>=K7vk~W_P}e zvyN5{2Uf_VViWORuE;dlYU(d*GyBG3&|(na{#-vF0djOzcg19cc<8uQgiw{6A)DSN z>WdY2v*};-=#dIf!OyCT8Z%v8CIuJ;O{Dxf?Tkzk8Rl7bDKd?2W&={gdNVG4kX4b6 z0{$#YqXTb$_WB| zlIam8W0?|r#Ig422J1NsNnpej_^Sw!j^OdEf(R#7TfSNZoV`nVXJo6R+2I=!FrcxV zG&@|hkXe$9wz=yIs0e=gXCI~0T7o6~xw5!m9Fh$c0iil}T##xZv8&62 zhkEg8b}hsb748Nt^(IeiA_QK#Dx2T{$7G5+GbW`8CUV0Da{XbT@G`L}93bi}0|%D} z2KoscxS<7_c7jbCXAFX^Cd*%Qv6?-J(K2KaS{+Ll`u(7NPTxvNEWa2_FicrJvvXPBPrsnv zvWS=yeaCwT-i>X2DOf%{|5k+ca^hZNfr>*jzps8#Ci)KcGQWfT!y$8`v?QO=vicNM ztJu5AabtpT)T-$DcVo7dd)EBbue}U04&%dV;vC!|s3MobWTURR+p}UD$?;JI9>9Mc z*1chCS3ll+%1aj=%T9jLC*fv{c=}U#$4zRk=`%|91z8jJn<0osC(T=ljsWl3?^}xk z>YJDx{ATPo=%<#;U|%20o9{HvZ{|&Oj!abcN9rHrPq6t-ngvH9lskb1XyVU=FXOxL zKTbt$P9sGXALjqyogF}J>#_!rDT%Y6G}Vi(jkEs%tS{)}cwsajV^vtm*1{7S`Ac~71GHkn$ii}NOPBe?mwdE*gCFRkJ zBbgh()pJEZ5kX$2s9*+gwEY?lsUT^~BpDg95PyUs&&tZB7b2vs8(4;yW{})=j9&no z%3!6dg(!_W{<@|~DkILfMaw0uW#Oi8C|PHeXV*%zHe+l^X+LIeP+4P^fp%WkBWf*L z$HEV_n=(%GxFY1S02?-F%BMivS~xZ-C}v8E;ukpsj8UKM;z>MxR5#t&uefVq94S^> z9+$A_R_bXy{E)@P?QO{8y(207UH;Jb#5}TivxYJ!GRwvN%s!%tQi+k_OXukT*y)vN z_Yt`Il0ZR%d#|MWx$5W^r}WWVw))BZR^CRBC?5d~*Ws&qxxG_;{!_vQ>z54}29QUC zVBVLp?u@WStx{d32q4*4qEbxF)NDaqJxSY0IpI1)`Kqd;tls=MC#*UP_NPIMNMwLt zv6D{|I@4(mRhfF%-7GvXfN~3q9EHy+zJ{8hRx`*luc*a??fRNcDcWx15UY-vd_$@e!I5 zmwJ`RO;)mO9#>ce`J%)(Glj}#j}l4#3uuCF0IEc@FrqFN;&~s@nriw8HC4PEm$?mT-^YjJUfmdI*v2|=YZpCYDZ&r zESr|YYU0xH3YJuzl(-Vc)NnTA7pP2~UH8~VM=*ckp-U>4KODD0W}PYMQMVC7sc$xz zOlZ6YhFQinti?Ra^%A9NOa&a?@-(@h=$&&OLvpXpU9I4J4);M}QJkIJT`i_C=N|Br zn&Pzk)?wMpO~dSnMh#Ke$gq$;1Js$kVy0UABC1T14~=J#!iJX`C6sTJ*L6GVT%oW< zxl7T*%|5;z6m1`u{}%S2It&gkfE+xXeZ0@g7P)g zgkeU_P?xU`$aChUfgQ5_h(M{b7m5<3D->?V1QR=%F7|y@-EKKHtVx23P<;Ep=WPQ> z;x#%1XV>hI&9K`_c_7#aU4inw9MMt9QDKgol#hzg69`fulRkHL4#?kUcj3(%}+|dZHwnWRm@V76s2H~z&b&4 z6tM&;B+eQEA&zf5ucvjeq6ZEoW8!mnEFa0?&F&nLrc(ApQs3b&h? z#?g@O9@K$@M(GpKR8vJ5X_6{E=lD67~y)FN$f%tKM#?VoLi>YZ`p z>~lksGHTV7J6xuL>vHF;icQ#qhZB7xcq!kf@qw+e$e%ZxICZ)R^aNsBZwYg*uJs`?1u&JjActuB;fScFGd=-v;M^aJmX1})Ej3S`vbbr>`M7HTE{DVGK+bwye@ z%q%7h1!|@sc?_haC6oweHt{#RRWDA0BgD+&ls;ocgIVm<^{oVMAqsYPivPhCStF3i z($B<|jnQ#QsyyTTH#1mudPo>$RK5ay?(klF85f^>SutHUk?q0AAjz(VUmyc4k*&MTy? zdWZf)ssX{Y@w6lT#ud8dOm2UYaT5+dU}?zW#W*Ie zEx}KJguSH|;d{*JXL z?Qt)0W`AAg^Gf;j5d4VEFE8KXR_mB^#rC%hqJAZ?`LZ^Jeq0UMS>YX8fRvKt<7R^zM& z{hs6vNfC~O94=Exdb^P^uHej*DLJqJrF=57IBm>dja5xgU^BE%C5CDwowcspT6+rt z7jGz-?-4F+*WeVA;Y{E(cO%MG>we9vF5LCQ4`$hep-A7LU(n@}UVtd>cdp%1`WG}~ zZm4fwu-s>i`Jnrx5uJ@8ehp(N-M%UAuU7m<7{l%0JA#-ZtZ3J?puG;&vf|Y675GOI z&_t{c{$o#)39!FE5h9tP58sHXMdazfL60?@cEkP%CceWDUf$uA1OY%C!Bo^I0L+>J8 zL55bvbMCC4{Sai!HJ(GpF*qQ|55^z4jVmPP2FvD<8Khl+Ji8!mR_%32z)OcgUMem# z7WJ{?XP#qA{9*H<@K+gWZ!_Qm_phs8O<<(WQpKg+f&S=^9{iFp%?03cJDHtwqGlbg zlogv}?`T7Ur-VF8R0U4ax(m{8V~Trg~VK@p$7n!_0s3NKZFH{_YY&tftE05SI{u5SvavjIcK`lV`&N$`sP z9JXZAbxT6+$85JVmzpBiE~Inkk4o%kMzdvO>oOGWKboV@rH*FGJbC|N+>6qmNLf-e z#qEXY^C$rEeF+RAeF=^pxQ3UwMp<5w=8g~_Xpetz5K{`k(ePFjt<*898 zVUJ5M`oQ;yK#A534Z;pXIPp6nV(!Vu)hAa$1jMt={(Mn+V`P5k7_dmL+L$5W+z_Hy z1DW_K52Fc5!F?lWmw=c)K&*1BM z^p2uJG4!7ncj6iggG%r$Dm}jxDx=<^uymNFvK1|;MdGIlwAE zGaT}abnL81Fjp->>1~H8D2e_R;>_eV;FQSk(>*f{e$8X;9QJznOCj5AV8`MQqhubh zuu8VyJnkvdt0_I`OAJYU|5_duO!Z$?Z~x=JLq>hy`HG&z*2UKAbKt7u(G{Riizxci z__`uKUR-1y7%PV`ovmfIr zD_Es>ffsoqRC40wcav)`zf+#Qx7n{Rztf{r6$k!Ltl#K~2!)7uA17cIYCn7^%1s z_~xw=REkMg?FoNa8dI$?x&3B;r=SjRUfBjLk7M0|uy7-B;E`IrNE}MhY}%E^G@Fdz zE{QuE%ub4BHI@QBmvroWFyW!h8%ue*^I&djsg%ur+47*`*SlveY$YwkB+O(isJBKf zUvqqSR!KST(>?M-Ug)guFRqSB@X-`>R7jhx#-MM>=UC9!TP?L_4%^e5sX6M`ZhsXB z;vKm=90@X{HPuI1Qg4JDv&B#%AlBKY%OV#Mo*`!UfOw8e7TxbQi za>#k4c9xcXi}q~U79)!@(38g?t4)O+iFFucPE}k5vjn4k=rLG(k5vlwuK_$#f6~{p zN#>ljxn`_zkV2|m$!~1yo2Y`VU3qR|tkr+j=#gTj*adNo?C+#uV^w>(mUD>H$-a> z6XFo$XR-^%kN!xyFWhFsM2e9+*B2x&kRhq!sOTk)pw+OvWZso;es7OhY;W12%t+bv zb)C1=PNs}N$StDDm0BrJ92HNvzSnGBDB#NT%hc^X`9N;z>=uLC)NjpN9G*bP5Sy%~ z$0*m8Dwl5jl8w|#0@8PrDSRlK=cP1JuBOXxeb*~%*-G?WJY_;->b9gzZW+XCA+w=j z{sZ?99rnX^MJLtvjSmx|-U+DZ961rnSXTOk4dddZ_#9f1M=(hw;>1Pb%s<6vtIK+r zRnoMWb<(()HR>%Bk@b_JU7@GG+r<<0FzC^Tik*m93D4;{t^sr{ghgI}!uE;f7)iO1-3nlWY z%qJJh3`%;b6Zdy>-rMW7-C~EITxm+3)$vteE@RAgez_GIWI&yQV7OuFiYo zKNQlTcbtz8-I?>GPf4CGYvoUDxps(S zoHc42XR@9(ife5yr2@7pWEYPDPngZOH?2na9@=9_w1yvy{YhgG{-C9$&gvrPFnz+7Od}tvsj0G7SjF zn0whM)|t@-bME)?JFU#^JV77APw)LmT+SXxtCBP2yix7W=w0*IWw$cj+VRz5_0)`W zG|J0#%LvNJ;IAa`{i;A`5J`>u|g^3wB$#a+BdyD z#O^40?@Av%?4-(77I%3pn&`qV!yTRJ@a`l`3Q_%@*`5&v$y{X)^u?aD?*-1a#TPRD z%M6?D9lDB+_$*JmnUI(0+`#Idv6(Gu3x#76xr@62xHR|eHF5*Azyy7v;?kOW&rK=w zA}t0JiJWRHai3RUkVf9z+Xd?wci!tUNspEH#CSUN4VmxN8w0zAK_nT$%;YtJPufJX zpVM~`!j2HkEWX9~L#KDd?X}Ke+@u*BS%O`evis%;GjYi^I06Hc+74Zoj}CXHHhXhR zo2|p;*~JxcNi;W)kI(k*WRp!to5LK+99@&O^mVIr#s$C0+Kq`Wv7}Xi8&%bu-{HG+ zs&}idDhoYx5X?_~ebnlV&jDMS2JL?Cp?og3Xqmn^w%s&tU22qIU)=Xa`y$Vq93Fd< z)ci-+hv)8uX)ldvat`feUIyxW+>DqE^5itYO{UAu&uh}5=gx>yXe`D47@MmYUy%8o ztSyNX`t0RK-J3}z`sdqZT)G9B@H@&N_1YnKZDs86xfbTyYWH3BMX5>pl-Ff026{h_ z7fVej9iBw}d2J@6ZtPPj3}t!F2kI;(*Xi9ZTL`kwvCm_XZ*i=~l74rQN2)$?YK#j* z4OBf~x2u%Rljzu=H&fQ6=Hm>Rkmbk_-DH7lyt83*~NSIg#7H2 zg;+ep7dqU~`fHzlzn~VN{MT&M=Tr4Wwk%`F!sjIIUIFNQI0`^2qa;e#a%+qxmzOLO zw&_=W$+8bUH~!I3RX~~1$?U&5VG)Y?@)2XY`-p;roWKYxn4O7hg{wt6LNjv|Lp3s2 za@amE7AD;bga19B8jFzp4v#r)X?<$3b@+2-b+x_NTGLQg(`4c!BXRitjG-@DbeJP& zHPT${&5X+HUfpK*1j#-imjwP4id{rvYwK%!JnhME!}U=pzKdPo6vJ(uV5zaXIb$Ir zfH=L^TvF0d7RqG~8j-62@2AYfESj?_dTtH3y*F7&swsf`QfWOJ!bhfj&_3(N8_dOK zgzdcmDaa_+^QlF1=s?rjaC%iOrUf%~zeyS|QY||$^ZC9ro6;U?u!GuPvwltO4GAah z)L4hdFKf1TDHFydHqR7+&9p~)Yga>!_yOn-sJ4gNq|O_jBs;0Ap}`g2cT&X@yCn{p zRJZC?3)~3Y=>0hQ*SP`WK&y^iixtElQxU`@~ViIiqtFBTH`0}3gEsrb_tMwzn(9OVB%f<$td@NI60kBjx#u| z`6fH9OxCrpQv!c*w73b0j5GL0lM#;>FzM)sa{Z<#ES__odVoPp60`-rL=AS zLrJfEFa(9VT;OPl^#BH0FSSCC!3&Tv_SPgEEKz$1ciiBMpeBnWYJ7sCuj@31(x@D; zPE~`2iIvepFeuNpNgqvrZ_|oF=N(kuhBlkx9(iev*Gdz?#YCpJA4y%?OA?N`6CSQpe5qX{S1i2y_s221ueyKmT*v$uiTS*J4#sdt zRoaF$lBrNFrt3$Lt{t2EB6AMWw(uowh~ zST<;dDqZjv`hw<^vONr|7d#i|wQv*Zf}HO^;45f!Iz3Oa?77s~Mx^)3I!^Md=+he~48G}(!;wpPKo z#wqhL!7IXwmzsew){_d<*BMeOAK+L-bD`l8Y%C1{Seef7hX>>@vaTFPbsaU>oy^N937?sIE62gI7W%icNSu{fWhK=VAJH z*MF$K!Wr}=VyV0dFwtH&SgEcQv4s8zYY<|Lxs=kQt41&H5?#@~XdN97Wt>xD@j(B(R+wPBWc zBOpWM@V=wfj&v2vbk5D+%j*u@5EIMkl&Ansi_ZB&57n=mw`9 zdv}xQy>ijLW;`CKZ2%0SA`KF9``oFXL1Yhc`tuf;1vHd~-(6O7h0-s5<5}Ne)Z2#_ z%RO?}D30L>?FqLGM#686UMu(_;^LG2IS%5GXad4Cy*6zZ8DK5v^xCrsw8-gRZG4k+ ztx}Kam8SGiRvO22zD#tdbe>G;jZy$Vmw_|CO7)D{@|ith(yU%zsiv22BURGn$#)8Y zyYp1EtG5%_^rzRHk;i3<=j!U&!@6nOMe7e#9Dn^YTjttZKF$oE_AFqb-yFKpBB%i^ z2W+1u27kXm%FI68fP0|-wPTLO3KdZJUO>RY`SC;Iztb`Qk3IeWXB+Kp=KLQvzW;2U zM>M`P@K(^jx_3eY&0Z53;%&8U=F3e3Vt$1+r)JIFQw6nK zYH+qHm2~=8D^^#_o8>-*tA3qXwe&Uf{rx8FU!C@RQ#sT66KA-L5FM_!U#@%dJZ}76 zcHUmT_&;!e>%HkJgK$l07%1#cZ@v0#5(g$BmryQe_gp1(cHrF)&arc8Km!Rkr;qx@O9%82GX;2 zN4J$>YpmV$`B_#D&2w(#e~_op&#v6I(~Zby&dSYVGD!hRtm0SDQ=Hfw%Id6Fo4d)x zVt8x(cHXT?jGbINxpNt7tJI2}Yx3**S-<;?KJo?ghp-#KDt)d((52!Kb*_JN;Za!l z)uFN-2AA4Qz0k7k$8J&u0>I-7OD#ro(vf4g#QWIn$$5D;a~Tg+sCdj=J<@kUPpqbP z97<_PDgavgA3-(ZIIVl@GwRPi=B|H;Su#cp5?qz_F~a%+tt{8LW6aY7pW_ErfW`qTxkwIF4Eiecn{%g zv1(&O`SB}N&(7+T-%yR=qiQ1CY|CQl`i9Wxm~}2i_2Y;B20G{U$QhdK%6s!pvGGZm z7tUp(wOgoh%ydznT1B!|*QLu5V7M<=H5#JJFc8 zV35%^{GoOYe{(|Ij5OU$it>ygq=fl-S&{U7(sCDsDyhq2i=r51F>p|9=Zdmf-whG< z6zE{8*{@loWFC;H4W>@b0&d$!od&m2=;K{dt(WYo5ogwSx*4gtPtFGb1((wxMwOVZE;q$N}mu)H^Ybb{TaSWc!p+Yf+Vh zX~WPTHT>Z}wpo48_{?L)VU1s3qTtY>Ue_!F5fsf+Q*6_))hO<2JA}_|KXmmUTmKmT zRfl%0=j$LMV+2^V4cz|Wdp4dDV?Le+L(Kqz{kZS#!D!lEZhGKPbrXET15$iK6Y>3L z(8?}=z##*u2ZlP68=i1p|18^{H>gkT$1Li4{d6n^-*Y5DoYm?(jK_$V*6~Gqib7%S zi6V-Xdx}SK%uE-#{mVP9Vi`4QC>xL8Bu&eeWmdRdJ7P0jCD%qgEn*5*W4{)ZZ=$lX znFqzQe!MC-#nb3xoW5>edq|~ASyqJ#d-&gUk^ySt@cSEZaLXelE~O;#KKR$iOc6c& zweBGghg0_NG_^5>$ipyQ!?DHMj2fJoHO49o84amGJLi255Qn$Ms^Z-wNlkl+ZCT{} z{?zN-f!cy+35ZTazHOIkzx-20({h@{-oCafmrxxyFCMVg8YB=5W_$0x!|^(dK#4S( zl_^`FSi@GrDNc?gw^*7{Yo|ON^Hb&~K+x3l!)AF#RQd9u2df=8%7xkCr?;9P0(@S5 z;XREU$;4wz)+r|Kb=G)e9J)k72 z<0m}c(=@29geKL2SPnY>^onyW{U@~Vm#En*%x4pnFHD}P?=*JbqWTDhBO>dcQSR-u zvKIQP7Ofie%W=~@QRT|E*W{oS;ArWT==wTq^kRmXHX+ff-|}j0v;7Q|X^ZrQ{U6m- z>V{}2Ur|N}9p`JZ)2-s4N7WR}9bX@Btri3!?fo3FzI9FrnTBy{?%>C#$mu{IMWRL4 za2R&NKt)4@&i-%0ooY#cTDdh9bjoz$`%O7lI9=2y3L9NX!D2xEcRVB>b(1_h@}Wc`+iweBQnp4PlXn&ay<|#;o&CiHD3M&ED26 z5gh{Z9KScWpqJDYg_(*qA-Cj$1XrKW_odb6Q21BgRmr|Bs*Aw}_=C<=eL{1bcRjK( z^Q%yL$b-vr)CBK^Qj=!poS4duN+&6@9JdIzu44%R%sn+^T7ha2U`0H!=-fly=mjE zLX1B7T`%Xo{RaNAN57=-L%Ptq*oXUaL?EO!W(${Z3w6D)5;Au%j6qj$6AY`A&u1_(;e8J7)z)jtSA#{TvB%wKz zCirO$_(8YiZg?>R@*3pu+d?I$oqB$d<2zF+z#{ut`d?4i@ck^{J3vks%+jwK8XEQ}Vs)-WZT`re{Azgf+*=025M%&$9 zG5@^XOy^&{ph=^PI#Hr?Bbr@Svn`hz{j3y+-7I?0jQl9=bY^WyZ=J08XZJBQoE*3P zETfD$IYX>(m^iHb>^K^&i8zMbRvR;EtCr?1?VM!C;^Xl;OfB?S21S_WL`vW=iCb@K z^`f~sewpb}4eX+4p<%M@TGA+3pPH3^a;9?{8=L9r#ZZN_a)%&iMYo`Yn>%f?I*aKSygi{oR`Aerlier>%L8IYiJ^|8 zD~6Y1Iu1Jdgptj}-KTV6abK8d2Qm2tSaUnzuuRS&7ouz}zs^ocKM`Bxif0%9x)HBb z0GdiOdXnnBg4Zsi!MXK3P0#!GV(oe$6@(hXJ>5*;rhT;#4>ybVs{P^XMno$|?TMDj zAqHI$M71cjUithDK+|D~UG)_9DY48J$Qi)1pT!A78rr zE45J`A5pDU0ydj3lZ!8P64MfMCN52ec~IHpY@=eeSi5cdx*dc zW3ARY`vIv%MRFAqqY{gUpilH9xD`rm#gpiw{k2}n)~s(e>DSpKLjOWKgZoav1pbKO zk$%?0yO;~?!SSzXH``zz#Q9qF>)~YuO~`N0C?SG7{+JL{qUfX?nK=-lwmI28zOJ%F zO79HH`I#_>#MMND6GvNg{*X&%DM6~o7b|1=k3#!j*t)?24i}DHT&o2C=tl0dWjo_M znuB(bKl&D!f873_BUC4F&iDHI>K#@)t=>;C+QuNCu2xShtN$b7+1DG~?q_Br_F8Jg z;lpX+!DN8I`CBp+2s{(({mU6i?g0ymr#mPQ7s1J}lRNx#nQ@NA#`qvD5`^|crwo4|}0Tba@l7!|)H zKcey>@xO8ab+3B6szZ3B5SW7mP<)TkqjHk(I7b7(=BLUifNM1 zz(1@NUyf&b!XJXa2;M+*@(K=Rc^gCctk2d@g5iByQ z-IXLPB0}!U9(GA@eEo040W|fd`Nsc1`+VmliTrndp8szR|No>6{I5BA(%8)Qwv;eb z95IOkaacdp{RLQHJQxa;lw_f#pd)h!`Ok@IM+;%7npJIsmfk3hnrAT^Cem~=s5I8hjn*;;0%@Hk?voDBYXQ~-~y^fV_pLfZ7}-kw?(|4 zIWUm_^alji?@T_6f&!G4VjjF+ErhPHT1k{v50@SqD`k(_f(7LHz|Is1}Xxzbm zc15k-%JBGyeks1)yns76hYwr<6)H+iyv zhrYt95iz4;#ZRYbMXXo4NV4#PL#JLL3)dY@j~VB@&?Z^}qg^@A9I6}g=UQ<8hmoI1 z|CzH_5P?J6^~JL*1gMZk+fDp6_xiU5Fed)G@&YOpK8|C%^2PhpD6ehu<{X=|5Z8^? zKx+IxWH4lEOk-(t73*wSi^L;m?gqBbVxe)W&Pb%PBTI38|KT#$uLr+#;pF@bn0pZR zdu_=jcI-CNVH zRd?8?Dx-*h8~!u|oQ%OVB#|lcEZT%tOJa}Tw}kiUhHUi-a;96^9Mb2*dItaOE_@ph zq{K6bZ;dk&diFu^()BYkh^rdFgwcrVAy?PS^}Yn{=>h3$(Q>LimRh*5G?hwE9Ub-P`KedCS`U2i zVDgl3LuTFWHq4*f!%m@*A4yswWaRb|D7}_{Y8hX`yhgz4CH0n*8=C#?wEwk5mQan4i#yY7@sbc{-z?K>^j|&3{mm~15%f>4lPGJh(GYdGokd;XfAhB%nlK*o$#mv;1 zq2;GMd!i#rDfL^_Pm_b!Hc0c)5pDfAgR;ea7bb@3s$0+NpoFd5r(*-p!TQO+&tCP# zQV>PDBUz`ZX#riSc!$=k}F6CzJBa|a*VO?%;Xn*SNeTd}R3|8#?LFQh+QL+y_ zzxh>aTMZ$m;94-iwn_NB2tc-(InF?CcEvc?7&g$rn{*r@H5Qs-PEL!J6#5|ix3|7c zC&GZM_E>4v6udIMuSKYUbzMH&H(4tRb997jAml@HY%<18l&qM<#{!P>9N&nweE1hH zAJFrP@%mTg!Sr22hB>c$3XESqbn@D#ds!F&o# zjjCmea22|C(Y$um(MO1|eQ3Iznq`i#h7(BTymu`x-7Gwe-4U5B{3Eu$Uy}lD5Q`eH zSypo*hK)N-;ZK`)S(snD{*c>xl@hw0a=CU9P9<6OOuKLuyX&fHO&Qc4j=M>ma2DJ4 zZ+hWlu&3dAyNW_ORSC^(qmhwo4gw3?dCxQQXt|# zKb1R4vea<&@m)mEz20WL}ATE^4bw~-XFtfXf`Ie`MR>m2;LD5XxtzF~cMyth%i7{5o z;}EQt=UfVgSo=9w&!GGg2dWp!&cs9Q7tfYlSx9w?=H}^e{8H`7s^2B0B#X9&8QYfd zadxEgRLteAqhUt}Wu0&R`cn;-WwZ=f{g7*}Nqm z`DEiLAIrrJ3K?u3(Hr;;O1z0b+1#~a0SjyB?_uI`3moX&wF`%=5`5s1-4DvWBBz~^moksj${Q)VibHgF{xiUt#@J^4>UP4 zsxr#OR(7&Epj@UdR5H-mT?w)YZC9IB%lP!wMd@<2wiXwc8|!)-YFbMg zDvb=TT2@+S?4IOeVTxj7=~BYTbj+RY^{`2t7)mhMw^tn=iQ}k?cF8(1KV~6`QaP3yJ7o^;MB1} zCQpDn={ada19!g5KZHr(IHJC$d=X`1_*7EeTfx8^=I`UH1I3XlXI(A@9UYrDnF^3M z(*X2EQz68W*oy|=BsSlZB2I=5LR3(DU(70;5v!KD4n^+zwhO#V-)Z%hmVDE1fJcTv z!UuXoxGI6v9eg!+)5fkMg_n++7WkGG-7y8!+`K|&t(3saGVUkLNy)|41=8U0`{^Lb zMc^KNDfLZjYKcy(qGYv-O>z{B#1R+9Nhjr1eZ?=YZR3QUcQ}z33Zhn1Pw7au(Q=S4 zZ}AG6P1_9z6XEI)oBt&lSKCB478@lnlY`+|+EN~PiwXr&VUPNNYChLs_<3B#hsQds zHH*-bW}aU9ue1&Eg-jvMz_ZfZ~GkSoG_n$J?S_+JxZ&H^0hdh zl`sg@$({(q2QDlua)hnH@p>=IrBf$b#&u8cB_RLWqzH+$lH#m=2tuTY-u_A! zH!1O0yW{x0adqw%yePq=I-M0zJWYGB%#|?Be6ZRGO_QXW2DD}+i-ZdCS`T&q!TxU}mQVeX`;S@XPZW>2&&S)WXsXEF`=$Icx>f}L z6YU(E{o^tWGeHX1(qFY{8PTD3FT~LShBVZ5o)E23P>r)hS+~W8K5>LoZ(A7leM8Oi zi#3V%Nv>VRv5D+bo1>3fr5aN&If?o-R$awiWILqi`cgfK?$VVq-tp%|OYyz}zpMn7 ziewR|Ube`3L>76-dNdY4$oQzweFC&TH*GgO$5W#v=(cBKjb#qxn>&H1nn$U>8{t1XcnBv6&orebVz@*( z`|&rX5BB#69veYHmdZR|ZS}+2yfRxwzM8IM&k|FR|@+|*Av2#@!;`D+C7`ySC)4G?+x#;;Bd>{6Y4>y zTPunX&^8qJqcL5iIS91s(H*X8J5lp^RS3&ggc3ulCbwSn1`W2S72FNx{Q@1Y5tH1o zeGg_V{LSNsglYHgtb-QlvrLqQgyZ>hR5d!qp^{F^iFL5B))2uk0ZGllD#aa@dhzX# zZ%f|E>}Z*j3v1@-_K1TDm};6h<5PV)SvrrCTwyJ+G!}&pp0x~~=_K3%l!2mTo)nexhIVO(VUxUf#8-92_&={1*F{Ytr|5^blBEOsMQYqBl>bj z?rc2Lx+p_GFx+kMU`kiaT(NU{3!s&1jtfRMP!P(aph?l17M)JUq@}BTTg#NrXl6fL zfbjFyp1n#Pjz1*J#N=3_K7-FFT8Fevq{f3s^vk7&GA#nKaRMScwF zB$)@ltUkxBDHI%;b#xDssc*YN>x$gS#|4ZBOhTqR?aI>4M&1spX$o9o%S!jkeK>W& zs0PS3-JAsJp(zW1szsLtXDSV%Tv{TkEVf&Yh+Tx<8&HGW@I6Wl^L5ToE;S@TH$j#g zv8En?$PCoZoS+(tbk785m`p;2<4Pc143)ZSXAIK%%kY$!*oke zt@~*J^|WK7o5RqwC8DkPYD(a3#n88uL}HsGGb=JTtytjaF*-vp&e$uRvUL)SHOwczJ@3ywY}6)Ex_2;D zc8F&ImM2cZsR~Vpz#V7To;OBWPbW;=+fN61+GknSKWWsU+Knq=M!g#gX&6;MR0Da3 z1evWn8;W=s-QD+KYcKvn`_OcFmb{P9^j7b4!-w>bg5Xrj}49tUWV?Q_Ee9u+J8Vh#XLO~ z12aqcnj6$a1U`3SAeu|Yhp0`y*#A-Q_gf9--}d7Dwpg{I<29eIH!>_buLIu~NiLYQ zmykjI!H2;?T{YHhWKMx$aF)p@AJW$vZ%%Lf!kff@^+4QEAb*W~dmv=de*B>PKS(0~ zp@4`wTe<%)qH&34bb!h+wtp}A6&Dv@Ny+>S9Bd%;uXu8B^b)cD-Tci|X)J{GD^@N$ zP$_xN)8r{MCvK-g=C%&5jO&x%9R-6Y+T=?exLezo^UvXDshI1Z^i=bAv7D;j{=U*9Kb;3hAk}=eheSAwfkwXK`~(Y*SaIJ$^l>(vL35L1s4)DP z`OR}xuR%c+Y^(>-%JXTgyrB#-wPomdYIo@9bK_4I=)P(qU!rcO*Ct;U5%5hb9}5VqQ9PLl@jy}Ba?D;zHb zLji3Mdd-o{03LEeuSa~~p%=3sRaS@rxsQ%>?Sh015m)OQE97swEIn1R?j0+jW|AwN2XEKea>M21?KO69$a z=f*Q~vk&??ewh?vnEIwGgQn~%5~*(zJ9DVSD5Wh0cr8}$dx7sM-Rgl;pm*o*i#>y5 z^!`q*54$R+;+ufUrR+&}g>K`S4zg?Ho=KincP|30SQ@2%5{i*JZuwAy;xhn~fBXV0 z$?D_p4ZXN^#}J;iqw+a>iH=duLQN<9{gv7*cK|4XF!thB89;RF4r&5*p}1K*HQAbz z2^28)I`c%YgVs%(K&Rb`Es-6YIqD%A**1n<;)NDzWz)sV^K>eRbNj@qTUt#{uI7neHeM%1X4{z%D-)R^V}Z2D|F8p$KGH$szz*9}?M83dZCUK# zr*t(V`!2qt5?1d#K7;Lir8Pp+Wt^rEye>+f0j*QJi+2j=5%3yXD+}zXN#r`sXFGGO zf%D7$gcKEB@4t*j=Bnu)e&@(CqU8Wm@h(wl=)ymftB(80K@2(IkoU{t#P85 zftQfR9|StSv`mrNqv@E9vFWMnx9e`@9hBFQU3xZA??!m?9RSY|J!B?e+O~R7iFsRG z8H5)D5*u4gk?Kl{NF)mNwGUENHz#$gYYa-O-0MYL@VWG{wg~f(nVUaXa@Kh8Oh8`6>Dv_lAFclix#@1L9 z6Hluu($Ad98C)6=`X#S&J`as!%_%9oJI+jYANSzYCLOP@9We#elF6K#%D0jsU=-F(q@tsgOM&Xl%@xV!lO77ug&QRWKH?m@%PZxG7d3I#>4?0wZA_f zgvMnE@;IC@PP1O%0Pd+Tum(-OS1M7K%?e&c1Be{aT+-NLg>{b8z~KKWslFsi*VR?j;SA6^ zXkNk>l~MMBQddTmTv-Z@E5Z2_;9OW}> z@9o1^M)9%<^}WP$cdE*^(|M*TbB4zvABaqkJ?5mly-v>7C9V>{hPKN&hpIi z7#b;*e~j*$r-U)iYY)F1$>up-k>Nq@Ms1o8g|O}S6%T+e+MO=1GvD5t5AT|f0}~(| z3pd;!k~HC={ga;>XUxDK7QD?hoI{G$pCoypmF|fU>=~HMU_>Erf|+pSNZ@tQ5Ly1) zqaODwFc@~7#@wi{e5j`o1X2|2CGwk+p$_zKKtPO;=cQx^5*G^mLh;l@yNy^6AF~`W za{3V$Rphx{6}-k2&Y773^n*f@3(?v`eCsO;9q!9G|BLXz8%U?iBe$h(CG%So!Gz{I zOnS#=R2(_P%0vKk82R3l&DwgHeL@IshoFwS%CLKoGjV0uoH)18!M)zxJd*Sks%kde zJfS3Sm4&)WEOA#`A)qbt>T$zcT}694L}A_|N+Z(2=sBQ4bl}J3ElHt1+oZz`8TCrf zdv{Sg&v}+=ymEh@e=tng@6B&Y7o`YI6^|bXGXEBuDb5!>la8+FG%Bf?C(3fgFV8<| zXZy_Div$b4??z>7jvmA`e;)<@YbgZSAbtRc{uZ_W;ZU_^Y|LgbKr3r4n+<+hvdl(tJyxmW?MVVdvI`GEk zgl6_{7ey^QUsChAP?COeGJ%y%V*RKS?%7W1%g!Ue|3%;YT$$XSOZAePJ-lx5RcV;Jx|HOuFxYs;rm@2RiGd9ZXBYdr{*9j zGB35rC?uJ$Bzei0DeXY`$Qg0NGbsu`&cK-I@?o)O0Bum5#BF;F#Vs^;-f$Vot1DPu zLIZZ0_3bZyo!F(<><|{_RbX<=+#xJ;Zv;c++V>;QU(mQ*aoD+ARp+qO0DO1l% zk25HRPT}(i_R7-^!J1J@1<`YicX}EXpH#SSpf=Hf~0@BV*Eks!sP|d2VhNT?By6tuVkc~w`J9rCW@Yij5 zt1-gawNY<^8mzW=2H{U?r&{SVu_zfPU@LFT5I;M20PlDb&(UQX<(G@NOL$?WiLR0} z>#)?g@>7hv^*H&9tA!oDyjEKSGA0_hrLqZ=VBwv6>+vEU?a@srRJ0*us9C$=f?2Vk zLsOmINVltaNSGp0#%x22PgNlc2XdxQz}o!cjvn4)?hTR|M%CwYOJ%2Kwwmofztt`| zm9Pm@VHQQc9hV5h#SAkO(JY+59K2M;vMu%-r&?3dy~JrB)UMRKl7iCz9>(Zz5n{(o z3ztC>AcT(#sZUIG)OLsTGo@6sUz-h9)zWl&%xbU<>$6CFRKe`Zw9DtOhKm$y?U1nP zXSac;J+k8evq?eEtNe>E*vQ-Tz?h*$CsK4XA&-mf;0oq1v@I9-)XqtURbWm4%)S7kRhCV0_Foz6jJo;0f-Cgf?CX`p~eHBw%>gW!-T z*I3=j?<^{+O_a1PDDwzI)s2~2%^eBHU+!UrVR0XZRoXupY-03+B&5HLE2Yt-$??~k z!nCKjx01yWOt}J4P&)UfHYImUp_iwnJ*#9rWT|p_gi)(Hc}n+P*Q@ri*X4N%4X`1c zuBcD7!Rkk6q52~2Dc{)oh!47<`oiy)%j!Y`FxO_q*bOmwYB5F1?Hb3l>?5@jDk$YN zy3EeUDKVjFz5?9%rJcKNwX+d)%J&iAqTh?5^ag-T&DYihmnY4b(qtVkW*xMV2SnM7 z%TB7Y3^k5;O1sjm%wkXRH0Qi#~kK=Cc6wQf>7A188wi+K!q1`GvgnUrml)OVoxh}p>QTSI`%Arz^Ma+!^#Lwz}|TUPa+9LM;)18OJr{(gQ0aaAQUT^A&aklZU8PLTzM#xgcetL(RWNterQ?qGwlpichlWH1d=%h26S zR76cYyz-BdJb!WP{2z+KZZ9Bb2{hGclxWdq!8zEisHt1p`z%o%X8NKMKrU$yNV z9{2-q2WWs88PdwvoE(U-jg8(~y#s1hk@9g#p+{_f#7}FAo9yqjZ^zl8j~Qc_AW{rE zU-?^lk10T^2ANl?7;2HL zi}}~SynS3zv}@pWz1e_MMTa|f3Q9*rH6CHXUQrq@TDy#pF-P2_$s8PD;43?gk^&9- z%SME_3exCz>=h)8i3?_?4T-TLQ%f=lV-meEh}|Ha6LA#Q^;j(nQS-R~`F`w6K{Uea zk_V%P))1eUv;qGGq|R2ilzFnR0ne&G*!6drml4Mk{KUhLKSFZ?I zKD9)DZU=H%e#GcoVC;+bvLDhER#FKo+-HJ6)GVA8+6^fLdjFAgHJ4gh&K|08)ox>J zN@e8G3r)Yv6;a_(t72E6?_DK`PBQC3#u+TtlesOU98Cw0Bhoo}88*b3Rnup5x+at! z!~ki>14?}1rD<{d78Vm~;TlY*keZv-iAE|`lR}F1u!6nTcq?`p&I+_9z$=yepf4k3 z4Ph9Dw8P6Cw=8V41zrx|SK#FZOO`!+;n+?cjefmvTLS(iU5SC(VVP zN)^!0;c0=It=nauHhoJS6NE7$vvcDC3V)Bi3cHp$f z%GU18oEvLA+TX3aba-R~;^#H|S}E&YL6ng8PX z<_#A$;+V~-%Ti{yk1@OPzj>bZy-zyoL6D{6b0o!xG8nep9!yxykmf$H{e zaG*41jDYJWooHkKI5RY63?@yOdg{*Z$HG_{sgL&Mjb#(Clky@;WxSAuqsw^%P7u6GW{E4Nr(#ySqedQMg1 z1`#noRNv6g1Afxz66j)TUIK?exHcnJ*4W~-1809EK0HTx(T@8tGhQAG(DMdCl#|lp zj`6B}o9>=A(iPwG4&vmv{xzwL8|eXx1uW;vw^QEFiS&Fap<+!_#TsMvkQwA@q-$H{ zZ(@#>p1A)iJxKt2!GrN)X8z5WH1_?4pYg`0ALVEEBuznhUFSz*5^N$(}L7we<{FX`gULmfRC6x;@6q z^0qg`#)p9!l{;LxJ}AAYFu_TWEFF_#WL|=jEK$znZ=9v70LD74eAw6#q)z0CT0iCZ z^FP#Zg&#et#9%-`6EHwPRR5>nRm$1kPTAB+#FT{bzZkgmEoWp?jIsQB*)B;90s<*c z3J63_33Y@j2(Tr#QfYQc$t5AmbDQMWji%W(SuG0M!(XY(vFLFOP8TpU6S3z8;)&Ge zCOvLq-$3TaOVH-WQxCrEF6_Z7l!r0^o_YJ%t-FT3Y+1lj{@yld1Lhg=dYJmKM(~ZH z6gP%8$+gM)Xj3-n+!B@`_GYIgkxXT9W9TwonjBiHDZMm$^jMOKWMc{O`550V0diT2 zsm(!(@rB`q>BhWMKxzC5fi(X#VcIZFy`+ckP{2T)#!H3fBzp3P#Ecub@kc}+JyxTU zUj2C*F9vnO_>_xZlUSa+H6uH77iXmP7c4}d_rh;?nNLU3TeU zi6Z4D7^Ej;Sd1Wwsm%ZvCRvyXP9SN6%W0u0jB_(m^ej^lAtc(t|=R(GAgHxG!yVn{c$mC|fwFJ$1{_@7!6Zm{MOZM)Xx z-|PjI8#sbYyRy=gr%b%IA%l4-S#pvb)D8x2!t7G3Op^RhK#W)Gt@^x-Hrqb!FNR+Pt}Ogkpi6VIE}hkJr^t@7PQ|pA@z_C+ zGG%(?3r_e~pKJK%nQaRLn5Ip|)R)@tAD^HL_%iZ zUL`r=8P1P<{!JJEn}&uE=A$EtS!?kN>MVOFI?Z~KXA1ZAi{f;$kBi>nWYatPlXRuQ z#VbB!`ft&6WVW|AmcrvB<`6=Bbp$bMt$LF>%WoQI*-zCO!+dpSai2KG`Gu57ge#Z-Dk2B!BYW82wx77o6WXJ_Hn@31dDFRa&I^gnGoY5wU0TI+oQT|r~1r(OP#$iSVK|V_PC@0|YFz7AiE2)EskG!9%Y^0qegJ+i3fQjDuTQVpv8yt;Bj8j}8`$HdceNZP#DWl>tahvlSHTd(V;lv}Th zJJ-T&*)lXet7qkqbXGdN=Xh3L#_OUdozC;3D&5Y5*&}k-e4$5w2#o}vpqDvUVwR-2 ze+YfEsg|XXYcnBz62v1^rTmp@$~vQn@Z(c)0J}SVp+ACMk?d}4%ftOOssDd)$ZPNH zTL%9^?fF09G3Ea$)c(&ZQ*FAg^OguoSiI4=lMc6|6~$g@sqB?c3tgxXAz{mV5vASI z7@+`2N^A`WRFeq_r|psubV{aP3MSpY|A1`9LC`@K{-}c+=`=uVM&f0TWXX%P_s(-} zW3}%4{dK+{XrG8@Zh%4qb0U~FSa@F6Yd!fO3C7*DgM7FT<~P%J`avMf-o$+^47{la z$*?<<551t37&D8xT9AmNk8sd&kESXq2Bmda*HVpr$@7 zoS8I(8e6SaBuhk7ahe;-A;L~gYLy3_ZZ`QpOe;gp6^quYig-|~u}AtvLXuppR>hHK zwnFx&KOk1uRlR&DYxyW}T)o{*byVR~Ka~?(exi>W6GnwhYB2=M&`5Se%nC$g= zan$a)^C(yxU)gLt-+V$`AL$NBkT`zwlUP1Ey~9)I<=45jCmi8Y(1yu$*97Y3Y6-a8 zmcV}3^@Vaxabeq&=pfFB(|a_j4n7IbaWRSg}fjvY$!Q13`b#wb%+>hAwA*z|liB z=x{~`>)OdH7aq_L3(0d@cMv$s22m#jyzzY-)FqhuGhteWs8T~gfMN)S%$y##t`-(_e*@kTkaBG z9G5impmFA|92Ot79vlELyR{oU8;aD?7K&j>LWu*kYGqOK9dUNHM` zGzl{irugP{;Xpd$rQ!n`;<;xHjyorg<&1lV9F}3pM-FP<{hr^=h>mLr(*p4|G9H?;S zxO`BflZ-hy@y_kPb70>%h~|7pJQ1MJV8S0EeWG}GIEG=&GsZnpu+P|(*GFE=?>o&~ zaf4DJ40y$A@Qn?91LyYw*ZRU4&{9nBQZ%6}`)#q2vv)N-=GJ2XoAP^`-ViUpZu6 z{@0~O-;tB@-jyl!%G zsyEWw{%f$?y^Jqy`2HfBzN-4Y%r9^H+2cPgkNq)oA{)hb8UuAIK)5G>x zcls`y{?GS&Ef2V&_wF~Jb#qVnWIxKE*DF)rdjIjL!|=}o!2jLX*LVK}{+-y@wf{(wFuQg^dKTVLPHGaf-|NDq;*H#I})vXt61NqmZjNmKBo499K? z5sH^7ye_dgDaQxI{Sl2&$n&juQ9OJ3ZvYc@d88ur9ln9mfx8g;Cl*x?#dylf7_X7!M&z4flsmznd z;n~^136`>)Fb8RG>x#L(D%|iH{^kBP{arlhwv>5zG2`AHU&Ib5iAh#v~dO+ z3@fPgt2ZqaoSek_4M?D6#?L4aomTddq>PNFB8;U=^Q;vR(O~{ldr0^m^wjUnMF)$zenPak8gndrcsmGo@U^$C;uG@^>^ULl_Z}5|m`Qtu?O15_d`ydo z)vzMlU$%x>OZ=v!og51YaZh+)F2hi&S&9D)tfJOHi@#TC1I5Vn0y$;j!-5(y6Py(v zn8?&w-;A($_%WbXTHEX3MA9tGwSQrK(N2FQ7OWi!sN2XgR2AK%Q|7Ar5-1@9A?uZ* zJhg+)4=!YiVvZ~zz_F&PAEDaV#lNL)Sz1ChP$1bYbAlEvr>KIbCB>v*57pLcDC#C~ zUSR>-vJp8{pJ$R?V;j~;*tkM3N6g?Dh$?O3hu8lxy)0gx%&U^D%b*<{sNQ1g4`LhQTqbyH=+>t8S7U&&sth{XvzsFb?Pb9A zT|BX3DZ7yEFH|!tG}jwi+UTJ{1sViNbWmacrtF}=G%-~Lh_Xsr7=%AV9jOaCwpH<; z&BQN}u`Ps`TiBb#h}bL2S4Y)WQ=w%#At_-OVY}05iV#10=}UibB65NH^{b`P0qrW} zvzV83SB(E|$wbWb~y?aiOJzT)V{CpCuMjSDsoMcw0g%U?33w5lWvI2!7B z_DYFuW>)(9@rH7UG;gD2I>74iSCIs9D#AR03B)YQbfMrCnhZONPyL#cEVdLXGoQRx zn^7@v)7@)(OL!WG9<+MVXEYSrExxWonxtNFVMg2>)x(1R>!JL>>;^3#TPz->YY^Y4 z9J9^9SGgPNIP2T{(Y9?!JTOpVws1$$FjH*k;;$t+g7eH>__FB(7Qj6W?enMqe4HshCLWN@&szehFs9PLu2QqgnF`sS+%yJajHY^V)lu``c zyyt6%-0BU8=qc}V%TeHQSc5b!EimplKP9InPe07W(uyTszF4f|=n{YKUo(}1LkJejkD$zQ>X+c0$>5F{EDDm$>uusYcP3&(z2W*`EIRm(M8^*4u5 zcD~3Bkx%pALhYUeRaR%i1f_ZqX+b z&CM%vhpi@EU6rEkdVRuZFUGfta&L$agIkw`NRmI3Eq6mS6KDDxvw^5io}*vP+Zkpo zYN@vbR^(6drx3H2bC>zD3YITcR$QDsx2+%;WXtyPAw7;>tjpwW%Ab|7D5j0pV?F1alsns+c;y*^Xn+<`H zXd^>lZ1AaIXGHejZKQjNS1seu5v_A(S;rtO4wkvl<;K0_NgG-_KRQ z(f%gth3CmM9iw%wrog6cleKF7jKs!bc@u!%9kL)?$Mbu)ka3WDRyjFu1I@AVSN}>18YL_4W5f?*&y2*c${xamG(<>& z!A9}7ku4PW;|tx7Lk-2oG92bl)6=|{(yi8;6jOaX%A9;=nrZov)NRM_f}0VcbCg(h-i zclnu!x9&{W0MnjWO25p+1AuXA`VmyK!`!%p(x1A?s|7F^qEU zU>sr*GtzGG*VsbRn64f=NDJt9$15xQs3gn$PNPr?Mux}W?(x%K8KX!=@d5amX;bz* zTN|igBE*|zxfgI)f0jH)InJnO{QYvM-~^cpc6s^jAG$Gb3;D_Xz@>&5Z>;4|5Me!Z zBf-RM(022hP#>_tIKs!pv&*r<0!4eoFbjl^O93DqToYh9wZG{tZ;XK)6S3J=`ZjR% zrU;v}9Q0dRTF@-K-zW?s;o34(MKj>OPj!z?Qv~fdM8PJ#$EJSB8DqJszFnayR!Njw zQqN@EV40#layWvpv75-S}ltCX+% zxWwj|&Zt}S&0!p7GOKdtyQQ|RNQfajHV#KLVd(V|pd5$ysWicArZH~&nq#S=Dc-EO zKqSG5kqN&Qu8X;LHn0j6XNbk#zee#+%-hl~3x4f-7AEU>4ZiC&E%JqH^FD^-or`B~ z8Ih06EW??ABwSut(YnI)aJG73lL-cOv$0gv?)8qRyOi{HY1B1GKPx+10 z%P67uAv-Z5S{5BvlO|BX**4afRy#^lWJ^1!Vb(QD%}j`~WE~yLY5HCqCx)li!UWRn zTYp!6z2o$u(?&n4v49y%NiLw+OS&jUWJ-PO59XEkJK%f8SLNaY#ddCzr6vNRdDSaw^t2Co4!pB`dwn>4jU>w z`%1nX3!PC>>S}|~sI@gqX@sze0sLqZ@m}L-rF;+)xuCMx=gGjTS=R_J>JJL+eHQe@ zNPCnT&(db}dQDb_t38~n(oe~eU0Eihu5J3px|6%S;`1jsYvVPjm$};9=6ctzo{G6b*Kp^0Xw}&Ztrq#(H^;4ch_LI8rT)1W8~qLV$gTZFi^SztpTgNb z32te(tK)V3XtRApMFQhTU$(>C)u zoEKVXR@?Pld-}qYtivtg^k&zUe9tI%JuiG)eYN6O82peJ8x%*l$-ausf+(e&Q>!~K zc5dL#c_4CgNf=kV#aw&IdfS(UVOJgTn$hX=dU4hY`-t__Tw9yt-%JgLda=L9#`MZE z7~)Xj>Y=Zp+dP!6SJI!~vcSKZAL9DCAn#J za4dl4J1QtX?=qA)fq2Wn(qv8ktoA2K^l!{pfDj|o^~ry`EAi`%lWA3mK;ssL_C}z+ z2&qD(qgb>l%RRGJW|6DrysFk~CX1BaygFL_T(jZaBR#o`WLrrnV%psSJ_5qc0&=_d zrd#e}3=ZVp{X@I)(KSll5)X_!Q*=#m)c)Cmx8c9>kTTo%PtL3wI=^~r#KKi?Y&9TW z0OyC>MJ|^?ejdA*J$K#QoM}+?Nj3p5g8N%R^K#|79pX?5{)EIly~;`7c_5qDTHMKyowc-GEXt{YG%ajI0i7z`*yJxEHFZ|X zJED#w`8O| z`I_ToyF#4ErPq%7{69!q|1m>-{&n)p$|v!xim@kj_xHBwpkAm$u;>sg?kp zme`Nrq?6Z@|Ap_;&m!QLZv@`J&|RTCnqRmr5%K)0tQ{e(sq7_5b2bmS?9}uviTzGV z56+iL*A&;}rhu??o6;?Smc@b@_O^hK)YMa$%hCe;~8!xe%jU(P=ULoGWT(hh?XP_s#soyRTp)e}M=&6RH}kBcM1V zBQ&-HsRu$z0r-+`N6r*l#ZK~<1G?<4&`By4;f7Q=eYg=$V`$uUN%3U?v@+Y`;;Z7V zT9rng`wSYFTlP?zdKdfR%BMxjd~a}=Ab_tOs>cCZ)$Q7|IaqGRjo7|xp0_ePuGuxf z_tq}UP(cdebIF`~R_E5)wontBF_95Lf?2lNzrF}cMp=|CJ?zjIm&^l1%Py$GF6?Am zNYw?KwvcIqqG(b_A^BRi#4erFkIh|`Y{_T0 z1c+O2MDn@$tD{fdmw{Qbw*bw@c22xSeFb#<)!ArJw&kJ&>SKJ%>q6wJUQ&v%-Tej!U@EqqZgw&xERHIB@lLrsJ6dp;%O%Fn;Hc?HPxi1w+-f8Ll+4-`CF%GVl{hm zql-F$d~{!Nv)W6d(&#c&HzlT4ljQaRcRz-hpl(v=v_FJi9DMd6wQKXibM=hf=oT2g z67Nq>b+mpcR)KrAPniVsYGF97xdG;Mof@EE&a270+@f$8Cow&0unyP0Ja)ZiY z=Tkjy#~yL=_z&JecNW6M_bo>sGvTp(KmIb5zyX3rs4DDAyM=Xn_sv}r@f~(Ip!>R_ zu~5FV_3p>$peVi~SvoYU%1l3Ia8s_ByDKEQx5qyE8vop%(rFdnKY9-a>$;Q;f%q-} zcTBJyHm^h+&2F#(j9b@|7yxRv@!}@yc{ip197f{dUOc6wFsrfGG9c}}l-OG#bGIAM z;`!ha7W6J%<( zMXqkevlOQKaJGn#-4+PCPn)d!Z1PX0Z_1v-08OLmwJX?O_f0MI^aOqs_u6hA;$Kx0 ztCDADWQXeoE_q>TZ{Y7uQod=pncuhApz!Cr7TSa=`MnWHpy&n!2HpwLu%dWuNbi3z z&TW3d7{IQ4`y4Bw6KU*4IQT*jJqkn2Vk4&HhZ26FNc~{Ve90>;x@%gHgV37~u4cSt z&;BN)iwW-L4085_fK5x>8<-<`ca!r1=9is6WhKf`Xcg6qKV z{z&LUk3R+PL}-5s`Gq2XY`)K!cV_erBJ+)#H*)8UxisQ5c|dGRf=^oP85MIywEMXb@SVPwZM9W6*uo#TCubW<+n{}gleX@9yRf^E-{BjXuee4>Bv{d8V-d*Fpl~v} z0A@TGyl`3)E~J4?e0`+X!(S};Wk4_sbNrv|3M;2E7Ujgy#vHWx)xedhxB(>o_bjSJ ze$;*jy0b42Gm8u1qL-@tIP|JJPCXg2^7Vd6Cbc4K9`dzHb+5xaR*JOe#YS0&*HxDFT zo=182J^9v2P;S{F(=b{3Sh^WGi}Z4&`;pC>?53Q$S{KX7dl*^$`gn|N*7`Fypgbjt-k?_ggV#GUn0>H^l-| z$@1q-r!*^O)LUm+@=uw1_r+FB&INb6nd5j~_P<4(F))-bmRN4RWbf@F%Gc@0N9nZn z%;>*ju^U10j#cN$lCMxpem%=z$WzNuioD~LLD>ag_{UmtKpBn@M7Tj1(S|g{X?gPM zPMj^E`q1ePpDmiT*q_ud%fB&srO}t2y&-vJ(wF{u2bc`$hAU2A=U5{kT_h8#p#$Q> zJvej1uz&f58Uo+afH! zTqvU$d4Klv-3n}3T~5}xR0ESbpfL7!MX;y3F4#mJy|-sPEO5fpUHb2i+073*;pbT& zaLkeO%&!OXx{6z}Z&9ioQZ2|%Cc6uQw>{lCMWmz`4(wds>H=nX1Kv=a=n3|u z1!O|Ee^2~%jPNN*fD`DePrwHPv9SeqjAta!q0uTuJl_rJ*Y^WWdl(5-~4iE!( zZI$>AVR4KSTGQkMecH6Y&0DR?SX@|Y@Ll-a(yQn%g|q@7Gl z6_0I6;bgn1X7Ck!F&f*yAwRvI*PU6GX8-Ea$UQ8_(2+pF5l0b93E%iaER z+7^v0`rFAxQE;=qtb+6kA#*wBVtoJaF5Ol1&rbzX|75P6J>-Ro?WB8IjZttl^-hj6 zPrsrkX-R&8o?7;=F9qgd?zWg72Pn0&O8O40IbYH=GS(#}f{0`S4A z+`}t(07f8RG^tGgAuGRwY<`dzInkbS||2irIQ zu7uVHopnus^o9GA zqAz@b>X-X7;eWuM@YFlW{Z9Lxuy?BarSAB+H!^)U;ozq}cF&*i^;g*pAXjku(_nw*1LqJ3y zt?fsy2?)yoqdz^qwGK$a)r|1|bz_0;KBNVBGV~I)UGpL4RR4w2qAc{rBFby${ zM~(@Mb8&2RXpUm+B?29bag4Uq7K$CS!OSp0|xwI=Tha_z!l;S;s$TQ9GR)&n*u zI!9&aAGt~zH35vFwsj3+5inStFrLAE#^QA8h{dBlc#R7S_lDzSP10vviO{8wEKVB_ zRV0kieEgiz&r>j^1M&0ra9~wBml&5xulUZ_Q;`&GYj`k!)x-Cg0bh@J9 z#H`oSS%#0NakiTqlsa?Dc|4im##UYls#%g^qij=iUlz`zeJTt7SVqc7a-+oOr#*3C{ss4Ejx%$@iM7T^eI8?Cg^i}p^XB-?uKsJh z{q6ECa=L^A?qChQ9Q;gnqX&U>m@2?^jm;n0UQ$?naym$E0 z&^DwFNhfp0mM5+3m-d62<>IqJC<~ z#xV!K?~7y3(3&YB85NUWIL|D-=bXLfJ715-+l_&wJ(7kw>`W3zMbdh*qjr}Zv=b%4 z+)=hy8A=Vq4oVk2&P3a^k8Fj*XoK53Z=i-x;!2f($Dz!HlXu-Sk6b`|g&L52{)^ z`=^EFAN1jo4|#`0o&)fmmd^yIx07P)Zq#@VxK9{T4xok&h!0<|t9arH{Y@rVP1svR zN0m9t*elAq8cfe~DBFfm@laL0s48*R`>s6%tI3Gexv}Bn636Fx4ysWtBr)b+N^F&})|oul=hpj1hrgxKKlgXaGFRiMe3j`(7+D zhL`=*7Va|&0p1?lPRgy@Jhq*lWhbZawDV6pPlT`CRokNq z9XVOGaOwB`c#2q>H*&jFo3qyJGRIo2SS4sRwYLmCzV(ydB{yR^H*;z96&_1?Oc)}j zO~NB`2_*3p-l5#ScwaFi~ zirLhBX+qE)!&;LpOC;}gq2Y?E4uMWlCE$jk4r%Va>oiaO+Jo37p6FS7fMw+mY>HRf z4ymNRp@c!46Gx1!g4*a4HkZ#y(}VE|`QJC}E;yO)`aiu)(?2^Fj{k2|=>OQT{|i5= zRl8Hh7RB&OE9oTWr4W7zU%({_SiMw+nLVy=l6#+K$Bm>7}6*-A9Wdr-~zA!_vks@)lv=(UMg8#(K+lT1fo- zV#KuTL=<*~%3q_+v)@W(p=h$>Xr2s$#Ld^`LYOMF^m-+i< zncuzTMFNcnl*a?2p5aBjyXA@N-9w^ZaSy&S4YuXX=T0lrC3HoJk6=dq3m*1+{nE?$248?EKFS+B~2n( z^%`ja8IyOFS|)#~(T+l=*35aX4i`;o_2>-i)O}%+*+b%Hf9YJnqJN^I57ZI5|02a51`rVc|F3BAU&q`33L2vt)*dM8n119N*EV5<0v(+i28Gae9kn6a(1GAX zv@V^j6Rnn+*K%ygY}{KrkO&knV)W}(wY6(rwW`r*)jBp1pxQNUs6E!TTKN7S=ga+k z-wzkWedcD(-5Zhw+55&xob8Z8zz7A4iffkh{#7)HrW zg@2-~5YR@=P-vXjj4l5Qy&h!*QTZsf;3Yg12)&1koEd`Q@emz# z#E7HHhz*60i~Q|?mxmA~_jyY>&sSi8l#e>eN3!JLZ}R%z*s2ZtC}4}~MOnj*qv9QbWFey`

    cF39OH z%W%-r)dl26TCMPc&-I`I@tBy=SJjtKzF%{h7VsUCN6k)aDVZ(K5BEAbEBt9InX%># z6K!1O`3p&o2dRlMTRD~vvec>n7iI4homseTi&kvgwr$(CZQB*w72CG|*tX4zom5=O z$=YkTv+r8>wsY5cm=E)5w%NY%jnPLR=q0>cKv~=D-ea77*I)1H7kF;S1 zxrG;bPcToB8fD9kI2kjSc35S6>sS*hAY}Li*Vno-VB+x=c$`X2wxC>NHk_Y>c&Eu+ zg#>FCs(8RL(5}1TB3q>RUrNP3;5Ip|X5b8j40@XrUj3O+mC$ zYrsxq$W3SCl##Y)9OMI~i|!B)O<)iWN+v92#!Of3&gDaRppa++#{Wp}h0MR?VD&`~ zDW#5tp;1fzqfPk3qws}qC&lr-9zh=~K=BU$lhP>rf_OlLdX`KgJN%-C5sz{S~moHj-A99wb&7ZU&e2q?(3pr8G^;JRz}kq)u)ZSEE`jR{roCn8^utG zOl|BfW0AcN*#Wk-1G~rUpuZU`7Y<89eT(26nt!M`=R`?~3(fuQ1aP6@j}aN5-aQkw z0$lbI30Vfjm1WjE#He#co7E|hXFfFz9EUTuAauV-r7la`MWOBaa$}rCb-$BC;P}XD zaAV;P^ytVokoaz-wx<3`ee88k}#t{xb&{6e6#5$g^3JLYLHaQ(G> z&G0IVE6Zk{k1hlB%OvV$!f=v;U!BkhZPTnh`0rcLr?uzWF$dga3$E6Xwy_@532!mW z(=Rt#v=u=(3Wq`OR#gjRc0-a2X;xzt3&p@k{F5MW7nb&X5Pe(o4u>1^71EV z!Q;cupVA1u#;Ef*BjsRahF`cpY`YIK7%d*U?8|jDWAv}j#Fp`UPk=`xKgzK>y9#|P zJPJ8@ZjpgMp?fU~pMghG_L(bv7(ao@e;KVh)w$kpym#MTX57?q#z!V5Yc51f`FJyV z&3gG3Z|ufW4dALT6S!@#o}ka?OcdfQ&X~O63?z1JLtbY3WV(CV@~#-tPX5Leix@?m z!3t^0?X`hC1y9X_vShiFbbKPIR&adeDtzQ^-Gjb92=xk&Jrp-?$!p((lv=L6oj9@qFb*Erv=tx2uu z{lAERn4k0pUB{TO%@^sjas&#$971Z1SSPa6`h*Ft8~iLP&6;1ePUlLh<4jU~AL^RY z_eFh5Bc67}zDY&FgxTsKO9-)M1E)eiDjX|>Jh0Y3eNOxgQynqu+b^03!mw*im&;qU z)Cu5x=^`~W6)8+PqtBDw{tgJoE{sF{VB3(o*tu!txaf!~=`Q+|2{C z&J2SM@vfHSc-KaP&n&QXxUoo#Ic78)NTW!Y#UecVrhl119;jurNBlxaf;o)#>2QxE zmW@7ASiZuFULfN3L3n=#91vwO*KEa%I{)gxP%x)L88p$FXg=(xHH#xmDVZV@uAawc zaC=8GlR1ViqFNq^>HCqS2#_mBkStFH7UuHy`2Bc2b9P0t#H{Kc5fQU8zsjh>NIml; z7*mNFgUry;Z6VE1zm+P(TXhKwB8wBhfltdF9;HuvOt0q*Z`QFA!@_O>dZfk*nJ$y_ zz=Ut~1k0n-vZ)2yo->OYM{goLq)7lj-+Z(5ey8#Q<*@wz1@o`?h$LH$zW#fg6CLu8 z9~A#5d?fn4(n;0J@xR48OEhd;b=A;5XS%LUkB^sKqg2XTHKmiG5ErZiN$E^CnnlaL zr#+I^S2sq zsh6poLAW78CwJ!?x4P?N#Hotov;NxlB1KT1`}}n+T2R$`1MkLmEl(+pXR2FWoM=ll zVaZNU?o8J{*DvB6y(>N&QEL^)T3Z7$M?!~I?+W)sxio;3g|+hXCFC+#}HB&jGprQVaGv8a(fYYvue z6x`^6i29-;ulZ=wS$`?ePF8Be*(S4X3$^Deg)&GG8t|WaxQ0m+xp7VF`U;VTInZ&gO{_4e*!;`slA_{X2RPvw)Y;TvDft7^>lt*@jnu%X zQ!}a)6Q`bZ&NyQ}*exjIWN(Ld>OvaJZiGtJM4VKEa3jmZvS>%+el_D&hEg9SELT5 z;b|j7)H6K4?6W7s#ICBI<4BVl)3V8 zg$HF$!>C+Th$FpOSB7S+KS~<0{ec!Q#q`6%hj1oFv6oV2TJdWMYcI1Ciaq-svp&-u zwLhInFB`s|l{+L;UB(NXzs;Tpx&G*lR8J(wEd+|g0bT7fDwN?t`~sVcF*aWj6;H$b z`5JCp*p+BzGXHPWRFa~|qo0U2weotYJ;ZFI$I{bGckG{}(|KqgF31&Jp=2C)@Snu} z-Bpg~Uv#NvGKmU(%&Bfph9%kAd)1A>sd}e%RSxNey6ULH=O#5dO4=RJFxAmIdRh<~ zS~inYFd1y?x(hNkWog!KaaL|xce4~5rhd6c~F1#m#6iCRROe6 zcC0yN?#>SgbmdRw$PcrLXz3#t$~+z7Jc*7HQTviDbz+9nu3}OCyn8S*XP)1ZkzK#; z8<9JL@9&Ylz<$HX9LY}@P$l+Cb7>RT9T_Q0F`0EnWs>itWN!?F7vL|FV~qF9(O49P z*JN-AGQ4@#B`Bv7RHH9$2yL(2$~bm5?&IRlk8pE7_p7+br;d@0QH;@*YbQv|xeA5l z=M&!Ww|4_mi~dGBLOGfZ%L@&lm^T+H%?A=n3EF6kqs_ZsKyv7lQb7b_Ye} z%p?uZ?(#V>5%V!+_3p+sMI6h9TU6iLr>Xw{S140n5q$QFsRrJo)t{%439UR{sIM6g z34#%>wm))x<{n^ryv<>de=bGvP@<8t(9`K)w_>e%c|&SKY*V7QGG( z|Bjpa;B4RKSO7d*5KRP)b2-{2vHNdEx%#ZN9FZJDga(T6vmiKQlJK8~6b&rlSV4Y6 zJ-G5ny5m($0X#vokt4grJ%8!0^DoMMS!nM;d4<}-I+fIhK(tPXNen@GP0A4fz#~di zu7s!bKS^Fbh#mSSx1c?ncwk$;hIG8S&7(ohT0MiCVdlNW=CUAq1jYI|z}rY@4C^j~ zOwwD(EpbOMv_G)D!7_yc;G5!&%6#f>y;J zu!b7O_DDsv1ZqPyu)hlcp7J>NOer5$a0kz9;7vZ8-*B;oxkAd=kdGWkjIv){EmtsM zNB!G%QQl9vuHhLIw`8Y{$koBoYex|6kLlafIt^j7Ua+hVp>zW=GQR2#sXyR0Z}D3X z{dA)doN}B^!Ylj#Sd-f;PPZ%34alAIPMmz}vBzyLLvIzRAILAqcP^E*>(q?$SwlWo zvR%DP&mU-67KI8tL6L98#hDt0+ zj8(?prIUn+cqeyF-He(0OCm7(*HsQ;X)2h`DS!scep!3#3~|{B!uieo7Oz*F{D#ot zGYhjP@y-xshRji*8EYZZBhFB-Ya!G#dx+v=LH%C5(lS}Ir=$G3@(gI%iy)?>2Vvd{ zbbGC9!uGow76~8;fOja@va)oyH?|&&1fVC3Xa#)S~_tLx}s!jH$L-EoWTi$mu3GTagfzJ;z4+} z8H#8ZUSnlI{J{5(fr|pErUBW|f+-Aj& z4vs!gfV)Y@@wmk?%$eO{VtFPFKTs{*4ib-TqZ>cX4nCyEW4z>Yj-(yZ9KEXbDCsY9 zv4tPvWoMZt{n|sX-Jt`3wmJT>r&@ZW#uQd5*&Qu*_2D{J*Bj}LzOmX2p>K)evM?S~ zx%)R#Y`xgG>+d(%3kLfCS>gFVH6J5m+y91sW(l*>gMx@7y;jC17rh&m9Cewxy0_X$ z(qgcvpfKJ$>&<~tilrr*=->PR~h++;2x}P>8LTjX(Mq-!`ZQ#LD(cH*Ztr zgqC4>qZezlc~J(Wd!0H7<&_5)MfQELOHe&T$8IX^d|I*^Zx^v@SL>1vQ>d8yuB?B)>vCaUP3t|dF&!xNGRi5qBCsW9+QzxcU3ChCVi1{7SV4m_l8li()hp}mv#t*4By0uRs*717QSPJz3#q!F z4&!5ro`P-bIT{l0NeZ?W_(adJ%_gtoG8Z$s>9QR&>KB%u+~uvh+^0P88KhRD2s|11 zAWY!u-~1V}K}WB&8zu5P@yx8=Ek)hz1I$$CSCzC~8Sslg;0~f_XFC2g^+SLv_!Id; zo~pHk!%R!ip!^U!!ygu`Ta1n1OurD*Z;=K#D} zMU^o)?KC?ct*)U5h_gi?odKq8pra+@KElxwA?ObaM@LldF3 z><6KqIFSQDNes`4)*=`A7j`(?c&9h<&Ip7B#**@dI4Yu!+(THrQ)v67j8P9mKZ)X_ z+fj+oMFSN>#t$;;=|-k+6xyx{%sY@ z)=RQ`*0+UseLp<^7nc72Vd83Wa?k^Uu%Ww4pOzX2RIVn1x8TICC37f4?D9I&WwN9A zGocW-iU%<~O`cF*UIbnNxePxbj<8a()S)BMHI|^GuWA*?WFG2MvN!cn*-K0)G;EGr z(K~NE%>$g0=B0nb5pZKyMM~6S7mVDrb7WfYj%(qUc4}q<2Y*UJU-T8`Ldd1*P^8G9 zp%RBEHw1t;uO|z4=Hh!6j792 z-#ZDc-gSLTDA6s`lly6p8IR+ssT+aa0O0gtV|cxER!4?#Q1T{5#uABC`%<_m%yc%# z-9C`89Ys+3XLnIVz(PPn$4Fl;vo5WSfa;b8&sMbP+u_*9#X3T7DwcBV<1^+64Rg76 z*T|Kh(UptDL}rNw?Z8#XGUZ;>DLNLjg6^)G4D7hDQ}~|Q8Dv;))J1g7>2xh*?t@-D zu^*aMGOWC@-51LEPdp$LAjJI^&2$shwqXJZ}v2GWI7g>O;q|zw{2bQ^6L+ zvMEG|Y}Dc;f18_SbjN3qeLP5t4EHoWU_!^N{^Df=#57pPzb}VX41KI>m>pmn zO&@J^pR~^&tFjcoAP~l5foLI*%z$z?fi-m4zDYPUw?PTLgl^&+d0BG9@UP))o;6mO z@jH~QeDguf|B=D}10Vd;=>KW$C2BevC>p4r@|c*C;z1GRHFnF&uozejnl%Lt=mmrr zLbT{t!-Gc&uwfal%tEhJf5H5Y`x`A+&(Bo|-nIl5`Ir1pKm#oS>diDXCznmLJMWuq zO(*dI{-0=lW-n5>!&mV1C{nH_$|?q@W)mZ>$ad+n`|#$<>BG9v5A9WHU^vlKob47S zQa8^ndui;ccD#u7U5uJ2r+IKiK{S&KG}LXB4@R2qe`0xNh8a{zu-=Ul&?I@)fbDad zDNMAWVeE4|aU___t1HI*5MLWIA&mug3NwF0*c9}Yn4^&#jGSv#Bhi=z+Q?C=8)MOU zNY%k8wV$1+iMo~HRy4-^Vxn;*W4T4=C#A^ss;pK~Y`9AiyjaX^LCIKm>!8}UzD(P) zsvP5tX#|-B#*^_UN^oK;RIN>1S_|l8VxvFwnSd$sWw(fF@!B5=BY=r9-&ymj*`?MP zj5xP5@!uQe3e=I?T(8ND1yNbA-0nR4FyKc%c_ih=@V8RW#MnR@kL795&+3lN`xNtLjgK}!M zbOO9C&MkNH4%z*bt2nzfEnAER&y0frR%@}t-7#KLkL_i>u-cu#e_Dz}H+!)eMG`o! z>ILz){a}yNo9)k7Khq;8XQ&70TV@5qU$=^T$7&>uQ&? zKb1G|D}n*gl+duf<2V&fZXRxWUcRRp;^>cvP#{GBff>yZZi;n|It=Z6vLcktT31k+ zs2>ts;s~JQ6DwNG%vYa1{th`W1>lX*jm)!5W`vsVG>qZ7Y4_oU3xoOj2Ux=^b`%Y? zGHU4UC4~($7MW-9fo2!9y{X$D>DnI+8)3$Or~QE`rf0zS>~(-|4p>2VSabuQ$6tR~ zByjHS(N=)Y+#QQNb`!rV@rBz6%&2gS#Eo```dB-9LKr)qav^W>wi^DlDeLf#Q0Em+ zf@2?B%*f;nV}!+&?xcft10rEPNpgSU!WggM@)-YNa2yM+zYZdgaGfu9s2w~wI}flE zW%J}1ZV5oYKQ(M~Z)8EhmAv6eAeW2}l*x)AdpUI3h`wQvZgVzRV(iVgGlO{An!6c; zyDHQ(#J~f7z;I7QeE|X?RR-?u?=XVPYjk%KCE#XDVP8@>eY+dZ+fcIEELP$1hQHE{ z=6#Lv!PBE;I){@b>VxeW1tvaxp~UMLTZlWrHdA0xSiOscGItV47fTkI5tU7EEjMdQ z^O1W9lbR?Try0dROP1`)XPO*Ik5#ZcvfTpxEKSi~5g-|$6Y-043ch3dk_vW$(Aebu z1p8P2<6UMuVf~$w_|@ zM9~a~G9XsbQ*ZB zXxyKH;xhr(r{Nqh3I9izvymF!-ttrx56!BPWN)`pa%Q(BDk&0(PhL==Zj3knJtYq29?u#jZx0V(ubOdwnF zB7{}Yho(GL3a<7ADGADi%x~FqftpmE#LW|DOVanTo_KC-V4p82M9UA~bG;j-F!jAX z0r)JgKkSNoyD*-0So@t>ZYvYC$;s0uS{w}B3dJW33jGcO@Udpv=Yxn7lrS@@)dE;V&7 zp`0Qy!m)i1Cwd-oL^Zuf>#UBhq|BB4TTH2Iq(0#t*?he*;2c*{Lm;WN(WpDfS5!P1 zS&Qk^TZragpxsXE(9lMxs;33mT-9dDmTa|w=kZ5opO1}d@enP6*XLf*9b6KKqV^IS zr_or$C|g2`tzXS99oK+pl(hvDj~DY8&AiJ*eMvQADk9vZ(-XR^S8j?<;vUMH157#Y zyXGXT=_r$GE!;vL(u%Vk-6{^EN86#_+zAcByR-2cK7i?4OByPRB#A*MLze=)GNSmv zT)r$=yE#Hg4&@|i{Gph)TZH?489<6cS_SSM#|@&(H#eYayzPqK94=L4QKJ`}4%TVm zoZ40uO1Aku=I+$L807=5hWtR zK7S>wxDK$yc=0zg`~PHVT=U2TDPF*UTtPD4){sbV(Uu5tYmU>T{tMD}KBpa-D@>M= zFG?kyG{RU)dC6KbgSgOrL^2KP?<_vcwBOAWC-sH(riOxo2sSYO&0Q%Q^F`^r87x;~+)S&;< zd+|@k#pr(s0d+eS6$w;-c2Ig!3}Pf-D;u#AOnuVfG;26zTHyj<>w2Yn?X)bi4(#jc ztu4-bl{b(Le>#>HShv!v6aVYFv+xZI4oJ{N`R3&D>-FuX6R(-c+@tv3fLF9WFHfdN z!_weE45mk8br~1$mxY08%ne4IQC{=|&7m4j`@=Z=M)cH!dZ*O^4euGNVITDCHWRKe z`)W;3!+o1T$$U`plfU^1sb*)WYJk%*#ncaB?pZE8@P(@C3c?&UW`V0v`}Rbn2=1OX zaz;fDg=4?o>=lYLDtP+>?6J8&d^I=THa&fs$(+>rfr@+xUTn%#FuL{!#$>+=q#4fc z-1h`rtU^~0e+#CVbhFmhR9?y(?mvaDAVR01zRw9Lo2N}=^TbNxm z!AUf?3yYc4Ow?fA-;g+p1+Wn@nidBV3mk+awjNxT2#4pkJ%oy4&n+Z%biQ${RaFMl zVrFaDw84z!t9#p$&N~VBGu40!t`upbgm`46wTrRtT>%r{IWAdg=xlilNsU+=3p%W; zd(Z6v#Oqq*Y|`#>T*&2%QUADzV%WnbTb9}ts*#PcIS9dTMPr*+FCy031hiFsg5m@~pv-4cT{MtR0 zLpk@=tZFFYO?q?LTAO`d%)Z7ZW&W@+8tiCo%r?jAfsoMI=G`J5qrBoB zH}X34C)zvL>bR(aJjB2FISljN*|PTgw+`NiqaYn=u68LEiX(yA>haeJ)0ldRM~r3( z_J$iTw@|%Ihl9NIEp8tm|LV}5)FG!#zXN^Xw^+e{#smFVdic*sPb%zYW&1y#?U$Vc zvJm3v*W_ZqS4%|Hp9<|7=*Ruyr+u(uYa5)FwQ|fD1`JN4Ki&yu^N%`bFpHun0zU~R zIDK?!LXbgrqSrgtGj?Y#WuI1WZUlg;4Yx&spOGU-ip12Tv(aI|(dCBfO%2z7P9xGl zE0oKHIg;xbMcB5T!w+CC^e)VB0$;}%vwgLufc#`_P zkGOzM843glLb*#B6#HM%e>PvFb62-l4VSXCZl(`rp%V~|@erzM(#;JY4 zXp$TV#cckz{ao7EGb55PSp&PqI)x|?E4kpLcuAD~At?c}sj*w0VCYHV7uaIeFJ5)8IYW zU-C5*aHBZmQo6S$tCA0-X}mx`GP010J}A$hO{6usAf<^xwU4~?p#2Er zJfc;JIxk_0*OOB1XIZOMmV*+-KK4MuHcJ6tM>1uD10wan{ZS-_EI^m{XZ2%sg0A9z>W$JC*k^AQwr-KyeT&7TTeUF}9CbxS+?ad{ zUh2i+^}l4f*AdLVzZrp-g*c9;c{s(bpjtWI%PTi&Sc)-!{@bX_8?Of!+&7um@g0r- zGeYklDD~e7J#`%w6bnrM9$ZaHbtO?$1hoX~A8yOqHb_Wt%0>J6xYbWb5{)6t^;y?K z{rFvHemC5UlTRa6GtDe&;oOUNfPjL<$xo=V8owSnN#J6Y>?!|~XRo|xzdQ$y-oFo% zbwA|ym4G`LEB3vCkz`x*K{!}t$q*S7N|!CkBek|<5B#>2i#DZve*?vWV0)bldH4s<|^t^WOb0;*kF9YpuHj|~ib#SXz7ug!2Q zX^Jj=Bxs69eq8ex+3FMo*W%b(Nnf7w*hv^9>Lk}wSKmH3yY(nxhgTJ1`vQeUt^;8@pWy+0|js%sYsD0rFi3}ts*c|l|+6wJrx#rQqQ3?!#QYArMgIfDf{oqt!^eQ z=&CX!AwnlFDvMNA#9VM0L63?A?C0%^j&N1s^KQ7`>Lh+-bXD14iDKeTZD8)~A*hlY z{Rcz7T5#CcNt?e=+PPeopi_Fq0i$T33Nh)f61(R5qgkA?e^>6E=iPocD#yfua8k}S zWhkF>hT!8pD=yWS4~7*L6d$UzbMwATWyJk>cH`|szIi^bBzH7T?pSDu3CHGNyyL>f zGuV>>I%KP3+qE@9j#{8sRhE^jKK5;CT^YQG<9`g$qU!ZvpRi+F9h#C=RU>t*Kjzi@ zmCJGcQY3G9auGK)h(laoRa%QQqT<2OTHF2LJLQ&U~cQio1(-x zt)5~#nu#H`oimSK%z7idMHTex`95~_NCN}W2?Obujdlzo|-{E7UXsJ zDd84{2IZL&80{{tDkz4c!X+Z_pKpFR=0PEGPn1zZTC0)=bVb#w1FH5hadQZR8!^~a zEDrnxNS;hm>{Z%d?D2*@hlQ|?P*-;b&ZNoDg_{2miogkizUEb6A}o(;r8PKbTW!RA zNMzsCQ%>AzA=61bfcT`G7>N~sQDwfXY0_9CX$lbs-P2Mry1Rk_`r6i}`_Th2*0}vI z!L+TKuI{P?Y}R)~1p3feNksZ}&Bj(4BWURjX-$_U0?Imge!4WD4&ZF@5isM`GUJ)Y zV2rt>ACeLW6vzw?r!B3M(GRsJ9f#y1J*KpFddL2tZp5OOyM>^`4*#i>`? zj0#q>j7+9E@=qbO$<^Q!W64 zavmrqY1W@bG?3HjA>2XOtC6aiu84o(TGcL-+ zi^0EwgM%N6U`^k=@yGY`pZVASCph?Tm6ViNCV3DhM6s{Fuuw8&pAb~5{veus!V*C$ z%5sXG{)-edlFi7WWlHCtxYWkP*xKfgCl@jX4sm5SAU`Rlbeie7g`ZR1*C(P8gim*y z>Kdb#SL?zfpJm&1#Bh}F=DVfw7UvF94VB%)1lHq>#V1oloXr(E*j#7{lBNk3?h}6u z_~I2Eo%EGR&Kp?tK;h=B+fp>7IDRqM_9~u&`Ja4O{II8%aCFbQmE1N8T+ZtP{;gDY zOSV-W2=hHmN$}%`$bZD5TiH3ajMyHVD&8o%^UiQ?|DZBhhX{A46ztG?CL8H%BL9>?zl9# zGlm8!zKoz&&&Khscx!-?s2WEwj5;R)kC7j6+^@}3a5jEPZ$9uDjiIy z$P;wvTU2Sp@;*zUc>dy}8#wIdX_V_r(8rtYB;=@jX6lVFjcHb_5s&Fjoy~^XC6lic z&(iNrL(vO#do@bS2vE6DC%J<9b=(_!jvDo%?r0V>xx}CIEz9-7%<{76YdHn!0~5GM zVSpmN9yhqKZYFz)!|x`BWxKUFX7wE59ewAASVg}~{D=^bFmBOZR~1z9vWrfE%q1@? z?lWeOHM0;NU%FN_XpbehY%QTMGP;7edZSUg`%_O7$rDPI4<{`EDG6qzsGo65T|X0_ zJ(fEqa_yv0Ub@{Hc}LT>sn}CUkf^@s)T|tE4r(-YHjA_rwD%HoQJ5KbJ7u<71EU5a zowae+yX~DYDHacObkp?>J0%4sLOMuc>?Rsm-mx)W)XN7B&NJAQd~=Wq{kRO_(Bs|mfEBo_;Q59+BIV7Fa-<>W?2#Mb9|09lcs1uPT7rFjTf z!!>EdNe=30wYy?}a${(Ct7F|o_(GLeE5F{GPc_aD) zA6kpkHo8!$^_L`i)MA2S z?k!EOBk|Q-RpZJ@xMDRYDzHkDjd`q_^ zZ~`m0D4}TUx;a&B$Vg)JpU@U3**{d~soql=U1Bxw#GlFfp*lKy8x)r(^8nK|c9?&d z_L4F%WdvUsmo`UT?O$`q@lq4FyZ_9sKSF?t9!q-xnJjSb2}Z&8AvKzJhXEI`e8=&j zGFs&+H`wd?f%7$g$NgD#7{Dp!a0bm;!N??2s9ZiqZh(sd#WSE%oT&_$PUFok^<N>r~5XO$4^%pbmSJ6tp-dRsaKa#&%8>w^nSD8)UaX7_G|g@yk?Mf{1j6sRajnT z?JBK0vkt2@Uy!kGtvxR90A!XOH)*$LGBj2^5$6h}xQ8)gyZR{RZ?43x~UCHszv`&FnYxSs8FdQ^cu91k5`E+&U)hD zrb~QJ+26kMiX1ay@seDJPn1-{kpOZUYF*QxGM~(gvewO!Z^%ICO}e@hhAJxQmpS1% z$cpwqYO2dnlHT9+grk8|tt1oe>c)Ogqu1d%@=N%jhT`M%}5kY?ZH{7*>e{bfD-}cztjJ~IbU0nHDv83d{%Asp`=w#8k@sZI-Is4Ni0|TnaDK zEOmHl#`MP}uH9IV!wLr$l=!T@V#(G%^ONyk3g_RAMTBkq;R}8bsSe-n5akGD)E_Ek zBw{>91NcIunQNLBTstff&(5x!+q<4EXI;X z`83GM%DC*)hO*7Jl3nuRK(K%A+v9OCHjz*K4xY-|1J51o?Y0(lcb8T?l|&9p)xSo} zd&t6TFR=Dk5@Fl!?DHSdAVSx&5_yXs;nP^()QXpaGhH&Zd8a~SM3MjAt?8a1&H+U_ z%x`!HBxi3G20*m_lB0E95k3n2Hveb5gRI z5Dxdrm>ldV@RZ#t@eDaP;5n6f3YUwHU|~MCQc_P(LcvarShEWHvr>AIk|5Y{ZN&M0 zI=41f0x-@gJ6}C%bP%9^HYtUaXpZjCrosAGE7b45XI2-t0ZkY0jP>MZt`B8QU!E96 zlhlXSKt7g1P4S#3JVNoN0ng%y?twRxuRZijS%<^3I3kifVQ82peFpRLRsy-X~PBY1J#xd)+dNVSu3&cq+ zWT(dC%N`5ABFQ@N1KqTSJ9*17l=A1Ya|WIraSZNrG(fP};1|Z}r=Jd?p5Tl&gO1TK z#$7qjteD{QsY(&#NYz`P<)t)Fk`CpFl8B-n3>hH^1yDG(NB-QE?gJ}x8!AFqTMLUG zXA{M+9VNTGa`H~&ro@-dKOIh43uGteKtR$356;{cJ=#Pf)Js5Jx+(c2Qae9$kQm2P)Vx}D*UTleiym%kXsk}^zYgnpx7w9p}_ zcJX5NW||FbZ53QbU-5*2bh0aGQ{;Zg*$>(%tUN4NY=_f#e>w06PHD+(lk{rASH^KxwL%@$fg~$lO=jSU?_=GpRo7wd4%n*_7S{*Bh(br@ZpQ z@5fFE1|{{FM3uM`6s0y~1Z9U!p(7ya-C*Hb+*0~?Oq?W4a-K3B&r1pfk1Oa}si|#* zSt@xe`6<2e1fQ(1DpuVZB%mouf#Djl3>$gV+yPKj1kL@RChjLx=_ZFuTw4>1-I2?U zM;;)1=gzRd9=Ec9^+&MUaj(r>$iJq=8h`extH+B40=eUw%$%v3Cc{L9D^km4aB$sz z>c*+TGQ8SLk|krs4c6{&b+&ucsZ6-(cIQ?cfODUTQ@~PJ82S;59XSolA$iFYNpgGh zJcC=NwwuG=8w^*9BF`<(e7R9o8OUmOn>?xu#I!x`W#hsc48w8_zuHeQewfeT9h?Il z&+4>$)9|?_jNr**^^lvr9Q%jeNMiMnEZ&CEhMkNHIFegy{d5^H4m53TbSZn<<1N$h z0d@m5)>^;Euim?yz^^_R@-wKxx!7}~m|f6IkMG2T+eT?5ESUyA`E@5O=rPfQlL2#d zLhwOz_=11^<)Z|ZtK77NRn_(HD^t=*r*2lhH&7&h&y_3w-ywV{`~MB_|H&%{kIEbY zZdbtSV8#_I#TxhT(|zRj*}eFpjN67ysLmrPbeUsJyG?jz6E<&gh%_QmVGZCe+Z}(F z5Om~88{E|hsurqGm8={;V|we}5n8vqC0(aLYviOqM1z`I_9OVUj5(Haa(=GMg~^gt zqP*N?5=1Q*w~!sNbK9WAWBUmJj>BazxsrZN&s_3~Fx@>|MD%16%w9T+!)00GYZ6MYXo!K(4is)Jf% zNel~tGI_}YcCWrZ%R$IJ)hqJeX}7}jjo+QQ5h(RRze(xt<#9?6y&JiK|k|5O^pe??Kw6 zF*~B3Qtq@=tQH43WcWb)6K4V}h&FO*mZz&nT416GGfZa)UD+ z?#Ie*sh@EC^V$s=7Wr4NaP`eK3o|O1AJ)6yJ3n=pFLNuzJLOMMx!k>Fg9C9bo0;n< zF9M(NAzT4<Hc3-vP3e>@g&6r zWLGRnR7vhuIMu43U=?|mX(Q3iafzVkV}gX?IGpCZv21aNpoYy>DBXepkwzEhmaH36 zt2)Cl^_(k(sPey)C3%V7A&&!h1c}qUr2+5-kDo_Q|AA(+p9ISw&4KpnydcX7RD z+FzptJDeWtaKFzG(kH|=MYHUNy8sZ%Vwao|GNU`_n_-UZG9NHWRw;7qV)&V^=C#77 zrk}|tYWa_{E zt=GkR-<)mryND8i@}IVj|8JJy|K&se*+J2w@#=x9hW@F~T`XCq2@1x6h`CgpTnai& z7>UIMHMa&TC(H;mEbT$2H>bC(t}*)aA|`ILnw$Hb|9TR4^pKR9OCsQz;2vW(X1C^? zHs_aIQ@YFjThFG)-)p~{p3n1uufM>3_#ZTJI!r4uIh<5WRB-WQ$Mc-nPTJ+82aHVg zGURbt31}0@*206PxHbvVgCiN}*<$*?py?dxYB^%n`6nQPPJUUP$TopC2Grn5?t z)RZ;VsE_Ro3fQi^lf$%FS-2+0GPG9du)&&6wbq?Sn3d%lE)gP4(}Y#puGakH2PWNU z#Kz2YEe1}18!M#pEw<^EX;YiLneXJVHh!@B&bmsYq+0y)U2KaX(js`=qT;df(z7Fh zj_tNzQPxar#k0xR#cb;`a^Y2&32V+`dLSvanCuy_eRN@Wxv3Z|(4#zt3uGKS&c8qd zl~egZj+9009kBaDRz9alP3@|0UI_i%vQj|3|A(jND(^x%kEu+y=yKERuv`l^u`vfq z!)Ubx3)!;Dj>@(&>`B)Nf2Vu7aC@KKfje1ttSM)_HYFP$+ZMl(0&kJQrJl2Hx52j^I5 z7WNeip{SULncEb7M88NVWLxwoakwdmny%O4_UU`B5uP`I1&q5#LBLG4_aD}zlv@#Y z?V!i~eg%Yfl8{2pL(M*st?IY+tsqv8Y&U!MHWmfxJZFcUpk2p;7In&tfO?M6`Cwh- z(@C6tK` z+_QY>jxvP2k@*j|b`R~KbkW$Xjg@tjreSr5a-uZk84HCG0Kn1MaVSK2K?%sOf zef32{{OU#)gA(3ue zY@FGmd*~DLS?$VyY1=8*iQSuiqC+)E8P>#V&}l)v$Yil+#_JSM7^x zO9wX+Ydr$O$84q6#@CM2$Ktnir79HbnjXr!S4C44=Ijc4`-p7*<=V$VGjk9+4Udw;XHxHv4R;u!!dK(#AKg7*26Eng3=mye2kw% z4P2Mr>^z3TEwilo1-XzZxM#M@RFNjE*p?Z{`e$KB^zJAmSZNIQKHpJ|i8H(rWF%^; z_7ZrJs$psy%NYpUAf;yzQ!5>15VlIvCoZe+a7NbN`;FS>xy}N)fGJ{c8O1AVZ%`r+ zst9_0Xc_^rRtK;d3}J^T$S!o!hanI)D;h(n>UT^?!s*Ue0Iv%#b^7D7_+g{MoX**B zetJe0CXDkpE;PSA)if@=p2aiP5KF-<77d?V>6gi8v3&v>Xi4Qhl$@bsOX$m1M%I)L ze|jIBW^>ci$-V)V%V$RS|ug9P9 zgQO|2ah@5SoV{J#Jy){wxB5vQ32}zwXC!#N#4ssVLn4QK-N#SLx6$@+cgc&G<@bqQ|DGh^2$-D+`N*clW`;GVPA%JhfOCqpr^!QWT7m4$fGi2d%zfXO z{DND*(lfR(6oH@IZI+(c`nU9Ty8NxqvTy!D z6RYhbYx>&Or>$-Lmaoq{Fny4zdBKfba1_)@>g;}DYJ!PGA2BXg5zDw)iZMMr^&0wR z)Rl%zYOe1d$k(uRI+rn1$OJ>dA3J+l4p^*f`gN<6$Y7qgZcjEv|b9mn( zcFl9Wao-U`>J@d&a60KUNgIh)+3SM6SG%e82+D(@UEX<&top`4I3mbUNK+93{dNyoN^K-^5gi!gFe5~3kyiVB}W06sQXQgF70b2o1eV3FJrA1U!JP|tm zJtm_eBKTjN13XhLK`lbrrIWHWiJ+;#Am(lATyUvH;1&U@@pc} zPPq_&JE%PKI-*09JH`wiiY4#6Gm64Mm!PidAWKPTFi4!y$}H*yc(+z@+h@ZsEcx8c<6J_|6*Ztf(f@VH zc99(u#@ z#91fR@VS<}94x#u^^4?Qrp-8G%+;F#Na|b18cN0dw3i&mSuj`Q~p0Q|R2r#n5&K-B5A&q(7BtlcTV-I9~pZgp^Sr8D00y6L0?R z!8!h;zu9+L^lU&(65 zdyHm`+Q&Xt0|J>tVZPr27P~E!X=08bv0h#dx4c7-XYZU}H^|-aY0gUa?EvV@XS=0f zPu}%vgTgQf;6!cfhBf3AYa18PfRBc1(7YtpQ#+@1XbC*)Dafd|vd`N25AJT?W#acn+Gq;aut_( z--!sp?0CuSY&iy7olFPi@0AQ4hxW=I?Sl(v9nXV|CtF^4y_KhPi2M9+?*4qla^8xU znkYE`;xGM(?s(taXN~jzK9STvEUx;xg?u-(IG@H^N;NcUJG#ZtU*k91d;UDzQn%Dz z&S$W@*yKzt{k4p=yduL4<6tcj-x#SdhXKrYXxtoVlQsxlVrPY@LlK%TsaN2qj}El% z5K?*AwboA`bzr(^7B;ta9MfjR8!gjFwg(CNFYJGtBaDVuzg~K` z@b$gf4b1-X_2V18y*Gj-DRLUrOLRXz*uQt}x^vFHe7SwtGyT!mVn`IO4{{`3VAsQOEw3VJY zmuF7hX>#~j;1+D6n6DJC!h_a9iSsSeJ6 zrp&IxkU)!PuGcmdM{AUNxWV)|HP(3Ol4Mc1;?&4YYEf)BOr*;+_!J#u2P!%;p4s^| zEwt5CATu5l%HpLw6^5?0R6n|YCrnrq#B%25pY|l7aFG6NPFgu4H`#q8(I(>njEfpM ziI@%{sLTBe_RH46fx)4iq6+i^q7lK%Rrr)79{)THy_z;%ShIR;s7tAhHj|y4&dH{d zzWj#y3*Bk6GEe~X(>(HsELn^O8!tUcc1Soh`ImbH{F*#nDfA>M{J^!XgOBes!O|?_ z(MEUkK*W6V@g`aIaLgvbZ90-t0w@JErd5v$U`AIo&bS%{0CVOx_%OC5DP|~z(GuH3 zQpC6%xwpzVL7(*F!6(O}W+9MySPI-5gVB!Zk7mk3(%L=u`y$L}jrE7yrmh#Q73Ys7 zO)o)o198=jb;{cQKXGMihOY@ZYkzhmd^09&e7-tfB&)P%4e}PsqvovZ}NR zgd4+N#(g%pw9Dq9>Vk=z0d!NH}XI7kqen`-8|1+lD3AbCZfCmSb-j&UoJtOA4Opr z1{H(B^-}}x*lC@0lV-jxX((JMpVtrkQMW6X7kcK{S^THXRE{GbSKqIf7np7&2`CbV zY<_4M`w0HQ6N(iLBg@!IU`?>5q-70*YsT>4Z|Mj(W39z*Xs9y~q*}F>s^?miMWfSL zKfq7jYE4Gk6L=7CvHDrW&m6yj9AeF8p+TVDq}@5kTVi(AS;p?fq-CR&SYOPYKDA>! zzSXP=T(^FGs=%V`tULBmcAGd?GKOV-;yG&VJ)iZyz0&2g^V;+k8K*IJFDf4VQso$B za=`wwAnB0{e75Ra&fV6CeR;<7%YY)D##cc%!RoDvc&mggcQ^2tvdcWs0;j|KB}Fr> ztM`=48E7pel2d-T!6275c`f?GEdR^5KQdTh3U)i(5 z=*p7C?PQgAwS;fy{@tU60Aa#TyUo7N$G(!FH{>VT=6;LMdXEvuaJD7co0PmMe&~0m zpF2I%zMp8BN#b)5KR{ZEh2Td*wpm9&&8Gtmm~$BFDrh11T9Eb3I&%Mja5A_6aQAQd z5HAP@J#s+)G_LdzoVh`S&iyc?7n@Y@gV^U5)GecKos@6!LbPi*V}>1Gyi1&Ih*iX&+j$bBpWHS+T2%VM}Il2`^4CYyh z*X|d;f<3(vy0(!kuJ;et>K-?w<<~+WLmEJ|hEn*$97$qBAd`!4NW=R4Z$J%=@8X#4 zyG^k~{PBbS|8}bX;Rf-)i-3Pt~8rx{J^f z#(uPE(`bs^0^K0S+OtFJGR&lGuNyWni%aH38rLx(yD9N)F;}xw<77? z(>47wLD{a8NSmw#=R{mffmVa23la908LgmSg(VlU5+uS>0F zyycP^2GE77m|`{I7na+V$tlOXI=O{D>=LO=3XM(Ltw)2iCRsIsQrQi(P7E9WX|G_{ zpkxqN_c0;d+I3==sL)R}tE=jg_oC6mVJv}OSySU8RdA_pVqYz!9BTqBur_g}Pe_*G zz(S8)k6TnVKsu2?E!rmuY%S~7H1!?E&E}AETCEn^u57R|eVRu5%QPjMqn;7nDc-Jd z?IN~AqCe2qz39rLj*rEQqytA%+7aE=Y2MRgNt z<;`Y6qA(^pO~(zZ@XZyjj3v#jdJE`;zfIiAdO)ysxlktXz>5Bo9xi@MDYHsP1!!6h z=73kuPsuDrl%$Kvr#*#dtvbw`Pf;s6n=-1aiemo?{LY`pD<4Nm?NVm8<9S^UN^aMm zzZ;)(V1>2nbH4Qv>l*|2`J)QE9`O^T&jEA57H&Pn9^|iJHed_e4RsGohv^#PUA`|H z_8n6YTxUoZ_8rx>rWiJ3ARTP)$9sGFl>Y&zcIF3X?TO|I_^oVVK@ z4j=1Tj1zAl#b3KclEC$#g4}KcwIB}6f>hZK9ZgGgL>RUujyr0z-#=|m>dp_U;Or!L z&Jry=ZbgH6f?*`45~E~6ssU9ZNgU!VZCQvv-imq0ZmHekSP`YRgE-F0jFghaOhCax zSMt>&h;U9yVE?DE-e`MOQ|`)VQsAP8%jyM%zqT`qPM;Q5s^#CtV=8EqQ=}-5(xWO! z3)4MJK3tPlD-P;GxpIRF2}ihFLS3)S_tS~heo|mcpK16uRIcOFE?c_+X3hH1k+vei zRFNS*YV{C4!=_t&g3vL-BS)?+gW~~>%MnPedTva?46Ym|S0JJ%N&`_kf=^kJ&#HpA z)*?!vQ__;gBoB@*On-QA`Vy}gf#05ViZY{5qK|EKIg#LdT%K)H27;IPVj;#Ap*zEl z4EC3^$+PlW9Bo*Z1w?TB2`AFbU6E7NHgYn}3bcP{yAGhFCBQEaE=97S*84v=`)TE= ztMr2jhQr?r$;B)Lm13VCuPGDZpb9)OvHZUm!K$uYF?j%|71UvoDA@HpoQ+ZMfT=a5 z^XA|td|2zVUyuIu;Wug2n-Z04F$IIhXC6k{L%3xMt|FL8~I&6r{X)1s-hv1moDYMpo*sYEo*b& z{iJ`EaYZCyKPN$tjlA#|Xl7+Co^fpy1O8Ag9zrZ^l%pGdWKEG}jR|pu`nf6QORYXJ zUd2$F&7cQG+2onN&0T0LLHil#V=V|HaJ4{FuZRDK;Fo~t5zB_7pycxP@^GtUwrjSn zpH@yP(VS5&hYaE&&e38N!4u*=3O4a%PL=WNl`32!LeCJz8@~_;Ziw%|t_6F?LEb6J zXBT9x56YqQxuO-xnzJ-3@g>|(gg_#p4ob%bBv!f|iz?Ac?N=+mwP@mJ?Qx9lJX~Xk zMe(SzA)6|p>1GB;pnKSzDt>&J+-Aj33QJUe+49M0yNqo*1x@%5L4DbE^vGobASb$9 zQz%WWBWL@>8xM@=z`X|wynGRw@W@_RZ5)`n_$V#<*DL0zoA`!XC6p}C&Rufc7x7@9 zeH#%l-Lu+m2?#t z+n%+6rEh2n`VB4F{%@DCqoJ9xwf=vxa25aSQQ67d%JF|PO&XOfzcWoRd?qo@0D|Co zT0+95S{=%Ah+wgBtm-(}xgf&KH`A<|wGEP*a(`2YMfqY%{`O2+7(_6<|L{$?;Tf=k z*@Zw7Upl(YlH79KtiQWkdeZ(uoMPR}aX)B>b7WXR3%N!vfOeSY=K}-+3=Reh)k3me z=SPK9CDAgOGja+uw={a!LBwSveu!F?2-iC9JXAVt>^K&~)3*@$b46W~jd`n~6FPQp z6sD?G|8+hWk=f|WCS|8f-eFx8Hg|zq&`F_QYVE!l0}DJj3A2|e%7(sSUutdG)}{i} zCbn7~#e9qdt2wFF68eiAgHea^et+==O_kavIiVxp#^NBtT+L(E8P_!+SN);&F=G)a zBKJbiOd(TRm5H$^CEq*BLBZS~+iQf4<|-{El2L3e?ZB-_JB{tVE?*KO$&NTmK4Ku2 z-x8;p9&gBcUH|yg71&-X#3mm25;_IRPPV(jufPvw5EEctp4eHbGK%%odXRvUsdf{v zpWy92+6Su_Vrso+XG(qP4j76lDXdck0Y8XAmQ2xiQ{tnb=pcfwK1bDAQFS|@53=PU zKwH^eEE@hg*`EH^!*od(3rL_sw>mLST|bNQ?X|R-J5sSPI!*SKCK{wAacHVZ9EdTS zI0PBRYSg>;+EvB;l@)NVsu5$&Lqo48LKHW?8yP?ajDLx<%t*Ql!fc>Y&rnX*$c5y= zk%z2}bO?h`Zuha(Y~V}lBEC}l!0G;kN9QGaC#d;Z7s)~u8#!;4P@z=MP&boiO_W}8 z<}LNyc4@zJ_R0o>5A#mme{3!afi)A#dykAT-N+`vE#h0CzRQf3gM@wqi&Yy1&251H zz0z{S=5iBV_ls{B7pqyu%^>pn?dXc+3d%00Y^a=ARc4@kqI{sb;G*1FT}JDH+Q5@A z9r*%6@J&sRD>hey?`aDBH)!Sk z4{Jrl+}O(KzYB)so4{^t`W*!H5BRG5-!VY67mHOWx<6v2y&yIaD{TpCLdT&vaV?HX z)Zd*jjg{J?5+?imxAsg2gd#-tOuvgIV~jaxP-r>MGi%QC(mm_W>7vW~>lv(DnOjU? zC@=^+g_+(+Z}m6+Z3l`1MxXR%mcCQ?40S$H{#r+Z)V(IkRSE`kENd(SJ$Z~rhnfzB z>X>BUL|jCt2$ct=UUJ`bxP<@u@--`H5mjZ$&akt(?p;P>TZHkn1v?c=CS2Qm!XytT z*Og~aSo!TvcI7Q_=|H`PX&t>tI@c8(d=_*=-II(B3)BivZ*S@@d~6 zJ~}0GJH4kC;H9TsN}hs;9Ms#0m34Rsd>X#HQ-+uAcT-jWo6Ntm4@4th7b`0?e0-ik zI`ZE4O_Jk9?h<~xdgMi141G6pfXE|>0y`XX0=u|6CX^RL>)#!sOR8qJ6v*ttmI@cz zd9UB0A7?`|g;g$30{9<({i?@aIhITQ^$n|Q#u$SktK?7S2WYA}GwAdpbg` za7OA}4rL|@30gTH)}->HBg16Q9&m=phw0J`F9U^}6D}*V3%q>A7znRF`8NYbv1hc&{qJk(?7P(X|HD%I=W9t?-|k!ZWpb0?P4(|T>Ip)=PtlB7 zYc%wFWY~i=?a}pj-bY?f8JnN4A0H5YYs16GB8(yQWc_x6y#*0J;*b&uh*~J~k(+<- zyNUKDz+$?RuE*PD3wTstR9q&oKrz7r)fzzZF-T8LVyxRU2kUcMu7Cw4w`$BMQX$_vCBKw3NI=Pg*lBGb&wcR;Okl&80G4Q)pcEKxQ_ zBo@yTsc+0@DNArn=p1nTq5e>pJQ(C`cCM|)2CE)|@!2YwQag6N!5G_7Tr81z>NUDu< zx6QC5Zhxu|tIr;RcOT5)Cjjv8(u?T-|qVW64gasn>V)@L9_%{tW+3bcTLpr^9vTE4LqjA@wnf-~;N=tOHf z(ca0&cV_JmPD>yBkz)4EsF|jf#)~)<%Y#nR)7?S-^HmSQbLRiST=mlmM4+P|@}ido zQp7&X1OU?_wi$4V+}06;{03>IG7~F(neQ0y?Vqqre9H5zj;g(dc&~4We``_~Hck8T zhHcPE7R81!q)E4E7g>#04kNGCYaGU*GL>)A`n&w5t%c&FYF9&hn|lZBPl8xnRsW?w z{R-33G*Ba9j1mHg%5DdScQx^1#JFof1;5e2-e=TmUWj;+Rdj&7u zl;?;n!GF$rsqODroFTuE50rJ%=(y&Yv~GW` zuWWbGZ0$`Nsnv^{H63$Al>{~&(Xv?$A@L5}4m$_wI=LZ+@Q3k)wRmgOQRL~JB9Mh% z93`QGbzh6{@$V-cS*Npjil#V!8{FP|3H)hV!Lz+o6DnA93DxiN#DqN^T?<=ITITP8 z1K@U^b#i5?)yY#c&z7|RB~CljfDyD+C&Dy&uyGnmRC~hm5w2Ke%oO1wnHmLMq7SS9 zhrKXsS)q`-_7$v#j@MgYO+P?xSBapnSRudVT0oR&XH!*52~{kMys1{5rjc+zaJ2cWc3ghg!JXT}>H3L}JBKQ>Z4rWGikiL_4 zf1%pZ?~Wgad;u zY0>uEw~y&N=j#8ez5J&}=bymae`B^zRZlym1&` z-$*3(quQEI*U%KwzHckJ2ZLv)Pw}7ET%STS382r3M*j71DoPr2E zN50L82@d3@HuCk`um8y&dZOm8)am=6t|iFC0H*WKkREbWEMs|W_imz8CrHDgG6c` z+ufS=*SJvIwCAMIWN93zZPX5nZ5B)jSR59+@$NHdlgdmdPQ#lGXca-`(|$|ObD>^w z4ECyR%ax5I3Mm=(n~;tAkx+7?+T%e0Cs?CMZV4-zG|7dTL;S|v)$`6LWl(Qa$lP4l z<|V1Eopy6)AN?;p-OboGJ++u@flwe$&y-to$7m;utrIfmFZYzgQR3#nekzVAl-e_K2 zxk&p(@Y>>`U`8U3eRgqX??SnsJh3L?Y-xDG&0?-ia$0e9W$nC6<6;@js#p<;)?{w# z;~Cy%q)L)&cc+F*T#|QKu;F5xzi|}$OARelByNl)O4^CPINk|}jy9c5XH7OIM(Um# zJ2;0L0+NW$AdHeibR2ZQNojdq+2tknrqsR_r&cz{M>Tye*{EUe^KGkrY!Fec)sMj! zE|5}?UZ*o&GZJ>ZJZttu>I?Xq zT(l{sX;reY_Ni#}H2bqMo|@FX6Iq**r01*f8!hb|TD3|cU&sSz=;G^Pnl31UY=lYtsC@6g|YJxy&#>xyKfY3PgP z$>O7enB0aBs?%akbioX_a4dpB#^ns~MmY>)5G-B9IE^InxG>UOdYAjC;??LjM^I-E zZVv2_h7+v(7>VhZ45@+?&cl@Ppj0Yq=P0(w3LT)Q~ zgd{b%L+w?UXm2Ca>eSGV?4_wC#DWgI3;4u^`BC;vCaSfo_aPE~Y`Ji`abptH-*Hp0hip%yx-F&LPMWAaOh#;Q zalqwP*p?h;`~V?PvnQEVYP^@xE9UONtLkV+uz=d=^+x9`lG@b6sB?~(Djjd>_<_6B z7N=NYkdryC>PVGm33X`XU??|=@Cq^Fz3=)Z%8mx#H4g3ggVTzjalQVTK8>CAZY#PY zluGTbu)%|9FKf8&Hh!2+m3$j{7hQos2#O&v4Qnw z(X2l-EqH5o$d@Al3Woyn%YhS{ANo;sN5vA~>cE{UJY4nhPv$!`tBIZp0F$UAqix?K z&Un|(IF|#XF9~_QFZ*3q#yCxba+V3Ntm#YtgR1$RL|fY~3h)7{LxX~hz3`-REX`gl z={-=5D@rss|Mgo9BEqf2-z3`F)giQk<)+Y#?2s^!F` zJ&R&J`)Kui6oZ)yG%~vyPThSucMy@Rz@K(x1brkL>wN@+IL?vh?2|iY*FGP(fP(0# zycffi=%Lg+*=Yq?M#^l1Qy%Fk(-a(tlD>Mo>x`!U->Tsaq@1%!$6}g!g2h%{S)tNi zoC7BOos#lCAowhL*k&x#gHW!FXk+oFq3B{`FQl{34Y!!O;P!4@@n+C1OU@6A%_LQ2fc_;0DSP7aTO7S__aDKFD|{sPst;YfflpBk_iDYL zI>z>1k*wGRTk=_)gNoI2B15GkP?Z-Bs?iNi+U9Srg%?|elEz(G&NIloJjk-o!9u2> z8#&3Uv4n-|ggAfvaW|Pg6zWCd1A-=yu?d>(8DO-WNa zd<>iU_O|;qnhR{TL;w(=INGRQC{wy`3HkY79qlgT^umoAR|JZN|~%S3GS?iFQ3 zEd~1u{Y3Y6g7R>T_lnNE<(=v>7C8%#x>SH;k>4BnbsEbRdk;}oMYx`5K z^k$y`OB*xU0Q2Vw$&}tt_E2A=*Mg`~*`2wx3ztkD_j9BLwGIO^E@zgcs+1SLe;rkS zy|nR$!~ggZK>42|xPO`n`~$&hz__6-Ie$jG42YEjK>-mlg2N80Rm7lyE`u9_iHi|< zHtLBFqdmtaGK+<)wvS#yN8rXUhvK&|U$r&9*=4lE6xf6>82u~?6<6Sj%nZJ>Ln=0p zFA~RXHcwa8t&P9uV|tsFz^JUD>S|?jylOpizIvHzt>=DAC+CC9k;l8%`yq?%Hu#6A z({~euE$cA{%C~$1o!bp<%6&ST&f}WYs}*JF8Wm*9a!Qql{)Xk^rOn^gYzn5Z1z~Ic z($@T`8Ls2FhsaBG=u>8>t9--Pb{n6&1#)Zd@NM-Yn)3hx{8QZcefrPAvpgmE%@nxK ztsbh+dC%B$vcfx+&#wNRE#5oVr(k+E*0Va4&!s>5Upd~t^M`zr*Sa2Gw$`7b>0b+n zs;eD&6XmP#8kmanw%3{#3cRPN|;+YFd1nc6iN zQ={ND7zDgj&e$zui3R4xDrFPuxLjGmk+nepDr5BkPLqprBnf6nEf!1n(Cd{m0+Mo< z2*MTJ%VK{~&!8TIR}@d8;X(U4d(_0X6f-aT?Lz97-tub+b?unN7Q1%*^@uQ{;9tPq zp9ZYWyDGMMLgm=#OC_hy{78q$n7Lfb01%nmyp7HE?4;X5JGge_Z+1IzwW(=9d(R3rorRh3dG)eOrMgqZPGXZK zp0azTh8|g~$%GZ7M{8XcNvl`gFKSd3nLkX5a@{Ny%EH2UTrRD>oYQIedSK~GLWFC& z@ncdz{EJXgGNI4v6bV-V_0m4;SzO$3IJFBxu+|`V>=psx7?XAutoo897PDQIK(Gn) zh{gb#`e-u6Mag_h?1Gt?$Rew_F=?D$ZGe{kXky(;0*za9w`z`}bO%0uj=w`9rDkGJ zkvdR(X!Ne(=uXLP+iC*YL8CsP!-b>Lgp-CivyspIDxfDi`YjHxad4?1E6Ip zk?fQ{OY8}I!gAoz+LOS$LRFIb%RklPZ*`)ynOOu2&;wB_t=S)16Odp8janf=UuLYP zfLMS~10jn3NAa3b_PW`q^l4kVeS%(G=D_KVouC&920WE>9|aKmX^K~P++X4J9M+Fj zSTm-Q$e?_{Ih(x(z{wbo^(V*_^qCZ95~5=}CB%G^=0cKlkVpZgJ|+Vx(f!hQSYk|4 zHkeeXeSJK&4H1w*y6d65kyvj614jvr$8bLee40Z#4(owSYhOaMxzGkBTMhFtu%sh3 z8uxi~c=fv5Ai@LpSqYpO#tmQnCISEGA!ed#4bY(J&q zW#XF&Mg@l%{Ro@P#}t!7ZcLpInjGsyaOmoZ8?2D1NZwCR53(FkY*R-A`h6dPw=+kw z-5qy}#0erj2iBc3MpUlgi=al!A}$*$e@BN^H9gcMjqXzvkc&||ztu_29~0;NS5$V; zmc$l`4+|`fNAVin>J3h=s-mW%vZ8c7Pepg$>M1wo4p^cOx6|_*bB)D~z ze8sAx7gxuZ_Fcs%WR@_R#8PbPTej@g&BdB~yN6X(hrDQ>nG@}q`QE07Ye9W%4}_I$ z7yvoF4c0pLf*mWAIEaN$eFj+C=7yM(*8W zIlF)VrggyS#E)FOaPct3z-8xo<4!J3m=7Mx&bb=)?65jot9jOgw(LY2$nr-Ff(yp# zEr0S#x2tB8Q`HoRt2O_$kmLm3;F<7O>FvEIuP%*OcrI6t)kEzRLE4gI=0s_loqr7- zX&#f|>Y+bL&JGTIMhPqa#lHrsLQgw7Ec+vh{QkJR2d((nKuWg$<#Cmo6ByP#-*}1E zSJTl?mIv3Lx7~K~QQK4Fow==L<8Qe-mjK3K&)iJ=nSf{nYWourV zb7GY)Ey`q(Y)+Km?hGOftci#q z2QM_|=c@#^SNg14oZ~RJ0- zoV!lqX?vp!YSxsvq`?V+w!a2IcL%^cITqqn@biJ^_#PmB!1uWX+G4w*Z-#dk6UbiY zLjG)e9pjLLV0$DOC7&bgrPv(xEX~qXgr%*e^p9hrIOPIc5jY7c(`+ax)BUco@7IH}O&@}G^z>^oc`F=RdCfUy{=OPGgg$ZKY*FX zW*nk1Bri>bA4)4>0QF*P&oza97PvbfSh#R`Y9OONNQr7%(-B>!-TQI=YDQcN>{arF6D$v$o(lUKHc{2V#dHhCVDo#=}DiDhyNm_`I^E9 z;;Lu~fA3(rGkEV#{{2@HmfKZW+MudFmfKcAkOv>{_BnCu$|Z;n-jKwVdWA$DS*Rp7 zZ`JRt`lp0T^9}EA>5j`I=ZGawckh@b{Oebo>Zxc$XQX6O`8UL*!tAAtpys;C#QxP_ zInJn8QKO!$E8n@rj2{WXGwyb;`?pB*{kq*=>B=e3Y^3a(Slu8F<19z$%~1 zrjQU}SN58oxBs#4{RzX{3rBZg&ke}eAKjjjd|9{XMAS`HmQHwpYPqRxp!=%FT>D*; zZ^jkD!H`{kIR!WY34A*dc{u})hunlOl{haDny25>Wg%g%Pf44@JL7J>gUPS?xwBZn z=t|2IOEbu^?Afkyn>r2H`9ZwnJR`6GnZAZ@4VfoJ0P+DI15lN3=ayhf+l{P~kUOl#+JV#?rg!I^4#lW(_T-&ZQ? zk=pGR>no_szu)&GWj6OHL!(i=wK&p*c!QCYdJ=14Vv(EqxC^ifk~Q;*r+K^<2bnjZ zf;aqNJ?7yVyP|InGd`&}W=uH9l99|hkN9g`#nyYYAOgr_)lb=7^4bl0C#%pAWk)Ax z7LV^|)g^nk1ZT<38mFu*wU7Q z{IPIJ!Wj?gRPa9Bc*m@_XYGs*Ec#2~ z8*i`BH})dh3Zh(o2L-wpgq<)L6`Ql?5VP->QJz}srby6A!19jH9EFT1hl*&d&T-F& zizFwnyhg^zMyirZy4Ut;@$pq?`Am!)kAIR#n`3w^b)eZarBBw`Ozgi5mMcAT2VY|F zcC|l&RNC-5FI|rpq6_t)sy2=9heJNTqYAl{Hqdp^|8-IGLg;!H9U?%d&*Wourg$Kh z=c$xEF~1kjyt%S6Z?M@R&PitfOIeP2#@GU?-YNbv5ZnFxa33nRkZ#>(@2hnoB9C*X zk;ov+;CNeu5FXr(^or z8F%A4IQh+z+@AfpAx7cZuY*7bbP$v< zmgoECZmqA(G;Hw`m2^G_U$CJqf^=oy;l9kVaOB3N-NylJh^oSa3zF2%Zz3DJ4 z+FKjXs89UBF|<)aY}+~wym!(ZQ-qO^!B32jS8Hj(6Iw}}>K9t#5VQw5) zb#7o+ymW8RWWs%2&PSk1kvB@g&1h(ADBiFxa;8GPa<2Ohd(oVqv*r*P9hNxLH}`JO zVhAxEVGb@%{b^aYDm~`3NST64MV?bDE08lMgVifO$Jlhl1S0YX?@cg~)n@GyV02cH zVO?F{w(XZfZ_NiLz#-6cOJySKKc$tY=Mo#g5f!_D%QaT($7F=$I5xznS#!)5PYfGp z1NKm8CK62kQ`iHSr#85lU@d|ll~3WuFqQi1*Rqn#>Ox@9cB=$F9xFBD99Fvlj6tWb zSet`RT$r5eZpWBgK~peU2dJ9f4uenLr!$CS+XtJQdzdv?`yR9rWSS=RyEN zt!~fy7!D{M(o*0+i=+t4d2h9tf|9H$91jN;i%1AhF(C}v-$7gEceKBA*Pi-JDo~2O zlSWpOy_C0(%E5aWsruxn%krvJ1;sOr^YU|HXqnNVd3qc70|#f9ESb3q*--KZ7U7I| z<|6@>y9ijHA$;Z-z1iflwe00AutbGBbTtDB4HTbcJGQ`*J z)vEY50-16OOQgQAMU~CT9dxt*E|%Zm_OA)J7$cL*-$uNxFoT{Xo<%c*^zs{4pMPAMz!(~L zg!5<6nnSblz_t*y19s+>(Io;*8K6Z@RHZWkUI7H*ZLQRUc9m)eGP-u)aL6e~Lo!X} z1Gg`{=U&6bIGs+{C>wBkRAX={Be!HtN%c(ua;m$W4MYHC5`7l^&4{U(1|NkaB6A4w zr8~8uEu!XK3ULk*g{GNaPKD{{i{l;g;Y9KC?u!PjAs??awD$ml`(5~#_?;F}H{6@` zP+p|g_)aX(^;)=!je%DKP}&(Ge&N4i?Wm~PV6=L*VN=@OzfiNk$fFl{66ov(XuW25 zki7ac2ckUFNHbwx&#JJ8dL1W>>P2hjcJ19eG1B*Mk&SAEsG&(2Mw8fr1!NX-GWCEF#>;B5x9NuEz zvO2mmEf0AD4$v^+ct1fbz<%#`Z4T9FDcD#iH>g}qQn%<(;iCD#_T0ap5^yt6&CUy5 z@^JtP4O?>8wep6w3iN1qQ>nl^+JCTL0{tJ(-toD!AW+wxnb@{}v2EM7ZQIGjwr$%^ zCbn(cc5-vh-nZ`AyXx!@w^sFEuvS<1>VDqmMJKRdfd4}wu$3ZAr~>%22V7*QD}_4#G1~=x$&+RFs%WLu-Q+(@pnY2z*y6F zdFBg$e+K*67l=&{)a@}YK;#T#gmY;LK5mZ1fA!3D7x zG}UtQ;nvQY^jr@MhJvOUIPWVfV|J7bh&Q3K@h^j96nLgxpBbuS4%%NH0Iu5so`RM5 z7=9)9yAx(pX+w-0P+Rr=!;QL3GiniZveSPuP z2*@L%1mb=Rf310x)`FRsi@=BfBI%PR6UZlg!HX~(1ZL8^6np#e@aN>p&X6%mm!!Ge zb=WTOW?kr5GR&B8C`vLOpImHqxIS-nz1?4LfBm|F=oaT6+GHAAQK-$=BaJdwLcc0Y z6`$YneyUZPV7M+%HLNaAHLWfiq34y0GxaIW|EHx|_s^wcGbde1<~*#96}6)8i~lLak{nD|wg4F+()le!;23X2u31WMGIqO-9SG zah;w_%#)CjWPO`wwPc~PS}B3PRc}E8w^?;*tzrCD zi}fhsoFOvdvFSwCrG3x8-KlyuT$=7&Pr2$nzec%^Xv2%aG=AX7dSajD>hS6pFjDkH zaJf->7^2hU>agRCdBx9%DlAmvO}VQx#o?0EA+i%p-GAlsQL8)w*S?jyXw%-?y?*PS z=0?r#B+G^SzRaO?n2z_h=?tN|l^{agBiqm+%5y0&_N1A&e(6l602l^3(_joGz0OS2 zg!Lq~3^?qOfhf9kshNLh_L};bNQFi?}||xOYcpRXL%Mo&Pp6GW3T64mCB*j&SEW(wcZpg`%e&=tRF)m>8$Wns_6^ zgQ_AT6!xu=Ino!7*A-Qvxoc#XChYQK1apkM_8kVXq7US7|c zjZCG)JN@4f$qDA_;V}pr^$A1*TVs(8l)$wS_VVEk%i;G0wC3J@Qi9qMIdxFZfV!^P z2^I>o5zo@)mLD7CSOeeWroowuEPE1Ugq*~2*ae1Mf(av6WElENFUb&S)}k@>H(Z<; zv`GN#$^Zr#_#%K7VLoF!e_|M&aZljG-X?#QL7-i_9cY46;r&Z!GZ|EZMwx{5Ln77i z0?)Sm`3Il%Jz>*?Y~yU0o1wH7B+V?B1xPrBAr>nu$<}`p5|z!Sy;LCl`c*~s>le-c z#G(Fw@m@t^2ciF*x?LJjdP<9FUtbeaS9EQ=2;lm1Mv1@Nr5I4*iSZE;0kcH-{P12f z46LL7jBGduKfc~&d4cr&I!^+^SV?msc<=Xk!*6n4XlZ?} z`e|<@7hjXmzNML|AieewTP)l=6;6<(JB{#>m~PD#3d64YwVL`84RWLs z#V^K;YNX_^5%I>1E#jmS<{%if#})z+YbNB7dq^0W*#||3O5WT?)On~9=NAlFf>M5g z#LqLc7l1o6h)zekY)~baGJ&e zbHgGzT7V3u)ymPLKdR&D_aMa7Ci$F+kh!*r69Z>@c{PHCj=hsm!L7q{CdJf~s zH_djXyeLsY8u_Ehq*7)vF2nt0Ux5T<(-=A?&xX5huBF^bp+g-hG(d=?*iPj)*@u)M1UcGx$yD)TN=i~CDsQ0d#Tj*dEnmF{X_n;H%>(i z8UHz%CX)JM!-&~5i)hmK7X~DM;Q}_7V8z|kRh;k}9F0wQnJPw(Nueu>`V0+R8HZ`5 zg-BAYf>ArsiBNE(hFm*!CR}Oik~>NYR`3lNlR(VO1;i*U>(4G65reLDEA=8q>~(k< zA?0m?c13Ra@@fk%fhMdlk%0Uc741#)2le>c0+yIy!&%Fb%g%7FvoofB?L1JP{LRm> zT8(C=aXCvQlN^rZI4ukD8WX6as2I5E%(GC4;$zbh#+biTd2nRHk-rs9iWzoBCib3` zS{*GdTc_ueNm3vLP()Rf;|p~(4KG#Zc@*GI{daCF384u=C$m&8`no)s_Hz4uZ!sbSSliB>1f~`X-l@urgU+PKP3I%`ZiPSAb6}=DN zzua)4+}ck*^Odesh|IK7hJM?%0X-!a+#_MiajWJTAUm#y$wH-8l= z_1s+|I#VGADig>1;ZS+l3Rv$(ck{(kcg-)3vD$qx^&vz{q*ZuR9mE^q@oFJEX(u$_5Ec#MO z(=vM5?j^%h)2=&n2-~JudjOQ8*S-<7>~zt)-|i*fT{teYnzt`n-XTd|`W0A^AFX-| zq}^TFTE#YNRANXA3WBo#`3r>#Mo5YfBQ&Gz-%@KEcWoDGrh-1c0cf+RBKPl(Czw=i zzpvJGc(_vdd215T#5UWcj@Mo!a1A}Vz#2JC2RI!hDGUhUbEII30jD>CfOITvfSGVrmdDG@h`y%t9NILVwv0qIRuT5I@QTpZ0{kRp< zX-%_iNoAeRV!_Vt8rH44-IIG%<5iv$z=XRPgkElyerrayY^aLomKg1xlk|U>O4xlx ze^Q2%@mBc8CwG0M_jXvC z^yFW{mA^%*`Hl_~0NWrE`V*n9B_5TQWIod#@!oZ26yM!RDMyBM?4&UFQbD_g))8$e zDMtt{R(b6wY_=aXM;cd*38FFy@zMjuXN7+9?nW??)flbc@%HBIJzJaN=U>LR_PBZE zl{+Gm((+B~rTKg_I&l>R8((#29d#*%>}}>=tf?5}vPFranZzfPuCvV9P|IU*E>#`} zoxDd1(YR$yR!!##tKexWam$1s>ZM~Ki{+iU96pL}Z6Wu>u_(Sc;Y0k+k7d|pHPfLD zPGuM=oSYzOdpQjAfQr_c9U=wTNG4jgS=)>|rkbMREW((9Fi{)bo~ym!NB>1q*;c@C zezQVJv^~w?5G2uKotg@N^VCT`y#&SG0HyDcX;s>&2%u&w%>B^4+w8v9PL-*`T6>e? zE~P%<`l~FmK-#@JyOjo;fT~!P=@?D|X{q{$*woqX)JJi(Ft;h;(XM*5bo|St(ca0I z&JU3aO3MT~6iPRby9OWqc%00^RKPtl-~;Wv^2|S2aEGgpIa#=NS^ypEtY7pPm|#7$ zNKWZ&eZ;=6V%n%`s#FHNM{c#;V06^p?L)&xx{Wd*pOmOy&b8S5Mpe>8Ve-IG5HmaM zM9Ik+v(87z*%#+)seddwUG>Z6MYgc#E3aTBCn z?Qo|h(30HDP*je{T=4?%D2|W8+N-XdW>@DA@z206xll!8KU%)ln zxglEi3cP$Hdu$ZOUf=D#`b>zs3f|y$~e=XhtgUPc&PX_6Qxxyo@`4jVW zpKUM)kgWxgE5GO2u7&J%rM6-vR9Hxmz!==Y7uR099?Oug!gP&Y|#hgqh>{U$~WTq+-1C4C1EN939=pjlG%(<4`;9hFS#7F zDuPKVib_hIIJU4T!eE^7Uo{)_suZTC$@f_k-yB&YZbi(P4WKj&r7TOSxRG|+Sw9&V z4^`99GWtoa=396p=F~I2L-Iu7EbLox1y1iYY`F4t2d}R5d%%~#?M8NzHePBAsN`Ml~kUtz#RDCSAx#351bt1SGpFwSv>MceGSoWb>QNrQ+8ziXK2xod2?uPZb>q1pbk{J5ivm`Q9`i0@DxpFPA zW(QZ|0K9`uQZ1()2q4cs1=>w&XGD3~*xHm{pS7uS&%jkLy31qc=Yls?<_M#K0VnBSl&w!i{_pn!OaOgFb?s2zg1OJ6E?c0nvrbRP!u~Vtz80Fz5TSE5D>Esl$>fz;4P11e8fR7;aU@H;fx>bNEQmr|@p)CYhc|*sB5&>VtsmAVq&EDaooYf_ zI02~b{OIMOLu!= z;EK(jZYK;wZ^vektgh;UKXUcR;#rG(Z@=>kby^dXt^sw4qC-2JVznTp(%=;4PJ>Uc z7~X%LWJ@=n0cu>Joxc5y-4aNe6tr#`hKZ_(5-nIzGnkD`l9Rdulgp9#9>y}@7$`hb zHyxI+dWsyO(1r~jS{i?&R5IPNII1yj15iW_QEocstt_l;M-3iCPzz9o2gTZ!5zv{j z(FhM?*dwD)(wPgi8c2qj8B5oL)xEwl)!slh4^jq(Yff`}@&dk4P4}p&hSwtlZD-ki zhg%;#^RMBk2DkL5z6k}tIV1gi4Vc~;aZYIm#`!Yc4`HkZ9|%`fW_wu9g#7++O+Ek*1Oovtj=O%R-3b)CSQ%omd;#e4n5(q zlHb?7kFv!Pl8Rz{ur060HYf6*R$SF4F{qd7qe}DFAQ$eEGOf%AtIhNeoEVy9AXSu_ zFlZ}<6hz5!HQKv}CFP-XmhnI4R)+nWhqRyM87Y@X|I~4`$dM~9^o4AAHj?NR5Kf$- zAA7~ycMg%R>L21>te&lJqZ-@Wb7@_1d4({yrMp;jf#BeRBuWv~Mh*EB5`gwSRz9uM zHi!9JrcSHDHN`NWR9>VHeord{S3uG3%F4EobzOcvv9(YIeEPxG&f zih*Fz*>^`4Nsij?_kV-+znq+{Xn#r`TYqo{%Kz!u=RaCr{J zGKd2w(X3f$Hd$y|F0L(UG%cxEYBr(161 z4C+(x)>Vkt!+cX;{=)g`Hu$fi{LB*6rzX6+dW5%>`MY!`2U-nui|)qM&vbO}ZPh>T zUUhJ3pgg`#$x_*nB|)rd30&f+P!VTUjiQ4oc|!e$Ue7YO zQo1b}m6Fh|s*>=YIZR1~E-%!bDqE)IJXsP;LEX5*-vR16-H2n6BJ~Ah;hudxWFxCl z!Dt?tFe7@vqnx5*k)qkI%?hCIY*CrzR2}_TC`Q6THnPLha33~%3hau(@;BY0-TAVe z8ikyah58oc9#?syrn$%Z%`m!3o;ZRSLoY8riOIxp($vu~Bg>B!V=>Z||7M_RQpJ;{ zS8R8KFt|h)=8v+AgP|gF}^MuHsU=vA}Pa6QNDbR z=LJ+mnkr*y1cB!ExSrbiyuN)cYs%QuJfdj#AYHAV?ZZ=$oB@7$%}nW%{z9+evA-M% zduxs>MgL{L2O$nwb)XeZrNg5O$v?D#(-UA28d*9h?C?i z+*SGm=9<0~Rz`mdw}<*I}FZg+;>l~_howe7x8yW{P+5R?@gQl zUd9XXxAKW1(sw^k&{IQ!uJAxnY=#AH?bv8tG|O6qL-_E3dT81n3f`sr--AFYl4cod z>u`dvY;wr?+FI5>=4S!Yaw(ET`x>Bqk_t;}E*wkYDV9y-%b66c#Q6M|0*NJ^#95Mq z<49$9(NvbxXXeGJ3i$#!1|#^2pyTlE6N!0y2-pf2zDVMj(!pgPcw%K?HM{2%>G^ha z#8k$?(5%LRLMJYYB6BPZ5+Z2kYgiV=k#V!d*yZ>QyR3iFIRjT0@WEhSTMi zQW!Nf>NH4dA+S=5r1SLYwty_6kkMV?-;AtTin?P&Z_*S-4hX~TP8{Zg3dKUoTjUH) zp#>&s0ol+WHe}fBk8MT@lJbLN9MsbN>`%Pd&Ln1vq?JaQ!o^nqkiwO& zR3hH7EFZwNSMmd4yy2?-SuL#)j_BbjjvljAT%R(7n!l~uJ(3&3-@2Wiw50h-hSOLM zwM2XdH@Xw2h{Oe(nX#AG8Yz`pQM}Ee8Z=fZ&caE|25{@FWY@8n%GIL0PcPWHdrKSF z`kp2)!f74YVBFd}3~yU4n0~PrJrAy}S0O`CP=em5@hau9*saeF^02$DMI%)9Qg)=f|Q!?O2xZd1ta-XvLZ% ztjU5DT0~YR&KSx$nT)Lm-0_H(&!V2I=~WRoks8Ad2A>#T8aCz_(0hLXLoSQ>=0Ag; zHU%vgVxBOmM1VY8Dd#@Xp)pNsFttLl9f!+{fxspF14_!}}Si`Xy< zarh_d6upu3g&W%Qa8n5C`Q~(HhT1>_+-b4Wu7M0|zmGkE?sNhNw8a!0thWlYrAYetz}c>VPP-cgqR}3?|~G;6Xn9$cFnT4B=`!FYZ}$)okTPA5~Z2ninT7Oo_hjSe?DcEs+mZWdR^1X z-loj6`j9Px%RTV=HU~3n4JIbS7XSLLOwc=v&gCN5*PZka6h(nMFy%eCb1zYS;D}&3 zHv-&j03e$S)Kz~3_KHzyCP*JLi2_-rR4Q+m6MM3qP9QH-fY5aWHso2#cBNxj_cB7n zs~H$UcRH{Rs~|ZfN+z=Uslybs&#$%)BrcS9Er@=t-M$oSoCVnE{ELQ4YWIKx|{RmU|oceH_)&aZ1~=r4Q%>W z`I!OF(e?)B85i|Vr5PMA58=I5%}WjNJCj7vtvhB2k{z4Um}FQZlVEt4iXDPpJ&=SQ zXL4?c9z=4AKLbY!ORi6|ADt-= zMd)~4i0}=5_UjBo$1+WyKfH~@*A4r_&nnRq(^k=LJAbw7Z z1I~;|<09?i#iE_!#V|98;@5vhNhVSL@Fh`S`N>YxX6jjAVppk_+-j6Lw4f~0#i>XV zlnu;HZ3sPj6|`rEePF~cuXli|p`EFdU8m_`q=hY0-zy==6Gu=_;f=h%MGncZR2~o| zays!!#Ok65lY!H|*<{Hni<)(S=5x=SGU4t3~crxyzocNDk%O;lrXG5;dcelY;O}tJRh9+KY}n1 zY#A`qhm+~CljV#0R1>bods;zW7rOHL*1$S1VnR>G{a2UhXS(?7!B@A~2JOfuQgmzD z9f8j9h!?94bUX&M=Y2%`T?PAn28dk;i~!05IjzCSEwTptChRE7LtrgZs9JHs)Vm!STuSo7Y2;7#60S}R{#h|sFqDJG1#|ZU)n%IoXN*Bc| zX_sQp*7ngUA6gx~FIIU&*xiPJTzKcHp{}%lG2SA*peCaSRg3Nu+7FSRzozA!RKxj<> zGW^>%Bhd#wmn}K~2b;4Ax(o@6t@7W771`H(!VtG$u-yw262V3`Y+yES;Zq@*%&w`!suc}T@BX0rOKCyN&*qQK9jBk4{70WUVy! zI1zWu*d84>gVAgQSf&GW8;=+-Nt;!Q`%`@#^+Dky>D7B;Vp>g5p8h}mbLOGws9VmQ z&@a{CoX};01|4JUrfgV?gp{C}7Eu=o)dl+Ve7`lr?~?rh5}3^}7rA|Q+|}jDhK`CZ zL0D@lQz%pNdSIQYXuDWDil>&Pm@9+VpcZs63-a`KU`huQrPi}!?!zp`@N&T>Vo7r9 zm*y}1d_eEl@sio`c#9LvxCzDnJb4H0 z(_s*zZ{8=8g{dm`Ti{7aYi$x=J1ob=&3?O zC?AF)IDy>+GB}a=NO0my?;ota>ev&N|LjnbHj<4$<%as` zv?4bfIMqX5{+)f2?J@H0c7lFoJMI|M>)g z7-bllq$(3@H4!~6!xi8Xsf3q6`OTNpuY@Ophc|n$Z-h5bnrdE&y`Ny3y2;s#9h5y9 zS@mS^Vw>6vV6sSw?Q#w$|3RHve0^`oEN`|F3a1 zNpaogr=E#BqtM;}8*_|0@ejX!22>*|{?2cuGRWxA^hSkwC%a2Tn?>iq>Y6q)G#e5Z3R;5 z{n{`j{e%tADg)8WhlYy5f(bEg`)$R;U<%Qb1tZuUyJA9dv3{%JESagl1kY+GV?D24 z0PwdQg6%5XrXx&;|BO?wL;r22N50v#j=%E=qjJpce(}Hs?fLqXKjZW(9H-{kjUV^U zti^YQ?KcIU!(GrN?lJ}o&>l@;y)_es!2Zj9J|J@1NS4D4N|DCB9_-UqWKddz#nu%8 zpu23YBmlvTW2!(a$us8e`}=WVf?nHY{Y=O1LM z*_zV+x9{se(@NR6@i2cswY-os) z(o0y&x!%8;$a0+KzV@8zVvX^3+79?7msjcBZ+l|~%Hz-LxLZKuwiFe!JH0s*(POu} zPvhmkouhP{Vq>>2kCNjM5ZtqOw7kJJx!Gec$jei(*Ff>5GPvEg%fxlocl`3~&T6y| z_3@C8HMA!g`SMKhF=d2k2nsMkn}UubQ3|%Fcj9<%Q;v(HVWJ-+ z7U>WosUxe`GKiEk>d-GjA19RAzh*^s<>cF!?sK2yhpDK~QpW43A4s&o!>6k8UGRU=K`$XBmC7H4f8 z5MkA#m~!q15>OU~>ap58;$qbzWO2NQu$giVqn6qM`IxK-S4lB*?2j_y%-3cr6EKbN z(nP_*s8w}xm_J110o@uC*iI(+_;Ba+3;V^|>(sYY75;-tJA~C@!G?!oJta{0`pg!D z>?}&RJHdksfH}I z_s(r@VeGRgs#OPJ!2jkhg8cC)UAu+}d8G4K=rI8iaPg*FP{)08omkh;IS8bT>yj)TWB*_p3N<1c_y|Mx+xDW?w?ql1b96eSIMW z3i%aXagPW&Ii=iL-7VIdC|1M*fVXNf?cw%pzJF*Zl`tSJwD--{^FWv2gTfFeotM3O zuCFe(u41;==JVr*{S_wGfT3nUA=h7{8RR^7%+5^`uv7gkw!3$ld{bHZPnBy}+csLY zEu$v}xkaji^=lZXWL9ATA=@NoJ88;E-a*V3D_0?1Oh|Xvn$s|+XCZKnXB*QKy>pd) z5Q1(0;$9C;A_(NL_{>%eOP8k8O$1#!zU@SOS=Om+Dad1R8>q~O_}S{=I@SU+I=tIT zjD;Dh1h580v`|ue14OVe-6T7rfhPmPMriICk*7hkuugm7W9o(^>NohzD$rOqTMP{H zJgud9zgSN2%y!O^6f=bQ50c!L-IO|gtj`)1d4g0K?mkeB^ZCz9X(OKXuE%aW;Jo4! zF029rSL*F*(av2ly25!;UH6TVQYO!lwQ&rX1_2>eXgs90WL9Jfp;XR43}&^ua5H+l z1te7gym4G}0v@fyvYo~-B#v77uGd%59&XNv7XVFF+mh|O+RQF4@l53iaaKbmdqQ^p zgc`=3sE0uC>*^T6ZbAZ3eGPerZx#go!b{E$u|cxPH(au9NQ{yTV90 z1W*_AZAM&`5)ABsi{vaF6`)EQAd5AFy?@wLl_Uk9gRK4x%L=-2hRWc2dCEow5A}Wy!2nx zZQAB2_K=6lc?Y@dz7~@Rma`f1lut7vQIdSD)Q7fWZO^iabB0sAuv5`=wg1Jscq^uAVfJk!*@ zsSrfkcKQAs=oRt}Plu&In} zQ4#6x6Y+Z?BtZqZ|Oi18Y^ z)xuiDJ_|!MX#?5{TErrN+wbWG2d8TMZ^|#Dj!{E!BPX^nT+>ul#xZ{H&g(w;KFL{K zs4?%t6H9A`Vj^8haiYu*B`n;gPG|GnF7|W^!ilC$X-tMo_4CI%siDRe4Yop0q-n!x ztVr>}2n3B_-XQHU*3yswA=o;imMSv@Z7fKIA^6+dV)+{-7Hh&H#mG<@!#8aQh73dN z;>D`4mdNlpwc4YI46D+1Bk(ykdw_Ug6^w|ukrbQCUY8bJOID}FK4wYHeQw7x&1o{6;jJnzw)b1{a~OCJwoagGlk zJ?io>mzW<2Ix2%W=_e@Lc|udEO_7m~nrSuK zDBFtzS$m%8E41Cyre%4!0x64^=T!HvpWIq!LZQEdUw>1RlX4pnwZVCpiW!CCS?v97 zl|*vwA5U);{SXU40#Ec@&14kuL}Ek?ybXxi^7?YonR2Lf6bq)B5zNyot>;5}4(E}^ zM*d7^6M~Vq&mjRF_ym6?<euK3ggZF4|k>+MG{+ftjSqMtE*$ z%`Unl^T}bTm9}Mt%9ttO$EnGy2|H_I0@DVg2rFLgtt4N(qs(E4K@hb2+=P zm)2q)sr=5!fEy_QH6(quD3+W4_4DF*_t2^5%PzOg_SoQF#WLkkW{`=hDE+a5u)vsC z{MMBR%gD_)3LN(LNV0;|=s(&ulsAM*;`7MU8N!X6%Mr2kJ*ranoILr7)OO%J)}bOX zd;U?CL$&9-iCRh&#}|9H3bLA`qAQ-6vQTQDPs_CW{fi# zXEJ2>y@`*jYxu7{!rI2C>y)<5a6sIgCBqSQPF4;^+Fv$ zr781&c&hh8*6WrRG7-zBW8z-(U7Zd*&pNa2_8O)ZD&nK3p zJ8&=ydE^yKshXILNfm|Oxl8_yY>ix{DIh9{gbQbKL)w~4R1XK!myl|6_Sw3dI)|1kEH?Y6Q%j2>v6@KR*bx~)EzK^NCcBGhOIlQteojbL2K)4~4#Y?ZV%;bqI&uPrq@|$O3QpXR zxr1D(D?hxX`A-xN^8+@!U__74?ra(!^&5nw4J3GKt>|!kv0F-cY9SG1k0-ECsop7Y zUa_@NgoLFq80Hp%LR!64YIXNzy&ErK}!Md(yL7DpvaEXZ7#R8Zw5X>_bL8hxiCV<8I*N%(_UPF*(S1$rA<2*A7~ zJs&2#=mI^8mE|>BGLEHzso%RWScj#pgVfFeUClohf)(sd0? zyd;HGacutd;UQJW+7z>G^S`N)rD#8(r)i`?FXl>-xL}b^5WTP~14A#aw z%cUyK1{+DmpZ~ewBF*yUg&JG4jXAmHh1$k8592$ZD><{!`VXP-1Vh!NpH5PO`dwo| zg%h4MXi_4(2A9+Jqo0R4N>gRI5j~wT6_)}I?@T2F!}LxHcw_Lr;R8FWnYn~(LhABK z>y<8q=ze3PzL`YE7lULpcyLY0_362CBr|1cIiZ57+PsoKRdc0r9?g8e3y-Bj07y8` zFTa*h3tdniU65d^AV~-&xK!*(q4-iwwZVzqc_C9%v8y|+mV>prVqcflcg3Jv3HBS{ zXCFJ|!r*=4eHYlJr=M@1BH6}PxT!2GNf?09WjdAkf{#da?ERl3>Q?@=fYcX`tk z=R=a?R1PC(Rk5l~PXF~lP^ZLljen!AfTWr5I%O`bExIwDeli}JFpb*v#ZO|!o1*Vz zhKHY6Hz93{YYB}sz>uUatdJtn2l#s$D!y+nxK*C(v_sc}EMg^x;SH@YMTTBS%I zg+}eXkzhCv&}oavPKhNP^xj+eduA9g+F+%&#x&5%|59$%;E~2#InVjg?j)7a9r?Gr zLFTIqU32R!pTAXso3R0+f69(E37?31op-iwT+_6;*(M&e@nq4u|6wdbp$*aqCbuj+ zAlZjroBi9 z{(W1#16@glKM77Hgkd}>Tm@Ep0A_jc1?8k#L45thTAskqby31{bZ6U<+{)v{DI|_PV?Hw^xQSPA zpd6D3N?g?p0Lp8lRX7deh+){RsqmOeR`lc6UyM$S-F z2T=bb-gdEM&8n6tF6WjwH&J1B@7m#s`uB024C_M0%M&?FW`Aq!RLIX8tY30^f3CCU z<|($Mink$d0G+_#Jtd~UeeuN(fXpqvg}5F4;Qs#3%ntbLDjv(LcwVgg*YTg<_%4da z95nt5RHsDG=?!M54dA4Z{8cmT7#{r2oOpc}OvYx;=I@5p;)Tm2KZo-sxQua+VGmK_ zF$T;B*vPQ}g4G{5tLTgh;4etuf^AkJDi5I4~l3nV<9*W1i+*K(Io|kw*;?p8*Z(?% z%E5~NdEUT{%Hn(=ZLz+}^Ox$Ne1Uuyqk-Z5n>WJy7ee9+m%34PgAKw;&k$D_JYeV( zQG!Q;6F53Bfqc}E>!+5PKC;fFxg~XXqNCdU>41|a^vODVybDchzEd=G3rQAv z*eqM50Z5Bqpe@{p3 zAUFuUXMn2}h8dl2KQXxfFz!%}{x=_WIsi;xrK=?Gpm%W0OElVWjwe+ue!sNT3H6x$ z-EX5&?cWeLEaDSoyhTW%2e$a19gY8ovUlLNwB3>gbKACU+qP}nwr$(CZQHi(-1g3m zo9CXY?*3|w8a>W$So3*jtcaKqiKi~ctZPUvYYD}6iOIC2zmo)p7Sbi;F_8^XV+kag zB0j7OH3)*TqKz&hOf&+@7Sv#AMACzfemQhs{1%m~otSfnCo|7i8sSDElb8BTl96fr zZAl_~%pUSy%m_o4G0iK9^vDcFPAqOmK}bbaLDnmYfGG(pfH4R&{6Ii1#tzto6(vPe z>m}EXJ=S=D<8^VpHN0PQ4AhpevL{>c0LxgfDY=kgx?zObz=1Tiq#%2gGJ$Aec!hRL9}wm+1NAI{9ZS!ZO5qH)sU5TEJ@!%;W`)32xr!c=eT>27}F9`WgJ;ozU# z^mibJ#|Qd!jbub7vjan-)$lZPRkn%|pv#@|Oc{-&qITK+OSI!Ci%n)FWz^-a(L()v}Eg z%UsFv$Hil4D%;>GP<@lB9eK1FFjWOYe1UT|FBTRKS>2k;CagE1%=dx?aWvS$%pnp} zvu6TzD;Wo5ZqQw)LiaTo25vbY;Km1R4~tShAu1u3XCbGOsEc^H3y)fENwBp&@F|#P ztU`0P(ITESJ@DEPJ=XiEDes6Z`#2;oz7L69B2<=Hu$M=-@tWPD<}*Bd(h}X2JzgebNl@5DV3iTL}b}-)I zD*IICY97WC%pj@pdhBLx6p10lazC0}86V2jAy4hdsqr_|K}*cO=v5oOgA)XcWKq8< z{~|l7W5`?MAjX))yi}6>Sn2uT!yjlAvv1Zcw%I_78OGQrBrP`|mK}aFb`#&nmJjyM9q74$pv6 z1GvqD4y@U(9L3*V29MYK56F7}vSZ&}2S~Wj!)x52zGB($Ttqr?x zXRxiA9x~;4$5JdO45O6Lj>DR6Hi6p!92>C!niq9w=ow)`eF~a)fak=L1kI0H^47z% zwxQ;UXI>)$VLSQnMOOj}ubVZQ_7F$`q)mz7fd`U~&!3Zr}#GASbO7idJBMseS3eO7L&KNj6EJWoyO;q*BX zE=|T8gY~I^Z}d2Kc%J9EM1maG`GkQ+aDWZj0X-z{So$;N4d#llOrWyckx1#pu63nN z2$`20o_B5Uo4}Z^>^iO8#CP`@zP3aDo4}ba`Vh{Dq)0 zh$iVQeWPzdJnsg$T3hmoZ^VU16AK)@4fbDa_L6TZA3rRN@m5$;fA+89`p*@ah%SQc z(&bWMzmt7*9Qd^DjPY_6XT@yKLl~?n*jkPEhDGmZhPs0Zm*ZvI;+bK*%JD6LzLOWP zijF7Z6qWHM; zrlBdG0I8LOS~GP*D?#InTZa#@#?PQDk-$~zQFKP>gpdV0>7ZVGh^G8`2(g zXY4$`393Bk=aEdYSQjr|H&BO3h*E}tq3zcvd-2f*oxqy98R<;Dtu2XI<6NICUYg+% zn=UUw&{kiUp`pvbH^FRYXObCw)_t}G{@Daaka@#{q7teEXtJk?xCDA4d7RtEP`c$Y zi~}A_HIqFsBzFpJ>5xm=fES?ayGfS!vCq{*<~y<}&D4@Hy4f97>;}ONHK)<3Ez*E- zM^`+hCXrT`K>H&p^1i3f&!3%U>|NTG~(a+;Arf^oX1dX%Hh!LLguTLIdDhC?Z}aEB&r^ac&`}+j=X;r*+%U_ z|GwP)j0NaeiilFlE96Nb4SLgI%_1>W?YWc0ec^87HT6P!-f);$2&s5LLnwXSIwIYC!DCS< zY1n>JrK<7=(}(`}e)d*m;V?5uQLH&3D;K`z>W)|UVfiNe*xeJHEA7)p?20IvjE&hS z2x^jpq@}}P6h@2Fgxwp)ie!!!XJ}=0m`>Q*+H|H?+|@xB9)P_GF%?FW?l{cqvit($ zu)5*{54+^qczqW5Afivj9k&G?pV`2bN5}g(_$Iphb-T$+jqFU zQ3zQYMRH6&RSsAaDi?lIFSXo%CQP$UsX7Rt045i{a@SoR1}6+u8X=lZIXfV@5M5%( zLyJtKSn?>&V|75m%|CnMFLIZuQK-`2<({VPA?{%+TA0wefof!H8r{96?NKWD!LWKq3lC2FipP^qp9usdWYV?aE|ZgS?1gI5621n8L}$*Z<7}9xh99(-2h? z>Ej@+if2Rad7#b~cqZw~%CkM7YO=j?4xe2(7dzB}5U`_gtg;nbxF6w4T<}z@xE9X| z<160fm_eJR@cGqoMXE;r%(f0!=^3bG5X|CK8C*UrWG|6{kYa5WLKwlJ}E7W$VRhnVTV zOql-@5c9=O(H_i5~L7q_4gh}uA zp?Z`IYKS+8I396nf)5~zP#nC9^Aw=U}L0fQ0K!av&2FARJq!gn39DHA2Kx)FpdO{k%?ob9)UD6!9=w7B~O3=k>%nU~!U z!%`4aQB?GFt>(GQyTmk6K99Oy6%4nUcRJDZt_=|Y5Pj3E=$)zB`R%fSL(E9tqx9c? zz>7_V#upURSNPIUsX??= zjHw^$(rA0ab$*yAB7_$NGRUj4)w0XB>=kpXUSaSF@f#}BOx&tGNv51{L}c6#-V17& zleO}4s$Fmo^owH6d`8x|Iq32o=$~G#9SxLkEXZHKo-zLS0O0@jYW;I~{yjP~>K@wI zD>y%}Yw;Tk#QwwPi}{Ay`1Qf@qo9~1P?*zzGWcMv3^=Y~I})K7oLpJ(Tb6WQ=2te% zFT6@xF~@}XH4&RjyRSa$Tf1RA%_~oTOw(onp#tO{!anbJw_bf;Z~yq-vT?r;62$?Q zCq10$gY1%oYK_=uq1&Pj;XvLD>oDG^;veCHK)y`2tBv|Vdasx5`*wT5?gqU^uKXFk z^JVbnulJQ74MRqTs)RctGJGqC!yEF&6!`C|CE994$jp-dAXF(jJ8XnYv4l9y8%4IARKUs}98U zmK{Aq@dnUKzAp|;c}t9@qf!s)K|}>wqLFe7 zEJcmGa=_3_kS9J*9K*fx^!eMVn2v<>$53oAQGWdfO4x?W@)Cgdk2MuMsu)0Ae!y~$ z=)zVy5C5vJ)7uMR!t>io^N{Mbr#02!0* z%jzw7@NaWvH+SML(L7r+bmj^bL5F&Z_F5A!HL0c1(^#78Eg_tN=Rk+BRw=0TOcWUB zr{D8Kv=~s-KK(W^;^mULq}oHp0jlf=?8{j5+Ff(tV42+&D~`{_y@G6~=bFOwwBztw zC)c6B>rE5QHC~cNqpukv26^<9-MSVHoxV_>i0RD5(n~rBh8DMwe0VySNC=Nb0t#8J zVhMc9SRc335zsu%a9t1rMTZ!-S&U!XRIN)H#crG2EKJA4yEp;J3k)UWt+C}KEu;oZ zYc(Ril{`6(AlM9VTN8U@=7b#MtV@o*HkSBKmku&6eH8>+n5{P*dM22YyzXHjgxqF$ zbTu&!MGh?Je6KW`Lrfl6zx~NO z2IqE*4QVkcO9`)vUfSnz$8(DZ)g^<2OPG{Ia_v7r+7F?NCL=MjtkL#pF!av4hR_zh z-1`UwB*?gPsnv00cqx22Ot9WDZ zEi`2Etvob)>xg7&H*|GW8+Vr+`5R$|(l=OV@+mrEch7|igCckI%J`js825%m6&}fV zL`V5e$=&loO`tilCu|>7dzaB6ZfbBstZce|avR+ba>AqslL@_o z!YW2H87tx54dthwiSj$bhH_K*^C!fwsvWmZX74L%*)}E~aaK3(?S2=TfR_%^rAoy9xb!NO=H%b|&`F4$?fLEj51=jMO&r z&?J6O`-p>e88vlfhJnabBeL+p5)&e&87`Q{q(X@{MVyG)sEUyJ26LA=UE19PQ)Q3y z&yyV+SKEXRFjrVcx7tyC7VT*Df(F;IvC5RH!265r={h1Z@m>t-vv6>!&^PGdw!vW$qCRDDrThmv=JjFhCt<5Pg|&_b3c#tDru z?U&A3-zl>-Q>$~3-{{w8w8`mfcu8=SELwf2!?T~!AfldaG^ba@Q{3aI315rMXon^- z;|oi}Y^xjO#%rkg z(p$+3_S0xt?O2O@SKkOvJn*##rRfJ>2SExijylDi{dcFo z;~qf@L7q%q11vt^y|Fi3?aP9=s@SEn=Qx5t36Qh>xq}D6cox;%FNY1FSnZXQg4e@k z!+(75d!!*efZxZDwzWN=ohZc$oSk?hwkej#-x|lbNfXEWSzB zF(d&TFbtkr_pmJ>i%D)y$?j-pJrR5Oox%;I8)8GjmrEED?_zR-fE=R_P$#hhKAT1# z7=pN&8nX`w@kf4eBm_}bUxAB8xBKUK=??>&GdPEcl(=qcRFH15rNxC5`E4?tL+QK- z2x9ptZSF0>i0!hGKSa;&V@Ym8HU6B-x~!Ju-JbUW<;4`2gwun`u*n@1FM(0o^s*!)Xl{@<;#4U&5jem7F`+-EAfcEyI5E)_ zTm|4x5&!%N?d^{9%PkV{we!fz%eKOYRiPn^}WKf zGa5}DZ)oR{F0NFAKPick%Ed~`*R}`v+0iefC;CYSadg~bo-p@!*hfGdgAxqMF;j*b zADo2*saVg!dr&x<0_X0(?#4v=?6__kM%%wX1jnLT9^JJg%*gUA)iBoGC^9jc$0>kE zSUlS~z_l_8VzS*}mQr#6vil=QI4Bu%^(OT6=@HE!CF5Rjo2+V5lFZE3-3zs4afAy) z6z0cs>=Q^=Aa7(J$%r~b^vBRR#>nK&wvqdV!EuZs%GB{mSb`tD%|HjP5ROA@U+&_V43LWW-^WUb4YQ+Ej zMC`wSrii1Xo#X!-q#L7Z;f1A!!iz-L*_z^RYHg-h>WZqNksaJ=Y!yx@XV|hyI1zh7 zJ*kd4l_bejX?i?tVnLxbB^%I~lGBKqNiKt=sX3@=e&Dh028kD7_X&VI+krXNNEG>X z&}+8qHQRB^^SVvd_nCL53+(@O_Xukq0_Qvkb|8dH?zP__%MOB9dO*bdsDO!Ec3{Np zoh=YJK*qW7N$lenDG)h;WbB(cfF3j#|E>Xg1kEdUdWMm0dI!STH?AKzkT&j}D3Cdj zHt`7=Bmt^N@@NG;OaI21v8QjJI-q3in>4@yx<~kE0eXg^P4Gw=1RX?~v4*a#e+Qkh zhG7n!OY#UBL;=b+@yQxQ9dtJCohuMK;5P0Z5HPzh!q_*ZpE*Fo*f)khxBtiZ+kXH$ z2ru!S2oy93FX=rI%$KlVk|iOWgpp{Bslc3^r2nN}WfsnYgodjKZpN5Jj7t5zpjw4m z^E6(QR_kP0TY`_8-;PE;2fXr9oD?xV91IbCPp2r!5)x^2EdHW`q2Syd9~*O`7;tQ; z^4DRYo;CFd&RU1!=;47peY_%5GD9-tNFm_U%4uPPxoFI=(#1Qo3-8Etn0tFW!Ec36n~C$3TwpmjNizOZ9Lsiy5jPE1Ty22y~0y6c5# zu^|Juzd0vUVk{_Ofj((&B9R00{HYE?-Aa>CSt*1jht4koV*fNhG3Tw-2l0u6 zsLg6t<@xDP`9%sWQ(P~pRY{VaX^@u4nl+vf34l&$u7aX8Ru%ryCYN4ku373fn=P{z zL6J5P^d}fvuJRVrJ}&EY&t}E!#VHCkfH~AG)cyF|ml$YTwm$6EMok0lDx2w<9ZO4* z;1{A$3wAMyyKAaoK^aw?F?F-iejZoinC=RW6vk7H>pytC-zKLy#AhrsjdvM2Q!+z6 zYpgRvW!)Y-3TU3CBW&S1OFVL`TLo)u>(wsvIDdk1CQ!E<2?8YGnBOW*OWch;#GH_s zTuEyrIjZvL&*fED&R*mNvvC%rOmL9SIq0_K`nXK!K9O}%^{*UN?*>WSN1*J2^E?KC zr=UnMX$O6D$@vxh0d6Yhb)jqYx&5=09n-BOn_NalHPKTg4(uKT0q+R0IaL;>oFp~c zs6fOP=*(Pr%FdDtT{p;8BO@oW)EY`t5>;L4%S<*SRms)U#C;!#-ea}}j_bn(VHV$<63YOFHi%=Gst zUU$cr#uF6@%tSbdkhNpi_Uc;jv_(0uVJBnEsA5ZHETVLmgHWy#>$OhYug9v4!j8Z* zsm)s~x-v6GiTQ*IZX>NYayaU)Bbw4(#)>YSd#0MfWwc8Wg*`x}5Z>%HTJ?$=(O6s8 z6WU&4hd_^^t&mq`G*cw==*dF6m_CKxX^PA$-NlkzajfUhtWCyy{dE5s^uCrx?Hq(c z8+TIsd`UW5p-rjL2*O0h5~NTmg$(660*P6p2vk~TTGB3LtT_M~X04hRQ9H-QST|8R z?&R8M_iQ+phiP)(kBvo5Rbo0z-p1xF5^XKm>xnPPmywC3wLMM=EID!R9_mA z^$bctEf$f2IH%jl5P92d2C_06%mIAYQ`hY$0*|q3_2EfJ@msSci?w{6<1T_ z2KkHONVQ{25Dv37N;l4WROq=`GaC8KU){<7XBi)7JtnlWrIm!dgOq&_p z0EA}}`(EcXe2(SsRc?DOuHWZPFl_w*008CNN}RY~jtYEk2ufT238XIseUuSbo>tg* z+J55|U-WzU^<{wSkBoU2NR8SXVGMv1yQh`yoj(<*AAo%^GHEkER?23d(-DmMYnv`u zT;B;N{ZHxD{DEHBUXI!o$C{CNLlz{Va6VjNJ%FJ%4oH&$tveGPpoVfNrdWu1 zNZXw@oFfce2PUp4+~n2|Cg!J)%{VRj1~e|V92krfX{c<8HISaEuR#*D3+h8{q zh#T!qfwU~pwEYCq0qcuNI2IVzEKsjhP`*Xt^ns$g8_m1OH21as$m7TP%+xN;;#9x! zU99>C%<4mFx2oj_IgR7!VT=wL%lEYHp$y&GrxvMPz40oJrcfp;p2V}X2hQ2*vuRLL z4_^|y<7v`aYgC5cYY(Wh6xp3Z!|i+HU)h-Z%c=-KepGfx%dWE5@La@mx5jv7pd2S? zTqJY1Cpp=hBNaP_X~*wD-C;^X*_%|JBy)HtKG~ad6>AYxBy)MkTpTRZ6U zR~#<>N?;?l6OU-`@MW-0m*M1+xqXvvPM73myZDlcXDaXLWq{e7w0|XiJI8n>qHM(x z+=)lGci=KvM@!}3DV?^H1hP9r6<#B}7X4&l94H6M%GxwA(31?ZBO@$y_#28LDih9D zD8*cp@-;{e4HY*CZ~(zh&K)D?9OYHA2XuS##_P_<^5O^ZPwZAVPp5YD zmtijW&lzThe=^Kl#ST`{o&+?>e@OTdC2`>Gk|Qbo>ygies(S5Mqk2+$qE@fG(Yli( z|1ZNFy@M~A8D==ab+y4^W;z?|_xC8OZY*R0_eM`Lci9h~i|bf5c^lDmgP6kPPY~Xm+M4&k zop!u|iH;Ta$%GaGjWmEGCxZ={WFi`NyQ2Q0MACAVc2;9X2cf50FbQlSFl==_yB{9s zk?x2Smf*^mnmU;I^Y$7lrpWF{cT=VAq^2Hdn+w4%UVU6t`EZTl%?14YDq|4hT{h{P zsZ?T63Zu^mYxs>27p?Ro*xp1iwoi&-5MtAPn!#@K0ujr&j%ZTIZS6I$k1OqneSQ7B?SGZMO;Asu1g#^_gat<9be98t2hQX~#V#ROSZ zN@l3eQN|F&MqB56i!|iPmFZG-zQFBCo`=~}Of{1v*A-6RjoTLA&~|spVCT%R z@fe|Gv>_O{?gtN(xBBc<5yK8S5nMQNbGX;Jk2zPk7YluRzF_+*?2x%5C&Z)nA_MHC ziSqU;11#FNV=%|VM_Pf$AQ{OFHS?C>C0UHN6zKgX0+4~z;Er0O)q%FgLOtvQ2-VXRMY{?#X&_27Qee_t`oXqO6+&TA61??DK`Z?74gC)J9_i zwE!dWaGiSViH!LCVh;g&1_(q$cGENoGzxXrp6zOCJJ(!lr8e++vW81QN2)t4n^CSs z#J*2-<%kU(S@ibd-(3Eo){d;IM$w?8=@lK>P2Bc{~s7xI?9f*Qst= z#N~Jfx*}m#RqZm?o*c@iGx5C5jBm(SSLm{(4BaZ^vpiOf6hue%omIvrsjBYZF(xyiX)Hi0-!8wZ%QY!>%1j_L5r@f zFciPcff?vnYAzYwp1bp^um+MMA*BVVQL5Uh1oq^7IH#qt_*LRFA+gekvCpR)gM+?P zPtHewyLKt~jNE+XXia72Rw%8f)}R5~AeCTJ|BkhuvqeV;wnb(UG7Q5>pP195o1L3I zv1ptxUhulCwxPC6PS8azN2&;s5E86yvuZLTT(Viu*#Igs&}<3SCDyb@R}mP@Z|CX_ zUN4Ckd+p^7aLr=YkbI?-njlvCd!n)E5R{JA6>=#?F{^L-Y1E5K=bw zy(vMHb|+WPPiY!LwazK4R;ywv`<#&jQqDPTyg#JbfV)F=(DHZ)Kr~18h;q*+`WkA> zfeGTxuoKJ_;X$&^-%?!Bc99nuwNUG_7wD_)!hhu=$06a1?$4?ZN^^;RC>| zSw>v*1%Qx7m<@iZS5)r>YnW)=zuJT?eqPoZ{~aYl^bT>sC>-f40s3$2!Kbt+F2Xq- zdyF1~AlVQAwXP`U9mhfFvD07zIOFh`#Y(&gVi7x5C$m60f@TU0dDP~;%#&Y-4a*JD z=cyJuqIk#wgzUa#mpw^#kgW24q>A~g1zf^;@aImLGEl(01w`GxcloDAI+@UvZ$R(5 z=L;CLB~7`;hhJ>(x#x=)Ig=K;`gUpq|}gLb#DT6ouni z{%Mz^_~Q;dREK?g2+DnklXhX4>`p}bvn0ZQVHoVLAlhxRGD{E~hWSF;>$c;}cRD9SudnY1m_GEK z`+@vwfYrcoyuUu;vE9rNo`}?OpY%c$W}A)PbRQt7g{ZrbC_7U-@fJFo$+^eg&3MGA zB$85wV*oWJQD6kgfLtiF!-TKgv7zR<`!rnK-8Y5Hx95f`x1J97G|SLD&eB#L+XP)! zpyaw#=R~v#x-xn6F8Mxn4c!sNPl(Z(wo9_ta(T_?o#L75arg>=3K-GFrJMAm|Wd9gL4S8tS3x3eIeU>J?qwu|Zu{!;mQ8JKx#9N9L()Lq-B_!mSU z8A2o}&g%jldM|f03@^M5I^xOvv{6?;Ks<*GQL68oVHE8U-pTS5h!Ts_3)+RwUA*z0 zhCafkFxJd?r|k;vlHEWuy0VBdmQAnrv=U30%36V^q%m!V zpEk%x@31$cf6I8tLEt&1)DG}z`S;?$;#OY>e!%{@*@sNI&QC#p{W^g9^-JbI+wA|> zbYNg;ZSt=bFJt0tpyX`l_+N&E|9Tp#J4#4u$Ui&8#zN}HSQVNgB2Wd0mCc$8TG0S# zK;;6TmQhot5~yrl4sJl^ebr1~a36|)z}^m{m{hE}mU>_5E}jbT=yEK~E}yccFaZ^( zg7|NGI9_x2I9}a&Y592nymi z2WREO1elPVMYJV@F~vYOHNvr!wR&cq*lemT)Mn;9cX*3pKuoSKJVfh(h03rVjB&ov z|2&;+gMbBCdCFdz*66awcG28Aiz5W*yfu?VH&S?2?Md`ZY&sVeAF&9oC$PW2YD=CE zEL@{ST6TD0GDHo6%23u{fV0!CQhE*^?3DpN!epcLdf}C22YPWvC{{-9sxXyDnNdtD z-rThsmA|j?DB##>8CuO|3bfPCQIJOLlkAf$0UDO;NJQ^)8{Ktm{+O2CC>^@qM;H*h z!rkOTmZuF=j9jAhm|C>8iWBH`5l35!6?*Z9CkzY>gt3P%#+sC*nY-OoKOAPb$##Qj zS|Mtwzz1mmmsiapd29vyj1ZnAppRB9hH86>A=^v&`KT_+pL#Wb2p?d|mm)C{j3~;~ zdUAXHMVG~?R6O`ijLtJ`kXu9qj!RE7nH^Zjia?8P<%a03JTq4O z8(YX?md!LNPy4a#1{6YtSJWSaN2jbvkC=h{h(gU4gF8Lz4GD)=x;e&MWAFn%D!^?33EQ!h`9sf% zZ$*$02#tvZaV~|MK?yWP*24}U1se}VaXJvZ)Iq`<<(JpU*Hn^;*cDW0=Pi2elnPre zATsIr_?eM>(LKmfkRJ*|&`sNA6~oNK2wRD0Ip^^?7sxw_y~zZa=g){`C3;FD3*3lB z&9w)*yeASaOX{ZLf9XV^H1nnaSL%6-yx#%a5JNJ+w3xmU6225j{1(ePA8>WZKULLX z86AYTIUc*HEPz6>P)F@;h#hw4*mRnf)ko;j=M;Il7Sczt}u0!9J8Ro|(;fzB_8C#&1UEB#> zpq15+V^=9@$qDC)U{dVs8cjhTgfW6#gc&Aw`6JmuGtco-7Gozi``g>!J@o7w4|gI4 zd?0|gGTVv*mnf2Da?Qa}OsI-oZAf72!#!7t9<>H_b>kGBIhrpmI)#3fn8;TnE&2>S zDVZsYGKz+L={p#kjDcJ#G}rKhjw|BB?|*cVGkDvI!T)6zcESH+K*jj)tudpbp@bv= z{|$QW-`!ZN>RddHMxnpv6iSCziO7LnC;YEmVQ=$ zq=+1Ln#o+FgB(56F_z3UO%iG_?+A+4LDO;Z^oIKt=UKecECU$#?Z6B@%yD zAGptF2$+Mkr>l@Stz(KFc`l(?CBAV#@>bH-ypocM3h1mr$Q0)yq+aG04OWOV5g;^3 zB#@a1Fc+>^Z)9{HV^AqU_S%^nhh&s5gYZ{P-`N$JJW;$H^mwRYh6@p@m**Quu3!i96yc;AgT&g!^omv8<0h~yMji(DYv61V;-0JkzF70zJA%$MKfoq5|CaJnFLm3Q~vkuN&!ms%it zBEn_n=8{~wlZYPghIMa*-xz-mCvQz;BWvwyXdw7$M^`Z-Cf>SkS+cY3R-xhr7mx7L znPf9XV^U8L3W78@4Og#v_LE5kmAR2aPmJ3aX;Ns+v*XFf_TUH_V*n}Oe5%aoT&SMy6)bNbPi7cs2YGfs*YB=$YUt5M(fYl3My^Ag1#h{H#v9gUmzrplfR%Mjo7I4bnB?&WQ zkvKhFo-WuSN{^P!BAfZ5Ic4ctD*znJUWgS5+L=Ke!pmJ9cl~Gb4qfNJTo-(?8|?lG z*xjQ^zL2%l#{_`$82|cVR8F2$McG?@ceYkfLYcSySO!ae*F{+)10x( zA$e=K=4O}Ab;0xdtOt5;>}GfFhL>IvPkJFt@#z5BCm4d4QxXrWWC>7amg2Fo!t7B} zfHkE%gGM}*VBY;8-DPmHU9~EK;WKv05*yy!Xw}Nar|M)*%XUt7A*0|xlbKt@>|h{1 zPT81+DlB(Jv4nbl*k8UX8R#Olqgt6&YO>lQnMEfv#fZ%fqM2d=g_(lvisGZ4pDQ%D zoj>2a(qjAsd1vMjvoZcjgPnVf%Q%T4EN_2C$iVnpB7jzsS=#vYI8afUevAiDr?C)n zty`I*|5(X8KVffxpO0o<*|4CT5_&QPk+DcpghS*N()qpe@|*jW<~336SdjMNA>%~% z;%L~l#zbm>HxZ1ldO*e&PS|SS9CMfY;gzW#vt#5>(_HA;>;qPJ3*W7wf-bOV0z=eQX0_f(ok}DHR+k zK5F^ZKsiUmmqjlA{+~1)$7MB1NfDLBKUi&9N2jn2DlZN@{m|7$e&CLz5pl)NNIpuV zU-ad3D77j%xx^!;Ni%oMq#V1%gKT!m-w(u$P3DVaOX+|1zZ?V9;g*j@TrSU0|4Cv$ z)jzL)DRa<4F&WbDI`6K=8en`$T!~}$kfD=AcMRiSt3PCbT8!4F2eT=rzVsbXjw}&O z9La-lzNiEuEaMY%8ndcTY(oA5q%o#OZ{)O_mEedeAt1lrubJPo@{K?QC#8~1=Tn=Q z96${`KECUZRHmQC5J_=f-lzl4gk=gUuJ`S4D%xYFA$M^OElfSpXx;kJ%=bAn!bnTt z@A^135z(QKDO4t>XEGL{vNWjfLomM$+Ir5>MObacw1whON^8o@Wtwv8w4SzD&sn&m z_^9Dm<4ZRdYy)1 zPJF&R&y@iwGpXt^UAalFskCYm3Q}v@_NY3dj}0dJPFxakgw!odX!zTZIx$PVLcQnM zBU*aOlBG+5>R75cbUAI=T;X#(J-!E^Rg-z|~+GR9Uq$9Ufb-n_q zGbU%U!c&)Hp;=#H2K~o<&Nk#GS+(oztVed~LG0qy;sfZ5K}Lz0V*dLPv+ntGSRj+O zbVA|7Df3&@Pi~3NoKXhx&pX+W(DX&fkQU=A#nWchM&XsRNGb}QJtb$22wX^8rSAlX zK@>|}%UfTx_pNIeiEEB%rt@Hi?dyAOKYzXEbGptW2qJ?fv{;mZ;vji!SK(0vI{<8{ za{jaj^uAMg;jw6QPN@1dN9}F%aN-iZcbyT7`bxt%P|cNwOIJ{jsrcWr-`Fs*wgYEJ zbsHBztrN(Vz)8>hXMELW@6F~;bWLm^sVKN)Nf#Ya(ZoDu!qQ7s$*_}#%-Q54O^}_s$E;ER^jUwO8O_ z%U(I;Nm_;~Eo<EjgOVX?iDqt24BcdibfJ{^{Ye& zlQ!7w1>nvD`x(H214`li7T?FT*U9J1 zTeavd!}$Ws18fgGeCETBPSsH@oW|ys4A^OKUyVw0m1F(^lO`+SndL#y>Cy*v6iT~&6WJ}1fRHk!3l%j%eQ7a>--v^7Jmj1876 zBHdcuvN}&`oUU#YMm+2-jUcIb7uUDh8f%w@xMEnXb)X_l9bR(}LGR>WQu$VCeGckL zChMqq8ACWPry~|GL2Xptr5|k;A=j`2+N}0Q4+DX>gw`J!gFh#jbbAX(s$|p;IA3vj zLyf#dSD6ETAToJzW}9+Z7T9Ou7~NWA=7?oaQR@|ft^RF}-7wK@bT?(|Q_StQXHQ)H zY4+X}?qp+W0d!~XXm60QWdmxC-Y9RDv2H=H_T6aSFk^rE*Bq$>nXvW1cIWP&_p#W0 z$=^g{@j!jq$Jhe!2JVRlz}R`geUZo5g79YUY4j;3x1h=90-dm@_PNKJ_Bmov?h3}| z3}khGp>e1VV2)u8ki^pc!aHzBLLZCox64RU>wm=N1NBZm0=cWf-UIce9*YCw%QZs! z1a3sE4YbPcg>B`~1!iN{1?0lU1KOKyfU8Dge4jKRckouRU*_=Xe8cIXDJCRh-gOokux)VqN{$~{QH`-D8*LC^4MfIzurxN^T)&($`69 z#Sh>=A=p(z_nqEfHLUut8uq`{QU8A+n25WPiM_Lho$Y_~#XCw^D*qzI>L(h#fgx*F zrpAb{)Z4&cs^oL!V+y9P^^SQHg z#GdQvB2Kopa^9Za=JC4W=-YemWb^xc-n#u|>Yx`A*sPAK-0d$NUW`TH9WDZ*0U|}o zD<~_%mx^+%QmzhRsM5`+I#G^a%+i1yGSfM46MkX}-==KMd09BWjf$N|{+4V|T%ZM7 zXML_ZL~nJq0Rq}OOrOqb1(UV*8%EdS(#IZm6n>FrsyQ-m%_Vdq+ag7&dsfQJ)3R>Z zMwel#B2l9zvqkxI+%~r0CsKjww?}A&xgD(%7IZ*!hY;})M^Win){4#FO37`t|~2KhIZ4(#UK;iG$LWKXoG%pWaljmYEVDt zOV*Zx>aC2~d9fm;o>-r)r3~XhqR4z3P6ztzNKet)+645;ZpJFd{zS9SfseV8#SH%VPe zMCSYWrqSCSrZ|)>A%dt`o%1vgfJMO358kF6d$V-CsExS>V$QrVj?_N%TD?eM!#)0H*Pyxy4{KQQ4#M*-08-!ZDg z>6Gx{i3A|jgZtejjw&<#-xl|x{a;r}-|&77JO~Kw{|CJPpVK1H)D8H*(0kkeK<|B0 zz}aCUm653pWvo=7VG@3O(@3F2D8*q>3em2$Odcgv+PMsvaY%3LtgpQl?@Ra!j?{EF zmR&YFpFx?wC0;xnZph{Rm}62;Oz-l%<~!y(<~`nUdVcQl`GDxv*L~`b4nm@cLz|`f zJse8^_ihB&%X}*E+h~yUbO2PIZC{S|Zp0GBo~;nVKGJO@(A9e-=D9WKFXsG*+$|B< zkx)ylVhX3Kx~ROLIdyslp@RG%1UPC)dh{rS)>xgSd_0{NOFhca=rIBxGZ&E#YKUDO zoVu!?4$}kK4{u!7gD4|3VlN8e1R1L(c2gNPU;-u>+pHHY-^MAe#7SLu#5{krR!$V4qKCL|Nbb zE0Y?kIy+RiKusjZw+nMJ|}B^N}!R~hDH!qqXn5ZUJsK7>1l zBd%_2onLxL9QhZGQ=(p6a6UFAVsV7ibplnTQ941Q0x$O5g4vH8e!(3GO~`O*e;r2`wFq2N+t;T`W92Z|8wuKcnv4< z&3JXznGX>~Bh#!EOCrr0fv6ydp>Z?lhtkBj&G&o4S)|@2xiUUKf+ZVns}VW)JMtph zU-}~3f2^6}fMO8#qTH|1s(Rau%Lm$QOPh=k$8y>rTUS;Ke#u8%zR-E7iPuL|S)p~l`8MyzI4xbXP0IMY#ZU~{=QewjL%~g*{ z6%7iZlboamq?N}Z@L=s8jn1XzB)}e7!MTafJPNp%iMQscoB&Kx=oi4Z`43P>ays1c zXxNCgHP*r-+9T7*zjxib=WoQ|nBjAMywu5-D3aZiPRzJKP5f+)+Dk z_938GIQ|;Y=Yr@lLA9uV^GI4kHwcA` zE)p2!EaVz}$Tp7Jz^=<@o{)n}hT_|)J&s=Df%_Sd#KAg>Jq?TZ|5d^GlJ(AN*F z{4&nO09VBHT!rf}cOc#;poPdxA+}f&Jo*w)dr75*MUGMG6#0e`8E>r4N8t4Hm`^Qf zj?geD!&n7w>5>3cBBvDzt=%B~^0>imMC25|Ky=WFyHLqbHSN=om0JpKv8b@HPyYiE zRy-6G&oFY1v2JmnWYnfqvn($pic3Upb_$O=YM~qi7;f2nCTW#6GsFbF&+}K9tD$E3 zWm{fPIG#;``M=JnJSr`}HZ&R90iVw(LIW0+z0s(AURk{;P?Y-d-)S6cj!965#MvxZ zc6UF&ov0q&!v*n2-aG$0dBGB!f=cEaYiC0M0pb52PIss8rL2*;sfCf9iH#|MLDbpS z)?M=Zs26euS||eST`WyZ|F6uWqI8rB=pDC5?!+a&9`Jiigc?J@Gq zT;J|55I>kS%V5q8seY56SLC|{{zvcqs8%t8ynLPVzv@obvjio?-9EJ9rU9=3rg~98 zQ?JOjb6Hq#LP%6@8Up(UQ5Gf(LI93S^>5<)z;omw^SeO;F>5#RiIb|-7X+>njL)UN;!eWGua+e z7@(to;x>a!GOzUSuET@nSn3-P52e- zp1Vt+&Is1+8HFKkK29@oUeZQJ^#(V4$>G( z{qUG5FR(jwC?-z-V>48|01V!iH>p z!#r%_=dJfoFbOF%#JE<(Koqi&m;^UpKT(_z66q*IS(z7d0_nj&r2JUnZF44anA(-3 zOc8&+3sp|lBy-2i6ymh9jxSiz*cvZZ$BgwyOVg;qf1MZib^0>DzWGIq$RHqm|3jHt z7y%e0jQ|SHKnLe<(B9V85(qQ}DB3##zl+q^)XB;I|Ekh#jaOeZb9^82)eBkndB_yO z)j9S^V!3*vKt4|Pu$j2glwNUmJhVKN7W?jId zA*L%Hmp-9Bi@pMvjyG3nWte9cyb@fu?%%z0t{=PIAA1<|Kpmj;BMsrkFI#c-!L60T z4KYed@N(iOSD^`d191rEZXBStLZAP*kp>_p9d6Cz@aOFYqErrq9vpc;ESxKTqln7?r@KgLBNy>SEI9&Azd(ixb?`5vq~P;7=-sJlz|-eK3O?rLr2 zF+}zK0kT4Fp{6=p@(@*gU7_efAz7V@Gh9}ydxFKBR2zn=v)e*|t zDx6puLmy1N>^Lo`AYwR(lLP>3=2<;we1u^KHA=6y*9!%I`3YcMci_T#+mn$?}n-?E;qVvqCUq~>2=PneQ$es^~ z(>3xW!S>Zj&ARQPx?>jh>i+&q*K4pUz=-NW|)_t*}x;q<;r0L zRx_$w%{D9ORN$8%`^@6SC$i`~3r3^Q|GhdMC)FPIgSHrXvXk(}B4f3)BxoShO*_Jx zLiXTYxH!!%nwD>xqR~)&&hq@plw{MTL?yOYs?avnH4&8(3O$9pK_Fn>v{cq~<{cH_ zFyUo!wd&ETC@zg4tZl4aC`FwPXtR zq*fuKQ+?#m3s|6syCYfK zKc%5u;0Ly^kqNkHazGOhuz~)~x)$T7*Ijnp=Td)bZM-tT#^L@9>0wC=gmsT`++7*o zBAU1Uxyp7S;}}dePejj#XTHgek_*T#xS*yWZT6*HdGpmr(r~Bcb5}m-ga}A6E}E0G z0F6Rd)ja{?KMz~?hZ4$ipX92XW*H5H+@bT`dm-^o(? zB#jwoHRwmH+|>IFttMD|tO8br!Khc3SlU-K17*xTQsv`QXrv)jFu+?nHRqaW9FVtb zwWwh-%e$!rhv!fqgHkyZqFX8*@ayl2FFH9Ba}wf7z%eh*pq$HpAuA8r4zG_FFDN z`~RtxDbcO@pv0@=wf)DEp}q^3#-ezARFeAaUW^sZ*%Aol0XpI5ZLZT z9>IE~(zVaeac!_&@~X{u4?8X@ni7rn)zCXf7wcAVVbf}C$S-QBF46*)5WVGdmz!j( z{zKK1V|TXYuMv|iw5}F&Mf%4vC(5xZm{3U)zsuDfX4m1R{F(+)S^;fSVOc zH(BT1D{AD&jtTy~OTCq#T4>+phag2)(iKphrarWu6F>kb!7=`Md4DoC-0yTE*F3Uj z4ByaA7rz)!G0PV#>e9!%&<1cWf`#0x;DXnUDgS8_@KUzGc?Zut@KvGV0|m1l*m1FL{mxYWra->3QoU; zmgWq7zXjLjo%mW~*fWxLM@~Ii^}RK4Jq?9QE49ojN-Au;P)Zx!!y;3t<&zh%NUF!b z?t}5z#k;@j(hXM>27Pw4*Yqezp)GqPcNZ>y{D@P#RrsCx>x#`S8v1#*ki-oqKyp1y zRb2GMIlxwfsjz>qreLoqLWLTZIsy;ut51*G=FqtwkzotcC;PKR(~`&>+vz!x#rN~S zu{Q-97gB5?K|nITI}6_bp|b!On_1eJGAN6PfB(oCIVhW&I2)TPn;P2#Oq5NH{&)Ld ztf8fXrh)c(Mav3bg{8RP4Nl5gRz;2jAuO5{miIGcM_AIvj9z47hR@|Wuh5KzeDDHJ zidyE7jbS)K?MH%U^0&n2^@kw>L{UsA$QqQ;bh zjjw8Cq=>zQf|jN#8aq+BBW(^|=KT7CS-GC5rA_VEMQi&dY}h`ikR>^|Flq{&a7dhqZ+XFK5754iGA&$#})6wLpM(*Ty-H~6xsrtQWoe#@5#M62 z-jRc_ch@?wg;<7APxzxdL*YzUoGEp=FbaWvJpbpn_}s!Ke5t?hnuqpy>)$@*$dnzY z;(BrYq5=o%z)52O5ezXCjqwQCc6e_}sOU5OQtT38yl6^7fNa@HxE^(mcwz)ynMPf1 z$2yAIY!p580Pg(NPw$aXDAhVHfO<}_lD~24?AcJ86XZi;T}o5DmvjlDnu@< z=)@OdoN?I;d@jo)gQ{0JE_F%U3#5aZJ2v0f=}8Mc3@}MLS|$v4s-Eou-qBb&qplL0f1D0L7{}_6_f!Fe|{%CrQZb1CdZb_GQ=LsE?+z zF9mexhIq%}2jhe*NENF*b_F^DiMh_8Soh28BmV&q!h5L-M$$<9PTxXbMgTg_agaNHVZy8vqw>Z6miXLHfeS%XN zIcA>vjt;H4dSxGNn&Hk8Z=!aEepVihKu8DV%6?wdR!!fztYu(SX3r`^N$Xuv??uN zE?@8!pc{h1M*_JjG@LijwSugYx|#Y ztpDe0>_6vaaeIKR5%7Pd+IFd1E1(LY@y!dTE{P$d@8dREQ7uAj$3_@Jh$;Q_@>WVT z;%SUr&vtgf*o}P=kVQikz!G>bjB;&8XHut&TPbRAJ$9UGbG^!5t=Z=HhIBxDt{>ei zu%thX8#u(X(Pf%cgIlK6ot~9k?f?E@_$B7&d*&q!axCWQq|)HuwdMhWWf9YblA`d% zTIE)VNNqZp^4S{2@Y(Gxv{}pD`%>oJ%$Kjk+qz!ahQQ6pyFz#kAFX4+q_>t+QitTc z&L4ss2YppzjlVpw_`v0XE_fH!3035tXnZ-C7LDNc+P9im|T& z3VdJqY)IeeL#r2(oJ&Au zRR+K@%K@N5Qw`lj5jfgUD)JPs0oNLXF2u$iQ{1=a!`Y!{AU^2Xjc*?qO1d6Q_F3*g)F~~go)lPM^N?OS;mlI?Wcb=;iPQO4PrQKtrAtkU~0>ca|B0JX?z9Vrk z?=!vaar`&u9gChB<#|$s2wep(x$as9OQRfHJ_og#TleePDW^GA>>Pv&1U5kkaTzn- z*9dm}!ryR2;xM9ic_-j)$=kb}%a4a5)xsS5HzY$2JrvK;_6W4OhYOk*FA$m({$FYz zVGleFXrAShGom{%kmXIN9Z2uBghw&?fAfbRkHp}}b-ZF#EfMq}e`SLlSd7Y^^i6=9 zlyTKT{H_$?%mOiY1edVm(i@uB3sJ%L$!S}6rzLE={@0RCq>2(l{q8(p zkpBO6p8xlf{m(^Xm&U5fx;WNX6%%w6^dRYvP+eA52p&zYjKYvPKmTEvAFwb8Hk}HGF2j}DsTo464!HrX4!$AQ-R{%6Qg>wmkXC{L@T$1wo`g8aacJrVpM7N+w_u1d_{Ct38obZFx~IWSE`bfp?&J z!)HLPRE}@5x0+ei88>D3kx{kLRhAJv(M8+5jF6euyEQIzDSN_;MZ|4%N!3Z%8UE8C z3*LhT(oulNghxwk^3PgS^3Bp<$-EV zBzRkUYU8Bv)Lk)?==!pQ%6xOS2J8AbrIb~aO`lR5wjNbJy@4TTJ=f0=99t<2t=!*b zxX-G#DM>Uqs=13@OnMF34@Jgqy4lvn3uEQ(5}Jj&>YJ+VF7 zbPJBO-~JR7t$vFRFwEz!G#<*TNZFHVKa+>^drWyP_mJ5|mVxEArLdkKd z4vA9)uf~S*^Y?qKQVyhXV{H54OUhjVB6y!KNK7i!2`2ak2WL(k<%ooiRUCKF#Q}{z z!na6)XO!OeIae{K>G~r|$Eljmk8!CL9c&G0zq+@%Lq3c@1^(dSw7|ubAi=ufQz!cD z@lG}N7*LQ#>h!pAL0xl@LZL)w$`X_{KTf6LX8Di z5dYC)vN4P&^83A1VVGAx{-z>o`G#9aaTwPv(SdTGu9I#bf2%X{*`zZTiPAkqK=hx7 zI9*x}ryw)`pu|u9lv@~n(=hwX7pa%ZqQ+=MyyuH!RjOtU%C#7$HwG+u`&j+{ROasf zWq9-EaHEjX5UtqV9f2Yq>RwQ}gXD!#z?C6qZ9p9w8R!Vr0Zi~8_D0*1Y9+)XYH4`N z@}?ST{rwzcO|2fqkn&o-0NRlAFF=mA0jHJJMbYKoH@mfM{Rp38m!w`pE?X>|`j4mD zVf0Hf)1*%L1(e4#tYUWe`<31>w#6#xTo&EfvwnYN*;Lu99n9Jdn>7br0ns6_alNDC znyOF>K9F%9@6wxm-}ZkX%!A`HU4{w_U;(ri`q6ZKBQ6mf8ToQ7+OChMl#bZk^*LI7 zEV9&b*ec6z4%IUZV^vKL>r@kusxl-L`Zfk5)(!Fc`MBxbLd25c1DT^9@XzdF@6<^) z@}bOqGQE<`hb6;fgp$~I%MQLGvZYj_PzU@ z!rlluf}Zg&++EY43)la!{dvmlUlE(0m6)2scCKd~Oea7qI7eXn9HAb2se&9cRJlnw zuW!qx6|#L={5p-hMG*Pgx(PuVqeFw6qfCZA}@(DM$s8L47HM6)*id55}zl4 zSso=;fl)_6v?(#K%1TUE@G7y}KCqWet+FnWBblHER`l4w>YHQJ8kX29`h>7=8D)rE znAXcFtBbOvPZA--x`P#yNM#~9Tj&K0pgfg_hf_$11sz=e#cw|P(Dl&B7g;L#B=tZd zbPa)M^ z!h6Nhn&PsFYWBB13d?RG>8h3YV5{9>na{4ZhP(P{Z4T4U+dVkMz+DrwY6bN z+O@5mQTAGV6Mma$n;Xl-`BKQVN=b|Kq_Y%)qkx{bSjV|fN0{Y9Ya@6^OpHI(}Y z0JUgwCbgu#bPFdohZdWpJzq=RA7xtB)I}lC2V>Bdww@m=#B5;4t(+u;=Dm10uIv3T zYJNO*7~aQ;q5tko#Rd4rU`D{dYjfn}GZ4A*jSN`s*d|tX8c278RE1z%n9ZN})ukV; zxDJ7Ud!*# zMGP(QSJv}J;d$}aei<)9!HJ00Gf7$s)NLdtGGtaacIxVs{yTi_!2`Ic4XvRraSWWs zazxfgvy^Y6h-PU=SsXfmOW>;l%JwLrBDpsDr{R}Fuf_gRT^dq_u4|g5NgJzNjpt=0 z`EVupLPNO%S|gM3ruKM3tf9`_u2&Jf(cniCSqpErJsUR(u*`mzPX{6vr=Vz$5J8kE z?5@Q07aoP&KwI{gsTG>e7*V2mo@P=ODRU>>eZE1WTZsl+l?f>2%cBbfHM%Ya@4`k< zsw~bI(DBrLOS>b6gsQ?lY}Pl6r8cG!3MCLSu+~LOWNbmU-y0CJgktRU-=hLnWecB~ zwCQF?E$}keLEm{^KOnkBPK&LUv1%xeQk}+rJyMLKAAN?>{TKOy0g8b?SFoUaP&&eZQZ6{zw#zK;Pt`FC-Pai zgmkM=Kf8U%`~F&z$$tRd2NLo?$nr|KR5u2XJ*WRw;jAW?Em6s|C5MRUb?@K2H8{D? zB^r~Reuog2HOEwv9oG7*71r`!Tf`vVybPXil@cNM|J))vnHoC-EP?I}k|t(O|3OKx z2UvO-{Rdh0zj6~MwPAF17Z(N0)6zNM8At^snYB=uh>)?OBjN`sei3}5kBPH8 ziN7jhC7_}!WSyGfoR=cuqDh`Qo|T_EVv;AA*jKjX?Koi8r$=SYIoXkt|8}xu%h@}? z;}>0;KtEH2I_7$1%%Q%Q|0WgfpURIz5VK-HtDWp%0~i2vM7kFy9E9|#BqkB*%x98q zc*VD~^3umYDbVjt++Cv-wS*@DcXhgF2dlgzV2Ge9JVGlps8%_DmFQK26KQ54 z29DsG1nJ=p>!j%^V2=_H0G+(*6Q=RQZ2h_^!%a90OD`r+%Uq|Vy*nQIdpqAI`ok07 z`P8^`Ze%qH6IA_^$cB92&2b*U*>`A=$R`dh?$gX0m*Oj^uuuUZo^2EsjS$2{#_aQ>L(WY!I7_ru zHI+IlD$QjHR1I&yX)S`5>3w_W<!Zz62U=a z@OGQ#sJvR<UL#uPn$V~kTB;O-{0BI@^> zEeFe3ODdIcew}N{brA31<#RL9(ij9`D!b}d4QF#WN&DLN(X&7P*U`p9Pts-Wj4(1| zAZ+0f>TNI$G9(Tkk#Su;fveA#Ae$eZ1l{=7hUUH{R1b;Clz4pYsm>zUo4S2z^pBNk zZbSnwGZ`n6I7BCx=o)p$PaF#>a;Lm42SaV{yuP2b5)cRqPT7uTgSf02n149q>M#nU zwrrOSe<6$G_W`#ZhCCcz&+%ycVfV~-fHWMuqms5pEEs6;_>A*31BUjLe3+gAMm2!> zKpJ?`#9siw?F}Q#CJ(9mhwtuP_E*N!Phwyb2O9sVmzCTPgCf1Vy|V7!w7WOf$gxp46z7w>YB>5-xG^}q$ zABQ;rF2G+>BDseQ*(UaqMHzV>`AEce>uKhfk$#segDTyluuNaLH~z4;Hsej)LE5nh z#pAh%Q6YZ8YxD53mDji8GYTwupf-Xpes%4~1;>y38?@PNF%7bE+&X~OpA(^t3@Sdf z(=h7cN~|inPmGO{!AqH&1@QiRFce>>0*Df~jHvGQWfL1Di)$AJ;6SSEIoQ>NIE8~g zI(uG)-$n3c1jVWesS%E@kqjZhhU^RxvDu-;6XcMs@ED7&fgv9OIkP2#xPk<0xBfmZ zYq!!qm28XeWII7h9l|O-bFEg{wJ~C`2%oW}N6K8%N~mR}+Xe3c=1ogT3b;qgZ*JN# zj8c2`tnP%;>bwklkKs9PGy+BU10T@+a*j@gQ+jTY z!*(`>&RgUREiw~1I%THS_8olKUghTSE#KA4O9F zN4XXsNmByJi!S#B*|`{J<{wd0)3A?t0yX#3?vcYiaa4>4shwiSm@i`PUecvHFw$P( z21~E#pULtJ^E73IA$n{=bRt#hao7o}va)S_fa*J;Z2|+WTDCd6mjRe;AlLB3Z3y54 zC0AkJ;QRyklWu+#@pkEVxYdwPkW92M#nrth8V+~QxUb-3{ypIhjY0fUkUHV zREK0SuX}oePFCQ-Qep%^JCHztSOZbID(mD`$5OsKZE{#-MP-Wcz4whncK%d_KVtdRJ?@(O_=bvA%kNV=6`~p7Gpw_~j;EJV1gg7fFv)+&~jHPc3RDLO$ zU6s<}G=^5Jq?y5&N*a$}7ziMC{go=CTNt5Kv+;X=Z&w#gz5aDJ}jQleI@7^RH$>~8&>cB-`_n|X~0bi*mfg6xVUE@>5v9ZuR zI!`pDk`w-GS~QLV`flpsa0vO}Q8*5ByL;jo3Vc*ejWN8&2O7j(aQ2Akw)3NJp|vL+ z9O)&;(~t#3;n&K=qo>!I3R#-dl3I#drIx-%Q_;_k>yu6DXxY^i|& zk59X}hk5yk(zdo~ucqiyTsbp3fpJcjz1!dl?SIhRhu_w_dLG3Wn$b6R&Fz99TP#%N zf}Y({MZsW;{!QXDS&l%Tx$N-G`B|fvAJvqnEy+Ku7*_E^!0H4&UJRiW$HdYQ`F|4{ zr$BawTaEBOUE)uR3hze;0a}D-nS5APt8!9H;LiOnf#+p> z76~N269Ue|n|Ey;iG0#4a+dBZjAIe6pT`F8AhH;=veR;AF7TfHSLcX(PXnYpPX8b<$!zR3e4L1^oqE9k|mVlS=*9gIuh- zOm8?YCAy4eVLzfS(cH80+B^>F=8_A)?Acv}ZFjfBQXzm~1p^tYJ zvVWy)nr1H{zv2(Zo3=`l64JOWw^;#l0i|k~&@t^y3RUCEK=w)>=W{(V(R8-bD!p^e z?Z(U@*@ZU4fw~?uX##Ob-mRt1OMLzsEah1cfDF4lJQGLy7I-ytU9llbBKHoGQLIKc z6t3aK+3VW-M7 z7!^bJ9H=>^$`~gTCji|V!HM`*maFCfyFI#!iR4ypM+$8}Da^SR!BaDe_K0e7WmH|% zY{PE?yg{j~qT-5JN{-gtkH{0!zshpujgphB!x#A3Qf2n@O|XN`Y4Zg~l@y_8Z25*1 z#jHEOZ7Kf}>u{{L#4pLkNKiI0EF|`s>-B@8?!n21(L3;2&|D3@Y$AvHQmug>Bn9-_ z-P9(|;Rv3@&V@E`66Ek-8m$*C%vACJjp|JiamojeF8f)P^uyoGMM?dZ&&^J; z05c{$X82n3BW9omzDN3_;zp;)M5s~q&Axaxvqkv{q=)IOC2^SDB<7ranch^aUz@3T z*8?Y8Fj`e;lF=W7vNjrH{*g)PutX98JoJIEPb*1WV3F|I%_)F zJWab80js+fMW3-EmY8Dj`*_OR9^FB<&9&bp_k98(hyMdXR%P|B=!%OX`7jLTj^e`ffIh?@ zL+eSUQh4Rfnm~(VJMWjitI~z%19ch24E&Q}fVYKaE+?c!BuF?bMIuwx??167$C5no z4K!VVeYj{&;#*t~X7cPlxb`hwsrSbAHx}+@f-bL(>=H(6|3&kFPl9lYDjqT7De*or zDF@r>N-CM9hggr{A37oF%?$(X`31|18)LLdtj$@!puJN+MG5*w=eQZGlzk;}3?*pW zcYyR`X}~co?H3{~&v)bsfm(0S&oC;i)QZ3BI1R`;0wxL>qnd-X^;%VNj^*epF=qvZ z9c3mQrjNY9S*A=2z=DnPEdQU#n4zl1NV8OJ9<_NqBJq^9ByyXAZ~~0@xOltom?Nfd zm+8uGu&B%ab&6BQH<-O8h1qNKlQyWaW6QyQ^HWt1`EWz}xE!>_1ZZPtU93N+BUlp= z5irKis#u8NZv;kB#2vN_a{l$h>qkP4mj6iaRaMP3#GO-kj5)ZBOh+#B9|HP`Q=1}> zTmnSrZ_lWN{L{xh^ug^Jgwktz-F>q2IyX4Cz8~G24xQaewNVwc$FvBufMsziOL=pF zIM0G#|4x3%m<=MF2MNc!X&K1Y?7cw&eGq=S@m*CG2}gxfi4Yly4Vk2&B=7o}J+Nj# zj6llLnI)+!at@d}uX+$MHhgrn_FyiPv=3{@N6L`U(MPkc^s-3fSi(lDDFqB;#4ecC zn9tInq+n;|s@&wnYQLV9(Obir35wp6G7tg%&I;}h3kD+Oh`Z@4Xn|TSvmf#qUs&G! zLa_>#I4owbS!Of(53isw*D_IS66ib4A#}F~$AX**jizD>j$yfh8O3D=eie!f49PQA z`@gHz?0O?2s9M=8cG4s#-LJ!OwpjyeWQFh6LsQk!b6tsG_IgB5G3Dn@#6I-TE=A5z zOD!?H5Q^xX`^3|Z!?JeUo<*&)1=!^^ z`Nuq);F*a8<&Fu&V7g7wLZQD7tD@Eu&9is%8PQMZ9jBCS$yW5X9MP_;dKqUn5@eL5 z`oJqsjf`)tV7}(0BXMUCXLMzy3O9+7NyK|XS+Rb63~7cA|mkvUj?|D_vHs{>|DIWBBnp(@_o zeqFKSn{XhC#ynkT*wrr1*^%~IpYsFS@;qE;kp9u(17uBJi#wV5UHN!@b;wU=)dzbb z?zOm^iBnt6Q2eL<2gH9csx-nA9bt_wni^@JveyF`41s{_qMk4YKuSHavg!h;P&x)> zlBP~X0l%X2KOD4QzJ)!;O-_nw{{lb5(-{H~zTp0`nno(;WTfRJQ~oiF)H3~1>?2Q8 z!TN^0`c&AJR6RRlpqwI}OS%8?tj{f6_|+=zBP|9q-1f^VE4Lp@K!h{u(p4pCu@u9k zdYZQ*a%xq!F1kps&QZJHhXWnuK|9&2TM(YD!Bh^|b51Fm#I+p2h#{kN^l}trwVn?y z6Y&Ppe+1r=m|`xJ{_9yOf;AyHoz7}7nEF!t#Opu|=dc&&xGuZ$zx#b^QKHLdep|Dl z;(~xk{ST{@{|(CihoR@Uc1+aN#?<`(1ZDqU5ZWP47+2I~^v@hxPZM_=<8)+HX&TDN zPQze9j9Q{0_CfG{EIbU6dB&9CIYam1DDmN`IdaPkpiDc%Y4<$4C7jfEdR7oH-?sfL z>@)1?p|R_EQgg{nBF1}scJ^;3$Mfd3&-2#vmiPUWuf8W(FaC>X9DPiHEHXdm;z~Ho zRhwp@m98pd`Hm7qECfpLzK|CdA&hVzD(w?#B;U|)h1-1z>l;>nj6s%fhWkjM#UMq% zN{|BNEtOZsHecz^>n|7|^04VWCm}Ba1H~IL2m{!i2_c(74bYd008Gt+wN$J+z zL|Y)<-e`GH7KfWOm1D`9T1}%R5{H8by5b^A9yN#Av&zek5N7ZOmlqj~fNOx7}@pBhBzg(D^E&6Gb2M;u+Z}jL$(`k9sbFgen>7VTjz#M&#TP zo==G}74ieDS^q%M(H{-1n$;FHj zt&igh-N|bHa&urViFG1m(i9oBNw(vLT(wtX?T}7aM4NgA@c?srt+izi`F8!%%a`|W zYR+Mti?Jgd9YgfLj0b>Dk+YR-Iy_jJ@fNRdQqg4}-C)4Lu^yP^QD`R4;4>s7fh5Xs zS<1RLa*~Otn#oAqpUETT@>A7vjo4Tk_8j8XGn?_^>LyeIH*9RFw$;anip~lBJ0`G) zQxK21Qp|7i3Wvlk*`|4#8t1R!9Sd6303qmrp zVrFB23=ONSq24C}KO28ftUXm6of4lz)y?!*Dyq!hw(bZdw=q?XGAyDtHcts5^NMrT zp&G}fY*@(C$$qnZW;=mK9hB$y0a_0{o1|j8e<)uG-lXRD0|PD9us66YShR-gDUKPV zS}Opoof*wPjDuv~Sx*p`)gSE{*^#+^onem3T`l^B38>d;n@UCfF)VRzWjbIJ@du}mbJvr_~vJT zFot$Lk82p_g6{u8**iv85{F&B-LY-kwr$%<$JR;5wr$(!*v^TaoERP3=p^0I&EUOv z-kCLb?)_SyYOShO^?&xW_iz7|Og`}^KwbNbWG2sgV-yC86Zt%!RpYzst!dIAQM<-t zYN$Y15P<)w=I)dw-7y4b^s8-i>QHJ-i!T9&K;PABQa?UjAmos?eEYhw2`O+2MpxIT z7$Sq8x8FOnW{$naeryPA&MLKM4(f334&m=Z{? zVR1Iv^+L~5s-D_aH;+0L%oP6#msEBPn|fhiVNALtQE^tftn?p^el986PA(_wCyw5_ zP-S^<71`+ksHjv5o%Pdl)z3pr9u;oNjx1fS=5CWZOFR7_64D)CEy*qg>2l4XN0ZhoN+pJ(+63-5=B z;-(vilG&Aom~GA|BS?okLf@|=zWjrbyP?wp@4`djfU9rL6j&u+xpUDU9w%P(JiF`X zG^w@r$;XALnRq-C((Ebx+|16D0;@{49`=lqr}S6TsO$ok?GR%@P;n_@KD61ojE*rn zLe@RAbPm-ojwW{ANds+5k=#4-iIsp_txh5mw#iPq*^!`ZTv#SDjpH0b3GGUKsT=nr zphR}??6VlxA5t7{c0NU{P;3y|?jub9Az1#XG(-Bmv;b8DceRbzzDmRdW)TBSS`WwF zqIsoq_&9gkKdmE|+(xRZ_*H6uoQDi#bkm*Yn_J$>9;?-dVBa$lIyB*8 zu{0-Y>%@7&os72nX^h|0fG9JR*np_J7A|Q#r%ntMQtZ`qMV?VNy!r8iHzJ)bNzi3VhYHp~lD3hWn7BSvYRz8( z%MJPYKZoXt$%$o7zli_YUs#m?o!R*RJ^}r2*V6xCHZIe!QwFKv_#@$8a78FcNRId% z?T0}@N~o6FbD~X3>0qJ@j~6lOH?}oj93Q1beSAaz`<=@6t(IyO2Z($tjl09%k|Wh8 zvnZ4SdhYR?<8^v{A6r%YhMlN6r`Q6{Zqt%c>O`4rMLU9(lY<`Smg_alv737AgYE(TVbCgYh4ZO)-c6A zp7u>_ZwfevesZpv2wR@6u6l*gGd)kr4Kgp!Od^iCGcv-w8Alh@V7NR8X>R3;(_4Kd zQM`>exCxRDP8IFy%G^j zm2})3zS=vwTzos3)fWit{K^!6gN{769NNhKSn4e!I-D69acYEb>syMlFVzJ}ILfLc z#_@6WYZ#cH=F%Uw@|*uLb<8Kq;4qxREq-^z%E*zmTt*9%Z-;vb+jk-L$rYMl@GEB6 zr0$ZTp=iYJ9ey466^q5Vy@p>qZhxe9^~KP>{8GX9eTGkRxJ28UEhw9!(iM}_^tdFs z)|+ga*4s`EhuXueLY-o6(1e!nyTg>Z-=RyBiWiX!(fxSBzVD0vdyPd`RZ9VX4+>0)1$9V$Xqt7Vn6ipA{^_C@cAVb4KlXIfdGO9_H z*2;t>d81av$@Io8k#>#EcQfp2`(815Rem;@hjGxVlm~s!Tx7! zhv8i_zv35k;+5>%H=h5^6yadv?DAiuX3YNuHz8*7)8c=1{(cP&o*K)TA5LxDSpsdL zj4(VjLy2Lk5(^N6i4riQ5-BJW^p6(DlwK@Z*j7{`5)O4OZVoleS}n`AH61Hu4kX3F z_)K*h8(kW8?x3!vl`hvJoqs#8mQ-025LB(LzK6e`ulb)DK6#&WeU2vrwqL&mdKHhx z@fqj`UfdFLr|)_5Ztadz_^!s?>`DFgPN7(*D{xG=YwrNiJEqjPcLSjNZbrY3N5Jj- zBO+}7-BIK_azwwsYvEUp7)3uEJG6mFh|d!a&+<>m<0cgZ?wH1GM5#;2HlAy7A%Hsg zgz>+syqE5Ktk^^u3Cc2#%5wL_*t^u8a@YQPYynb0Wy+8O*E4(`Di8M@O}!<2ylS&CWll%d9(_5YbQYcBY-?&XL8R&ZSsb*hshxT7)SRe z8ttpj)K3}FesmijE^O57MRjDv*|?{5M<{qM)IJWygp$yh~Zb z?J%v_h^YTeU&}!v0|&La zk$PhrU2qV|19w3#Xo1jDgVpx9k4e06IcInrX3hPXglp=m&6Xd7lUs`Zq%;j@_cDgY zQm!1Bk<8KGtW07C3wFH`=ikh4yIvF-E_~NJN(0OImp?kzcyn zI^LX=g`WMZ3A=7v3){9GauRtL9QySLiPs~ZGA|&nd1K4r-`VKm^q$l5t(6okA7B;4Fef$sbg#eg}R#C5@B$kQx~6zdIihw(*A*rVVD`d*yKkK z)Y6dhYTaIT2t5J~B}=9O$HJC+CmuKVM2^UC!y&x9wx;NX_h4~_h_>v|eIT`bi%*G+ zWQ4*vq=8zM;#8@vquxMl4lzO@t2l^~FSmNYwyae9B)oz@80d{Psqt|k7Y!~AS9~Al zGD=VMM=xfQDgz0_$1-awcrR(m)O~?{XbE+8uK=Q%`h={+rzE3|R}JbVD><7o5{jzb zVn^_FN^+<#t`$0zZq_(;+Qj{MY%LWP_1f=i&*Z$U;_;!Pfe9X7I^8r5jk6~k&jd{+ z;17*XpC3SI21T&APufbGv=k8!^H~4|Yg%xBDhJ#pyhl0i1^+Y9bmufzxm%3Lz0 z7Djf}tGJC?^;Mv;^jJ2P&WpkvN@-QHF_SDJ1VV21HgWZ%Eo_F}hoRV+FJ=q3D^uYgft6pW^tJ{rBF+f1_%@n%dbo=Qwb@OTZZZ== zEhxow%8ga~rPAm4L<2hlM6R(5V>pd2ZEM=c6ptn??+EV-h~?43##CdzneVu4S;^^R z59wnnHyJTVvo+L+4oltxbm>rFUF>^KbjcyfOK>vRk>4aQ$|)D1?SErKT;ils!OG0f z1q1HoJG!%kz*t&n?Rlb394gT`zsS@dPe04zrcC%**Ol*hp(>#s{VDIxtGG&i^lUyO z&?q_HM&TLF=os|X^J zyE>8%ume*uD1Nh_;TfOPtrHOEq}v-WsmD!x))J9ss3tg()~z~OT;ZCJ99dVQAm-(? zCl?>l@huhXXo0iUHe{P@04~@8+Q4t!B2AU6;av?mADzmqxZZ*?2iC92{-6Rd$tjRW;T1{B>3(!Gq;(JHasd9?RS`|J$_YU8rA7U^=btJO=>m%3D!5H&1sW+4B= z5=q9mfE|2F1^RFOH!=4@f{VY$2MjzX>GmNVImo&^QD*si43gl|+oU4oO2@f#@vX4y zwhwP>mL9UgO%QG5ufH&m9-slvCAf^a+PBOR^mRH_7|8_6M@$oq=@+`W(rNb6N`5xP zF0YB)150&6Oyz82^rPs6rnN&ha+26eV^-s)BB6rfIq|uq9{V`|> zF76|5RHCBlj8B8dSz#wAMB5>x!dZ$K2_W;&*?Lc*Z$#$k!o>FqSnH74%l>M;3ZZ=o zLZp~v{K9qp+nW)yvA-V6wXfZ}MALun)iF}2lZ*a2IoV#~N?*>EDjZqN=$v*VwxA>( za)0lT+5O9nK2KV*t@0jTZ_0~KA}Le)lp)+21% z%XP_xIUpSw#?yF5`tow>D6gI*otWDXNG&d4vnEVb7`Y^I36um=!dn5B#p$GnsQdvP zd5Vrm@8voqjA3mFmK_|qC3^;l1)y6m#1oqru#gz~0V8i;rGF8*`@_~Vkh%e~%t6R^t%s z!NaUx-z3ig6_@{~VDy{a4R{lOb z^MYSJaLDiW$Q1ld33UT_e)Ys6c0P$H{gB~(XBhvHOMRyh5Z6<4+BP*Vey}c})hV1% z;UjfUVX*WlhlzGj`OK|>qBs`vLUrbXZn8gL zkXP7GCmMixW|5gn>O`!$xKy-NjC9m_{)v^9<-9~>el@})#C)t?42Mk_d}8b3n*VFC zed*1i!uTEkKaXf16RLJ)Uy$m}uVs(a|ECe{e?)>%{MX=&q>GD_i-wDdncaUd0?V`< zl<}oe{IOVsX05=EIa8@JpwKocxD-L5p&YVac7;?nL5YjcM?GuM$1A)AMFscZWPik$ zW$H9IKS|ZV6v;V>Yo7GW?v?d4{Zu#B`9ox#znD>7>v`*)R+jmBy?+vv``(cTXlpM{ z$1d=s@WTB!E*Dv&8>t>m41f|agY`v(Y<$I;NC(y7c6@p)b$g5! z-uhz*mcUHVLe)Th4ZtW0AMf`Z>tP_?kTZV@uEi=5X3l4dN=H|M)b&oNBn@18U>YNOB#<8U1ktE+- zvlfQiK;zS6El=|5ChpX%(l26=5@UxSC50= z)1#QAN1bQ7{%|ncxuu876vBGOGepREt0tQ`HLgypvvmNOC0~DEOy6W7Us@mQ!c5~P zbzAHF`Xqv+&XdLgY&@ZlcxfiIi|GH?QC%OUunFNd#y;g6R{_5@Q5GfB`K$Wg21d8X zpcstb)|(|X>J9fgN!oR#0rIa^bK7e1CtXW zf+|w%EGwVG!AkKLOtYM(06^Bln!QDn)V&wK_qvwHWt;UK%A99E;n+K0j?0h^qvkwiQj0@loK{o=$A8gzD5%y3ue;3ge!yAS@qm$y%e^Z?r)oa0AWQEuVIG-D zX~P6qRmkN=+zQ2ql6-1V(8x)r*Cfx@DwC90*&^i=VF1C(;pnJ`AwyECl1IItR~KFf zn|8bfyyg~_P0qEBSeSg(RL+6!hNYA|(IF?!;%2vJ21r2T_}hQ{4Efc;F(fRnGt0O4j${rwRrWyAm%VJ)Of zKO$HzXSNX2dsp?I&Z!JlPI6CVy^(6wTr{`9=RpW5*hcI+enXk*YV`;*TogejRQoiF z(w5@e`QX{T`1AOc#6bz;=#D};PgQn7XW&JtiCK0k+yg*i6|#W(wJ@~;So0pYR0J!6_(yhqf9pF zNQj)JmE#P&1{Wy0b0VIHG4S{{x99r}XbB^w*;i(a0BJ^X^QwF}YbZfX+J$2GdpKwe ze22cyf^?-zRvW1CRtbAW1tI3Ofm`qW@eLc7)n92#>P%G;C~;vq8odC?b~ z8A)Kc$eSi&)Apz>w|02x#y9M&1MG93X|y`9wE+qGVIFFFE$Q1csZn)E!GfxAiO)wl zt=R1RGO`wQ-f~Gwm@yloUo9-Hj_|&X zB^F*{hjDAbZi&RLLuQ4FBXgmx|5jHJm+3D*0Jr|dNAw5mKM(+JL-;>`2%nC8;S#>| z-pBtJz4!mosubPqU2XnHp^DM%t^8+8^J9(pxl8wo(E2>wrPTiwOh`O%haCZjNaxJR-5;D zqG1MSM$G_rYMUw*W$RQPNh}@4pJ{pRYRK>jv6G8DxyGSHgU9N}kAZ5}kq2Wd^%s0! z5Ng-=xnrfUSGBRmObz{}!9?1aUOA051AyTkGhfdFN(+1Y`CrbM)y`hZTE-muYfnzQ zh*dvTM0*oKcmm2*Fiqs8rjr|I&IJk8*S}B6K7TCN$12KaQ{>HOg1PEZ>i$&u*ovjK z&X9+ha+j!wnOZxA-|vUm%KGGbU&%|}S=%>0)>1F2Xb6Vl8x=-wIpmPGKaOI#Q3|#v zjWUe%?Jir;=F)X$I9dJbP&60oLI4_a;Tp|mobV;6oE0;={{4c1PS)gWufAZriaGQ+ zdThh{>NCjuA^t@wu;N&dAimC)AvY_=&f>Uc72vrFV?`jkN;bLdcSvSeG}Btqds-C4 zXmMMC^hm=Z9JiBBQ)KC}ig$ZvgCoEye>PUz$jtsaumR>OFs4;z3bZ;`O`%zR%;W107+a&Kt+!e07@{`TVyuf80v{ypS= z867kCMc>(v4$vR+pz!66tgxsZd8Y9FF}nWpUHBB4@=ZpH9!chf?#Z<~AaW9pL zTo6N3-PT?w2B`)Bl#%=^*AhqcPlc;4yDQ1wq|$n4z1Qy`z3%-gZ}VR_xBdFK%6hB* zc8wIetqaYDUHP9wl&c)^SX_0*Y(HDFnuU%)*+uA0G+Qbl%iIW4c_jZ*4Kk%Bdk)p? z@z9?YDP#-jB@Mk8Y5tU|+I5psX^mzJ!SAwCN(xi<#@7H?MJ3-FUc!faRE&VUK#mIUCrd+dfwf9iSr%wv6L5Pi%@m=ZI@EQmhi5qIikv10>(H~KE!Gi9tgWu~Hl@p?4k_8O)GZv^I>eV+(lAuU zYZ5X9lf_4I+M_KP330KZ1#Go-1^Y8u)GtZ451cz0lQsd?aHe-3pTY(o*@n}|>YSxn zOp2EcRn@mnkOex)|-YFI6grUkQS*+`@e2lRSTm{*bcP=TP4^$ML~Y zBCoaKk+xP3CsT=WEH;S`Ls5!dcauSfTC+#6SH!1fbU4=w-*o01m_lE`s3O1|T>52U z+Gf^Vo*(I`sy<{$%HBRXuOfh^Yva*JbOgDJV0Adxf|atjWeQ&ZS3)v+_r&NFXZb!L zNb>~GG@Y7C*%|87s3!e2ne18q(E=lY=`2q}SmWL(rb7!2LUm8?YLT6)o*FE4g@Vn< z%4OczI=hICcjNA=2d84GC7q+a`j};K_?eQs3bz>NOv4OzDbCs|K|MRtLN|zvNDIwm z$F90j&|K2Qum`Rb}% z@Q$&8%b2csaoksVevGv|fL<>-R8KRVD^gC<*m9Mn2&QN!im#L#HC7YW>uL!Fw`fG3 z6K_Mv*FQXDlwlVdLE>P)8kzQ1KSKGYH|TU58|-+wpRhxBI^YGz^&pKsTOy)y%P%n1 zFb}Wj-8=E ziI1{`uXv2ioL*4rB;U4jP+>`K>u*?gUd}Q)nS}5Ak~!TA|3}eofoDGc{tvj1xK9=c zZle~zgSKl#Lo}nr+v4EQeM-nr;1pauNQn+i!GzLN%cLw;b>#fSf!eS)Wxby?PwV*( zt&oNIL7t(IB?^+Z9TUrxjk05C5+>gGn8u!oK0_yL=(tfXOtaDUT&pfc+TLvJPs@<4 z4kHka$N+UaveYc(@Bs#M;8KPy8RJSf6uNTgn7w}pE0F_BYYp~NB8<^I#%1GZl_5NZ z>I#%4V=ZnQ9Gz0t=^o0jk&IE#9>+Ji$R}Ucg`u0FFp9;V%PT%6y_K&S-ds?+>$W!F zJXo93YKX7V4Eu8<*Fs0|RVj6EN!w~;sXRN3!3~m8zFm}hhU$O`e+65g^2On%&+5#7 zx?ipGHQyT(t1^!{(1Rbq7fIv9ls5^#RKCG(U0U}QlO3}P(bm`1o3qn`T;2=T_cQvH z^3#pBtR=4R`}Vvbcz0`N7PNyhYwqPyR3r!_G}JRkIgvdCVJu*>X0rbhe7@1eEpJsAlsV zwX|KeG5<~v@(&O@HXY#ax41fogunM3Pb{Q#oZv}`sh?RVTlaZG^>PA;hb;Gya2sS^ z#q+`JGL{#V^%xT8lf`?=L8jD4-&YMOIbUoGwy*!Pks(!Y**O$TEt;C+F~?kv&o4#V zQg)EYR)z0-=(k8btREe7(I$_ctH7BA{9TH zGu)BlpJRCNaJG7c=QK!A+`Stn(IUZ@b4V7mk;RhLw|E-#Vf-7NfonhYW zuZKKCed}emxeJ>=Xa=?yxMiq2S5BS~eGvp95X0u}DiV!cYtcF+?;IXk)RFxtr5<`H zX~$3kqF`Yp*oyFIC7nDR-Y$unTr#lq5*@=~DyHNt2&Us$0YT1iGm740rm^r+Q(e<{ zP(T*f#@PmZADBZ^==?bf>rpl5(9wz9%tedRJ~92+QB|3=K|rXSGsZn?Dq+~-Hb*pX zN#^Dydd*SivSq+p0KyctN+bJa&OlF|S!Hc4o}}BFa)RnoY>_+01LS7#7G9W*}UYK%G8e%9baq=B~ECh{n3U`S=N9>o^zb*P=n6UIclngREgXzhNH;Bvb82v z2I4VVgqb2|YXW+lCo{&1^DGthV;z%q)ouocyM;-#OBiUN1(w_G3u`tKk*N63!U;Y# z9%DjdjJSBN_+icGLQ_o(bZu{fB(+N&P294a5GJ6+Fr?E@);5PoMyq+Rc= z1pHf)-_aF!?gAk>w*-!(@Se~{LyXf2`s==1v)RtjcBB4nh|oI1*QBx@jX2kqvRm|E zr|&dlZWt0!m~r3M8Kniq&$7s>W9KvWT#C&Fg4|?Mgh-QpanDl2BxX8fH4c zw9$r4+rGa>0wZAf;~pJD3Le)jNK+o!_$KhZcG<|WJ0#YF&5j%ts8@t$?l4A8qPk@ z2~x8{RT$Er%$@#@`|jXwTThPZ#R;#tdzlw!X3aEx8!|N8x6|2`wOp2xRoEnCZ!P|B3qDXjvcbCQ*F&p+P<=utY}0!2K(u;ap?rss+lKf>kO^ zLvt`JVlrbg4CYe1?(^b)cLZLwX`OUTW#I3h#vWY=Vw}9m++gP3l*p6?)c;TA*PJSo z5usnc2waM9-#Gq%i*Em&P^0VRYoP7+5zn!hVrkSf-e%7WARCL3N-jf8Hn)((oWr%#~Pq9WL}tg@Rh%EdRp5)+TNIP8c>40ClUrvdo>8 zV<(J9Cq&<|0T0ACLmX9w)jgko@2@N$-#;sR1K>&^ceF19R*s=j-{6fZf&EgqPA?vW zZxKPCOE&?yFHT`E#Ds4ux*t0**>?d@MoTxJ5smoFxE;foL8NG-E>&=5^%{1}7)f|l zagn5M8%y|TD`zq~3Y}s_>5}H1wImL@vURc{Tr~)~gz21%*5i7ElYJWAi<>uH@%)#1 zcj|5QEU#*`n^#JDS+RMj8qdUiiu3a~9cGOR_rvjbI{oZimfhVjHrOQB4<@xOn8G$= zN~4T(a(rfuu=H0OB`E-Ycc1nn#0|h0Az@3M>gKjV6R%D9LCPVhW&4C+?DHS%*MzypN`L_*prTxOuz0lJVm;TO?s9s zmGANAK1;J}jze-(L~3}OllH!BfPO;q`O03wl@w^jOU>t+EH?1?)r|AK@NK2!zP6r# zt*)JDMeiu{%fvLQtv_Mx-61B6a&aT!0^CV>o0@zjY}vD=(kFOi+~``Y9+2)cFC;;+ z{@Qse+Q-gz9@J{EP6>t7U9N(duuFVoFmC)MZIjtD>)3~{EE+70w`P1Q~`YwfLQt(M$-8FOX}I21(e*r+xCgTU(>xRd zAFo{S@3GlbMd9aeTd*-*+umNOeO^4poDg~!v@l8upAt5)zvu)I@?s>WNjI+=e+o6m za6)fePCs=$pqcx)($c#86>#U0Z*xd1y#>A8^*NlXV1fiI%;t0C0D~qTsJD~q{!BbxX9kFjx|jr{;3_&3CDGz zv}1(I^<12QSX?NtshVPI3aj(X<=fm^Ud9|qr7b;oA4^%Mc$g2qel)9uWwJ7z$=)Ni zT8!K5h223J;Vn}yBUedKbe9<=;siiTDJbmLMgel!l9s5Dm}2)zNwu?@M1OJJu@JEW zkgqxKw0dR|?G$X%6(x9MVnzP@3w+ zi?(t(EpU#0og=hIsqW|=`^)9_DM;#kl$J6XwDpNsn?R|C zO`EAX773*7nJzV#oE8O49yyETuQ8#z9p-E%2cfjRO$(`;hwkeCG#bEN60dtwS%xNS zrn4|>8HH3{snBJGH|;N5PmxMyIt6A&j^8%w*T29}=Hr315l4}<^`_#Qh}k^R-3mQW z^_=G8!~C~OgUg+avh`j@Co{k5UB$x{J3}hjf)iNzzDPdhTUERGSP&NEwAmQ zp{ve6P-8J8+*{R-_vxcRNck3Hy=tQof5qU?H;YL@mm9HbT{K;V#={P^*c>)GhVRzD zS@()M7gHgAD=KN0V~)79S{Nv?qMdchWV4adiW3l)D*9^7?aduvD+q*-Wf`X*--T~k zeUV?gr0EewGZubi^~izdI{OIUPlh|A;HWbW63`znSco|99{U3SVwr)%s@u}UFHiAnj9@=({RugRJ zi@v?#9qTU5t8P>-mZv|~@WE0?Y$)5c#2x=^h9d$X-S$H8kBBTdgfoDBHc#x<9vu z*MspU3n-=tV^VIW`Wb4e9N2`d6R;GRonNPA)tg?yxR`o8Gq*HuY}Yf@40JP0XGfwe z2IDN|mH2;ERCk)0KpUJ>GLYyCiKWV+ZK1)1yme!#7*KjVd0jEhS{^;i3cmUSQvT%C zl|}a9BX3YA$8rn!;nl+6$gU$lma*lhMYHP>#-^|J4B@CTmEx}B5WR#DZv}&J^&h8e z@J~=4sd9*Uj=>BL-hRsMVLnH>aCKdR8(5$*MeUVAIL6_&YVkQ#WG?X2O-inNYdAGZ z9>b*)86d#l7dO;`;bUM%fY%1~?9;~reF#b~1 z?)=r1G`^e~2MOcl1Uo8A&%V1k8-NaAiz;lg)g4Y~g+u)H8gk_8OgmPQ>rlh4x5g>; zI_a5kUP4ljjcN1K`RMc)3UVf zKkwR31H2R;`g?f(T|Dm^!7%g(5Fq~R*?smRed;&k$8W>F1tLLxJY%;gM$%7q)`JDbN;P-h zvn!%T!$#mv19Q$AQ-84D7&asBWVDu^6N~9MS6D+h`Ngk5*14k_B{onO z9xEKn&U=&PC_%rnN5WD3S68!}c6OUnsghpTJSO^1n!Bh;a?Ci7n<@R3>(q{(Q_pP0 z+V;{XhgGkSBBeXgv~huGT`O@7#YNlk<8>yWGbg9uiJW)|9iVq;cK;%z1#o^dldK z!#j{kvZK+kH`;9dI6V0H9Ynlp?lH~Xa2g|yD`q$n`n#8=$YVFpK20S#wg%-wrqUm) z-q;WH2`dA9deSw}jvGf8O7lJqXwg3D9F5?4sEK7FmLeD0zgd;xL;Vnp_V~+l(Zt^` zw0jX4^yBc2If?&AXf@QmrSlc!^2pD%B`oI#ac9(3WzuxQox%fcr9CM^I+_6P+5K|c z9&FG*qS3o^n@bID*mQSIlU*=(;$`w)02cSf-WDhTsui3tVxs`aCRrG=<$b(3#%z3} zD!2(+^B~#=U*--CR-I#bf`1I1F4pb<`Co09*KYMSr?V85Z`eHgA;l5eiCrmvOmiS9 z9PHxO9%GEPDz-MC)`Ok=#z|(+m~D4cD57_5`^V5aKNXohz^EG+4zjGv7ckfXuHO3j5(*D+`^s-C`XGVgZw8rDLI4{Fy`IEAC(eYw6OiZiZmeKFNc_XF zy0V&yP5n_^?g4slexd%U@t-Ddc&tQbh3_!_+s~2M_m_U)=lV>nvzquXh~WJ;y9l9* z%S{>1aX~w#pY1<}k-+<#_aiDhORq%-(3sz8W-YhhM{BzyzzcXNcrqUTHr({@lq&2@ zKD;{T@4TKkj~EdZB%~1(*dSmK7yJrKK!z36N}t~=L#qa&FWERF)!!G5(<2uwCV}y} zAF;hnJ#N2i{j*8~3xIN6GuAosIMv7zSG{ z17nK`{=f=rJW& zq)VDX7hjPfCSMf<;{BHfh_UA_aW;P@pr)Wuf| zVO92p;1@QKNc6=k9-qSZJGWfr5Vl_jEcef?p}Vg}dHKwc`pCT+wCYA}d;hqD3nO1( zjFX%W369pDKHBlV7V9PYKY#s1zp;q-j+uy?{XiTf7js|h0U>6o;Rh|ibz&jWQHLQJ zaFnNS3;-?eRgkP!uR`91dd142&Z&lAL8zZrlcutqEg5(xK&0_M_$8zCigJiRE8mlRgzTIg{ z#?3a>;147Pw2qeN0kCwS9(8yUe)P5xtwo^sr9~KyH2rwv|)A9 z-cS4CKNVsZm_GxTTprPJPS#UuSN3gJougez#dM?W3pHJrZ}7kC-yDqyU{hU8|Dt7 zsZ!_}yoC!{E>o{zvzFF*?gI+OVqjhr2`uf8=#6|eYa_sko*evg`t%(^W`PZ1e*1hT%o***rx9jO|K z@GvJP%T}+oZ(%(@4rSZ{Q}{y& zq}s|K2TgKI&@kqn@7NRNh-~gAIzKjxeI7?DyqnJ5B$#Sx&{A1NTqNB&zQrm;Gk$0? zo7TQ#c(wYt+8LoniCyZ_Hxq^fvYw!WSGMaxR@^awQ-0gG+w>N!glHJOdD}{j1+>OM z6J4~1rb(E`3#kFQ2WOE{KOWKp=oiwlcQ&Gd1oJl>;Ub8%I}(36PHK2*@wP)w3C<7~ ztI!=$MXo0byr1U$4{IolQd~CQkt$};6LI3DlMj~x9xt{gNvIv^>QFuaNXBZpU zCdW^bi{x+HutDFwsa)CTE2^Uc(c-FAO^$BN--wteUqWy+deldW@2KO2r(WPczV)Gb=YS_xN-H|vPPAj%2mHHiWMbC4w$Oj7vi8ntMWGkw+ zbJ1#xMak~f9hfm?_)j)rnm{LyNZud1GBgNFS9+7B@lN@9YrY^OA4*ju#6`$NK1{Z}%b3a^v(LX4i|-ZhBt{pPIa}Mt zer?{Ni7^7pPcIAyH{uu{dn2{@mL#SL8tAb6mUjo^7#$qM-o)@l+w8G7%ZG7I9ulYB z7Ywkq?!zSC9MwSuZ?&6y>e=}mfN z))nToaz>kR%nz6$8VIn#3NJ)jtR4|SIpEkp$%nL#&||04tBl<#sZp>;Wcb8;e+f7diZbu`|Xia(mq zom#88dw$|t3b`9ve~q>Bk|+7n2%-74AtY~r^V0v)64{t|f!G+iILz}(f;KrbM9TmU z76`FRw|^r}-L|W!10GNG&kksDyC_G!u@HoIIciW_#trK>Y(iM)#RAW5D##Rp-rNYS zT^Q3+H|lm;o~0ic&y7y4)7*|?6)n0IYs9xVkkl1sml+V{b%s||XfXuNf{-}j^+ zv;4;bl{<`5x-S{exRN7=E{YQRGdGau&q9C1p#%yg?Go1!m>AUz&($AhzG&XOm;_VD zHO}z!-~SMfd2KRP>sKLbF?F?hul07i3fD!J_;fQ_rT09_?>&c4^f4*{`1_^KUD`odis+ zzz&r#!_tZu8&_;Dk+-fcc?cmqXY@8Nx&OK9F4o)3Kjy1&#OSN)j`ja|2>UOQLk$yC zdka}dOQ-)ZwY>gB*Tnc}TQ>)}>i^r-oRr6HRn%L7)+h=zhF7};x33bASSy$BPBa&9 zDuU^gybGH=9gk(_>-*;RF9cZ+CksFjATqY|4o~m_JBHft!!5hX5hlgA-u>+FGuwHR z`nDWvQ#YP`hda_jSg@#*UYE;yv zI*dZqYHYZpb@TIM(C}kuwDCu^*imhLXHRc@yZyj`{Fiu^9b&n0&U%m@yzz_t7yydQ ziZUsgQ(TYQCWpEl~cEHwSiy)x3iCr6GBzNqz+;CIFBHobdPb@D>3{}gE0>%eB zt1|CfvRNQn6k~x;OsKIp3@WW%Q$YrB&BnIEsEGx7Fz7EGsiA>53Pl_|!ZYW+4Bdy$ zCx)TxH2O34H<+m9EHe+GS9Mj~r0t2srySl;^X230vFKk7*u7ge7331PmYvmrw1{&y zp7}Y*`i^59rZZ0VbW)6Y!S5-T)syE(XN}pml&ZDySnjjNx|9ojw6Qk%BbN-hRkL2S zY%~b;Fvo>iLch~{HUO%jUnRkpP7G(R@&ctI0v9HpZHDn9hf-`gyfCZ}17BNlp~zKZB_3g8QmVt7}WL&9wA+ zQJoqmWJ@RJYN~cQjIjKSh08-Mlza@MQ@eHl7h~_(ooU#$30CZ+V%xTD+qP||V%xTD z+x8vXwkt_Zdi6|qzddXEnJ?Eb*z4T;I!K5Mu4lp(Ry{;Ud)5x|deJTq8)LDEDA*K> zR=t8KhU{5!NZMV!kBVDoR1`ScK_+_TtTklKr8kC&YriK8XMgU2eS5YR$P7(HnjIC} zv{r7MgaNx%cc|Mv_7js6q=%N!-d9(I@Kvm9NTn%;)QDRo-|G3d*u5VRQZmAv=e>?n ze4sX>VDP#UY|47Fb$vCHB$>4v!pN!rFIMeooU+x7iMT_MT3 z9n;IGUb$^%Fk-^^Mn-|vaFl1)+NoEb^YaPkk>#-*+K2K-G8@%4nEx93+qaF^tQ6C5 zyIDOFbW@6N(ziWoS8J-k0E~rZK&lpXYbLIUsamMUf--wGvX(cIkgTzSQ0>uugjI-9M99^@V+A3X07uZi zvdFl+w0vyASY3sMnHsNp$R3TzoMHEc1o943e7wS8H+$^r0j+mkbBQ)oPUP6Mi42Er za`Qn?)!E=pKThNT(D_ME?n%2s=*-Zcs%xS@mq+$Jp==bv_DWW%W2Si1GwcS&YzW}D zgP4Ebn0$68Tq4eZk+#T*=XnEcKqD>UM3VtR#{-3vaYoLFrhf_=byYm26^{)HoEpm? zjAeFYOluT8o6DWV`A%9ikD;j`$Q1Yj6bd4gm1R_p5d5fHQP=qu%6&S=zXf^Z@KPj? zA9bMAs(V5sdr0bbP&*h$y`Hyk98-n0f6YAyTb4``2%b(2*o0W(pz()4Pi z!j^(p*!$z6Z-9nzSCxe#Fho7{z1J8~KS9~H{C?MI^fog+WQ{k*6J4Ql&4Xm5t0z#O zVIO3TWc0gZ(7;@Qcnycj5Cb`ZM^F}{4N$aGu;y%meq>0NsiK-W7*HqRO7eUp<+6`_ zDIiyxU0f~2iF@Rg?=p?(xq4Xr`mYo8F1cK@LEaCC@uZR9X9d& z`SARSj%k`Dhih?A-!Q%*d5-F)QfOX4Cdx`-Noz4cp{rC`$$Cwi9g=vF-=p)W3&1^| z8HKQ01BqRHq|{~zFOf-a2>%scXxiC9-v#>h3-m{z%=sTT+y9Cq2;1Aa7+Tu>4?sah z+8&wl_jmR-i7e!kC`_`tJ;}K-B5IjP0s*9BILADd#1>hXnvujo%Av|}Kf(bxQaMuC zxBoZAum^hz5+Z@9gT~y@;d?LU+{fqNHs)V(4tCsCU0I_R?ffn$jaR5MJJ+wdTfu1s27!zmVY#Z77n4ndbxA|ds1&yY85$!IE>wqdv z+loRz76R8Y76&AtB^AD6m#iq8*YTsZObj^A{FhMU^6ECF)DlH3vZK@Ls!uu+iigK~ zA}E_VR8Jhegzu;P*ZZpa#YWFg%FDTYk|w1(TJ8)&AFW&HKhv|ho`rDJyFy7e*{o>O z`yVcgB9D6Yqo#ws=U-}U1aY2Z=}3o(iV)DN)Ho5j!yS4Pv4UQ+6;Pdy<=@RujJHpY zI|6QvRL@o>jf1#94k(&>hVo%0V~rZTA>}uIq9@GScv9VZ(r#`Xb98xYL#nN`tdrRneVQEBlO}AWZ9`(&$Mweu5{;NP7g%5sw4`Qb{J{3Ogh- zWOgJEWOkgN|C`qLv$)li0`b?cW4d3zc>m+n{I~Ak|5ergzfX=9l!rDTgDiFe`YYk1vDBMJFZbCVmu{F8CA*9?AbnOTlOj;=T5}J_I)EgLSt<==kb1_01 zr$A{TN@?i=m>joK<|R4v^JZl;i!$b=0x+q6-g(6mlSwtjEW76Cy?e*Pq(VN z@5;#Gfi$S-|1w19ef%Dnv$~Ui>p=f!v#0O#rv90Q(AVGReJ}m|MUDRr(#LieiNN|M zkN*u*SDCThR`?k593Xr{<{vG`OX{v8^4ZnkRRi%C}7n$J%$~&`+OwN=kAbJ zcyGv(NU!2<0_EO8iK-${r6Qq#ubZy|O+RKi{5+|`qeiTB|5Qg&P62-jE<+HvcqPm$ zqk=0*GhBr%QrKRBOil*V$wQx0{l(R^GrNh}D=-$&rdpu{>|q zggk6P1-Uhj!H*sgJ`0IXpq8{K>YXlmcB)aY zYxfINra*DmF1m{C)Uw!7ypjk7I_uD|q3oDKupBN5)ZyAUN&5J%+vzycBwTiu{qB2q z!)f4(prhI8j|^>OQSVmvPEvC)FJmoC5I8(D1=U;O6aIi(w;DgLb`8}TplUtY$mdOj z9V>(qu0`Q_QlF0Ri!CTkB5GfP8rnhRmUyyF_dteS$$kiis$(J|5AS@G%1DQ5X?nPZ4`dOwPbF;4RNQIJcss+!Pjl=ogq1+2)fDU3Ba2;% z4*dnFFk?Zv2BD6HnbRYy5*K8(lvGF)tDAAMT|2zr6>`MtN)`92k!L80P&biDH_2vL zGV_zhkD@H4IpXy)Try$WsML+D13N@FH5^?gG$q^MAZ3hVF!~+oBijms!a_DRU~PD9nv*(JYaX?^#x1gI6SNH;;2}A0T5eYi^_=K7v5>dW zz^O}(@|yX(FQZEL`Boq2JpvIWB9A~;aU{DX0t(6WW>3(q9t8D!=o zr8zGa$_l7jW+B!&7gC}`>slkwR+IU#m`H4Gxtm(bBs}xQzetHWriU3xiH1T& zWFw%A3->Yd$C9l{>?8)0i;HE}NxL)$WINQv;MR>NT_5BI)=gc&f&c7>x}tDtvmJYp z-sqNY5wK@EC2i4g{CPl@@pGKR%%fGCs7#p?NE`2}P?;!ABgj+qU>ByYa;2!|v`1W) zRl`BuuHi|f)L=d0E$vpfQ+B>(KK^p*qQs9!$b29?9!AJ|KzuJJ#AkL4^4W@)uV2h~ zkn9*r=02udS#xTdPVO?T(>hcp8+GVUmY?DcPBXSg0cCbAWxUy7KQT)=@&+rDT^3CD z5V&DJ_Ir=c@7PkdsO>!2vN<uhzN+QsyJiBR@-j09!YoL_d}$%N=?w_0bHKd5>(| zWmE2@LO@%m?9z*GG*;YhJx*phvM%KzL8tT1G3?s%>ev6Be zKwgAai{upghPIT^ssFiz1dkZ)9MEW_yM%-bs5*4CR@u*|oSSqOF;vb?%aVOh_4%oe zHm)Ho?aoJu8F$w>k8(GejFM8F1NTnmdb=>%ykvP$*i1X>3f>wtfv-R|c@*)NR)OXi zw)0a*`^cr-O~b!CbIf)}cTP(V!K&@9Mn5(OC$0tw--LsJ0OBe!%{7~EYC z0z>T3@tvm-huwnV;tak6ojNH1?J}4uPmelhx-A^uNI2UfS&&vjY%o4+k-@KPNignL zHFCd~MM3hBj{}Go6-AlEqWGd%qASnYYC@R4X>z@%Q8gk!dQC`*iyv3Vd#MlfTCw+73;1%)F=93Q4h-%%CIa?SYlOaN zYgB7o12awow3qid!e$+$$fxM3n4X!f#a<(-^H3$S3wNxPn}TJo$ToD|&&p(}G;YW# zJ?d_|$1+yV&8O&{gVlfIkgWK{=@3D$k2h+f|3ca`xx4G~J1U%0VYLv!>&U^3+UA0J zUnOYhs8`9_ABw6HP`8>EKFSve;#2HrbZ_IyqqN^y^qZf%nU4Ovv+kT_3ny=aC(kE6?$fMpoU2>!#u6;Ao(-&xmmfaHj49`l z4=)~+8WyrOn=Ve%xkB+2sFaL8eO^lDPHTu*qqplYcp-I?8W&i(TiLtc&)h@cfLher)N-9^KZ)tF5d#+rD3$A8Ptr4Wo`SoBqgIsw?B+xFrqfDX!xa zI%^6Z(isVRJC!oD5=0xbiY-0Xj~B5ot;7W#F>_ffPtbDO^Lzk94mw zgwzWqbL+X~-&8!(Cz&^yCe_5Po3U+7c~P_Gm{{tq$~m+yt*A#m0o5tM2a zD$99hA-1%Nw&akZlGui*!xv_|9r3)|tO<<|ieZqU1>ZxnlozA1#Pnm1Bnkl}d7baN z1GkkQp^W2xv84nf@_D(saZ)z@bg^dKDr+&;K)hHZcv79Q#!0)g)7+e$xrfl*QAOAe z#3ha?k(dk$L+|R?@cD)N1f{<@c$pP5#8k1w%m%Z>T+I)G-g1RxR|{FDSxVWwv(xM_Rxa!})D>YKmo!s_@}!nLPt` ziZ{%e;;?e>{nLWYyOBxl7}(kZ;hP>ZlO8lx|698OCCwPMc34YiWWA~ZnQ#$dpLZ>w z#|NwER&PK0p&xJKeY^QowmpMyD@yv^1*THcna=KbyzDCLajq))Uhl&IUT@IW1E06? z_uRo(dk#|S!R6a8J_%d3eL^2d{L-}5gQz!1J}G*;QQaT1TQ06AQCrrtBUXO)BX2KS z_@CrrUt(5QkjkUtJzCleR{K)FF8B>w{c3+1tj|V2FZd1Ej=*@oZ@Ye&3>exWb4L6- z4Z!zTo-?rW%bhLiseP&DsjC#aEwBEMG`dL~Y-q4HbX4@P{MLRsKq!gX35}M^zts1~ zqJTMe;MZpkD-DkxO^q7R_rQ>0c0Do}jc{s>&*q&RHgd}uZmNvounso2Eia5|G0X(| z^h$-L;A@}nHZ@<-ms6}6g}I@a!je9q3t)niGs9w81dBQkHf7m!>a;?|ZOv5M=zr(y zJaXzZhQMu%s%?*uQ@FV)5VX!{&KpB-|90wh$Cte!%)|1HD8l7{(jQglC`@sR?aOfL zlaf|*@;$zRwsOM78(u9ha+=pH_lb)1Xb!?#-0{RPGWZ7$5_fm2Xm1=~4;r_B536wn zXc7pLI&`GDwI3(VApFbFL_ogCVGn!CuM=mrueFwoZXs%@X__90^f_t@##11{&APENHPrkfsi4r-g5rezda&9c6ALHwwUsX0#o8BYX^17wXyB;y-PzzHYlJrB^o&FE2d3{`q2Red7B)W*}M zWx*070>s26;VVtFvx;OVnb1&Dbh;gI+#{PrI@fS|O>wgv&IvRXHNrVX5T0+4BAHil zVYN_P?kM~D3ylqjHE)vkC8m_Knu7D>J|O*pON&~s%biloUU^^UjuNXD>KVVuXT0zmfBXv!{YUD2sLpa*7tvBTU)@(g>L(rcE!X8pYU{|Y zwx=DODy3vs+@+si{Tufax6>!GG1(35>5SATEp}t_;va8kt5Db2gwXTN-wyXV2RY7T z6c0V|xvX@7-35g&A?0ne2W3gRtA50LB0^uWBHMgcgBZeO0Z~EZW$5& zOnq^;5A2-4(G)~_et6!yzd?yh)itrwD=^36x-8F%J5Km*I*Cj9*r-8@{`*zW2?$JY@vUr_Fvh z_<)z*jQ9}}63c6d^tP>I_dmD2f9<|MFIRVgo(?LpwHadc6p)e%BM-^H;kp>%WM#~UE!2cWgqp!Sv1b^zn(>e?j+@bW-m?g?+b}WboAD9C%R_WkYumhhQN*qN$gx47B#q z;xP6FlE-9NlNFZx6M*bu%+EnFD-S9;%b9GZluihT!of;gI9Ub$61K22&<#uIWipb5 zJyQl1RE2jn1fxo8h~qjgCWlN*sxnT`uQ3dL6T!z!V5&B-@ZuH48G?^qz29AA!>YYn zcA8xqbDYi?Q^T1Aj+y_Zm5v#$1`=*t2BX$c;M@DfV{C=Cu;iFo;M5O^$PY)HSsP1M zm{iF*Ku+Ae$YMmnbX))?8l#z6piGdm(u`()MlE!f&ozRKNR3<& z9)`>y7*@Y^!6%o~-%61~fys(-gF2@XPq>h2bP{?Q( zshDT4Umxz^fKOP7-PX?Jojqnl@Dk&xG^p%yjT$M_adYDl7-J~HXTviA?6K55yw}ataj?Z!eYz4IlS_(C#^S_-Btba@ z`rY7+CQL3gA~6W%fpl#B1~+5hndkSu7l&qEUs?(mf$x;$1_u{9OesJ(r13=-dyj zMCk`R!$$6cHe`%hg&oS_h-lRm&8B$Binl;lVJmsd=RfEhd{4D0>v`(CQu-RcBXbMs)wr(2&mi7IT( zr5Yv)U_oY_Y>7s`(8I1H7_(ZpR{%l6~nT&dM0Nvl*fb;FD~3Uw)Kb{IT@ zSE*yyrz~7`qKYS#-1uh7RumWRy@D9c0Ph~@hgMU99@#dHSSk{`5&2h|IosV--q-9> zFF-~u#$t8WTBe*TR(IH0oHRZ9Al17xIi!DXAw@`4rn+5B(o-?ZxdpqV@1>+Dc|(aV z`0z^v4lDeQgVkon<=45l5k1oMbq4{G2(4~?MfjNooGydx1)gqBncvDmr%=dAI{TQK zWwrZ}w;C3tRU+wP>|)Y_E`j!y7Ft_KcK$VC#$AnUye^sxp5vlfkx}InR*bfpP=lQn z0P|qSN$0ey#`=qC{4yC>BhQa3QH6uwmF{#ZX=|6OjpVdxDruZs|AZ6SMHa!F4Gwv! zl+oT@RH!{8_TE6wd?%qT|M+(1GAD?$Via~sjZf9Dt38!@tXkslpxcrWQK?mA1-}sI zQnQ!u)g@!$i0HveDV{Yrv;LpwT|ZVdOR1nIkggcNE07cmBwM7IOpz|*KNW3_ealyC zNHWcO1g&c^2{3k?xs*^5Q8)P@@>EU`brkL#tIowdea_^Xh^6m$lO2C9Wozg&id z|J{nS;tBA&AwF$`x+B07NF;hozWkEB4GR34Uyy&uXP~mY5&XrA0uL5|dj@v3Ah#vW zH+;s|ctKR6PHI4uo0W))S{VPWIC_%B0O@O!% zPWZ`np>3xv7|rPT1$AqGkX2hr%RASI1!u;21d9r0CBal-kus3R0Lxq}kqKal^r(A?1&x8;5~1>GNk1;c+_Xp(k6 zQdT=d8&wxeoBuAc=+T1mL0--H$EW==e#jU$V~Qj|8Vaecu`d;VP2xW=U;rBt1JyCM zxP})`N*F^J&vu^BESrgEnN(Ul16W)vww02gw#n`i*YYZPHFIwgZ|&Ny+Wvb{YWVLq zCUzl&)Zj13T({?A&u#Axc8=HIBN&n9#6Ja#dS1!}eM%L9JJlun3;L^zyYqF{=l18f z!^gL%9}tXRalm`*R9~szk|lpi{~$X5tDNrxEV6vKJH8`?d@FZ<(>uP~13mKFedI_n zpZ`Jinv(5Px1{T-nYTu_WwLsXee_RAQL1OOdHn)I8q6f^H~X zw5nD1#FAI{2$j;yt#IOHr{Iw;LCdA3a|V6V2|2AeIu*NT(BFqqf%GgGgcsLACNd?s-zgD4`P@=&>GJlHg>r&~D zb)oj^?PI5hOJ2f<#b}sI#Er#pD4Wi`RApRv6EztA;bCd^it8h1;;|Xt$f)F=uD8^M z!nkb&B2zafv0#wBWgHT_T|3K1yj+D44%NY#7Y6Z~Mwy$COk*_y?vW>2De^KHzKa8O zkwy>={VMWeF#{a~4(;Q%!EC6<+px?ajL){*G8=jF2bhZ*iB)s+pcqDaH#wXQNRV9M zGrnSfD-JCl0wdAO_GuI$U}ML45SWSG&+jye@7Dbt^WW%`jccXP3QU7lbWrM8<)9Er zq7d@(z^F4e`}t5!la*@1A8p4AqJw5s7JA9xkN7IDMFI_6+2Ca!HTNSTFxBvC52P*oIrBd2(@QN#xS_*aJ!pC#M$4~ooc_jSox`kw*O&G_Wgu1xT4C(+k*_VsNRlu{R zmjS(+1YNu@N-yG~T_>%92rV;GC^4)q2aSO( z{+2Ol^^F5hq&7+FDS%Q(aO=xaCg$RMCZ`iX9>y7rYcv|8F|TnRC-Gzk-P1>~;D(sO zz_1iaIS)mFtn?PbPco80=W)dpqcDF$i7w3ug(K}R8WpX%z(^fZ=-@?B2ztjr5?=xq z$UN<^)-V>WO~gW?P{&6lOiahRukcy=SUhR%MB$zs!H!;Z_g%`94VJC2pB0}T8sn1} zi5e4H=tjHR@bIia4A`dss-(kW7MF#&ygqGJdSBnyNps-rn*s#aO~Sd2=XuCTNo z1v5YH@aXl1ve)(3g9%Z1LMp{$LcrX-Mll#>3j}PIJ!nF$>iwel4TbQN^~D%Arr@WD zvr#57$F@(KAh`Lwu%t3#OP88F4Ej(;FxlM9;zB66_QIttkRFW)wS0g zwO9@$);Q~eSpoJA;15x2cj*T=?=02HoNlYjzXhUrug0^Rchd(P|1OfB6v|rHJL{B737)^s$qF991unU9dk7sP7sk7JD z?7b(fHg&(6DRGYcl3j*IXWPfr3_ZL-?+iG!T8OT^8z=kjMC}bZ|K`u0mDsbDZx(lA1d6$cf)qvVc{b$bee zwM#|^@$PREbPRr$ipYxti^&*15~c(?6W-_T?G-lCT+R)Poh2BzUFCf>eS+8%8c|K^ zGT3JB*u{sogb;aDa>vc;hrUbbt@0;h6C~#L2n4!M`|$P7o7FFG@0+eyyJ+kjDrC3r z9^x$sYj^0?>XY!heh?qyCf|B5&cC9q48K}XJ4P8bWG^T2j$fqD?NtGg}13s)R?@JIw~KT zOtgekP6I9C+-bqkU@x)5!$Fu_z!_<;?tUHRr)vzdkC0mK{F()ZBYW*VXx_3%Vjf7z zkxSJcNA3Ni4-!)e7v%bObLf~*!AgOIjf%Kpd3ysoVh&uq*BCX5pWq?-J86vmi8{+S z_@AVT=jnj3WCom{VhNQL6rcy%(kSU_b&8#b8%*3N?Oeg3A(cx^kv%xk(Luqu!QGWN z)n4Jl+q*C5KkA3dJ`hVME^<$6nCr%&ybDj+m|{7xLsQZ9rqo($Q1bH_iMFOBSJJZ< zpS*^#RCD2WS$lsafZ%1ke@YJW4mJG=yF{JGhw?Lm>ow!z#veQ-JDz0@@x$BM-@_;@ zO7d{2K8a)W5An@QbiLDp{hC~OWU1%LPx7t?MAmh+;1`Lw!NrworY5f`iWIE#fOwdD zW)njzUT6%U4%wD`rYC^ivw5iMrR%PysCuSq#aD4s5t@|l;A6rc_-XFwo=^IyVPm=c zMYMl)|I~BmZZqO`Myo8@*%zsKMdfN7tL_fWN!XG z*32x!+S49!Y`_LtM;Op&Nd7WaXKFIRCh|?^(Zx9$=gI=MsVAK!DZ_J)mwZiKJWu9J z>Y_&6Y+eVm2!9WB)bLmuQUQ7CO7zb9py?rNy$jH8)o&|bBhV85je^QF0BkO=Qn z!8}AeV%*IVQcBrE{fS&!`xpd!^E(1e76m%Ph;V==flNu(6k29KkI?W)mU&yFRl|~` zgXY*UEXOS+jN4a9J6URj7M*is3)p_@aWY96S0X3KBdQ&5#f!x{ zNYOF{lX^$AIpfy#oXBq_fMMpU<)mamVont%5^%yA7mO;&ioxvkhc01>Blg5Y~Fca=ATl7(YVq*@HW~H)Sami3*EY+p4 zy)t9t`n8Simc-Z1p22KrfrYS^!%+ObxL@?-p?RdoWF$n_smpiiFuSRMsqbXkU!XDZ zAkUpb^S=N{PLf{>e^0(YO$3%x8WD~mq_+Et=RmITcwFf&Bd6w{&;}CRiC|7!-TB#= zl1JK!@z=NSLUwML!K7+X&m(XcjB>oTe#<($3&PCJ#1yy;5w6LH0FdnUXy~b*j7z9z zD?BJZol7sjfBMN9wdak>CxpqaN$;6!-w%Gl7ioQ77h#wkVAKreIM!M534X$Yqnhkf z`4xz}kanE$VGT(%1z7Oo_HWC!EeKYkK(^%zk{kW zC(p3LRD7egI09K=K2+J3bsHkl9kiB>ndAqvvk-6mKtM+ZNkdc z`uZlC-Hd*CeX_23n6)g=rLGCixCfr#fH>1pE*mk(H3O~6JuHfK{-6Tbu1ht|V1+gz z4n0b1GruRKm(G80$EGw{Q1&G4fH~TifOiJ%#B|_U$;}Encp>zpNyP4my9+AXb>`ia zz6yun_g~>`U8mp=OJQH&32aB*>o|Sono&LxhTQzcrS*(5n;c;Js5il)lgCeBd#8x^ zgSDV8p4Ap;$9gn-tWsf@bar2z;}(pems|AF*bB@(^2L5~+y|v~ICEB&%7fX|iKzvp z#Uhv2PLYS*WfY8}wosQg25{(_#Kqkr+!n!g zVRjXqwU!*S_`OO1x1^wzr^^z@Czt16<_%BEl zZ?qiPgX(PQ3A4ZC$x@g%G1LpiSt##Wh4b3hO68349luiNQlW&Z7l!1gZ_NNXvD+y- zRX|=E*~Rfv+2cDh8Z|Lh%fHX;)AhjK{Zabp+EQ}9(b0-8G%VG&7xX=!dMoviO|E7h zS?|pjphtA7)~w2^7O|dU3y1+gX#q+}YL-syREixptgx&TPc%uy7Wt|+m&M$RE*M6z zjovfG*sloHaFWdWRML5EvuNY7cAH1^DmMwCfly?s$i4+kmc(F>yxU1s=sejb2B!}WwEbyFmG|1!p0)o zX9LS8v45ZXN5KK$WQGq8LIKrFY&1`3Y$=soh$kmBs_~2M5X{aiwb#rW=z2K&t~c}q zE&ZP9vj(?pioCog(aCuqMBye#i5AZrYE4d}5zYme%9!&c6Fm<8H!diGAz5Y`}P zG6|3WdHle(>S&c$4F5KW`m9cNM&+^J30gB&?gv6Q7{-O%>=?+Vs?ao@t}9o2ku)I! zw3DKs|0CgV*S1~CM?Vvt&=t?@f`0={DW$@fzY{tX^^5ifh#WgPR7b|KNyd@`5trLY zow}JMJ9c)is`p_sxsY2-(dmDw!Fpfqw|?k(*Z25^;RuvlBe~cHe)tfDsc=2q9k%lc zGVkf4_ot-}x_{nfwcMr2c}%89^GfP8$~ryS*3&**q9$HLSL+Ga2|>A;{T7|feTO&8 z7JH-1ky`DNs)){%yx5J&YPBzi zB?w|Pozi8{1A!xv_0&x@HJmdU-pEWR(>>?JIPi@3SrF{Z@e`@r_p1vwrJV}+Bf~GL zrVru_y@L~A>Sq5aJO8|38WJ1LY6nH|q&V}| zU9kQ_4uF*>hJWz2#`;M9PX0Z>HRgXW?v*R1i0b)K3CsUvy8maD@c%*US)(j#za;S6 z_mV>r8ThNP042UXp&dU}%TX#BL{Jq0Q33_5R@qy!RjP5tO}8TKyNpOt^`IYNPrP83 z3kVSG~AdGnP1*UNn|G=3H;%3XbQ4HF)e$45|qQ!ZOjJW^yR6OK}~ zD^e^aBNM!S8JpyF0h@6nZNOv$mdtpm7FYQVHox7IU9AveJd(A_0QYN4)_wpQhz?gk6`JMYyG5X2;p(rv=aA11haNrol*Y;Q5E6g4a5>Y?{-n29Tm#mY0O;oG9iNk$ zK+NtdMu6?f`aW`3`;?~d#>{SN08M6Nor8{Xwpyy+S&dV3sH-1$OZ>t!N|5|rZ6i3j z%FhLn(Vtw*flM)Nabrw8#D&KJ(EhJ@ohJyH+t3dIwTt}!#}oD6XF|%}=zn^mTGX^1 zkX13h*<_R0ws#h>QRq2u>btmvV&eNeEAIL%v*Qo{%ibzFl7g z17be-gNolIGGX(98PM4k#Uo2L9-(k^;Ers;(9`uJ&Nd#oaP*+}Mvwpx%HXtn=uOVC z&_|p0{!sPxJ1lIyxQb)qVpXX90OUza z(o`j;iM4Q3i;N>Ss1%X}C(sKSzg@>&Rp|CJ`keTQx?Sm>Eu9r>>P3W^>DmH9pd*Y@CHe%#lgVqa(7{ z&1;yTMu)OAhRUW+P8Lp?(T^t-%?j#4%z2q8krl7j$c95qY1vBfu*yS)FXjx-W`&jc zCMwQB91FTZOuDNiBV(-$GFH+W($Gh}q;8b7d8Yk!@*QsgrdO94qMGRUgmPqqKCdWZ z^D@*R4-H{FgUU!@dW?7~5~=*!roAGfaQCmDCSf_zx4%zNDW>vMz`3DZ3C_mqV+m)z?&`6X3tVV}`V*>GUDEyB+3SEzy>JZ^PRQvl|MPM+#Jm8Z zN1JK#L0?Lhj1eK(z(T`xLo`PFM>ug%({5I$O;S`j=;-CHi?N$~?QU=Bo2$n2$$KwH z2kDaZ1QW>{_wAOD&W1F=bW|aH-=OFU;7}bg%XhLlb+!boW(PllIfz!0j^sB*RP)$% z;`;T)>G$|PkMb*wQodTB5_69MN^THYk_9Qlfb?HCOYN)R4yN@?9Z<9FLfirh%>qPT zbCP9UtUgSgn7S;=c#%G-jKXHd8{8?2Y$m)xc(!h8LXHACs*Yq*W7L~?M{;Z;i!x+l zsjTSdHG(2!T_&Tm!q6VrCTli^tXnWg$<4_9Sq-9M2%Uj% z3Zvrr%9`x>xdr#gR)dqyOHXT%|6-SB*Vv76*yB~=YLf<|3|yd6exhL0rIvVpvreNd zI9x0*!x=62=2Ih}P#2$X$~L$J2?N?mDIqdDm~&dp91wgrTIcP-e9^)*h0Y$Cr=5i6 z)V4yp&bO^H|4J;54UV0n2!7ifI`O6QB}P4XLVQtZhq$tT8RfE;B+pTPf37P({jl{E zq%8UYQZ^%B&3^uB;tH`$UTV5wENItf-O#(JTB(NEF+fTX@P^zmRkn;{m~6&35R}Jh$CC@?mFVn7^2-L+RI{;E2>Q)B&&DBa>yH+ z%ZHq?TQ=6HLYQt=_Ix%rWnBuoK_=$q2CnJ3GW`8C6OXAvKCd{8eEKG&3iU`1R9-vK zo0P3Yh%d%0ULY!`aPy*q{v=mp;Q}2yRX&pCKEEJvk9UmzU6{jcwEoFUoHZx?#4bUe zg~cB{1ST@1PnZi48FkPQA-N)bAU*d{4V1`oW{TY_r?3>XUo?ZdyhPG)%c7FQ;#Pbw z<`rx*b}>R%bDh7uX-QP|%@zxZ?d-qis@vp`zlkF)ZY@*k5AE6d{2>L`05=lxXpc z^a1_+Q#TN@D<*xPg$bA_Mvv$pWW*WRv7opoDJAX4;fYl?D_Jz4MIpZ~fMs{wos-#=GU5y-D!{Qr@``u}vr{6Fo!Isf;qw8`PfKo>kO zB%M#IKwv2hr}G>7fVxnrq*xT1N{Qa$fF>g9uw@0?5B85g3_6|Ye><1h35Pw9p;Q&% zd9QA+uCmze#hUp2eE-7l0Rbb>sqs`B)kI(7+3lHOUeeihv|k^@3YuO#yr8P*x*-CZ0_6=mv!WmcrDIbh@SQIJX zw~c$K8H!bo1O!R}8-EyR7z-Lw{z%|cgi~UI^A;7EjN3o*3h)1=D`~|+cLk~|BjZZ$ z;FJ4PI9-z+v-!jXWt_sOt)7{H)o$^|0Wvod8Yj!5xu;$F?1-*y^XTPr(F>{aom{`l zpG?SZ@jSekF(KhgW1-d7KcsB+TZ4}=$DcddZ1u&fVCPM}t!Fx(Z9R)ZoPOnBC=0v; z0%@*#029J34*d=3?rbCK-9%F8!KivT{mc4kk4cU^<6IX%*d7iYLEZ!v-E1Ig5sZ)C zCeNTwn9ftvsm${rQk`s&`EW;zzi26)5mcH(#)9-x zd0u&khTmu&lzR0rR_(*N1E@j+My}nI;y2uX9j%&_@5-j1<2&@HVBtS!yDB)@n>(30 zJ1dzw*gO4?$e$8bTjd{xkKYZM2_Y20KCOZ!5bC24R3Sj^cL5@bh(e%7&4%VsPJH6k zjSU;_8^jmP7nsjwGjk36*L;KiM*)B3b%(osA}Ims(!|ltw)Y(GoO7=8>~`Px_qF>k z<~OovyRBORbi8mw(J?DfK^~N$%vjg^t?)g@-0*cXP)?iB2?uFdc~_fDkJ3YGa9;Zi zx?OinyJ}52LgO_tzch2V7~H8wX44_JH#ychW^FN3`;q3V8!A$7bd*mZ>7Il`W8=j$Ok*ctyP1u3zmLbf})kre^y3}0<^C=JI!j!el6n3f2 zP-T=`E5@@MH0l|a$589!>Oh3c24hqly$TzrO(&h3-!rq>)<>^+n5|DFAwIM7iu-i7 z?W#^2c^ZFL-e$*>VVmM7I^-5;Bti=!nZk3n%puV^MwCnZaipYt(R3MQZ%m}cN(Rr* zaQkjao3wIe{S>OBos>GN$%2tsVwghy6s*Gx^*d(3UuCNF$hJCHylYZ1L~^Rrq@yGV zi^0rv2di2y=VlXaQVO*mHP!5CsAdr=u{_G@F7+Z4)kc1TzPtgMPEf#bgSSpHcxY)X zayZhpE+%uy0J$s0nj#!uDJ2lplFdSv8B$alTk_A@wpnDL;?h$PJg8918RbgQF*k31 zg8}=!@uQ;R&d)1t=AjL-hHBHI>^tJ`XREqC_;gS}SM_ytl?JObb&j2ruoCY^6ym!V zoe9!J>OC<|H~Wc--b0BMwFg(n)?p@|*5W&w364!s=WG|$Q_Q&TXqZO^*XvJw01HG>eV4RC8{l96mA2}F^AqG5}5N2 z`|lIYM6C0hRG(ukOklnn7Dl5~nMbP9Ji|2eFHqv06E|C0Fq{>V#}nc$9l4K&VN?;1 zGKhWh4FA$Vkgcd6m+((U8IOlptP&8y+7ad#iT?r}kTwW8BXC#>V^kT#C@e>A&MWp* z*@#F<76KH&DaDjAQYzn7m@t$K9*IbSGF~yBCBc%#&EwrEL%vZ=OF{Fbz2baz<)K=spoLR)@-Xjaf;S~@QJi~oF=@PIEcFDaW|rBjRU!U*j5qKz_~U zdY8(b~X+=E0sQ6?ZO@|A<`&e!vkMC^G1il*?7N-SR4^fOf4s)M$Y zOv%iZlJ6C5=G;)!_j>NkCVI}z=ITXpl1kG&erUbOQ=s}n%PI4I$|O~J#UP3+GyrZV zzfP;CS*@Grm$~G^+h^kX2K%p%Ai?Xe#EGB%O84^+D*ea(`u~nX`=2%7!uGbdmM%Xp z9*7{u2T8!+57iGOFx(iAx?CO8VvUNR@VPwyz9-^z-TRIkCO-ka1or{$3NA zOy+JehW2-!S))gxz5K5Dt3oO4zrc3-aVYLQ;mj?{CBmv#3k>@b-uc8*FBRd}zGyBp zBy?aa`id2^71~NhNO)>@T*jO7e|8c!Pisw`Oh5rh|vJciCoEjF} z{|6UBe&`|?%Jk5|-J_hyZtQ#m5DIgVKV+XjfFN9GOKnzX8)bIszd2&c2h!Vf2pe|o zbjvMryMSnNdykl1$ND4%=e+KVpzbFijb|W+Q0^Jl?@?(;!kR~d@B2YF?LSTnFdk14mwUcwr$(C zZQHhOb&@}}-LY-kwr%$0taCAI&RMf&uD_e_Vy~)QyXt)&9n*^ke)J2CK!1?`S~%JO z(|Nt0q8Wq*0;2gJ3rE4w$>l$%6jN%_&e);|-)i#GdfLqnm#c@e4kkm-VFJAf#*j?I zPA1xsID#H#B(jIDk{uOGKlJDQQA9f&PN#4xlnM@qgMR(|R~QuD25M=oc*qw(m!^_C^sK^0Q|=ezNnC0cs^Y~I%2Te^m_p` zd(?Ag^f4iNyUdGLg8LyLg9w>>fLV%sUeI3#v$XT>ESO5Oo?9-(KD|Ji%nPoK>T=5cDxJX*tB>@gxQI)?qL1I41#}H9UT$~87U6BLaKvMuOyT*e_}s_nuTsL zP~Zzf9~;uBF%mb*wJ-Evf$N(@><w6UzA4n!26B9x!)N*+8EI_l6p`6zN=p|r)fy2ZC53ay1GdLx<8dlCw-)Wx@9 z^G+NTpG``yw4GP#)`@O+pNxv-%3n;57mnd7mm283X?aneu9wW`;P9L#E-w7 zx>E$vOM~eAN*(5u>m?iAQ=|$6Nq7EL{-L!>OB#y;OZm)GKq|OKw9a{t@%#7x7#t?n z?ujaX7N|Cx|IM5J|2#ncciY*L4}^=((9*wbOiBADFDd}-hPCptE`W3b47LipK8kkZ zPb`UgR3~70ex16yv%@KHHVB!3Bw2vK4I~5!HlKa1zws9XtpE~1NCtw^uCxTCqy)pz z&ds#3HQGTezBXWOm^p+Z0?pG&76<>eDO#NfY*rRKcpVk!J90Sp^o&&x>sKP z?Sl#Yto`_XdAU3DJ3fBD-qe3%L@lo{FJDWPd?^0TQ9aS|uS5JW!f!jp!_WV$)B znWS;piY!NM$YlYXVUu8(OzrN5P_+8H@Tz1pibS3g?VkM@ zlNAS`Ia-Fbn_@Yn@k^Ux!96Lm^5k)!VLXJAs~AG@Ea0J(_9|jJ)G5s(KY17c!7CJ} z|8;UW_Ng(PD*9Vs$G^$B9*c`PP)ySZjn@XC_rS`<}&rA47>qxO0*rI6eTrJ_)~t z^T@3&n6rJ8(ET1{+GVsMZM&9H)BIsM8x(S1H=HMxF#9Cog!^!%?H`En^?345!j8)* z4DLMs^fzbxL2sK-wI)^%V;Iey*KQERd{Duo&OG{ z`vmXrH_-4;LiSqIvGzDjSKb!C!BlLVd$I7&$jHug!td3D77oi?HLSK&%C;p!dt)O1 z{e%|taqh!xpSB6Y8@F-q!!89JPyCyauUW9S0A)Ukj@Bl4EwUHv`b_sn!sA8YZl1~b z_}emZPkp?Di36Dt`L}y!y=i10&p()3HlwAlc*l1T>0K~sUGfhB<~aA7dKb=<7bl*w z{O+UZ7(>xjT=aKG=roo+hrm7ip`DYHK0}21M?VzJ>lK`P_A1uehXL#`p9R95{ZxDG zd))BPYC`{$=FRkK^(_1!+r;zh|ve#S_7_wA~0LaaP-F*R~Jr)sJCB z-x2()zauL9yUhfRy7CZl=dK+>`8QEEk74dA8Oz(zuXN608==RWC@y1OL5o?pfA5Y~ zy|grX6K4}9G_$>3b#6~!9tI5NfW3wnvw&?7cjG2*lh9&gkTnc%?Acm9r8IkG2KU9y zv)MxkxUjuRrGm2qwDVYV5M_C)+W3%QJ+*zLJFljI^(AJjNHAo#BiIpaBm?S;HnAmh zKsC%N*&CTqZ0nv(nXj0j!;Vs-B*vrlO{z#;Q;EQra<_?JeEiR!jH8 z#nfo5H5yBQom7JX4H@f#lCUWS9QB>%IYvu)bMN{8l#_A z-RslMvE2&6?#8{P#T{eV8dq>{Z)?{w(T(o|LC0`^KdJnCS3oh9Pr!a3R~f*5S(_X|!#Qn=0_m_Tt4fq=>gUtf(0fgyE zBpM^!3<4bEUOs5PuOCvRx5{pW4gk@)_ZOZ=`(x>79&hB9djOdgOEwy)Iq25{(gu|S zZDlz#3RQem;-h>BrX+*Z(yE)iNM#*#c*d_-*+5_A*5oP7;lscLE`8@L4Z);$v|5t7 z$MXCJti>BxV|R~Sc5(rEc)gvhML%V>Klfu zA2Rp7g1ItIDZ-UFSv{!cTOg0UmBmzRYS`S;aT8C5-81(RQR`CAlzKa>!uoNv2P=Z@ zx>BqRU{LVQO6Ke~bp3OWfkRCTRS(lUjk}3nD=mS+WjZ|ddmf`lQUE$`@-@pb$o1SDlH-0rfoJE=@VJtv@_uiCJGyxUYV;x@YP<}u0VmT zRD#rrFGIEf;-3g}Htw9p-YYRQ518s>FQfhGMki27{a}K>Ab1psd_?&7j>RWIs?VX@ zo&>_Dzd7%H7cOpKcC>KcbX@)OaS8xDE7eNeJ7K+L1RravYo#-10J+r7VV(a4ANZ6| zLn4D6MJS8ud$BJD?sc6}jr9ZZoa-RD54@3qJa3*5-zbm4!#v_I;Xf)15FKk&8O^)` zeq{BeC}-bD;d*qbv*-bFSo?(eOeAaI-dme&3Wo-Z^8SB$o9;ue88ffq(slEfHsvyq>M(LV?!+^Q;R$NMnmZ@&v+ll;O6ea zBw{jq1JlWenhW6*niMRGcgf2+eX+TDo8%$&@A>OjRX$8}$MKC>abS@6_qQoU@Kg_SrH!O!|Jkk_mf~&SLWe9U$ za53_^BV%u)LcPUfHVrQuj~5+OR@eCh%Ni4bnTi`}*HMKuQjQ=!x#<_VsOqw+PYQJd z=?t)C4`ANgJG4%@_0b;-Z~$#DV$L?_bq}ndD9dq6Snupx>+Qp}XsIp!Wzr={K7f!a zV0Q$cGC3v-IZ7ZSz?nPVnp;TB-#~=)%A^ID8UAjei82r*s`dyWw^%`2cbbIxAaa>S zb5yGJVBxF6QY2pST@(0+n+5Zx#Xywq3|(HHlBZ|!-W}Xr%w4LpiMc207%RocL)MSdG!x?Kig5{1VmQP zjZB|FX#L)mg5a94U9{Zgd`NbnLMjX^WS7m|C z5LTJv6j|8>x|EQg@yP$sAMP!Xl0PL}<~@dw{R~Fu&>Sh1{xH>`ni8IxDVZTfWgNwl zXiLSCYD)pLgeiGIA-#k{3&)tEw7h*VHf4Ygs)Tw+9z@ z&Xqlap6lt~?5Q!qg!UGt*p%Q#lLT6&Ij+xV3vjG+DoKkb7%a-NilHSQq6En4h{k;` z&tYnuMU?Fu3mcvJw0qNJ+jbn#96-N7@zbp;Ksu@A+!@_$wLnYnkmoj!+eO)*BgcyDBG?JXOa=dEqw#k8`&Z^~{1 z(J!bpEZQjJmJUN_@Uc)ERve^PI;Vw`J6wpc{_dO43s}u9NU&xN{lxWPSdu)qQ1)EZ z(L3N#o>G4(=*YGSf^F(mfms?0`;7isqtr%V!E)%7j!G&NQc5b6qGg*zR5Pt8TL9%p z25v{|VdL%V^*lH#F4B^Xm|*_ za5w|l95mB(8}E|@czOvc06tyQF%Grg}tV^DgjSov>oelczOJG*$IDzHe`Q1hC zU6ggFCpv_D)o1He*Dr68GWMW>WefKxaViWusriDEz*|gJUguQQD=K9in_9v38W<@h zW?JTkCjk%%^Bzb>xp+Xd^Dn)8CEiD&%?k48p@M6J)gaXaSyHz%*e9qJ+|9#gP%C#HwT9G<5MFc8^I(E{7nt19tj;<8U^ zUYjcXrgHyecnvGa+ioabXHiZfoMr2jvVv?V#^ln@EuUh_bB4Mjq`^eIrvKD)xiO;^ z=2T8UGiPnxL2~rxZ28_oNJoPoBO|ImFKAQvoFCw&npMTCx@;=c7mGVz)xl(GR}{?zE%}k zNbcefe-U8meTgj!Xv$|bNe2_;94p75`2u>fp%~0NENl18WZ$*do7-#XW+AufPS&z@QnLD+FUj?2pH*?FxasJ?v3%FCtDUHq<;?-`dMx%`VL~WeP9+ zzKnE^tl}PcEUQOFoM`64YK`I-LUO@aVSe-_cSfe8zA8UkC;G4=n%JUyHb4}qN0gzV zzKJYDBWaySmN_lYTvjcI5bfp1snReT-xl-*Pdme%wW242rAtz^AhFUDapB_#&-iH3 z7!*V;>@W7FjK@YdUu;E=rkB!cLL3z>xLh>u^W@(594XFT&EC1lkr*R8+29$F*H_OX z6EEXjJY2;>kd6!_XXqI;-f=~i*SX$Y50$7H=B1CBN6P_7c#VYpzbN#C+Wu}4AARS2 zPVNZetM^_x?!(LPO}a9!E|o8O%)Xdvbdc!VI}nLaHXmzHw9w#fK!Zkrb7mefQ;o68 zJmYuJQDSr0tHS(}<#}O2)i02nVE3mi6KxR4G(P$VhjD`Nd-X~I2H3@P0LE$S!D%uG z`oGhIh+nkl)t0E()y~2`70l~#Zz3OzS&yR)ST=9c<~_gUFMsB~FI0h3S5Ja|kj_u& zoqt7z6sayU-aAU0!l*uamHhs=*7JwLRt%a68uR~R;8aXd#OYql-T&YoVv`^v!&g_< z`|eCCx>Pg`62Ugi{c5uDFq|=E`hZ5FjrgVwM*U0wF>Wq^v^F;1wC+Z_W*9q11#9vB zl4YN{mXc4i2g_s&i2X8YDWoCjaT_~FMqrdvQ*};;xW+GTmx5j8pvq0RzzLnlSg@N1f zakJZ+sB2iuX6a9~xM16+j2FK>WE%N%8l03I&wSf%EB|C%QM<3CsA^YUsQ#G%5R6|5 zzs}n9{+RxWFZb(*{kSrCFmJ^6P3ge5g{EzX<%M)=Ehipt^79R&(x|%{2jlx}MsmUc zE&e-j5Fh;B-AXS|GJ8hOOA{elbuecwAWlXOKX#UDa6K7U-O2xkfccw@b5tg4Z??4V z+?PRP;15}$wVe{JeEb-YkS`raZE1C*5u5fFst!#EXQ-&Gp|SE%rSDmH_6nnRfgb4e zrfIm3zNpl;hQ>xH6tiAhRTEopjx$p2v2Exv=9KCX; z;Tp{)k;q0%T?46^gB18`SWaf@_T++ha-f$ADAijqD1J^@mHP5hb(wXQW?g%IT}73b zr>W?SFp=-n6|UVAXMMawi>-7+?v@*PzbtJXIqqH^>2{3<6%s87o+NCxiOHimCZ8z& zqIvp#ff}J-#+apl>|BAnjN$!-UZHc7^Y10Y8D9tPRc-PryUhlab7PawkBpSQ+bCI7 zpKS^+K1&oYYb88I5V-f~6u=2G1ZgqlS6~AvLiHy!mbIklz)!H1)wJmXi+Xtpu%wd8 zUSm@hD$ZNMFa&Q}fMhZ1FJu9s9Go?px}*mH67S?(A4dIPQ&nyTZ1ie#l?_NU{&jZk zMXMLF-qrDdtIt_BAcxTLwB4rV<2zMFg>KdyZY(ml;0(xhTV<6kTPOmqQ1`<4XVakm z6=|Ld%fM~Pk*DK8($x7K_&X!Oqlw%U8Z~!-eC;nIaOEw)n6`6z1rhcPYXXR$A6Ki5 z78*@OcaD8?6|qmyMm36X&Y^%bq<50(Cb%jZ7x#k*OLjELx`FmzY7d#DRI<`=!dE+e zHt1kC#iC#6zWg|I#G2ymvwH&bI|+m5F@e+0;z?|S_MeBbLxEW?doI|j-=3L&pr3sI zQ5ojd3-^pHLzKNVL~7c6@dzX5Aau{1Ryr-b#5do2o8A~sm{v=?(`wBS9R~D>VH;Am zJP}L6ssD2^nn(-_3zo?Dvd;*F{k?2d`kN74ky}Prc2aP4bv1F&VMPBPs zRXyLRk$u(1DjrJ2qVMXq>ImL%d+a3er>(5bN5-${h8MbfvQJ0T}G zyKl-|A#QxRVoSdIt1t9t3{Zlb!?O2;^Q#R7K78rp@IIreFPuQBH|hP69nvPGyXn$} zq3O3iII$pAHi^M^ZKs~`DBpw=RlL^ zMB^YRlQPr2Mx)YZhF?VZaz+!2DtSE#J=} zDAUNuwvI8vP!X&+s0bc^S)j;^=>vS-F)E5qmWPWh%sq@0K%u3Mv@RLEIy=XEryFGI zhaJ=Ha*EBRiK6*>=p{uCvHcl*cSR;X*XVn=54NmzftTKU`)8jHt08q+Q~1IGdqY+n z>b#^7%yVAnr=g0;+j1dli9>zlXyZMDu-N2MZ(pwRW$BJ4Yn=x++t(X^GQr6kyU@Y> zuoyS?hUXuHK9_7UkmnExy@=8|(`8AaR}kBUFjcrgf`4?y=vk?C4X2(`+M>(29&3%4 zdN5}IV@7Rb(y_GEWwC~%w|>?ef5vD(V#4b{*U^*~j5r~o`G@D%o+zmIY}4=n?-;}}$HvX)l##o@ z7{+ntjAaQqV@OpTQ`VOrZTFZzHEgj>IL;+<4?{&JJjoB6J8QfP;({FWvDiA6F@cP3 zMkbM!G*d5ptq^+o@wk7aPLstCv6Nv=ER7VWkn62_mXm;pd54`Gn%n zI!FUtSwL2DjhgsD`E!EeIU)Nz!l=$TD@soY^XU68K7+K*QV|OD=R6;oVEb#&ydSX! z?_49MwF5rrctv_37TwP*z2qO7e`E1yi&+F0?e;}AmPt-x5>gSWodvjbmI+A%Z>!W{ z7^Q&Iuxc%)t(zNxJjak4XjO?R44<%|)jLO&#C^x8(O(mS3DHERuYd(iC1lxv-lJa^ zCgEQYLio!uSWjc{mD-9OD3Jd9GXSXe21+@IN?)CMEGJ>RLLq(AFm5z8Us86F^VFR7ge{&db#g|j=90#Swo zEE7sTk#fY;UREDn@#Czc)7l^B5?H!;wX{U1-4LJXC_4sPGU|tpA2VvM&y)+~opv+b z&~l(@X~bIebkZrIHs@b*R}L;>^@i`c8b2>krPEklis?^j&Ruqy(`rQg1uXsjPRHfS zpS>(v+Q23F2tPmPb5X!tQ@989zB{%pU21u^0&gYvdtucAG3jEwDUvf%J0lG)r!y!^ zm(z&-2(=BT#wK+krTjUu1#+sBZhci$wx-c{5wRUF z4uE=x^{iM)4uQ+?^}RLdsJG5iMbQz>c5&ns{!GA0&=lS5q6kD~=}08bf2awcRtGs# zLj#Csp~XMxg+S0sPx$4botC+`x(9L}k?4X2!p;B83AE7$NA&L6`V>9%>pe0Jwn0ro zKC1kYMdt>>2s?K7KkV>n$ps)$;Ala@uI7NP;`e|Q9jk!@ySU=1zn{L`*N-(-BLAV8r zkzKik4U(R-ufaYtwg8H6hIl#BY%f^T`;T6|1zP6l?@Y{8i#o^bFtsq=?i877(FV#3 z=?bQh9fZuLw)98LB_GK`&Dz^fPE<^z34hCw4`(W7kzE8+DP}vyOPV4$|0^O3?-U$@ zy~#p5Ek=@pY64$&Rrz3(7H=jU%=l@>MI7ebYE1p<=(JbRQ?ggzZm&lRZE#Un1K%INegt|_m4?QDbc#cErFuKDV#)i{=|^*|b0 z-3BL;VqeMIHSW;YEHdnU8HS4SyiK10dsawkiPFc|o;}Vd;mHvJ>gfFT;YAZ6mv8_mRA_H+U_FI!|$N&@T^{)7zJ1%_-ShdT6fO zc4Hx(d>raC%RoYC(W)8YnNx;^9(v+T(PnA747P{QSA!}Ik=+axe?uZ%;gw&y>cJ5D=UBb*VG zvgzv-O~R>y&Oo1-}YLNPxrPa;+l?1v=rFt=WT%}&2o!NZ)NEEi8IhpA}WzfKWoJM`#7>zKVkAGIq?jr1sf8OMWPTiDL zlFA;?&DeRa0k@lN`96M~Hg46;c~{MSoI=DG58n=KW7J$l=2u16Xfs!*Wv<*L9?XqM zZsTF?fL_HH5ZJ`bQSRUHf^W}S8=!BEKI`%rZ~{}r44C2+v0BNQ36@NFJc?;uGyMqP z`W8phdARE)_M5ottZS~vmv!+*^u{++px_;ICma)wJ8;%>ac-DaTX?ue%8yM@O;9=P zMP31#39J*w{R?S;Qc8&(arUe6{>_~$)Hbj)?^9dq&o)ixk)-nA%*oG8PB=%aaF;DR zrqZ@7H6)}HA48B6f1IOF+R0d%Xl%5F=(oFfUKb^o%WV~7Ud@2x@$(XG(luB&Q%kq@ ztf+?Du13eQi9*a~nYrY?(M3J1ME}f@zv0I_!@iCXpWsu0(EkuBylG{TR+P(M5pmAo zSmZY$C%lnuQCoWTZi+7kuUs-)e>7Jwvm5(*MMN)zdL8(6>%m4br$Wik2uSmw5hZwN zP5X$7^mwS1L&gqj@zK_U@2IdP!~jaGYA5q~sATM@V-5H4bFb1$U76MJ6b3VuTV`MDj=R#qvil3Ke>z zyY{s@S$h*R`vR*E6LS0CnN5UOm~E9K0JtT2u3e}lc~8XD!9o|{x@)`3u{b3P<)5*w zP`;S!ztk4WB2V?eTXcOkJNxO+D1OmLXb+ku}j8XCM?Sk4Na6%{i@FOOfQ$pf=8%Gc{L%bqU7a*u1% z+s$20j z?_XaRcsq2oprw8s7m=Fz-WEa)vFjbm|CNgmT)g}KglIn%To%(H4u;^?!L&JP=Skn) z=lg5_FL)8xJ}%UBRT$Izj2+App<2{~Mxx^fu5JnMlpnD6t&n^L(R&4QIWh1SpXZ9t z`wsNuY5YELG~OM5=0&oBI)jy)yPaRyZb&D)lnvn~(8>sixjG+mW@tfT^~kOO9s3T5n8 zqbJ6b2Z_iggqA=kF4&>NA{AF$c&LalV@&WcGS9=AQ*ro{>qhi*EdNh|NV zm+fDQ)*$Y_++h${qrfKVbr>L5K>Y077_UnlcOTEffvrnmXPN7!NPjL=>!jgSb-mbi zJH)iSfqAGfA%8m>L2SWvJ?u+nhTjQHl8n3 z7Vbi(ju8;cwjP>eAz`A}3C~y+YR#__A^Kq=#V^w_ib=~ieR3*M>C3LfzqKm)h^adG z^bcg(xR@0<4QXjFzl;mxN@IuO^bCL(dS}7=_Zf(HK&rRt%=fWsgo>hw^jM?{n-ri` zban|@cZ&CH?0Cf(m4`Tss@@(nbI?quzC+D})0vJZ-dLX$8S%`CcJlSelRPmz@REB| ziq4}+SM9~_CF52=?^XZy{D+*Ji+6>}0nqLgy*tV1gxLEJ#J`8e2>pvXtr=FGZADnG znOYddEcUcSU1L7+;%~sl+BBEl+SxP%T-_+E%n8pZbgL^_i{4d&tmoy{Po1wIz4-^1 z!It>jy6XXxTjYW(gLhGtjBe2Ml6+9xTzHhp)?=b==wf`oOs>`%y5-U}JuObxBbC0# zJkYx~`Wt9oe@pr;hh{?W-`vrrxWCur(I=&DLnf2%^vGGp$R9mVF{{a%M99VYA@X~T zv@S`Pd>?&3k)*>DtL6-2-by%zRQ&Q-pB9<2v>1ngLrR8xSB8AJ+x*I1VRhwv9ngn1 z0M(&a+RVZktS7i_Vk}A`c9}BU-0wA+6AAph=2>?WYF*MVm*`C)_X!}^F*Yv|@Uy(j zJahycUn=(~D(t`o#J6f@kix6t2YU2lg!7k4>|A2-e^6Uz5Hoz z5?V&>+SKuqTgFq5Z0~73J3c|Zc2ea>;g6E;i9J(4HMVK_Q(r3U?txi_-OAVQ12TwT z%8^Z5Dw5;Nu-?ilQsXP(?xC7635h?RL=TVUwYba}2J0X_+FBW8OyG+{ii@kwTAhYC8Jbo zNO@gQwT!Ykc%ATHIkPM1{aro>MNU|7yTXrt5VV4+iJ2DRH0h*CqnJ2456^wyuCDdVNbU_z|;2Tu!L0SB}^?xw`tX?gDaRX@ z=OK5az&Bw0@S;(oTaeX*4{QeRURH~ArvT2wZrB;-8u#9v>*s#P6DP7UrAcy*!zohT zmDU{*z%BcU>l=*=$*q@g z8^L}(cX9YaC3uuGi*H;0UuUiqXAmkB{yD&C&;HD6b7LPDw?lU6VSW(hO=Y$uSavfI&bq5L8GfI7qQAQUU@9tfztm7M4X|X1X*JLwfqOjE9A`tzfmU zmUX4ChPAd}bu*8ZL`1QwwW^IPetTud#`$&ohA#e=u6xHuHR@-s^Noxd8M=hh7veSV z3HQ!h_l?hu_aEnznJm%*tnb9=_0yjF+t+N2uNIHb_-M3+4LazBf9+@gPH}-h{L#Cl zd)()|A0gPYOJOt&oR}NnDs9{wg17sEoqy4&8xDzrU8-=*h%T%hnXy+w%R9}kvrDfGX$EjZQl_63(c20HTi@#P`CxhuLMOHwlc`Z^P;RK|bgudXyEv%nT_2G2$naQ$UYzRMg=lon zj#GS6l4{-VS%tc_>WbB3&QWUI=d?~k)$gp({R387+X7djn#35XPzznc-Q`&(6o& z4BN4Uw0J8AG7H)t!jT4O?fqOy;Ie7_Q}i32y@ocz~NduljQS0(vpir0M|| z;uCFH+V{%I+D{tr&$4nGD;1)FPNrmoK?$s9o9_l3X&O@|^xE@dhZZrvNPIDht9xp- z<6SMb2zAd_YeTu3G!!Of^N8oLL;6xBzpNFfb}GHSzqEv4M7=tFwOC3Z-oM(}r|_4> zjeIj^`%(O5R*<~fc{?VUPAvi@NHWC0mTR?{^yEk}f5KVMU5R!jZ>dMWUQXwbhp$8Y z82V5s8fMQ-N7FJJ4!eI5DaWBfTPalR%QJOXX^F9&j7{zLMvNb><}rbeoXzfkU?If% z&WS+hVBFM+A-7H5(XcVl;{F3`CNOlL=u9f=?CF{5CI9V_7}EoNz0UH7Ys zi#l<5^^vj@9bXFiW*{}XpU`InqmRUUA|PQzWuQbFeLY$Tps~imj|B7lYoI_OkRywX zz6xI=_P|Re={=aaNRu>Y?Y#Mdhycw@h{oG2^g{vPYkspr_F^CrO%rEKNv8!3(nSRJ zKw!)GK>{WinjhJfX~q7Jlciq|UVJKzc00<@UY-vPa(qT&6ILA{iv~UE%v73&-$^D5 z6=X#720neNvz{Dde&dN^VY?3E?L&U7=2^mjvbGjJ>blTDMOu;81+vPO>$MPU4wNdp zL@=emB}vNsOf*HWz`nu-d#lDNjSu)lg|lBVMX!{7w~1Fb~tZ@{7H=^1d?0C(s+8rHV=EA;JIIqZ6a=+yh-~UqK zgYW;R+^|y|>&cc@q$k&MD+JOMRaEdonTo$RSM60W1%H2LH`BCAZ$!nXZGM=={?`yX z^ck-fFbyUBz8KE0n@g>S4B#ZT|3_5nyKo#DgM zaa1rR{1w*c-iW+fY^j2+Qr+@wD(FDcFJz2cPMDgf>YhD>x;yV)(~nVD8m=zQO+rVV zK^8X<`YxZp_Xl7s=B$egNo1hEUkL55d}RMHLFFH=KgNItZSQJtAm0hSHRM87ber=0rG!WK zlqImIsFpxMZq(3w;pnj?B&jiFoCUWSyF%HVvGv|kv`7*rAO0&gX;^@eeF~s+-mTPdMe|tNTF0ZuV5y9B z5alWn!>np0KG<1u*-m1grKZd0B29{x(sZ@IrG~DuKQ3JN9E)MA!m%H~(^9}n%s(rL zu6nfC^SuCtQ;qhg#9AVABBOwqu(g8f+&^0)g>vAJ*WY-=A=rfJcyhm*5PTXo=d&EQ zcH%$1M>E5;&C?sJ7>4#h0>czeiPTjAi=jlR@&AkpmA5tHU#$FyY)Z^^VMqCXHxUnj zBUw~aruVm;dkZ;`dpxG;9M>y04!?q#xjJ%FXKRVu(>MwE4Mq&KI$#Xw z?`t9^JPJ&3!jp2#qD+ zUr4-;&V19r9~0>49;3&7x}`&(`dPR?=A=aX?_5L{be;C+6cAu*F*Bz()zx& zwb?qu-Ze?Ee6}NT&X{$$Dj*BeQ9AnPmBQ>*Of`iTm~wZ1R9JN^wr}?S%e)=v-|ZD5 zGni_proCG|!fFSX!%~Aygbp*$V~F#74y1-Te6ks#hw@l79no0SD|MkP3scMM;g8=m%HK0~D25=dC3>pPS{9Vnihx;MFY zX}j*sYex1RK{uz}t^%V76qHv%r{4;C7X)o#qp?vm`5qNA!YpHQwkfJeED~?3#GTM2 zv4-QN_6nF?0<11-_q6O53thADV{^Wbk`B|QqAvUfYzCE&l)aY;=fLlcp(;tV#ThiY z${{MsvgKsj(oa9{ZXoz-g(_C)sFs4!lz1tMy(0L6TN2J0FDLrEuNhEXR8?|HESusg z3^TUDCZ=vV!pdgl`UBj?c=7EER>oxQXK~)!4mlb>5CDfLB66F*!W0PU-kXxt%nE{%5KHOy#a1OJmiRIy3RZ+>j7{ev0 zQRRsucMqru06r#cI~iXALIss3;E3aRT@5uBLjGMIDLP9u$G;%+PTKjUUl#YBvOJ%8 zSm<9$_l|0*ago0x0cRT;iLiJ4=k~^ zDiK5JlPZ⋘O7n=^R@K*@;NB30!C&+F26$90vHm#`~Nit~w3+zgr-`shaS@33xjUQXtt%F{+1+ zH|k(AMi>R~5gJ(K7l`tTOMp3X^1^IYiAbZ?a1^j28`&(fJ=mtXAle-1g_w-uD5N&Nr8*gFPC8a7(HGqE$VZQGhmY}@MCNyo{=?AV;xwr$(C?PTU; z*RHSX?Dy2E{r&0xPv1{fcdfPV>-w1tEZ`sc^MqSMkdh|Mqu`AGmitRc`GkpahS!Pa zAqDO;>4T*g?z0ghf+(--6l-U)HGej2iy{N-_?6{z;D(^;yE&w{9Yg2>#&D=5gKlwc zWm$()>=L5#{Ey1!*`~6Hs8WX-CrzlSHh|mI)qKNu3ZpPi)t}Y|9l?Nt4ux1 zl{dtgenyZs%>>`iu6-JvWD(vyjDsCInw081@2>QS7u>P~ZS8*7hJG8j|478#nl4SU@b3yxIy& zUW=w)ievzP1V4Rex17#n@FGN9#VP1aQH(GUyMI(+HyEAs!FBN?BiN;I|HFcM=9X~z z>~bV9hH^S)v^2O)$`|qMH{&N=|FrjyvgL~K3&<2S@~b1TWn2NmrT}gZW zJbHiMTE6CL|93x}{jKKrUpX->-`u?vcM(SWGUl7)2!mbQ)(`DLz88?LFJ1q3sXNv+ z#*UA-;&0^JI9);Y8?Q8gs9#Sv-3v0&->`oVdkVCaJKd@uIR7g5{BF8w1$+N_+?H-HrdRS4e6~u|dtd zu>^&!QYRWkNfjkVnZL3o=(y#3+_+TsLu_*C#3^7*7;r=3`soF8rs;Wes&4_r_v1G( z$%#dyZ>O}Af5^<-6Q#x2RWJz{QcN^u*vE%cR#`L2`c2F<>HAT&l*(pwlcUE}Gv)79 zX^Y`J(yQkj8-5lt&HYN_(`TeAaUsnjPf0jbrWr3s*Cy9~LA8#F2<77g zNn#i&H+Pj5aTT~__)`j?a^wTJO;Jjs|G1vU0ro#qjl@8}Y~F;}?h%Bv*}5yun*dSm$EH6<32-O9YMuj$j>fE9Yi6uxubgmUH zbZw8>VY7tr*cjvGj}|8MYiN;-@Dgp0)NH5qQ3L3^QF{%Ko?hG=Y=)1PyG>;%Fx;uw zHH-@=Rj52pOQGv9Y!aff*l2_;sm%7O4KP-hMZafF=(KX#U@|rfV1q;mFc!U8_)Jjr zewXY1p(pzhnL4`_VO3-nlvhO9AkO5Ojbp{di87{3?UIHEim)Rq)0uS`*hvA1W&Mm4 zyD&raY_`34d=u{^EeYkN)6PR-TNwRD=#0sia!-lL5s2Zc6lew;Ieo~|9H-y4%QN1X~^47-#8rIy5y(Kf{1>W zZ#CGMpEHxGC$3&nQg=AC+POLku9ZNWK-H67We*WCjW*jzBT!U?9l|(=7%wzk-9cnp zkevldoGQXkA+Pl{)^pHZC`z9ia05Cu@w2GRic>Is=lDRl{cxfagv0odoZt+wU(cvF zLU|==32h|k)#!*S3pa-I%euB?xKb-bN@F3L zz_R!UE&<#7rcCjAVcI53DsNr?8MCm$NIn|X(do8VpmMD-isc2h*LE-#SA|W}D_Bp| z4fP>rx}g-~7YD28^wmoCXac0$QZrXwlVo;S6+yiLwaOqMqR|7jOSU*~qa`|V_G;K9 z91gClVzIcMsnZYZhHOp?^h6o1VD2IgI5Z~R{MlFZ-Ik@1yb^JiwUI|IHX#d_@3zV% z8u{?yn++?IP2bh+xJ`AqlA18yBE$^}BViDMA@r;wvkr0krtdPdkbs7ANo0)ET8q?R zHgGZEC;eGiPATrdDlsw!;8QhUUm3uAXcPwM(hEL4LKf)!YfI zQ4J>u*Lldj1!T!2t=3!-Hgnh@mTwO;#5TwLlwWIR5ednOJ(~y27pX|)c%rj_H=1+^ zW5A1tgc)iR$sh8g&?}r>bU{3}26fS>Yo^}aleRn8MN|-y83I#f3&`+|AaXruFx?qv zA>>Y&sOQ{RG^j{Qzmoq;l_OTq02ptw%NW5$LKO9`IDY%|D*7VTbV_lzZ+J2CPU&AkMx z7s=0hC=qX`uG+=GpP2#0ino_4^iSW!hakdOHG~q*TQCX8-ypAIT0WRy#(sXn20vLo zkP8&fDx6pb`1|oam1kqC^zxhNX2uFmbWaA*Bbaws08nl(WP5q#QsIXs*`?N0>>|6X* zlL$5=;QIU|LLHN6u2nrD0|^#wTQk>YuTRElFfB5e5Qh}}(M>qE$Cj5p>Mv zEa`N%W}sLf7Iz=Ps*ujGk1#pv`K~H=4{S7UF5BVWa3Oy`MMUIg8~n1~u%?#4jEuD2 zGE(g$Hz265Eq-x~56_ZQxyXK`jss}Y?|RNh87{aRa12-<#lHeded&HMn?T^JJRPY6 z(#||xGuSgv#?WIK_^3??qx&}l^{aS@G%4#SJuAe+s(N?WA(wu}j-2e;(ojSm?vtwQ z@-&Q&LJdl7F6F&jya(tqRB3p1(5G+KHahkYHc=HdykBz>0lFsfT!d$|(z%P=)=B0{ z-2A}6B@27bHDf@H0NFn?QBhlR)aa3`^m<`iXJ?c9wveGER|dL_bleObYirhNxDf40 z8+8nG=+ivpEz!3DWVC3_E8HtaRw@EYK17O|W0pe9C26xJ)eZS%_W3e-*8%w!rbax? zI|jTCUle)SNh&gM!A zj31fR!Lz_#;df?TxddTJpVG4#cdtJ}0kYp~k7?0Ej?bK==VjDRim{*1oNjBZ%RIrI-g%M3L+)J2hALAP+DY~> zSU1#^ncr1kr@HMhL|Rl&IuTkb`uHnM1== zr1eCSxJqyiFXX3iwcauEBV=3OtsSP6l`aUotHr*?w7CLoZ`bP zZ2h?FCqWRMuZEN1l7gPRqR&I9g;kg5No|O*6@}PYH)bc1kiQmt01>@js|Wn{xWAqB z7+R=DF?W8%nvs%G*wyauC6uIxyH7e z&ubMo-E+!^|6L%FZp!f&uX8#g`Zh%QHjaxqme>XCm*n~4M1d54=%%aZn2D}PTnN^( z$FWZ`zvr=Klz21xEVV2&CmhUboQG|1iQo5VXSBW5`onOL9S-$KUAm4ds!X%IVC&JVW1fK|hL#vfqZBH2db8ecN2$9e&vAV56pW#Lk@uda z+eBQOLM9Y_Nw-(XhKQ_A2meiN?EP#+#%M^br8{oQ?@gqF8yf1z{q8nsiYCH6>ZD#>E#>6EMIGeMi=FhdopcFFCpWCCl}M)| z4W-j@0pX<6*XR)V4@g*E_`}g6XWhafmj-T3=5T?Ryil>Uljp~2`UjZC<4^rin0q(= z-460vzPCfc-ejE#XJgA3AX-JoJI%E|xJ_;+KI+u7g{+y5?RwONmJ_P+GkxgySM ziaok_+8HpCB@!7Jr%Hc#3_z+CXN-bJQ#IAQB_3ch)B@W_`qA0FSO^$8-~Y&w{bA*P;ZWE_bfY{m7VX{TnzE-Ugl6;4E5~6 z>(D*i2c8XbpQ*=*aj)tFPP?w`w&L&!xa}xSt6$vl2W@8St?YYzSMb^ztj`wO;h8Sx z-g(8-H?!@C1;3cS-5boTbq{ZJBsRy?PcR+uRxnAU+Beqivn(d8cckeWO6U0=dF!eR z6A9#Kvw-k4FYeZUQ<_&p3 zMbtHTub&J+=wh%IkW1aKI9wIl_0-RaJk4&tCD<780A`yDb&bE=)vnG*W;n2+FO7{~ z)HJhtV-OBMJK%X&k5(53mjiZ zNiv4Zv+F=~e!YKT`EpEjX%40)0{Q2s@kgQ6vQdsOsS4C(j)xox|2;^srUxiMg;h|d zgL!yDO;d_a9lK%e^6=YNd;6O|?+*su1l}~8?G$fR+?L-t^34v@@3mKqJkqB(F>ESk zzVWjgeOuB+D#EkDj@iTuSJ&_LhbV&Nr*rb;L5AM5*iP=Hm+^m=!-R-TxCk402@Q1sJW)?5>U4oxgL0`IV2bB{+U;+`#QFL?{BHt!9cJbnVN?$^cuGth z)+(E3pF+_QLk+VP2lt-MB8T*i1XhPNewg6nuhuwB^-x3yAbI^xU_ijEixH|Hsk@sw z;wc79Y*9O$y)DEIREo<_4{~qUhgwM$)pWo!+T~Lc6Bk#Y8wh8T+@I2^lpy zDDVt65ZLFYxNjeS7OmHhGT;3~cnwhu?Vy8dL6EO~pgTN~)V4{;p{<%Zrm!G55(aS7nk}_e%jpPx+fWT{K#gcg|Zbr_UwlYNl#h`3W4= zEoa9?cO$}oHTw^;Y^+@8l(GE$_J0J0MRx3#hmhaCp?rbS|5ZAVP)jFJlK+@fjz;w+4@;iIM4gL=m)#FK`f^_>ny#X(B-6_L^fY)7 zJn{KlMV|Tk26Mm0xhJ7#50wSjeAp7F1^YPH>zmEj&s$yAkIzDWe<61qKj`;R_^Awa zMPWvnDPOo-5!)Tr2kuk%=1(an4Alqlr^4O8-;-WuW8IJvF~PWp+Z&+I2FQ>yidgo6 zsiI6u!p^)nX^TR$%4rUWbrnap1eEJ~CjUEpG0^`_(qn3oohpX}lGnq{$dt~`G+N>D z6>BXTX#(~qQ)O-h06;kE^0n_cxX=$BcF3#)r28RBaTqE}SR5WoQaB+r$LRBO%6A1O z8PS(H;a~Tyknw0hxn{c4*4aHcW+XCnw)-JvrpM?q$IgDWTlx#|2;dms%8co>$o56J z%P3LR-@>y(4L={k$wGr9j7jiV%p6J3Chu85x=nQtmQ zX4BO{wK;FHJHXKl;FmitVqdIL*8Kxu(QWBQ+(=`S-aV6AUvers7meunYcUCaI4AX8 z_)(q(e~TUoQg@4^sxsYd?*vhet2HjA*hLKTy2NZ-sm1lS%WR1w%$Exk; zym0XYrlnWR7hUHw5QFVzJd_^oQMpPiatt}fCy^UE{Z|YtzWg^;8lAHV0}To;qEo7s zFGlM(79p2$W%!|-8hRzVib=FO3Q-PTR?C@iz`iIbKvDK#zhfKKw`lkpdlSYn=J-s> zRvfv`nk;~bv~WfRwc;#p*!ec)hq1fW3C^%Ge~d&KVKHTDG9CA8mTGyP8n?$|_+qp+ z*V~;maA~O?0l6eu+{|2>4RQREx?X2R>42lwwAcS9*5Hi{$A^pX(V&ZG>CcfxeS!1z z-p*Xz!zd9f;+)2)s6q@lJC*)k0N`TKNeR#TcG~34iWpE+BzgWU0RKNs#MjG z+DZo;i46k=CpG1JP4OC~Zr9r75sQNPKy-FK zDIcNZ19y2dc5Bh;6HbBHv(sR?I_JDnOo5PEYI0;`{_Cg@oR@0+M*QL36AL;C=4=BPO>4x_t9wNl^a`!^k!5p|5f1Xxkj4xU`Vx_S z)fa1`-Wk0T+xh#m`!5+FVc~WH#D?^SLYy)QzOx#_F87;{4z$HT2&hjyxwiJ7%e1^l z`xw0Fv#n7C=L}hN?K@Sd4k^gGNW&+uw-w0Fk?57hwV@g54@ST?X0*?op7%$^L{!slVOYosRcg}Nck>p5<{&Yr!H5k?I{hm91V{uQ&&eKOWYw{K_oPulYb+}rohl9MaIs}&ow z`Bm%jmRIWQhv)k@hv+hYgfUzo0=eQ8DM$~|3zrtbvwrbvqY2jyfn!|*JW`Pua)m`tiF&)g0eSs<8(wZo2Y%x4rS+kK`G=>_ty!VL?Q@^&1fs?_=P zZZc}fqce@FGP@KZgiIO$kh|7XB6@8+vsDpO6pG`Wu&lLtG#`mIJoQ>+48+%wlVYFc z*_r>*3E|wD+mXTt?mBeBQnH^QnE!j~SBxssVtj^LGO;HY`>o>nitq^|LSZG-3}IU* zA{$K_8H-XBgJpQnZ9A71J}Pr?%I*iT(nFaWJ!fz$Sel>YC9!Aee?X^v@4i$vh#}Rq z*R)QJDqd(A0r%xS{h#Oa*K&FK-Ude0WUw%M9GlDe+7{L{u7pPL9VrOF|>(@Txp@xC}>^T zs9uq$CevsoCP%EC)TFLczNBsKvnWyixcTPmf9N+gaRG2+lj7WY{_OsAI^NhlAIMh(5BV z>Yq90G#d00Qe?zpW1Q&NEy<)O6-z%Xf}LJTT#6rp9v*)l#ATdg5r!r0D;|*4N3Zil z__1a7P5JfHQ7tJY*_6_y<`T=;R|BO~DNT#qOEZf+O9N;-*+EE2fHvt+4<+Hy@piHf z7jm^S_WsCIm&6JDO64SkGPU&6UIbq8bS*;tSIwm8`#sai&kB;dp0hW)3?7nvrqvGkW-5oDly@I*HAfsA$^0H>!`$q?2Q&HG?EYZBmB+t;T z4S;GB$GM{bx@D!!GGCtfpZe93Od*{Zu7X`vv^!&QmuTVC*pLcn(zxicrXB74_-*K@ z(zC#DT0Y&0iym&5y)6{g!SBRzZ&U6P=KQVcQpL=%e^8?F&i7@F zW(5^37EG+I+%KP^TRQ(pksD#*VD=BBKdBi9;uOXbr4ifT`570Ka&DmFu;%-GGTRCy zD1X;MaVCu55I@2uX^KNfg%7B8Z+tO-Q>9(i#?-P7F~(e_W5v6Cd*i`D8k$#?QPiMH zP3J8TU<1?_QuQ_XF4DRqrBfVCBZLIrY6W2wuD%-mE1E|Jjmp5b){}h~Ar)h`S#WLQ z!kHC!6hWefC!O@PC?!N3TgfK>TRt*0c~oW3$txN%A!*maFp|gb`D1qyJ|s#NXqv)% zEp415fxnbaDLbZ&AM`ozu`(04V{-XV!PErLF6?~LdWB>Hlqz+`oW!{68I&dq$51p^ z7_?ru@Xpma-pP``TPZHSM2@e|>rw4w>r9PmLO`%2^bT$#YO8IY5spf4)GQAR)5u?` z)4H&}fdiV7=Ppi=e2ZxfLNNy6-1`1e^Ar)W_cRm)IqOn;9jE&=SvFZ1Tv+n8qp&x{mmq%b8G;Ym#J`yBeX{wIchSbCI)Y(ly>z7wkPs8 zsUo{Z>>k{2NDV=S@A$Y(C?iTq%I?IlOFX6xmJMe%6`rs@8Ui*lIqI$sDqJU_>ePWr zG--bqrsNHk<<(Irzz>`=D8kDQMPPp~k@WH_c^eecz4Q!Hi3#fcz^!jG)}pc&Askox zDN~bVeI+WKfkIsl9qmF>l%RDxa;HUBoFZVuXhZ+#C9|f4R+lAJNMlbwUUtPt7CZK4 zc`9zzoxl}Zhr%h!`z8EHgrx8HEf0k2H9z5jvQJ+$gkawg6`Hvb&@C`xvr|2l2Kh>A zShf&yIaDp4NVqhL3l&+bzJ-e~&rv2-&CkdQE#BCAOB9}SwE1(EyX1>=%}#Sq0Fqi>DgcSec`c~#e*AmKeTvAC z)2ttyml-EOkgt@RMGMr#T$W3Ucz66RCC-+d4u@`NPF*Q?nhl#ogX9YP8oHJLT8j_> z`;x`4LoUUx!@qXzac=al;RI3iz~@DAU6$kuQ%R{~Lti(wsYAtG^SL#GUliiDcEM&? zvlv)_{jU1R)b*?b596#uExU%Kb0NEilrsn8F7~pu`~9OPUNN(9yGH(diAQ@y=XRlhNJvNF@XU0bT=)J}NX7BS_EjSq zoP1p~q@84IP%G@Ve4zi#QUIl0d3jS&;f9NkSD5@3n%isc5AG)OAid+lK{&UM#@(~@ z4MZG*sS!j&VeH{GxHi8OUy{m{?d4a?KR)rW)Hgh%8($4GOG<^!ueW3#RZ;Vl%1=4P z0O=1#W+JM;1v30)PN)GLawPpNo$WqxMI_o0N!pq@*?yqh?P1>zEB1iy6w82S4OSgj zJGK34CEYkYW41I-g>XMp1Qkz3K!SNO9B)B_7-u=RqkX@sG_k^~O2bm=R3XT-G!P5U z6l+T7z&I87e554}x(k1c=hU+@xV46N&78=*Nf+wjUKUkzyb)a*RnqO$C&OfZQzMa6 zO2iLQcjBN>7+pqJbF_@mx&1*2z`AEcaErZX+)edqlbb=W&MTGncVLxOf#dO|5gYp= zV(reeBi=hT1yo{i6k-!O?Txa!nL|AIvNp$>9cR?0n5zqRAw;brs}IgZ^Jiku2mWMV zTkVZbLmffJd5G$fYPlu}FJFHMntn6~j9}kb<>jzxC!4#KBGW=88v>SgaepJ{N}1>; zyPC1B&~V(V7l(~bG=XQ^!L~Lmlu{>`Xr@8YU@)3wg!)-$Bxw=EC%`NQzM0~3=k^Gu z;wf3{3Q~9KpPc#XQWP3%_8kdxZ%Zv9ZI~-~RNEmHAz}<;#uUKbZ%mFR56)X39ho+o zbg+LK*UqBG$6~@p;$v^(aV|Ojj6B~3M6*#PJ!#SL@yG#bBj7#ClWDY*u=yl~+So<3 zCLrokIoqAu2sD2JzQLZ8$du++l22D{t>Rxhi$*0O1vezPAtd7BhmQ^V*aMrzjIxxb zfvi1UZvVE!l8QpXVO7lvk(y~b*+8~R<+!Z=6rS`E17|v(*V&P1;whd9-Y391b4oviXsFOBc$JVV5KC;VZZhG`@}MhNZahya}G62nkdU(KgC>EHxfa;#i|fSm8=$ zCg{m@8UP#z*U)!=Mb@8n>3AUVygSI>;OiTMk>^;|Ru$C-jWtDfuVp;r=T?|YQ~Hf1 zwN?IV`zL?X%154YHoMX@&kmG)iZ1Rk$Osb|iyoHCSOYXJs8Z8&W4$)Aq$GwHkOQT0rGhOzglo44!$n$m`N)lG*`WgO>G&GK4cPB0DsnBhGMqzn%q|b6b8ceEeLn*sX zXVttv1pC@wG2@V=hUoB_(Tq>WQ_9I%2v1gcyl=Sf^@NAqevCTANi!oTzwRAkPnGsi z0`xSZz2w7)1uH{fZyyjh);6U`?N}?8x^tp{W4Ne}sEr1fmTkEG2PIG5YqQx8gMH*# zcxyEEz^B$HVw^jPPxN*70|0(OU18UHw|`V3r)LonOARcPJzH22nCUTNX0g zqlQz8qq?W}>6C|rc5ahs_KEIZ$`Ws5Uarr+K3_jMUZkA78Dh8!K^U(e#!uD5h)2$B zb_eEy{To3!B+y*|Pk;q$6|3p~3VJXZOf<+2& zs3zSlCt6|WWH$m@&G?utUpW)uH0$`Pn)$k#b>`NyX4bP>k`9+)z-_U6;9zu1BDZwR&iPlRgbp$yH)m>MJl6M@zmfAJl9+{k)M)~Vh~+0LkERMRt|Vaz3Z zI@InSTYjd_^;?;(`iB89^o`#6bOC3+tAi~)em2ZkFLyE?THnGwJB*3q2(ONIhfJ?I z#)TdIc_4b!fKL<1l?A}HVl@z-suRz#YP`aF#rfT?b`tS|w3#xDbb2d2RSsE%g;j4r zA?esIvyLDEOIx}tifG?cQ5=CI7(S>}fSQWahRCf56_}StyPPuqr1qsy$X>?r^hPd% zeaNMmsYjccFL%Nz7)V(U?1wmU2&RfD8A}K!gESOZ>tfm-J_0T@-e`8_H(#{+!NZNU zDVfc#j(IgC`jW%7bf&hp2EDc=u6E(`Ms7c@MI^(`8QY3r;!hc*DT` z7Q2Oq9e2p_rVEdy;oE|uF$z!2lT|p**#v&ve48QI3FRARg3Uj4E}ZQp0!M#CFy*vE zgThbL4sSh7jbVFJ?1##B)%4jk86G_*EzC$6WW;z2X=}x~k~J*r-XhAta*pUuhDy5v z2s4KwS$d?;9ozm?8ary5Moh1K_jyHPXLvqz%lE(o6MW{x-n2Lsx8_=NC&1L6fsizG zG2!S}Lu8{0ecc$eju2U&?Lyow`<0^sknOP(y#T$!d9bXm91yx3@Tt`K%4{9-3(fH|%Quc$Co7%5-nSyVVQm9_ z53Z%~^6F`$?d@1_rE06PXZjr4v&}K_e_&gE?bx|CYJ_TpGDFW)j@ZrdEcX7PWEU{v zf|&#kg6%s}7k!bdhf{_UI@4FK(8K+VOLyS1hep69td*>7<72Os%~zkT^%VD1xLb12 zkP17Jg7b%O2w>ks8>^6BVdNTrBSp4UWiLX zOSMO41}UhnzK>Q&kVDJjT~slAJSOg)n;D}wR=HBS5nP>ASwJ15oPCFVNFAe6b%%c_ z1GD$^^T>#ta2O3gYRI?@M_ac;Vse5+{|ixg^dao~4we-q4{JcVr$;-06-Uw6EiA}VBz*FaE;v*^|J{8kJdt2G zBb#^<;o=ikzCoV2V~HQ0XapQV3aL${BY1SfK;E{w{Y!jXD8$jU4o#|hek`Ef%UgE7 zH$CWLnq=g}bw;Us$a_2dZ+}7!9U4)ENP~3~JN}FkrLCP+LFAU7_4socrP2bzC)r#I zdz@anP3PC6>2PmdC zz^}^@8!x}m;xZDGv$WMYqT#kqUY#1a9*<38?m%|CvFxlI}0=rCPxA5<_b zWP8b~6_&^}AHLZYzQI=9;Vemb);stgIj(`@8>IuLR~REwgY8Ut3%`qm01Z)s?zkkJiL zl}$dSAMq}5K`K#M2y*YT1OF)Y_Id46TVgUywOmR!SVz_yG^*VkQq=LDYr=3;SU1t3 zB6I#{&>g3gE-&^ax+Nz2zj;fg>?~a@jchDE&7|xcTwVUVgc#I@byr=)_^Z&w=iy-l zOTrw1jY&dI9vB9L_I2S!mYCeXrtLRVg2bI6t9@n~aqXgZVacMlwQYH$_Ib!=MF|1w ze3@QBrR`-^<>vF+q+Qce*~7*=fRZf%mM!;jKBo1$>wOA%{W|^CSiC%(#|R)9(&$y> zBDz_(KtEeq4Eo?+4U??$6KBMX9hua8{>|cGnkS z^QC_EC71H8*rkd$#h!-!Q0>>68+*THd%Zl4|BD6Iy-*{KBot;adbCXOg6OLgu)T}p zk99D9P0>pP@wa4G;idk-Ps!>vvd4hq1JREOaQN?sdwA^jk|SYm#U#?BXW!q=@hA7S zi!FUNRRwBoGsub&tD;ivh9t!qi|5p!QCsyM8={kIPLT|dU3Y1!Pl#VRO5K5%O%3s! zc`z&?#R?2=m*D*TF&k;f_SNG9UT^Fw+YXyRWkoOR$C4Sqnwa~CQDs#Pj*`zWb2Yf; zS(Ei%@-0`On~(Y}eb}isc>AR%PxyxbkjqG3y)~#bPH-NM0-l&YJB&AAsbkw5iY%D` z2L2E9EVr#Mr z8|J~Yjg*9LMyK1txLgYz(B^Bx&-8FJzaQ(ajSG@YIg|bg2~r$L(@?_%WwA&fjUkJJ z1ySHEo+r9N{L|2h&%} z*=7Zo?c#q|)p>CR?{*z+79XTq)==h%j{Hd>VpL2aKFp6%N@K!|t$=4erw&*YC8$X+ z^opT7;I>PdDAYsgZ*~hrQN@J)kg+9Zg8#^HF7KfO`-kGLw!?vh@9alz`a+ro`@1t0 z$7tV>t+{n{k<)XMlI;RsOBlAG;IuX?ZQ!F?$_&i0ZW-TnCe#atPToZ@lLn^^Dj_Ym zX2VGNcsK{cZA5GTjDE$^p zDO`*w@-@@w*_3S7cv&q8H_+6zv*#Ra7{VNqbn6oG5O$0T-A$|WI~nT_*6^ZkSjq9( zH!h5rDz(g+%sR32I_O`53G+l$$#)IX`SQ3^>~fUwM&W>%Xe1Xn+L_YW2p5|{`Dl+U zh*lTcWDsQr>YOWWp;S-;+S~bux?2`|d32|Nc`XTV+b$ZrRxu5GQhn%Gfh||;lrEo| zBNI)&0FcN@AJOJpuoqo%FPpK1np_x_qe3YHc~+62vqG5>9$Xa8kFv|m(C|(cseDl# zcz7oZJu(+u(iMsw+vY95GmuXx-%IV}M>iK6@{fkNmgR|cWC2)b&7YswzndB4eW0jT zsvt5@PDl=qr+**cChNBa4gkI`Uu&n6SX9&AmCmv%mFFbPO0FSu?>$vl?WEp&E*@7iT+%AH?=x zR7GHke3>~3w;W;VLTHXlg+YV-kSPAzwF08yUnhj;V|9UE{#btcvpHy23 zr?6_hB{x5FG-jKp92L6~eQte`WmunIi#u3Xsn0CKru1Hrbk+Qd~HbE#W^_#JIQafMGoJy@L{9V>Gf6Q#^oA>tOl8SUzU9k7rM=cLNb4 za9>Bh%0Qo8jLj(`@4~ucd{pPvW}JT(6zf=e_u3+-nL3Yfs8t`dZ10j_HGTUhb#xF1 z8R~E}g#8WVplQhfvIZ3fW~Dse@@Km0NPPw5DWtfOBnnk4} zl#)*ACJ2mRlqR4K2=)sZ0&7B@NlH?~%aL_LmW zzoKku-v=mWV;-iMv+8IPHJYiSt8NTmDS4|mw-|oH)_3W>rOwEo^NFL#REqaiax*`Q z?dJh)4CNj|gb(vqMuoa;4N?=j$Z^ox`)Ez6U2DNRTg82f*np*IG~p&s3yFn5q&2N` zDgYw3ulY2M+tct=SX6r@t5M0A$u(Zo2q4?7Xgz)(VzTjf&%{&gww$Neg4>PgvJ2fDQ0zI-;(S7CV2^lW=XvJJT3H0B^o?}vRk7Gp4DGSb z;p&J0vQT(V@S?=e(AI=dxd=%lMlr1BHj&?xD}B?(HiMi+@e_z26A31ZN(YMwBz~&D ziv-X1U~&nP_4a6m`O zN`O>{4PI$h2jrqet_E`Mf{p`Bf&{bWFuSZ^0L>PdTknsersbhY{9kF!4iNJ|eBsvS zk|90p!3PpmQ5F%PIa0q|3u4R?r2-vOL_$ZOo+8j`wSrn+>&huIpSzy{{nll5H!V>Nj7v%_}v*dSjhm2bzr0DAoZecdtscSjB=N0^j->Z+dl z7F4Dk+Kz$zVTa#$4zX~K45uy8M|+-S{fG@kEVnPtsUE#L1yT+@L7}bV|NY(H%rC`EMwLczJ#~MCa zFDy*(LpIpZ5zLsP!lKB|dcHle=)#wR%$ssF%$PB`qnE}t%gHVo=r1cjD$%Nz6)7EB zdoU=59T*}YwVxSzxQyfhkndrwl^BF zAF=pmU>i-x{PB=z)Nx?1Nc!f4CF;0^(!ihidq|h_TL%o7o5RLdc-jlh(GDT|u6{_L zPi8EkViiJNnw+d715;@TFR*N%lT)u1rp%3Ct5i81bnX~9Ce!33+jS;2j9ybD*a6Q~ z2VQ*B>{eeNsd>)X5h_fk>BhG;-cc14iEr8LF#QUI_e)?b6W?Ot6h(8->G^*cd&eeC zqbN%|ot3s#X+3GX(zb2e&Z@L+JZamuZJU+0J=N1Q5pQ?I8$F-yKd{d^_geefYrjMC zBkMnDL)xLePqebcmmSk&ay7^apCd2x1w4kT>JiDVOr#wT zMBLNs5q(|v{8v>n*U4lN|4%z_|D$aGr8WIv5&Tm@-%x8@d?U*qbxh8ag#9bb>{Bk~{>9yW$x zk%^ylVFzql^R*znogg0~Fr2JKiwCH_WK)-t@2?8LS*z7@Wd> zNA;aS`pHAYqZ134IgScAl@Lj;tnp3PFdcq=%BF^@Fr%fmZk}^bQN`)0+fM8?i(c}O zXikR26WvsMOm)T329Q}-OUsMl!FPJSRep~}EmSvQN6Gg5liaxTX!dX7-XtCq&pLU3 zZ#Kx#DC;bMWYH(TP=!18pmjr^J>GwDLfaBLu3U4AcVdOf_0W}85QXJlUB0t+EMX}6 za(?MFr4%ETF_)*y<7g|YLSMp1Ml@Pv$JG}uTUwR(Z=tETGMYHNp+@fQ6dj$dwQI`u z5H$|bmYJmEB6Q{>H+GRrz$=Nlz*6lgf;Oa4?*=eCk5AIF1Y3rj^;k*TyBAJOZ|7Nv zla zk*q;#`4)Fdb{qYMZoq;bU6lqmxjX`e@34W7YnW&Qove8aVqUWz1oZ#|mk%Fd$r^D! zdkA=1b|5$(gj)@p88p?I)7KSsHsn^iVCGiptmjsmgeNG&aZ>S2ej9`*xPwh^=pnv8 zN((7VWN_^e(ur#{7AZ*h!%?cm#`0)UKnws@p_ystC`44mz+}Q0iY1hwjK!d>$&x#< zVAw&U>>`!zZnP$Xv5)kTVEAA?&p)t?JUYWTyM*|ik>HqC;3@}J&Ho1Z zuf>gsTmDPwUvUTgGlrP|$KsZ>`v;WS8QQ42SlawwU#Ln&S03#j_P3B?r;SA&_9HB! z{*JspU_6p2ev#eI0@Sh(Iy+=b(%o>)vRHdj|F9tg+ZbNp50PnSLmH;6gv`&iuIH?^ zbnmOlt_I2;KYVbF;RUc^N0wM#in|~?ORtSy0$GZtwu&h}aAl}soNcYgXTgg%w(a^t zE3M8UU40Gy=RzAzKL@hPW`%5x=ELo7hl8J1Eu)gAwq%>9cc`(V%H^@}WxAon&zlH= zRAKteh33a@*v#aoFd_a{$l;IEd)W#I-B|OZ3aahAn|m~~A2SA0J_c!}SNEiLp}bb> z=k?o;pHe@CYATadX0YGRncR`6?oALb=X<`+_$JflQIAh~(pA8u@R{yCba z3;c(41labi*vB`63E7H z>RewxP-nh0zI{XS;}&+F7VR#4P#Rr7=ZP{d1oM)~o4Sp zWXulGn5|)^P%qCF11*}xokgO9{KP^YLL9+;v4_x$$7W8v<@_KkJY@N!Nls1CmcZc? zfF-I0hoKJ97cy8?E{biDA*Td_S3v*s(ob8eyC1%?@$E4web{U7W8>(iqLpa{J!62s z`q7_I|FuH38#Qf){#ED(6bK0W|5%~_k%aOeSFoYEsf3}OiOv6oQ-LbB|Ed%@+ZIlo zb~JK4GB?1!F&AL;Vif<6PR)R|l11xDv$;+=PdLA9Ygq~FkrIfv*hjh%$2oqIqAoHz zO6X`y2YUYX{r&m%g$A;`Yl#SDuofWBCDCULrv}KytVG4$MDCu=gqVL7O8!+$Mj&U< z2=M8~VzbieGG{{~M>Dl#sTofSdre_1I42>iRZAH!0vNaf7uu&-?+c~z01BJoHgfLL zGU|;glSdts?!46z5N5WgL_au`K8lD$-jXZ36q$FW;9}zNURly63H8VAcr`kZA4r77k=S+2Z>^&sx_i^Yx6R!u_7L?uU&}0 z`EW1LuV}|tOOtmq(iO&r%I-Oc8ZhX${_Kw1h(~=3tCzEGj~wB1*2-k3?6^iAb=T{L~qp)IN>#Df4|SNx0)gDXZW6mXk9K; zcgeN>Td`A}q4@`1ug#-x|C1+Zk$DH;gibBUUF!mWi1$_aGjvCgv=d4Fh9`^aCjsav zaw}{B6%I@L!Kl2;?aD6q9Y@^EPhgN^nox9<<1^+CYxl5f`+(|HYlb``n)+XRT(e0o z|4hb6#J}sHLkZecZMLl1S9@}Bj@v=)AnpiVp=EO?Lyl zBaH7Zji<3Ya*An{Q3cG^TLp>%_K?Bwy1>g{7Z5@}9sv3m(k4JIppp=!i@X#yI`?T| zAejIDAcRn^t;K1sH$(e#c4N|4w|qUxSj0i$_XE;-xA%_swfB41^~d3FLT)gFCLOeX zOnj=5qSl>Wa6uj0-2>vJeL-+iV*Ej3@B-qb17h&diMi;|K#xK&@tacr0zqE0-CN}1 z12iap%Uxwbd^kgj>>qmzFHu^2WokL972twCa(n_cKlMv{m0ye?yMq|icWGbB-(EBP zYoKqVVF7&{2tKr7_7z9&LR5Y3OEb^#J_11)oCdql#EW}>z`kn1)xPGkG_IefK$UBHySE#!(AW=Yf(LNLc>JENf2+{(*$6WG68i2b=GvwQ_Cm ze(>2$$;g8Wvtb>rJyr-rPjazq(@9Un7y?<|-Rfjc^V61VD zZJLP~t{VKc*x){LnMQP3|0&p#Q`p~tdHmC*gRgAWz{`8VmC1Y6eG}qVDkK41YUv=2 zultw`s#R4#=(o5LQFOkAH(hMRQN30|;0{Xc2DqA!FTjj4u}wnR+lRKfWde8k$)u5` z7iVcPdDCmF%PoHdM?%AIya4Q6-s~?O^>5B2d$)IP)Q+v<3?r?ib(o!vnd9OIXEj{v zBc5BTA&ug`!OVRC+T`Tg#f|OEoh}BG^46wvlBt|1KrG9gQrS@-nUHcXB&lFuo%r$D zRaEx{?X|XZKKN=3c0nt)V zt-j^tKp88H$a2+TmUv4H<%b8637#;Lw!~|PI`g(yAu8vO&d%G28Bg>2ZGQ}i@ zf2q>l-7CW)tS0QCxPrCmFo}c$LVR}Pk!!XwG^@%Zu%?0Km7ALuiPT+Y&zY+sm&KEP za)J-&%KFJQHbu(=@zl=bfV4W6k|6ujje$-BkZG19ne-VaM<4sZ^I=LtbV(i}LC*G+vfW2jtsJ{Jxg5L{dsTLkMTN*?vd`8B7*`SxpjY zSxt%wSuW|6dI`2Pr)Dr!X#~7NYTCl(IBw;$U@Lhx@(61gP3nr1G?M5{`26BZwGDIh z?CCBz8BNj(8J()M%4AoYT}p{`85{e?WUKYGS2|+e>}kh>t|)Wm>3C<&SQ#$)X%3W; zIO^#7q@wX83QP7RWICt3TN<({N0^zVRH{ouL`J$Hu#~`p9Dma_)7?)L4cpk@~iS3Aa%}OGc zx~qO_Fxn%XdL!AdLAFh&n$a|rHS>Zk67rss1{2`ewpR+PdDxm^EOez(m-57VxluP z303A**+Q+7p=1FSVuE6!8dk^Q&eF(;&iMyKg*;6Q+eN8#Rx%n2OUyehwASIIk#875 zNkK=6yRZ8>g`5u{>okC_wbbhaK4$~XrD3!~DZ0aWns1J9L@4ypi{X9!b?M+;CVp9& zg>VpRJ@tz3?Q5Wwu5-fK&BcYYk}6*svCw_iIj4r(-|nd);^#S6&hXl3?|DN)`ozlQ zSw6=&v$e;noWj_fQ5Hq(Uiw*t_lF(U!&cqDsY0N%=Yn&R{YbUU$T@#pds6sXM^Mz%d8(U(o1@TvQRV{dkq zx5cC;UO6=6>DvdF&PH61O`Ok0W_Z#lh|DT6!2DFjmX)XBA@?1!$M>9lCX2^s^J(i$ z#yrX~MSZr!@+Tl)+amE-{=ZDpe;pkHby$G|xtHid=66dLUn~<^<&V42n6rHRNIy_& z9EGEm*bURnqT=MwQzxEf!Blmu)7Jz33}f>4I>7jH zML{~#OSt@=phBk+fpOqEGJO~y2)8hmVj#yx(2J0gqe6mNzL zPLFBAt9JeC4Yjhk?75dtuEQ~b9~B}7sh}krz1b587eHa;YYUvjt{4*9-&>y8Ebb~( z++l0FO{8fHy1(I%Tpvl(yt(uTqWv9qvHz}67Y3%OyHarUgMLLB_(7Qq8e6tLU^jLt@hjGj^*5tw1my~VG zp;43C7i~MXlldEFc=JrT)A5F2*v2zuYmx5>e;%K*7uLp#RS!;i@K5RX-`n-N>Ogv| z(_f4{!f#{6nX%W+`{+w_pLSt6%C{MTg3XZF><&Tc-Pr?;c%4S9*8aNL#ze(C?`=%i{i#6YR=iKg97OhDtC^=b@P0AOk zu^Y~$ZqwQdXffHu+tYjd&_)^0aLq~MHru^4+Wq&n z+|_mAyTMn>t3(c7yk34}`lY=&CUr|lfORc90;zLW6wv4dnP}&$>&wX)scI)|XxNoX ze!_2Co5Jd)b}EH?|EYC4UOAtvSw!0)rEZo)$S?_g64A&gk5=8nq6MqtX^S-R|wL))NIG_4%`lZ+SVzq+Q7ZiW@2K95f9Y)@wnF zxktR?NN=;mvE|O5FfX-!T7P&6l^3$562E7_-b7jR_4j{+sQiWHr4jzQ5K71(Aj1En z;QB973;%!N8Aa-v>gfMgHNZl7VdMHc3Py#OmX5OHhT%nw{|qox)MOo&@bZet%PaAo zUpPqc-Ff~bIwWVHI7XfnzWXcerz|IDr^+$F_^_KBD(vd|Sat39T%p}!_4Rr^(*wdA znM1l7J%Mu3pZufRh&g~-Celodb!BkXboI$&|ObBJ+LtI&|TmUMI%0}@gA=wwhd&W9X91yp)Iyz@IHm*~(HxE<=BMlEGEL72@#>Eqv7(Z_3#*99rF$VOavZ z3V+8%XWlOQ1tA!{8P_Q01NJ9Gdg%sH{86Ov?fLZvN> z@+&&)BpybZYxI|3OrF*Jl8%tOz4M5Y$|gk;0He0ESbuy@8(y9jTLygn{R=Bgq4 zy;`-yUrzkEF7|0SQWeKw-vxJg>zU)W~-O2QU^^( zbG4rxpsv~(vVzf>myf1PW0cPoP(ZUis7;eRNr=&9bitLSGRL4dlPli0L9^Z00lTey z%g?WHOUzF)Ov67+3IBdqv22SQZl}m>Gghq~u?b{Tg5_7e#o<@Kh3rti>%h<=Fm>qJfxOQYm`uc+$nv*++oV{%(p7q^~Mm`71tV#c9a6G z95h_5ix2m~hxN<@?oFm3oaqcZV*`#;h|7Ya-Zo*r;Nt5{H|D z%mZ-@ImnPs_erKK(H$^ZE#t`w(9)mBHsjETlqQ1kH}w!+TY;J1RqRqx$fX#sX(%eI z;@Ut=2^F`o*b)h5R^k#j;~~iNr=>Vw!jz*`=-oH$U%w&a5+iK#H7Jpq#xLBBVb&{P%rLuBhpwY1O2 zKS9TW_s5^|SH!Kx1|QWtTCTv3zkDHywDxR;=;V&GOGL6$Tj zIP)%dB8OHINf?LV!EtwQy9|1Bk59ieACc%>T(@PPUB%?fzZN4Ea;5<)1`J)%xd~q& zMO_8=NVV^aRlWR{YQ#0KODBDPXH6bv7DJ%Fo)Gtu1;h355EET_9)(%w1;1UsUJ;>A zspA%($W>eb1x_$|Mhqj0k!bT1h%@ZhqT~!)LF!=y8$UYAvENw@bExq$<*bR`H@z;- z^zT=fnD6dQ59ZhO0~+ds9GH%Y_kHTfp7Ka8wxb_PsfYctYsDx2%5q>3y|?F3&0%Z< zrNbKC0eNmq1DKYM>2cM$@qXV`e`Q&($s?5gwngT|D;n7Crw?6m?~N^WmLw5eID|oU zg~3sD5XzaZtLNO+PYQnz!B|rUS-mlLoE64IyYlv8hL|qKy3-XHMuFM#Vpt*ai6lOg zVG&n2| zklq7>97qx=SVWs>l&2>rUs1cz8dmNpIPI=vugDa!;H6lx!AF~a3epdb=^MIrp8Ec8 zQ~wWmQy{!A&t8&(eE(M*?nutbSI$-vCv^YexNa7+g{biynbVY8)|2^crVE zO>afvSuN@iciJB;!)HsZ-lO_a;xfXf%biv{l06n>O<~MuwvGV>J;fyL&Vq<=Ma_NH z8a)RpB<&5n?2j`pfnR5~C?)>G8n=VxH$))pZfV`_>G$H^tqk9J$)4?|T^8j%N_g9V zc2yGj$qv=$4jp>PB<&{X3CoFski%VnqJwj5=L%UTw&m zjDkhLto0psRbi*I3Y~}*mN30Q#P76@WZ9h#L!S4m(?&i zi08634`(snJ&V!VPr^^QJ4YTntpj9>tL<0FiW$gg`mp72v|X$3;7SY94d_l z2PW!O6)iR@G6Q~$0dj=E7ny1cll<1(WT%9!&)0GFSyoCaxUv~Hq*WZ+cr3+Rww)JueNCEw zF`eJiOz}2T0Zy)ds`zltu?o&DoEtd1*0oH4MWi6P12FD=8`GPqleL6*jN3vW=H#O< z7yb!O#qmu$p@xeA{L z=%X*;?(G58V1&`w1N*bc*xP0a%shz+1eGOK9l=slsO0z)_(h3xcs>jGtzT1s^3aAK z`hudy7EsdFN)UCFF95QguA9004R%MJNV#EdP(O~*-xuT%(Xw^8zt>0 zj${r=g5;)SiCxg774!BLF=CIto`v>?WwU2^sP5V2Mx9OboWXV-U0$ifZmF}%!al;r z_W?Z2U{h!}qzbh)h4osMyt+!B0yS>k^=g$q%|XmJ5QuN%z$DD{im03V#tP5N+D$%r zARKC5U*b5a%D5L+NQbnTqUyzHQGz0Ve_|4iS~ay=w~o-7u^Q5d-oxup@IDKxzg82G zf3?s2T7Iz-YQNvB4NgDt{)XoCh7FY-Y%5${x|VL$8eZ8#*%@-?KPsSGsdd4e2(hkc zD7^#@-R)^EBD9D!XvB;9>Q{^ouvOZ0g@m+O-%2nGLs*ecEI+VBescqX2#w(u{Iygu zMq%9mQ&OIglJI(eI*T;#Bae_{;%tKBD;VDMcMicCNdJtJopQ~}V)f_&I?c*4BU^hi zFNAZyAwEARzMQkdsU6O*7_SdOxZhA2Qp($@6zrMDmNnUHp8gHYr-I};BYDc+A|%6~ zYKb>BwplZYAJLN1rZ&jOOzlM(Q+vWPHf3M`En=t|(M59RUqT%^-TzC((0?>>g$$kl zlj|3=cd|A7KXm_;CY*=%lJEB?-I#hTtv~@82l9SU+-U47*&nn+WQnXGPR{883StN< zj=Sgrk(4MhXdzGZA9TSf;`*IZ&^7?=4)K%HL{@~_YFg`-dbD-(`C}AEiUuewBSZpdTDx zlz-fBogX=i_V|L28i^Wiy4PR)aiguk1g6!rX+&G)X+HK*TizZll*+a<0zBF@ea`lt zrGL2?#pSA(k0z-=Bgf&&T76o?bI&Jx^-RXay=9eQPAkQ(Wnla1{B)L3pPoOo5m+Z^ zcV^=f?>a6^5BW2h3VG&O?FD!k_teuAxizVyFchFq5qq>yiZv6h5KScavrJ~M$*LJs z5@1KI03^bFIOXXSid_bcIW?IDtJ6P8iTJj>7lGSb*DEDTL$DZy=j~0tX>4lc$RK2- zPq(Ja!C7@Yr$bLRx29uTGzb!a>D$}cM_-OXjj{Cb zR;#6GU`RJ#N(?cNLvtqv3Y$qqb5D29BC-=Duk)bCnvOb~NX`7*z&S65$7#|enjYM@ z-KuL;N@9EwfHKkfbYlEJQf>c`Bbw zVJZXz)1^>0@^z1@W3^$bQ|X6sbbX|0xJ-(enWTqQLB(C9=$9uwI$3azb$+UyD-RFR#PvtpPbYT!-&Ynn;H zq$I}^c}G4`!8~)tNOwOS`5N*Pncv|0+gXSF6c3n(G0-s`tyT*r1}=$Ai9A=(DaC-n znqgvv1q!#3I4aiNs>#Vzq7l<*kQ|((L<~A^@>H`v{$)wCR}IyTQ@3;~v$7_u95c?)&L$a(_OLV>X!@8L_<|X(Yig;u>rh+A_X*RrWCYo6GWt*~A<8wZnZi$u4 zdC<#i=M!K=1W;ZZDFiNrY2-!eXB#&jfx7E^O;o6c!9GZ|NI)g$sLTAr2LX z3ofOJWsYuy*Q8)ED4*scwr6PlM%oVg7XGFzY5c(Z7Xv4N2SZgc-Hm<1vCS73MWCu# zI`S{rK+iG>w_UBq#HY#6oY8N0+q*)Toe}1PQ5oa9_@M&lAyXbv(+k(L?nrAj4N~^1)~%-(O3}lsjU3Z& zg%ADn!*a&Ih#y2lm2U$xLjhpE5m4{%=&nTAVGW>6IMdYAqf0@Y4N~;uyNdcg@AE0J zGAVY=N2l;P!w_thI59&&{5A)nJBB?QT)n?2N)EM~%9{+t?l0c|Bynkx3`v^q{=;%P z_DUVx$`9N-ou*;q4sDEmHOfV80%&vcr6Z@c+yk=w442b-#cE{MWFo8#m-Bi_FPAld zw#ypL2J7ikk$iQb(!}Yr#*FI+fA<)!LcaO5U|Cc3lqc0t;$<84hFOI&^?fdL7Cp}) z%@bt%MN-8Y{z<)b_1uwGTMUAQXPHE5?C4{ATD|s2%S~vzNU$M)!K@(adzD`%E>4}x z8Yy_j#S)JAyUQAbI?Q_U+Riy@s-P|^te{eeOPiq5g{g}qT=*tP`?b%8&)RY;+s;uF zMF_Mu&`%-NkSq7#&nA_Ij@hLu>Sp1DV4G%Q?3q*KX8DAzQ%B5J*x3uN7(|tF0SzDx z_F%-NP57w}hd_B9ilr;nQ#X|-+0SY?G?m_cj9B+7#gE;o-ABhxE#?(Q_bOhVxg*KJ zqQ6-BEq~-OUadWLfL2t*lS^@(t|T5Pn8;oej;QabFBDQBJb-; z6cIc{uR3^l%AwCmkUw|o;mw^0yhHwUUKs@BkD0r?R{F?+^i@itCQ#(89KqeyQpBFz zmGTx(9Jq+Jk7Buc3b8X@HYo%n(pEi5<~Iop>sou%XFa*jaO!gjbK_wNCaNzdpsn)#kR;8D`RlF($LuuuC~iPtBeF*v`YWz<1&t;A5WG-K&sovI;6mo7Dz^= zSJao*RRn5Blvy}AFs9dXwI^IOWHAlWWiRg6Wmc8N+i!QxM3f_XZKD_G4XoOl$A1m% zDMdSsWpe0)D$8mXrzLwgD0D<5C#@%|NonZ_#mw7E!VooFwok32m6RTeol0)elU3q% zpq@shhb5HI_M_RwkJ}DOSvb29H+OfH9Y)9eIb@Jvvl}zjcg4e@cZ4~NKI(EeoAc=_ zif&$!d0w5{hvle`b)>wA#o0^156)s&zU}5u#-9||dJeU5r)=wki<5Eh#-sg9ZaUb4-OrW9BGt;S znl*GazMi19wo_PP4*x~WwPpS>pm$iJ6JH;*)lN7~OWcrBEfgYBdBd;eYyR=-_MRUxx@h|c zN4l`47BPl)fh#jv>VV!#vo?S5ju|CLs;+CRnixKG{6vqXDFc0TuXBKuA3alZHB8-TK3HYBJs1 zHUT5J658%d?K`8GpF^W1bQbv6&)WKmeUSa^FhE9#*&Q@N@^7_!O45>S%$ClH8n+dq zCgLg|k-8Dj;#xi!j|0XI&#S6`1~c#g3or&qJCfPoy0TpBcOI3T*$1j#qMz%$Ppk!dGlwhWxu9$2#oY+^w${K_N{h z1_V|PZPDJv4Md-{FUtzjS>S~&srq`OmWvi)hD1G^L23h`pJC=@HI#Nu9p#n~TDp*y zUgEoWiGB7ncEZTMkd5VxVGstf3D_jdOczg}nHm^R()IPP8PxILq7X4r%G0Wl622lC z&GBz>bxnLBi*xUcBhU^7ft!zf`)Ps)E*wOm^s9?b@O{|>r7^AYxn3oE{1F*g|5(3c z#YNFgX)y(&TkPyq12`CAWBOnB2(tY+cE8NZ$3W3OGEeRgBIU3u>G3L#Ps@7;nCm!Itu}dqDBXZ~s+0}6B zEzAebRo{6lmOy{W4LjlB%nkDPz`_17no=ZX5sv(rsZY%a%Y5)avwl{HpqAgJh>V@l zyaNlrlzJx?LTm7Yi_uh{+@ZlBItfurfu4|tl-$*?=t)7i*EYB&W~^u(cdS6_5RTEE zt-;^OI|C!s%71ho*Sbp*TWCig`FGy(1IjfX~Z zRt7)D=O-f0nJyQ+mbnd2T;fKXb@-vh2K;p-@zmfQW za>0opV^0EYmZcoulqNz?`Q8dE?*Nkn?ES^XEjo08M)X_wJL|_=5HOVO1B&ZK_6tkw zm8$;+Xh?ZOpIkR12aAz(FtO(F4V(O^{bEbGc?5_c`00aI&`=c`@8UPOl`pg&@0`b3C}^&^gnB|y zD5oL?*OP;;?FsutGv#uDTsEFa&R?*UC~XMt>?pP>q&z6P__vMN*(f4<{mgdf3K@Eu zedNw5*xC5I39$GhG+1uenJLILBfC8p>{1K_`@ZzdJ@rh4OLBnZh1F&l(7$Gf8k$mXf7b zk*Qh}jFJYGf~12#L#NrkQ}8{MEu5H$@;R`9*vr#fRn18il>VOn<2rBtGYT8>}*&o@!8~>+LKxdH85A0M>7Bz z0ZaO7VspJs=D)kgvweSZ5+juXnI&Zh&!@i%YN><*49gGKdf8~H@}$b_il5t>pLClu zxofLD(_leM4sK>iXrf_m^^7U!Bbcff8V^2kctW#$ahIlZf2sRDdW$yWt>s*b{bgU$ z&A1E>C{J6_bwjVvcfieOJHRC=S4+f&NCOH_z8HTFZu$tbCf;fd?I;RQEQmj-S=J+>UdE9AC$AS?O-^3k6?asWmh)&ec&gyE|Wv+9^$-N5B=by^Tx&NW{Z9umue2oY6%ery8@L{h4r2v zFSCX9)Q+Jq+S*>Jg`I(eLnC$F0Z=_3*z6*muDU2YFg9eV>LMQ@9H@+7GcD)UPX<)j z^Etz?q{y)Jkt#TTpPq2PVyEg1rF@|tEJZH^fKA{0*?@G>F>;nP; z+b9ownX$*2r_ynrHxl5T%sJ)yz55>YW@DZfaiTvaj9jEi*`?9mG-7H}?j5TDLw^7R zxd3q~wOzS3>}QA1PbX@;KTDh>);C!$`(%*^6*geoNfWotHoe~UCpitI+>TU2~-$KQ~@*Of45x3KuvY6<*Vww5wqh zGh=G@>)o%a14)~7Vo0nwK2fVdjU0#aeWCquWlyfy+^hT@eV%On)XO8~n58ts>Vhoip8Z>u$@`TX;S+*+;=>|qLkm7<|+`wRgU-%5d=IF8W zgZAUhuCrFnt%9YLPR+v~GS5pOTHDg{Xgq>sf~?h?)cyk-XBc5Ro+fErI1>F~I_zyy zpA-$Vz9k0UYRg#-cbIj0jkY5T;C<6$7{012%L&COUgeEDPYR%fS~N z-*IXk*jI{&gePWrc(UixxJNN9n}Y)ZIUd)&AN)W&p>ZRVAS*vxU! zl1SCVgsAlx^T#K^I>bwsEgO zYp(S{m2g^rfukwKfhH$^l^6ELF8kDn&bHHwhehXc$R`yde8?Gy=}y;ZDi3;xtDU6a z-aK0`sxG-qJ&d^S?k_KfeT`3Dpl*0Ey)wlR2o}!?zgGF2i7M=L%DWAx?v;Gi=@ded zweY}l!;UehCfz0HX_Vw~gXXZ5R&kspAX3etq3mI+HOHm1Q2%EME#9t*)evjvpqkvb z8s>Y@UPdPdWcH#({-d+oI$lM$T{D+2+Q&oPL)7CSJik~w4G^g71`K7s`y$}&ykEz_ z_Kb`eGV~*GyX|Q3|7`>P6*cuo#Xqhq`hd9GvM`{L`U!X6TpuuAw`NY5hx>~Tb{E`7 z0@v)ACm^opHNPgia}^hN|J(vV8}!`0fQcOo zT?_@L++gv*h95Sxg1o=b7-NQNiaQ z$F`mjSf{?qf=z^ML~z|Hxf!ldGgMfr+!Bz*P(p*$J>@eJ<_FBvLDj7KYo1DETebc@zG4B|nZrq#*cOTt6D}C;}41AzAMN{-q;=u$P6d{@%W0)drZa7 z?2`TI<}*16DF%p`G>BMw7Mv-F9mXGI`AWcsv|w8-wD=M3DFVOxYvcVVbAfvi$7FpA z;~d@>^nbnW6P54{gd0`ScZbp24iY&d@|@19}QG848F@j#~pCHwiDH^tfnb=O)@(A^Alo*4riF0 z_~{#j>`ptKW2+$Ab*d06vZuNp_wl)|)laj9{dBv#cA~U3C&jvG%MY)nh3?7;ms4#A z-*yw5J<`)%3iG-aESOxAkGCpkt|0|Tfa`szIRFS33I29e=a6flEJBFMON}fok2Anf zFY%W59Ea>mn~CX&QJqv~*-P`f`eTaegnlLk{UplHEDB?j4?raVoR~^gl{b*Kh?qCr zk5?-*>1-(0=E?OlQNk7PnL!^zxei(y;&-wjOE-%cDc@2umlY1f0VF_h z;ZQSO0jcrTmVh0Np{k{trIcq_?g6qN5`UJ_SdWgnh{@Q`mLMQVFPUKZd&q__BaG9a z0eNk_xj~oxcsW-b>1f__la{OgK-{-lSz&=@>(PBegx|x$xKD<03Ct(7u&wEn$+E9I(Z$xHzW4D%P?_yCYOb)A4IC?tjT}KgqO&{5}xC zrQaj)u%c>B+Cw*K;K6Pu<`_U5<{H&L~g(W%3H1d(n z={{1e#JMXqXdpZPEn>BM0B;9*sr;>UVEdG~%(4i$)o$y*4 zf_5T#5TNeVpwlx@AP^TsGMc&a{$dHO9&8P81|P?ub`z+9sqO_2GXm-GDDf0}p@05~ z0RI97n-t}Ai4Mi=yE2!s=0*@N^X)2fd>IbnFHYr&Y>VCRq+|viBe8)azw_?|iGW3P zqER=F^2Cb-b#&B?JZB;NQ^Y9~{uOF%uNoGsI|JLuM-mNpV+u#mw{bN-H8uEH*%e`I z=kklEBjBH*;GbT{b7n`U3dcSbP=Cb9$lSw6?ekUpiuUA@?d{Jy!bKo#Qwz=T23IP> z8aL=c>77SRA87%@l@D{GJd$5m=7vGjy4@|{dVo+>Olg!F5fG&Xj0y2nFGG08~*=A+EsvMv3z~HySqE3ySuwv8tE311`(w} z1OZV>8l+P~q+0|eq*J6MCB8-PRq-O?|J%pcSA5v>o6~2`&d&61Et95shWQnk=OOP{ zKTHz9X~hx2Y2s~mQIF_7H0JM}Q*q~td~m$TA7L6~$&t0my+QVwGjxt1k!GE_1%2Yd zj0~}DU(5Z*78x%T<07f95d_BE*|@N5b^ab98oJa&=687qf%G zPU~poH|~9k7SuE|eY*UR&MSBHwbE>hl@QArCyQmank~uugxMOWR9r7^(nhH|Lz8wb z+(i!8blID=9rw@TSSu-{fio+C%?gZ3+M;HYsgOhL^ZoD3G&>>(Vs=Qa1w@h)M;S(b zBoHb|tZ8ynp z(=-ZTi5hU=oQ1We8?Z~pKRaY-9eUscTlS?gSX11N84a78#z3cL5vd^Pl4%*_#zBik z!Q~6#=H{NnkwL!wu|!fj-55O$gN89E)7Dz2wz%At)cQWb9M%#3(k8VNcVoTMwsu(7 zeShM`o;&-K9tj&v4vRZQf~OFTMBaG*A@UGT(un&K{SdfRvXAarBeK;A+&b82Mxq)IRazlzTL(h%?nNscjA%btL`lrr8h`;qQZO zo7?eMtP>w|vR0%hb>kiebjJ;{vfRvuf_yhdNp0jDvOaOwqJpN_#PtogIfZP5HRYR! z{7ymC-L}1svXHcndMBSwI|o^v$6g1i-EbLeqR%BDi*7ox=yeT&s4>-=-$=yRL>|ZO zs7U0wUB;de6Feu`Nha(1RuTVQhRTiG29!rB*0KCLQrYzO(JPRk0*PH6&Bf*z4`M1D za)y#Q30hv3a@4fRJ~EA}HnBl8^med|b`597e<+{zNN-xY zn;a_-V_^>u=AnCIv3z>_G^aF%Rr#&AxoqOMP{>h|;^Bwa+i6?qI#!#XR_wrot?&Bl zwHHA+EUW5#nM;7I#Da|1vqJ>)N+jT6fBNWlTbYXlA~RgFA-LfO#qtD0W;`*1ZduD9 zxzAq6Y+mnFE6^jNQ!~-ji&dzf^SYr2tqNs<(Yh%5HzE<_)^QDMIaHFx-gr8;KAXiE z8^M4R(PN#RV>TVIVpaUvxTB1fj;B0PwgT?s=NqYra`_acO)P2Tz2G+N~F z46|S3-o)oSlET!BAQIn4*?! zr#$v9QR2WB(vGPQ)Uctr>G%&oU5x&a@)PfnYL@e)|ke1wz;fZYyDkCBt zMWhlF^_V!6g+HrE8h}!W4oB}qL`dkrEeVQrW0(^H)6*|J<%xjeC_J@cRjXJ&U>3|7 z;z>=4wzh1w7*eEYyVbvDD5tB|XUjn5|GKgpll8$-3uTNcOHsqLj;?^=el#@oeyE!p z>hoR^O_&Mw@kexx2%M^3crtIL!Ul02>d4>UO!ZL6>$-dIju8+CVmYv-&_zT$rcEAa zBYl+lvwtrxfkq$oJbJCiVj()%DPduR*x zM_ro(chok>yz9HHl5~U4z`*La;9*j=qzbcH;DksbT9_kD0;}-tZlZCa)Ym)XaNQ(M zv&fGlS*$I{y;+s}{5I};j2zhTH+|Isk|FseOi7;?8h(4qkDM5}}g~Kq7Gc*ZG#bbnj1C=VK$2(hxp?VMzpLCm%Aq-m~6biA- z?k)-X2Ja>zouO5*0`Ic0)_B&3kBsj-!Hhj6WLL&G^y`|ui#pP9whQg#>`Be0pD?3? zTY`RffGEkHDSf|AH2&^Jws?Ri9O)Lj#1X>XWBv!ITLe_CJJR>DRnVAC4C@}U&0-$a z@UBVvpy^@afzrhWK29UsLN^avrpBwQG!% zZSV%Y?eL}=qR`Oh4zr_@9L#Bl)P3{v*$Qk?kkv3IK$5sb6CQOb8x=A z_^2=;4ml*_yVn*tWU2lBKyb5@qe@IPyh4%&7*J0wj7u_qmQ7T zHDsG=ZQ~J-`l*T`jK1dNY=%yFXjhEhXlxrm7|H{8%Q|}GQMssYW_Upj?6q2HEyQnRBu=tT)n4tnTq?7`t+G39N!$hUF$^z6@D1_gZ!Yrj)OYw!vN*u=bqj&wkPJeMyMM_VQnL3-UfBA zNavo+o+Lk9YW)I<%N({}NueLRi6&Ma&53U9(*F3#kb;_A7EZ3D#oO$+#_!*nHqF^Q zPiW#Ql$y&;RJ1`r478EO@8ZH^oz2Q^E2ju#CCJ`amRHDR-*)5qFml@6_ZTk{s;wu3 zC~_|Xr^8BF;l+$KBR8R@0*8fopoMn|qbENh+ith}y@&J3Hk<_&z79eXYaH3T z>O+blz0P;x=T5xzxxV0%F`DZw;l4m^k>-9&{r2rg5jb}ylQ73}dJBg&TB!Xf)B*8c z+T(%G6LF7{m9uYB+|ImM@F zKGFpW1}117sSG?u9z*CzLpfAkMe(>wbfvAU`SfKJeshw%xbKiwZo=g24N&ljBhxI4 zCj{8nbr<_o(^?gG+eDMw*d!epG`W|lmC_%e8merX9#dqmO|Kg3MoNg6aw1c53{%LM zW<}z!cXJxD#)M;~h=#*0Zsi6%owY>sDV*jw7HwkQqKAC-W({phc@w{`V^jExVOkxN ziAWRTgnDskCB=m5rrvP`uXdB&v0)SOrqpV%sI99HVou59#yD~}!AF%_)R4Y!j4+Nd zebpSnBH3)a*--7`9Cj$EFq2z(LKJv<7DBqUG+0(PEi26%RJoNpK8<*1hKA;o_6@-i zA`T%$3Wkcv#uhyUc5TpJ%tGMn#D{h5)Gi+mpWb{d@G=Li3zL|t)r>XeMtP{%GxvqM zxXu0AX?kOdz&wfmv7(R`UAp17<+1kocQc6*CHmTuOYLX!3p0neAB7pLCc;MG47}i2 zueTw4)=JJUtgBgLWkA}=nuuUN)Q&1##q{#7+l;%b=-QSXl-S*pdYHR+)!80kzF~N7 zwAaXdW)a(E6xseke5I9U2F&aOlAi2vreONV{Iu5KZKVd*4NGjT56}9i4I}nD>*xiX z{P|fwx@q3EwULA$I;;{MtWgy=5Xdti9BG$QDCA&VqFC)2?-`8VWw^CM(yit(;gey_ z!C|vYwiQ>1^r>o>{MF9-x`E1!hhFr-gIulHBjZy_-ceQi#N+wbRPrMR-4816xXA3l zJz3f4=R6{hPuEO7NgFUZd9P!@_zC_qV{AUY0L3_Dao}dN(xB!03M1%SwV+e(@P!@t zjz;&gPpInB2}wN~@7}3wD<9X}DA=!iI0WiZUrA3f8RQDjnsa1x0{;R=`m?s*%TU<} zKkH(OBM&l+ahtiZmyV2v zw;Z(I1~L(sf;ry4@t5Jnsy3%+Bk zsj|g!%U~7O`RHh-isQjBOR5;g@f-vGz|WJN!FcX?u-SQv2u=Gy*LKEjd2SbHX6;7D z$Q;23${|?cD0ep%$z?VQ+yGfp(jM-l~J@5J0ZXbMRSN4;g_Y;SAidvs#Qa z#&&4J`4ACXHyJRhv{oxEC;T}r(Tbd;Mnz14k)%dEi1YP*264rsrTY*QV6~vB`^qga z>vu-UX99vA6oP*weCbA}<)UFBs2gq7S|t`wosg%WA4eLg(H+G#4eQA`-xFams^5=* zo5=|1mto{C{uW+c?CMmo;#7Ej9W=N%cF3W<_yJM(_TA8(sQ5H;3>)g60D0t(nTgbk z0r#{g71eGMXF9-#tT$8{?+2Su_c(rT#j9eyyG5LQH{o+KEZhfb#EM6Xfmvki%h|37 zD2PbFA2dX$`%q#V*ob8NJX1+>Z8Zfj?^=R(VN3|A@}*$cSekjaeZskpTspJrx}Mj0 zK{pe-piF`8p*k_t!1wdsYuYBUG7|1O9>fp!R5GUbP1Zf8aPlQsI)q@_rnWteRME3I zcom<-*K$TjryOyX8?Mzei)qAb@$%S%L}*61m*v(Z?oAk7|0=5&`freU6ZBS2Xjb%_ zkx>Lt4umdt^b_*8mWP02gmHka1eN}WqUC(W{}l(<7N{BAI^Soow6-H@{ry|*yxlz7 zgQ&vR_y%1FM2xTjQASMSqZ?xg9q)R2S^4Ns8G^Bzpnd($vgi&-LOnQhGVp}JUf$U6 zO}W!s?XvOlp2Nlz_-JG9J-A1=936)lM)W5fono7`$YOH$8;*<=F#`itiGJHOPQa{ z>S?d}9P?;SEO17NsKiZr&J*pU%wxWL$CUdsCaqGKg^?~Vs;yXU&ng9PFyKVrn6Va` zSRo1v{dBXko4o~BqCaEA@R?DgXiJ=&VT-!_JE%!am$IX>JBqlRV_utB3{`g#=(UAH zPO^ehnN5%y6{m|MUz8|4wtUK>w6zE$X#`FqK;9zT2ct7@^m^ElE}$l{RsI99r}owAYK3cpqTN`kp71#QDE zhd0}he`j7O&h5SH?Aqa)=&^!(dKvWA?%K!){%R{Ogp(&)MSEXP^anXzOe>4Hhgp+w64n5)dTruBY`Wg+|$E_S}?x|-pS+c9S@h?8Acz2el@MG-IK}fd&mcf zum?6wh3W--0)Ec+r_Hz6cBqqJD`OSNbt(B+8fBo(^{CCX?hm?R7NM1w+;q)F(cFEf zZhGGc+Ym!g&zeiS9(U^wY03*nIgR^0BKaB}q77h&?;eh)94J1<6SVT|jWf-wnbBr8 zRl=ZG@GbVL#Sa?u=!8>-#af*qndkoaaQH0iq%PToJ|u+z!nURy4a;56t9_hQP1w$y zyin%zeKga1&=l+OQIs;Ux_-FRI?Qhx!3{+U2pkZh2=lGsNDJVI@ltsd$v@2S~9E*L{@O2uyo`O%rr@{F`G+DIn9b1ZL5}QReS9!+Fk(#<$XHISZq zkWC=A%a?ObUcXWII3dMg(eU0`KNWHfp2{Q4+uoj_tEgqeNSL7Ydg}y`{H?gupw$Z$ zZ{HQ`!P3I43=>J|I%pK+2QPfSrGRx7q*s+op3FDR(B=+Pw*K5kmwa<)GuDUH7ujd{vn$xZV@Cr;yJab89{#9TQfW+R_Nw z+|oTLndm?&LxKfU8Q}3&4e@*KfOl4crCNMjTAxKAEy2lEUiyilsVJ^$FR=@JG>D~N z5jAx00KDQ+EG@{Xx5``!>N&BqzJ!N-m;L=ep-(8$8@h?GiQwwu4=c_I{-E5q{O~{$t%n1tv~FSqT6?F$Kw@!ndV`@( z_1xyXZKAyp7bs1-CO6v69*0pZ@8AYqn|ySP5ecq19Q&F1b;)7f?v14S4e7cfJk4^| zwqy1BG7MTPY+lK{d>f=F=t*kU38?3ga~ZJ(H3*6$AmSB|C^7GoF231V82H*cZqqSnbTd>XTfM#mI4e@`@bH@)#O$KzS+?QQd6 z!|vHc3q!1tFECxj(i~BK??7;{mxIG?X(INpM%=^&-}LGk8Dq^$YIP`EWjAG-k$$P2 z9aF_9q5lAWD#r^-whD#KB}^-#tND@)(;1rU#k&O*Bn}UCZBg0W=u=pCAx%LG`siB= zlFY0)iQ*y_c4hdE6V9&c3)XFqHV22=BZEA1*mK&9gI09J_?uPqi@F9LBE?k3@1cu6 zRrpAw%(U>ig8bt>WpUzYlRinZfMHw6y+>qy!-pT)gCh^z_}$wZ`&3gW{6Ztw2eNKL zciLjSMpu@UkVcYS`jSE6x>|`ko84q52fnxH93I3aCj|ApXlb@cI?r*ua_hif=l*Vc z>u}rZm<5$wDM37@x=b{#G)a&j@P1M#HV99U8d%HPFvug=8R-5I^*iDE>1>m7+jW8H zb8QKYULUdWsa>La96fdPAfK|M)IzI~MLkt_`EWK9@@zZ+Gs;xp%@*t+CHF=fVm>6j zXxd(QkZ|M&zg~SC!nk+sDm1^! zUVfA@RWgD?qaa*uXM;oWBs(E@O8&#q1r2)qoTjN`TX*haGT4p+e5caDk8X;(3S&BY z!cY4LXb5eLQV*~{p}jXAqBTMVJ@tzg*Hr3?zzza;y0@0c_@a-(AvgKM&B#KSa(kvn z97jTXqDWD2ro6o7{d$@r_LQgcBun`F57djvDl6VpCz##fm$a6gS31V!XO3xG2u;`p@Poxl`8aS5i4XYVl|mNF+~|>E$0Af<9F?K% z*_zTVwTY@BtjTRT!P->_NYnk#xJt;B#x0m;lt^@?1^~bP)`Wd*xm`M!WpDN*V|{r6)Qi?g4}S)=70eSgC^WaNIv!I^1)zy}pYOv*AQ?$MpT7K$NrPm-(@H zk;X)Mo)U&bdUQ-(C0M~Y%-zSd%Wri`$!KJfcPwu5zd&W(6j_F8O&H}scbRIoT$gQv zV#T8*tWZ0Qn_`w3gL@6bc^Ca{oJgD5n%OfM^=eDE13~B=nW%-^0))b_2Gq$-G1Ur~ zo9YcEtvT`L(_Uk7UHed$`Rc^xaHufLAY(ct{hG{MQTLXgAT7D{-yknhMoa4dLg--< zYLso!d#o_;ueZWIA4o*TQPi%SwR$i?_L^7TVyDEmJ-69c@1w_dXmLqiitkQ*63>K%tHV#bFyFkKiEwyXBiB6V$$7@n;iQ|p8o>I+vlV!{xI;YP{Eu$%RqY8^MoP}bFoH0_Gh^Dq zXQ`bArPXGs5Z7B?q1i+h6njCg7Ma^klIq8W_d2(r%)7{S;?6;>pAaYp6xb{Z78{WP zu4u16t%o~#P{=;Ejl5AaM~JPHJr*vaVtw4*#>9p*7~Vsi#d~7?P~l=38JxqpA0*cu zB8$`bHenyRI@4nK49)Jc+kk6AO{p}4*3;h{@{fHVduZF9HO*y&*c%5@@%HJ`tSp~R z%I07N7O}r?fB_N%I%KQz2xOkY0YPvjDmf!@DlG8=u-1P9nL8Kee{CPEOu*dsS!n;7RSV?&)HRcD@gm zuTf#4-zTdEP@mT;FXX5Pg6K6ksT6R@KXPgCQwN)9E~tl7Fs_7ix#k*HcFHp(ZKLzwy!v~NT9KeZajO%JMZe?tgS;H`E=fQ%z@TQ~Ba_%JHK3MN({CZMR zsZKj5BGe7W=11TS{?p6u$KF0Syc9d3k<0_S@D%XB+y|-FtN>mfnVDPNloj^kY21a| zcvm`ZDNRFC;T`V>5n@^6NxBpa74#+%e>r^4rpsedyVbPdjo#dxWI{fVUT%)9RWyorrLgcrX38whzf7?xY#g_d@B~wq!s{$l z_YO^MOCvf#!tlsl-yDu%M=>qs6!9?O41X$$>To-_IIugRNOzk5F>}MKK~NFXz^(K# zQ|i|RX6!g)Z0O0oQVwOV3hwx5Je9Aoo{<)w;rX*S(X7bHQ%^pp5Lb$Q&VX<%k8_90 zfKAY}*`*=YZq@3E>~MeKCYV@DA5O5DXpV1T3|Wh&4MV+rBZ@B=lp`60MyZ;})O{aNnD(Mv@yCZ`1BB8yA%(gbk?!{S6@esN$eM2$$J*ylVU97k%pQ1QUZf;h~n zN_X5>{Dd2G^$X4Qw+Q+&Nbi*`AUWjo z;0&72GD9DH(Kr_M3QU!%%zrgFCn%kpZ3B{NUIu7kKQSjdO4&{R(`GsQK}R8ZPo-2}6frayRR+x1V-z zzZHq!BE1Wrp~LA%xe$^myKKnf;IrLQz~oQA|ccZ9!khWthaFN0yM z*PrX~p}{swbgaU<92^vRR|2l3JphnDhb)?%$W{)PgHS-Le@I;4Ub$}RM;wP z;t+C~#()wHvA|>KX7Wcz0UOvbUlwCgwuew>C@j!jUppTy_B>HTflB`%9oG3- zs`HcT(&Vdpk#@0@IAp6GQiloYbj-gDM*A@>$CHZ3F?D(@)QyCQr>?SJ9` zu?Z@73~L(u_y`m}$3eTqM)Z|e-`R4*vAe8_5IN{E?n{dTRc7?fg-Fkeg-<}JWO#DYh6plH zRi9U4pA4nIW%gGZ#pSGKCI_20!pmT;Kd5zr4neQqx0I+L|MF#uTBCNfM&qWdL57s7 zuB-eKyM>OZ8j_1fXheFz(!h(jmx($n>jPU`#?U>r8eCW;GZa)3ED>Qj#Dmq$3z#i} zv1(FM>Zq9cJaQd4na+sXc2Oi{W-A}LIUMIeZ`iqt-=C_bPGwGNJ6%@4v!U z=okvkz?>_!XC(5pk&ZKxc3&Eclrf9SBu^Uo5x9Aqv7_8UFyT#ujKkXUN%{}MIF!aO z*sI?6^04U6xk(_*cP5z+iKVK6rM?!gprzVyNB}3t>DKfi#dU1zxJ!v+)p)D>jir{+ z=bJ=N%9RRWkqgV^2M0OgWw7MXoxW@xZEqjJ5kB=6z}5etpQ=Ck{B)RwSCDwhM^Nb4 z`3_R>`#17Jcwatpt`EJSf@#q<%Sh#E($FdMCH>6Fx4^mx840Dz zNxD4eD4GrHEVSyan-Hmx$l^>Y`PRejo00Gw>uKtv!OJGr5idcW2Ya>m=S+4=7pt3SW_#01z_q+s3AjQSF-yD3q~4DAez*l{~A{5-n6;(fV>%=9N8cCtUTu16sdqwvsx zVqt`Lb@FCR2xMTQMyVf#5jB>>)GOe75L$4j#L%-eN+avQKApD=mFILL)(JwnmPqMI zIRt*|(p*c|%#O+Y3=J(E!IN;@S0^qktn&-VqJz@!8qRVEaCtVQF={YAoays5TkxQ} z_Z1%)-BzwWg`Qjf{EEjXcGGGC#0nbreL&vMHs!iZ5?D4o)l%mjR^arO=N`|Owa-)p zn}9v9ZZA;XcT`r-D|AdJ``m34rBIq;^xOxA=D%U;_T-r6S0yH)Ph z>{UFbQSdbDVO|O{>gr=oDt1fjxEr<4$kIhr;yp8R4+(92urqu6bCRNV9rXm})*L)- z>Z;bkmp`cqi#5WldW=Zj(i%mRpZrXX#`;Ky(2%=#S{=)Dw{10}*|yuRZJbkesR?6o zLo=GAYb~mBeuz`&*?~rTjz^WIT<*I7Ez-5@xgnDW8AFbB1q~x;6ck<7+E8!R^0Hh% z>rohXzA}J8rcfF-oR!T%?4n0u_w4p!EMyG5MUbqf>X_tPOYruzTRY};2#vJmUTGjP z*Jf%NH)*sF zuFgp*3HDSNb#5Y_P0-zDXzzGj%b9Pbx~hXJzuU}uTlngOUR-@S z?UGfuv{%!tn7f0Wc!}rhPixWc?W$g`fj0~V+imuGVYmb3yXZT#qq~nw)_nbtPQ6*V zSCEw+*$;L^tMlX}o#kN5&KaR!(!P7`?1wLzcgK^-LEe-i zC#UQw#fA11ZfSZ%6d+bhbqAShMU8dNYQvcD#GRW@3UH`1z$W zu=RZwg~?q=%7#Fe$vgW(*|p622LWU=8@Q`#;?c9^sva*t`zrZcI9CzDN*o79Owjtv zY!ecWwNbUm?H|No$ONL&kf~Kj1L22p?MPc|_)#19I-=)d@iQCXSQ`Ago)z+0PXg`y ztkp3WlX$tK$@mi1p^$;tlsfs9D0fa%N8~ZwM6h0AEvP-oGYMM;UNLgMwII*#3>ThO zbmtK*3?%nT4xeVTML~d9U**VzrgiHpx}9A}UhSd|892K*n|8-egGbh%i7mrN z>(yp?ZrfImf7uadKQh3YXXa}iA!@vN8sJ1rQh_TKtI9ZKH+;Icfe=F5ahshAR0!GR zb%c8J>^L&tS;-(CSgk}|a_;AyW0(B}mYo_|9r?HYPhJfKg^2~YvGR)c+ydpIs1sIp z6gnoW5`7!n1T0AyMT>khT4RtDU0y<@oup63B_$^anau|mCxGN8%co6CtpbNumW9;b zw}gcTBONUv5`Wmg5~gjE!Vgwyn8Nb%@T6iXr<`T5)q~4uK7mj1#!IsoT2D8$-wka? zN3K8c9o_7;gu}#hSn%j`y7WICqyV⪙J}(Os<^FeH|S1Qt!O~;$y(;~ zK;n8)*Etgh1y?uG38wma9mup^iwUd{A=xCMc2I*DSi3gS4gCAt9B3+K8UzV(u>s>< zo2oGPP4xWJ=|0dsChcjF^1}!)hlJ6oumjnl@gxW^)*L^oh><$2=Qm4_fmopxU{l4E4{Tx+QUpFal~lF&F0gzbPPw2|o6cX0A@M=G zVbgX_F`~8d;@Tm2#r%!7`(C%mhmZBZyRil3Q=pK9Ny1=)Gs$Wtd|~FEyKonK=L^YB z?%(NwQE17DtibYJ7ju%HNnUuR!M^3x8pKcc#0kQXY)lKUl;q{$&SH{rm()UbZ%Xo; zxKN}U{Dvb!j_D|B0eSdaY{-*{_bO�vB0{!f)Qf_AjL%>6H`>bb+SDnn;u5AU=S; zp<@RwJ#`TND4v_$6g}i5oe^Ksgm{5qrHE$Y^UA6me7tii8)s306B+T7SMH4;l~^(0 zHGOk2`cuzhT;3(5)x}tcaM76-ha`1A)F@eDaAI;~G7(4JSwPp+x{F5b!)`tz6t0R^ ziFlN7>=)DIqO=qGI$%J+Hg5&5_XefTUh#n(ZfvMhSG0VPnj3HMd>pa!a_Zag95CB> zJ*i0iWZ~DoXW!qS2i~FtCh5R@_xp8Wl5S_>Zf?)~?@al>zf1$%_Wfb|w_gU%IA!{F z#wnAPwYxdf_k&J$9lSuBLI5=P~(Gui2nZw zJ381gdD_{oYs&s$cOslDTcNtC6NxV3>ZWze7EbM8aW=&Xt6^v3MRO4xHS{>E_lEq|M z4ig?p{3QP7F1RqdlbW>0OR5-5;ks`kLdg%nPaeA}9b?srK$cn8^em1Wj0O~~j!4?{&p9`rb-nl`B*oRLTJshn9N zwUkwUZte(usYR}62BzhP*8 z>!}(QBuZwX?mR@u6OXHw^%xHI%_uVNP{n-=UHgRvF8-n+t1=hbEcU_TcGW^3JKj?^ z*7Q0<|Hwe(Ccak-l}Q!ynMY%yCGq!Z$By69JuTC1YkEptC|1@R+Ez7~B|xXv+8Z}n zjTH@f;8$F%)B;yGR$Ek1#OV8M)}oDFk`J9l!NA2M|DL;}oJE{-zTgNUd|ryQ%6xrQ zpd$SlIt}_hA8Zn zqjt1U8AZfL9EZS{XUiq9=~9Sj_h$5_4dU#-`-;%kA+G-x(nFFgp4Qa)f%lZXRxchj z5h%v=cBqzMmTu za^8EY(K8Euem*lDDa^V+(=ra-{l}y z{E~tBDc#Hi^C?Cx7MX?;;#4w`!thZ)znv`3iZSl9`^{Z9*5eGjKHQ_%?B!%;9Jkg< zxichE(6U%^n@CRQ1ZK*KNGWxPA1z`-bfNqR^{(SlfFfmd^zBFWng?GtSc^dT1wU*G zl!tzdM=jcpB-~)8O_1UDStUApHfGNxnLB-o5!k;_KNe?``rP@^;Jxm>!D-y3BbQX{ zvQX9M82D^-c0Qxv935Tr(@_OSytt!y{IOza+W|YtVJe4fXRf~NlUk#n8#m4Bw9zSp@o#pO5LTwfRo_ehgieWy$s$H-uRvIff$+H@ho)UrCTme*xL zYJoP|Fz3~iHx-9?>Z4lzdh+?N-Eu%FrE~4OlHU$4jxm?I-yUZgWwnLoyZJWgmPG(B z$FOYtF|Q=!z+$Od_7?KLK^z7oHrHI@3(qN;D-dJ-fRD`oKh`=rJ2;v zHQzuH-v1I7*o!EW3lrQsn+5~dYw48*+7ZtNb$n-TdV6}yuyUzeHrTSxcT+l^&9%~k zI>)MVd2)H3`{Sl}_47)5@c7+$3Z-{dP28b$8@o4@o)7fPnwXSkyhnM_53C+|^f^uW z{SN$z&}Wgj8F*zmP%sQ|IDichkn_&<*MI!;7wq>RP(ap2&VTrx^?uw21Hud3#&1~R zHVFJeF|eKyG}|DExEBKc|jgb>sQMxo_9`jqVq3JWsnT(8#j z`GK)NNHc+7A=j_Ge;=>4ouloQgQonK@K>pN{*khasTH8KD@hVxAX$4ANe2fn>#I2@ zxj^jTDq^M%c6JWumGh67@~dl~*UsOrbG>r{p8i|=boq8Hwbi#{sjsAZ(Sf|LqH1Mw zwGEN{JHD^#sJx1lv#Eu(?bX5~zd&vGDr$Bnj?U(0Zl+fg{1IHg$`2^{f0myq;C8kS zSBFZR3q!K{4TV3&8QKd}Qm>%$S0UzNKwbC^**``XvJ0al08WLuOiC9W_4-z;e=T}{ zjwu&o;PY!y{bM*Gxxo6;RYb25N$@UEqXafGxJ;0L4kEy`^q=e8Yz=-x=Nj?jqC-9U z4WYjX9T(kUfkWW$t|bBF^Q=%>4cG{eTOy0h@M7UO-0R{#)d4tcn8>{w9F;dv2Y> zHB`!gvepqO1*I=w3Jd)$=4ISo&S_M7sA?%7V=*8>{V2oEjYU`a52)uL;?97s|JmQ< zS9?M68o@sVNCT%bfdHoX{mSsy*13Z8*XS;9;&uC+{uRozbFQ;*SeNT- z+3roTDPRj8fX=x7MF74t{F`1~8$<$VHMrjXeY&JH^QLYF0K9Sk^GTQon((vcScN+Z{L{s~vZztK_x{Ki(AHBR zb^^1c?^lMu%fYqLeu^@e8=+)PF%vsr50SuBiQ_xQxtgTE{0ZfsisOevUk)J`{<;Yd z5Ss)5bJ5QGz(D^Xwtspv=Ma+C=C)>PU%S2(z>nsZK(%{0AOe$a-YOuueE?A60#C9) z$i7-nKLLKj(r~r5b)ir&alA71w3@H-6#c4b=MZf81X~P;guh^7b}8ggp>UD$#zmA@PJigGe|#J2?N|zqSqNMHc`~6#qvPXK8;X zF8zxpepBC+$gJ_X1Jks}z}f`q3x>**{`VlifXbQwCWD%h-B%X}lp_itUsMj}i@!%@ zzg#&@62na&fZBH#$SSHASS!r^dyud8{IvlN2v)(|)kFProQ0n;z03ZZNXKWXkGzYx+k03yx0m^!y!DH|0ayh`q zTQicHfSS4h1Gxww-f%qt2~SgV$MZVlasa_C)h=E@O)@|{V7b7-TJ!Y)u58=DdguxY z16p7J5{cLa@~-VykpJb;mbjap-PaDch?}d`)!na}ZY9JWAm|JL6|d+8K+Ntd0M8$N zNt-x-n?;KOPx!4}fsT~2gNrKwXKL=^;_#cv>?CXrk1BpxSx9lze{$<(&FLJ z0Dp(bzw);{_5T9AoWBmrgoz%2KL^0xg)fLx5eUYA$=^>i^~=#JbZVp00Lyfz__b|* zd*uqWA5Rm{ONH}J-_Me~9LjVtr#}~P;AQ&ezp&auiSl0V$LR}w|>_S zt~d6Niw1%{44}k|7IW|TTEIVT8~o39{Xk;TG_iI2P1BpXBzX7)2>PeMquGnkB5t2u z3s4g9I&SxF8ys`%;iE=)O@iP3h1v2(BH)_!~i_l@2%yc{(ctfIsK{^E?H824H*{}tia|M`Y|Rh67*H%sRaDDDXmT{tgTjVSrGkk1YD>w{w%dkcrl zx$8pNq!f#Uqge$fs$*!?%kT=&SV?SilJ% zzqdfa4EkDJfS3C~4RWzmgE6`m#<`V$OQOFQnq<}?V?F?K7YI!kV}-lvwLpMEz|tA0 z1XRr(9h?CbU9OuP5ub`4KsN#al-31{yKQwXnE%F#9|`7rVe$vFZ;GO?u?7Zk@}Huv z*5+D+P&RP}`rN;Z7~r6XsZ&5ZvEBbyuX1s?7Wp4ud|i%a)IyXN0k0|t76M#+C{vbs zEy(|f^>X!z6zMjY0Kwu2D7G(J$#&(naL!HfXZN}sW)6&d$pWwvc%ak~xnL!^UDty7 z260{KfxKsA9tXlZ6(GTjVbG!HT5$h_!JirXg9)~#w9cKcP!|SD5B>`r77SerQthAB zUldbR%xuE8a6q|=YRn-PCcd^HC9vfZ&P%AHMi-z-Rkfc;6KDPtg5cX z+SS^`*4oSbYb|=YscR#kSKJ4983ybRaH{WDhJQ0CuvHK6%ELb|FnoOv;rqJ%ANla> zOyBwU(P%S?hFui^cnApVV)jUexE8?A3HNuha#;^rBn2u>E|OoT$zssg0{PdRFBcNo z#Hi3AF#hw^4Hs>=6ytj2fn`E~D#U=f$Zt9ch}xs1M}Q90fb#2NtLY5)S`Zgvz;z?f z{<`!15TF2C08lS7W<+u=)UTFxW%U4$hZ>3jbO0%U)NnDvwlK@4NQ^os?@h%sD@5KaNHHbtO$_ z_+l9}K;Qa5hTt!KE&BgSrI(%G_ABc9m@Ih>AWHLKf7=KCU&b>0KPml>s6UP4yE$I& zUUY-hvco_!q6S<{{a<9@JHx+N`+o#KH<-VO=xl)n``}o>2!LlDlov*@uX}L>rvJR% zaQ<$>xh8+R!q_A;m*pK22uKj=ud@IFr{9zO-xe2#;{*#S0oudC`*j6{gWq3ZeYMPM zuPfevIs9@8a5pZ%5iS-<H$gtYKHGuhJWw>m3BQqRaI#mOWmy%id8Y978Mg*HFXyT64q4|6cJDyLNm>Vz=cq2 zVbcVwXN0-pkeY^QB-2w1DgS`3)~j8`DBX5T63|S{v|`r9YKH`sV1I|_d-s0#oqNu? zbZ6c$I&;6@`Tm~oocp`?R0%|z(}&BiJ5qu$Cw+uz@3(GoEH797O(4)sz%`q#8?J?) zT?o0&2-gurZnHxmmzma?(|m{B}^|a8uvm^H0Nx`w@9`66^e?CvbeF>K=SUVxO+yc08GpcK4Z^!`~+do_cG?uq)vfQ%`N7%;F!!`9QZ3dUIULulku+U z5{T|GBlwHDZdCyy zOgDaDoeW(UJr;IrfWK*XG^ei$1oMw*$92W7>=RjVc^=fD@=xEcB3~n$@yGv%yt)$~ zLG~Va zKt+DH?VUyvGYqdsEqlNlada4p&{&EXqypr8ipF*+OgZIWdClxog?#A1k3y8(qXMDV z#pD71_xFAAh7CK8lkm1OZ7Ry{Re`X~?=2bj@pla8v z)`g6)MnuN5SRq9d^kbuc9y&kT$Gm?2y$VF*I%E^w7Q7Lvf~RGZI2c=Adtu5x=yMIf zP@>mQ;TCaB0%e;$zIZ?C4*DL$#-rB*DhOKL5t3XxDuhKF3L&p$4L-3Swdtux6*V`~CY86uJzoiNst-lhm+u z+)gQBwyv@8?cG4@g88NrwBEC=qbYJ_-yPf5uoP{<@6mJ8ay|eQ8vVuzX>)Ww)0vYW zbY0ipnXndX?p0V-P>6;{)F56<+Q=jA_}WQ8rRiJtU&OkXFQy@Yy!&6LO6#y zX6SM&(%D{lPWO!9R7l8+O}{uNb}8b#1<^pO^aoF?f$AJ1pv1m;a!dPeY!{JHuZNQL3ndvo^f$~Te*^8eq-7X86@yWNqcIjb8lCNuhKJ z3w6(@V^S}|D%cP*SmpE2g>nsCP%~#1*ZZX)2fYelnaL!qz0X;aZ@@F79kcek`d7tBZ%wS%Xhy(0@x-Yvepj|3kLNT3k zwCl>taLFF@oT$w1%WkXfkjfS=;4=M#N5sS8VR$k}8z}oNeYi7z&xxfM=N;MFg`llR zze#Cr@qPJNJ?4~7tUk4QG3LrrK+z{!JAC+1e14~MFVi^%@yk~ImiUFKFyM5A6*XY? zKr2XkVt`qx**Kyora!c52a}X)73kknEWTE2?D>K0-I8EvMHe|mfmV=oOJF`wLDcrr zNMvImx*DodG`6*Q9jE)7Hq#!<$YE^erGL&`@hGEbi)qbsLk1>pMP2D1;QSn;D~as3 z#s~@0O(Nx#^z8IS-AEQ-1A_vmHKSUWB7JFmoS7tnHC50oKIZ>W6`6oUzA@CeF@0|g zA4}s1u%1PuNjYh%e3x||pY`|-6uPr8W~|GK_-nWQBUuGSpHr$7+^b*b-(CzWwt86c zICe^TVWMfKd&sH8zKvLSD0w92z77Nm-QKT$NQR|OybYUqVs!z0H7IkVzQ@i9sX1eOm-eyq%_9t`_*8CY?#XW3Hv&s#p{^==vM#^9+rtWb-r22err=_ zMHS3dh=N16MSUGIFf}^u;5M%f$Dz*sKuIP6=66Jf;8_N=DZ)({k$7v~=JdJ+z&_^D zyC&7h@WmwBf1df@&Rn4_?NoXef zE~hU{M(f%W0Wu`QlT)gqWgjkjCLVg>6$ry-bJY+Tnu}K`rO9??Hh$8KnzI+~r?bNB zAQ^~7`l|Qc{lf4fg4bpw4&D3T94>=$`bolET|cWdF$NhqILHVPHC_gwL|Q%R822ak z(o_>-fG&sYVr2j_zN*jN6%!fXV->7Jw+Z&d$zAfsg7C$wgS*zX1*8A#KiIgW&q|VExFVLHC72QywuvA zuWor2ky_(pOo#FLDj?Ez?uPnf{j;(65d}@4i2JhJLRP5&2-v8$nqH-SAUegv-{;rJ zM0-q3DnwM@!ZTMAu=w)@KW@5P6Bb}G>#YOohTsl3eqg^+`9t$(DDrn9?UM-Z^)1$M zEznQdfd|#qUllIKzRLnM95A%|vfIXQ>#H4jFayW-CeDu0b9Z?M{km?q+v<1g?oRAJ z_Sbl`@$_Bo=9QJYoAqb$Y9bJF7`^Kj_7_Uryx2FESG=ne;X2Vs`!2ij<`CDt-Jd<* z<#{vei`fZB=2EpSCN`uN3sTuD4<08Txq$d*9=SBI+hQFax| Date: Mon, 13 Apr 2015 20:43:24 -0700 Subject: [PATCH 751/817] [Minor][SparkR] Minor refactor and removes redundancy related to cleanClosure. 1. Only use `cleanClosure` in creation of RRDDs. Normally, user and developer do not need to call `cleanClosure` in their function definition. 2. Removes redundant code (e.g. unnecessary wrapper functions) related to `cleanClosure`. Author: hlin09 Closes #5495 from hlin09/cleanClosureFix and squashes the following commits: 74ec303 [hlin09] Minor refactor and removes redundancy. --- R/pkg/R/RDD.R | 16 ++++------------ R/pkg/R/pairRDD.R | 4 ---- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d6a75007a6ad7..820027ef67e3b 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -85,7 +85,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { # This transformation is the first in its stage: - .Object@func <- func + .Object@func <- cleanClosure(func) .Object@prev_jrdd <- getJRDD(prev) .Object@env$prev_serializedMode <- prev@env$serializedMode # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD @@ -94,7 +94,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) pipelinedFunc <- function(split, iterator) { func(split, prev@func(split, iterator)) } - .Object@func <- pipelinedFunc + .Object@func <- cleanClosure(pipelinedFunc) .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline # Get the serialization mode of the parent RDD .Object@env$prev_serializedMode <- prev@env$prev_serializedMode @@ -144,17 +144,13 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), return(rdd@env$jrdd_val) } - computeFunc <- function(split, part) { - rdd@func(split, part) - } - packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - serializedFuncArr <- serialize(computeFunc, connection = NULL) + serializedFuncArr <- serialize(rdd@func, connection = NULL) prev_jrdd <- rdd@prev_jrdd @@ -551,11 +547,7 @@ setMethod("mapPartitions", setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { - FUN <- cleanClosure(FUN) - closureCapturingFunc <- function(split, part) { - FUN(split, part) - } - PipelinedRDD(X, closureCapturingFunc) + PipelinedRDD(X, FUN) }) #' @rdname lapplyPartitionsWithIndex diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index c2396c32a7548..739d399f0820f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -694,10 +694,6 @@ setMethod("cogroup", for (i in 1:rddsLen) { rdds[[i]] <- lapply(rdds[[i]], function(x) { list(x[[1]], list(i, x[[2]])) }) - # TODO(hao): As issue [SparkR-142] mentions, the right value of i - # will not be captured into UDF if getJRDD is not invoked. - # It should be resolved together with that issue. - getJRDD(rdds[[i]]) # Capture the closure. } union.rdd <- Reduce(unionRDD, rdds) group.func <- function(vlist) { From 971b95b0c9002bd541bcbe0da54a9967ba22588f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 13 Apr 2015 21:18:05 -0700 Subject: [PATCH 752/817] [SPARK-5957][ML] better handling of parameters The design doc was posted on the JIRA page. Python changes will be in a follow-up PR. jkbradley 1. Use codegen for shared params. 1. Move shared params to package `ml.param.shared`. 1. Set default values in `Params` instead of in `Param`. 1. Add a few methods to `Params` and `ParamMap`. 1. Move schema handling to `SchemaUtils` from `Params`. - [x] check visibility of the methods added Author: Xiangrui Meng Closes #5431 from mengxr/SPARK-5957 and squashes the following commits: d19236d [Xiangrui Meng] fix test 26ae2d7 [Xiangrui Meng] re-gen code and mark clear protected 38b78c7 [Xiangrui Meng] update Param.toString and remove Params.explain() 409e2d5 [Xiangrui Meng] address comments 2d637bd [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 eec2264 [Xiangrui Meng] make get* public in Params 4090d95 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 4fee9e7 [Xiangrui Meng] re-gen shared params 2737c2d [Xiangrui Meng] rename SharedParamCodeGen to SharedParamsCodeGen e938f81 [Xiangrui Meng] update code to set default parameter values 28ed322 [Xiangrui Meng] merge master 55be1f3 [Xiangrui Meng] merge master d63b5cc [Xiangrui Meng] fix examples 29b004c [Xiangrui Meng] update ParamsSuite 94fd98e [Xiangrui Meng] fix explain params 48d0e84 [Xiangrui Meng] add remove and update explainParams 4ac6348 [Xiangrui Meng] move schema utils to SchemaUtils add a few methods to Params 0d9594e [Xiangrui Meng] add getOrElse to ParamMap eeeffe8 [Xiangrui Meng] map ++ paramMap => extractValues 0d3fc5b [Xiangrui Meng] setDefault after param a9dbf59 [Xiangrui Meng] minor updates d9302b8 [Xiangrui Meng] generate default values 1c72579 [Xiangrui Meng] pass test compile abb7a3b [Xiangrui Meng] update default values handling dcab97a [Xiangrui Meng] add codegen for shared params --- .../examples/ml/JavaDeveloperApiExample.java | 4 +- .../examples/ml/DeveloperApiExample.scala | 6 +- .../scala/org/apache/spark/ml/Estimator.scala | 2 +- .../scala/org/apache/spark/ml/Pipeline.scala | 10 +- .../org/apache/spark/ml/Transformer.scala | 5 +- .../spark/ml/classification/Classifier.scala | 17 +- .../classification/LogisticRegression.scala | 18 +- .../ProbabilisticClassifier.scala | 11 +- .../BinaryClassificationEvaluator.scala | 15 +- .../apache/spark/ml/feature/HashingTF.scala | 6 +- .../apache/spark/ml/feature/Normalizer.scala | 7 +- .../spark/ml/feature/StandardScaler.scala | 9 +- .../spark/ml/feature/StringIndexer.scala | 10 +- .../apache/spark/ml/feature/Tokenizer.scala | 16 +- .../spark/ml/feature/VectorAssembler.scala | 7 +- .../spark/ml/feature/VectorIndexer.scala | 25 +- .../spark/ml/impl/estimator/Predictor.scala | 16 +- .../org/apache/spark/ml/param/params.scala | 236 ++++++++++------ .../ml/param/shared/SharedParamsCodeGen.scala | 169 ++++++++++++ .../spark/ml/param/shared/sharedParams.scala | 259 ++++++++++++++++++ .../apache/spark/ml/param/sharedParams.scala | 173 ------------ .../apache/spark/ml/recommendation/ALS.scala | 49 ++-- .../ml/regression/LinearRegression.scala | 8 +- .../spark/ml/tuning/CrossValidator.scala | 18 +- .../apache/spark/ml/util/SchemaUtils.scala | 61 +++++ .../apache/spark/ml/param/ParamsSuite.scala | 47 +++- .../apache/spark/ml/param/TestParams.scala | 12 +- 27 files changed, 820 insertions(+), 396 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 19d0eb216848e..eaf00d09f550d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -116,7 +116,7 @@ class MyJavaLogisticRegression */ IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - int getMaxIter() { return (Integer) get(maxIter); } + int getMaxIter() { return (Integer) getOrDefault(maxIter); } public MyJavaLogisticRegression() { setMaxIter(100); @@ -211,7 +211,7 @@ public Vector predictRaw(Vector features) { public MyJavaLogisticRegressionModel copy() { MyJavaLogisticRegressionModel m = new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); - Params$.MODULE$.inheritValues(this.paramMap(), this, m); + Params$.MODULE$.inheritValues(this.extractParamMap(), this, m); return m; } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index df26798e41b7b..2245fa429fda3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -99,7 +99,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) } /** @@ -174,11 +174,11 @@ private class MyLogisticRegressionModel( * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. * - * This is used for the defaul implementation of [[transform()]]. + * This is used for the default implementation of [[transform()]]. */ override protected def copy(): MyLogisticRegressionModel = { val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index eff7ef925dfbd..d6b3503ebdd9a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -40,7 +40,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { */ @varargs def fit(dataset: DataFrame, paramPairs: ParamPair[_]*): M = { - val map = new ParamMap().put(paramPairs: _*) + val map = ParamMap(paramPairs: _*) fit(dataset, map) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index a455341a1f723..8eddf79cdfe28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -84,7 +84,7 @@ class Pipeline extends Estimator[PipelineModel] { /** param for pipeline stages */ val stages: Param[Array[PipelineStage]] = new Param(this, "stages", "stages of the pipeline") def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this } - def getStages: Array[PipelineStage] = get(stages) + def getStages: Array[PipelineStage] = getOrDefault(stages) /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an @@ -101,7 +101,7 @@ class Pipeline extends Estimator[PipelineModel] { */ override def fit(dataset: DataFrame, paramMap: ParamMap): PipelineModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) // Search for the last estimator. var indexOfLastEstimator = -1 @@ -138,7 +138,7 @@ class Pipeline extends Estimator[PipelineModel] { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) require(theStages.toSet.size == theStages.size, "Cannot have duplicate components in a pipeline.") @@ -177,14 +177,14 @@ class PipelineModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(paramMap) transformSchema(dataset.schema, map, logging = true) stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(paramMap) stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 9a5848684b179..7fb87fe452ee6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -22,6 +22,7 @@ import scala.annotation.varargs import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -86,7 +87,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O protected def validateInputType(inputType: DataType): Unit = {} override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType validateInputType(inputType) if (schema.fieldNames.contains(map(outputCol))) { @@ -99,7 +100,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.withColumn(map(outputCol), callUDF(this.createTransformFunc(map), outputDataType, dataset(map(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index c5fc89f935432..29339c98f51cf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,12 +17,14 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared.HasRawPredictionCol +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.functions._ import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -42,8 +44,8 @@ private[spark] trait ClassifierParams extends PredictorParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(rawPredictionCol), new VectorUDT) } } @@ -67,8 +69,7 @@ private[spark] abstract class Classifier[ with ClassifierParams { /** @group setParam */ - def setRawPredictionCol(value: String): E = - set(rawPredictionCol, value).asInstanceOf[E] + def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) } @@ -109,7 +110,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 34625745dd0a8..cc8b0721cf2b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -31,8 +31,10 @@ import org.apache.spark.storage.StorageLevel * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams - with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold + with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold { + setDefault(regParam -> 0.1, maxIter -> 100, threshold -> 0.5) +} /** * :: AlphaComponent :: @@ -45,10 +47,6 @@ class LogisticRegression extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { - setRegParam(0.1) - setMaxIter(100) - setThreshold(0.5) - /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -100,8 +98,6 @@ class LogisticRegressionModel private[ml] ( extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams { - setThreshold(0.5) - /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) @@ -123,7 +119,7 @@ class LogisticRegressionModel private[ml] ( // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. @@ -184,7 +180,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { - if (score(features) > paramMap(threshold)) 1 else 0 + if (score(features) > getThreshold) 1 else 0 } override protected def predictProbabilities(features: Vector): Vector = { @@ -199,7 +195,7 @@ class LogisticRegressionModel private[ml] ( override protected def copy(): LogisticRegressionModel = { val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(this.extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index bd8caac855981..10404548ccfde 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} - /** * Params for probabilistic classification. */ @@ -37,8 +38,8 @@ private[classification] trait ProbabilisticClassifierParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(probabilityCol), new VectorUDT) } } @@ -102,7 +103,7 @@ private[spark] abstract class ProbabilisticClassificationModel[ // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 2360f4479f1c2..c865eb9fe092d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -20,12 +20,13 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Evaluator import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType - /** * :: AlphaComponent :: * @@ -40,10 +41,10 @@ class BinaryClassificationEvaluator extends Evaluator with Params * @group param */ val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (areaUnderROC|areaUnderPR)", Some("areaUnderROC")) + "metric name in evaluation (areaUnderROC|areaUnderPR)") /** @group getParam */ - def getMetricName: String = get(metricName) + def getMetricName: String = getOrDefault(metricName) /** @group setParam */ def setMetricName(value: String): this.type = set(metricName, value) @@ -54,12 +55,14 @@ class BinaryClassificationEvaluator extends Evaluator with Params /** @group setParam */ def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(metricName -> "areaUnderROC") + override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema - checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(rawPredictionCol), new VectorUDT) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index fc4e12773c46d..b20f2fc49a8f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -35,14 +35,16 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { * number of features * @group param */ - val numFeatures = new IntParam(this, "numFeatures", "number of features", Some(1 << 18)) + val numFeatures = new IntParam(this, "numFeatures", "number of features") /** @group getParam */ - def getNumFeatures: Int = get(numFeatures) + def getNumFeatures: Int = getOrDefault(numFeatures) /** @group setParam */ def setNumFeatures(value: Int): this.type = set(numFeatures, value) + setDefault(numFeatures -> (1 << 18)) + override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index 05f91dc9105fe..decaeb0da6246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -35,14 +35,16 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { * Normalization in L^p^ space, p = 2 by default. * @group param */ - val p = new DoubleParam(this, "p", "the p norm value", Some(2)) + val p = new DoubleParam(this, "p", "the p norm value") /** @group getParam */ - def getP: Double = get(p) + def getP: Double = getOrDefault(p) /** @group setParam */ def setP(value: Double): this.type = set(p, value) + setDefault(p -> 2.0) + override protected def createTransformFunc(paramMap: ParamMap): Vector => Vector = { val normalizer = new feature.Normalizer(paramMap(p)) normalizer.transform @@ -50,4 +52,3 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { override protected def outputDataType: DataType = new VectorUDT() } - diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 1142aa4f8e73d..1b102619b3524 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ @@ -47,7 +48,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } val scaler = new feature.StandardScaler().fit(input) val model = new StandardScalerModel(this, map, scaler) @@ -56,7 +57,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") @@ -86,13 +87,13 @@ class StandardScalerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 61e6742e880d8..4d960df357fe9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -22,6 +22,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StringType, StructType} @@ -34,8 +36,8 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - checkInputColumn(schema, map(inputCol), StringType) + val map = extractParamMap(paramMap) + SchemaUtils.checkColumnType(schema, map(inputCol), StringType) val inputFields = schema.fields val outputColName = map(outputCol) require(inputFields.forall(_.name != outputColName), @@ -64,7 +66,7 @@ class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase // TODO: handle unseen labels override def fit(dataset: DataFrame, paramMap: ParamMap): StringIndexerModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val counts = dataset.select(map(inputCol)).map(_.getString(0)).countByValue() val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray val model = new StringIndexerModel(this, map, labels) @@ -105,7 +107,7 @@ class StringIndexerModel private[ml] ( def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val indexer = udf { label: String => if (labelToIndex.contains(label)) { labelToIndex(label) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 68401e36950bd..376a004858b4c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -56,39 +56,39 @@ class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenize * param for minimum token length, default is one to avoid returning empty strings * @group param */ - val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length", Some(1)) + val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length") /** @group setParam */ def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) /** @group getParam */ - def getMinTokenLength: Int = get(minTokenLength) + def getMinTokenLength: Int = getOrDefault(minTokenLength) /** * param sets regex as splitting on gaps (true) or matching tokens (false) * @group param */ - val gaps: BooleanParam = new BooleanParam( - this, "gaps", "Set regex to match gaps or tokens", Some(false)) + val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") /** @group setParam */ def setGaps(value: Boolean): this.type = set(gaps, value) /** @group getParam */ - def getGaps: Boolean = get(gaps) + def getGaps: Boolean = getOrDefault(gaps) /** * param sets regex pattern used by tokenizer * @group param */ - val pattern: Param[String] = new Param( - this, "pattern", "regex pattern used for tokenizing", Some("\\p{L}+|[^\\p{L}\\s]+")) + val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") /** @group setParam */ def setPattern(value: String): this.type = set(pattern, value) /** @group getParam */ - def getPattern: String = get(pattern) + def getPattern: String = getOrDefault(pattern) + + setDefault(minTokenLength -> 1, gaps -> false, pattern -> "\\p{L}+|[^\\p{L}\\s]+") override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { str => val re = paramMap(pattern).r diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index d1b8f7e6e9295..e567e069e7c0b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -22,7 +22,8 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Transformer -import org.apache.spark.ml.param.{HasInputCols, HasOutputCol, ParamMap} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} import org.apache.spark.sql.{Column, DataFrame, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -44,7 +45,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val assembleFunc = udf { r: Row => VectorAssembler.assemble(r.toSeq: _*) } @@ -61,7 +62,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputColNames = map(inputCols) val outputColName = map(outputCol) val inputDataTypes = inputColNames.map(name => schema(name).dataType) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 8760960e19272..452faa06e2021 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -18,10 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.{BinaryAttribute, NumericAttribute, NominalAttribute, Attribute, AttributeGroup} -import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap, Params} +import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, VectorUDT} import org.apache.spark.sql.{Row, DataFrame} import org.apache.spark.sql.functions.callUDF @@ -40,11 +42,12 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu */ val maxCategories = new IntParam(this, "maxCategories", "Threshold for the number of values a categorical feature can take." + - " If a feature is found to have > maxCategories values, then it is declared continuous.", - Some(20)) + " If a feature is found to have > maxCategories values, then it is declared continuous.") /** @group getParam */ - def getMaxCategories: Int = get(maxCategories) + def getMaxCategories: Int = getOrDefault(maxCategories) + + setDefault(maxCategories -> 20) } /** @@ -101,7 +104,7 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def fit(dataset: DataFrame, paramMap: ParamMap): VectorIndexerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val firstRow = dataset.select(map(inputCol)).take(1) require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") val numFeatures = firstRow(0).getAs[Vector](0).size @@ -120,12 +123,12 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // We do not transfer feature metadata since we do not know what types of features we will // produce in transform(). - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexer requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexer requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) - addOutputColumn(schema, map(outputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) + SchemaUtils.appendColumn(schema, map(outputCol), dataType) } } @@ -320,7 +323,7 @@ class VectorIndexerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val newField = prepOutputField(dataset.schema, map) val newCol = callUDF(transformFunc, new VectorUDT, dataset(map(inputCol))) // For now, just check the first row of inputCol for vector length. @@ -334,13 +337,13 @@ class VectorIndexerModel private[ml] ( } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexerModel requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexerModel requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index dfb89cc8d4af3..195333a5cc47f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -18,8 +18,10 @@ package org.apache.spark.ml.impl.estimator import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -53,14 +55,14 @@ private[spark] trait PredictorParams extends Params paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector - checkInputColumn(schema, map(featuresCol), featuresDataType) + SchemaUtils.checkColumnType(schema, map(featuresCol), featuresDataType) if (fitting) { // TODO: Allow other numeric types - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) } - addOutputColumn(schema, map(predictionCol), DoubleType) + SchemaUtils.appendColumn(schema, map(predictionCol), DoubleType) } } @@ -98,7 +100,7 @@ private[spark] abstract class Predictor[ // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val model = train(dataset, map) Params.inheritValues(map, this, model) // copy params to model model @@ -141,7 +143,7 @@ private[spark] abstract class Predictor[ * and put it in an RDD with strong types. */ protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.select(map(labelCol), map(featuresCol)) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) @@ -201,7 +203,7 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 7d5178d0abb2d..849c60433c777 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -17,15 +17,14 @@ package org.apache.spark.ml.param +import java.lang.reflect.Modifier +import java.util.NoSuchElementException + import scala.annotation.varargs import scala.collection.mutable -import java.lang.reflect.Modifier - import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable -import org.apache.spark.sql.types.{DataType, StructField, StructType} - /** * :: AlphaComponent :: @@ -38,12 +37,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * @tparam T param value type */ @AlphaComponent -class Param[T] ( - val parent: Params, - val name: String, - val doc: String, - val defaultValue: Option[T] = None) - extends Serializable { +class Param[T] (val parent: Params, val name: String, val doc: String) extends Serializable { /** * Creates a param pair with the given value (for Java). @@ -55,58 +49,55 @@ class Param[T] ( */ def ->(value: T): ParamPair[T] = ParamPair(this, value) + /** + * Converts this param's name, doc, and optionally its default value and the user-supplied + * value in its parent to string. + */ override def toString: String = { - if (defaultValue.isDefined) { - s"$name: $doc (default: ${defaultValue.get})" + val valueStr = if (parent.isDefined(this)) { + val defaultValueStr = parent.getDefault(this).map("default: " + _) + val currentValueStr = parent.get(this).map("current: " + _) + (defaultValueStr ++ currentValueStr).mkString("(", ", ", ")") } else { - s"$name: $doc" + "(undefined)" } + s"$name: $doc $valueStr" } } // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** Specialized version of [[Param[Double]]] for Java. */ -class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double]) - extends Param[Double](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class DoubleParam(parent: Params, name: String, doc: String) + extends Param[Double](parent, name, doc) { override def w(value: Double): ParamPair[Double] = super.w(value) } /** Specialized version of [[Param[Int]]] for Java. */ -class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int]) - extends Param[Int](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class IntParam(parent: Params, name: String, doc: String) + extends Param[Int](parent, name, doc) { override def w(value: Int): ParamPair[Int] = super.w(value) } /** Specialized version of [[Param[Float]]] for Java. */ -class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float]) - extends Param[Float](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class FloatParam(parent: Params, name: String, doc: String) + extends Param[Float](parent, name, doc) { override def w(value: Float): ParamPair[Float] = super.w(value) } /** Specialized version of [[Param[Long]]] for Java. */ -class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long]) - extends Param[Long](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class LongParam(parent: Params, name: String, doc: String) + extends Param[Long](parent, name, doc) { override def w(value: Long): ParamPair[Long] = super.w(value) } /** Specialized version of [[Param[Boolean]]] for Java. */ -class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean]) - extends Param[Boolean](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class BooleanParam(parent: Params, name: String, doc: String) + extends Param[Boolean](parent, name, doc) { override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } @@ -124,8 +115,11 @@ case class ParamPair[T](param: Param[T], value: T) @AlphaComponent trait Params extends Identifiable with Serializable { - /** Returns all params. */ - def params: Array[Param[_]] = { + /** + * Returns all params sorted by their names. The default implementation uses Java reflection to + * list all public methods that have no arguments and return [[Param]]. + */ + lazy val params: Array[Param[_]] = { val methods = this.getClass.getMethods methods.filter { m => Modifier.isPublic(m.getModifiers) && @@ -153,25 +147,29 @@ trait Params extends Identifiable with Serializable { def explainParams(): String = params.mkString("\n") /** Checks whether a param is explicitly set. */ - def isSet(param: Param[_]): Boolean = { - require(param.parent.eq(this)) + final def isSet(param: Param[_]): Boolean = { + shouldOwn(param) paramMap.contains(param) } + /** Checks whether a param is explicitly set or has a default value. */ + final def isDefined(param: Param[_]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) || paramMap.contains(param) + } + /** Gets a param by its name. */ - private[ml] def getParam(paramName: String): Param[Any] = { - val m = this.getClass.getMethod(paramName) - assert(Modifier.isPublic(m.getModifiers) && - classOf[Param[_]].isAssignableFrom(m.getReturnType) && - m.getParameterTypes.isEmpty) - m.invoke(this).asInstanceOf[Param[Any]] + def getParam(paramName: String): Param[Any] = { + params.find(_.name == paramName).getOrElse { + throw new NoSuchElementException(s"Param $paramName does not exist.") + }.asInstanceOf[Param[Any]] } /** * Sets a parameter in the embedded param map. */ - protected def set[T](param: Param[T], value: T): this.type = { - require(param.parent.eq(this)) + protected final def set[T](param: Param[T], value: T): this.type = { + shouldOwn(param) paramMap.put(param.asInstanceOf[Param[Any]], value) this } @@ -179,52 +177,102 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter (by name) in the embedded param map. */ - private[ml] def set(param: String, value: Any): this.type = { + protected final def set(param: String, value: Any): this.type = { set(getParam(param), value) } /** - * Gets the value of a parameter in the embedded param map. + * Optionally returns the user-supplied value of a param. + */ + final def get[T](param: Param[T]): Option[T] = { + shouldOwn(param) + paramMap.get(param) + } + + /** + * Clears the user-supplied value for the input param. + */ + protected final def clear(param: Param[_]): this.type = { + shouldOwn(param) + paramMap.remove(param) + this + } + + /** + * Gets the value of a param in the embedded param map or its default value. Throws an exception + * if neither is set. + */ + final def getOrDefault[T](param: Param[T]): T = { + shouldOwn(param) + get(param).orElse(getDefault(param)).get + } + + /** + * Sets a default value for a param. + * @param param param to set the default value. Make sure that this param is initialized before + * this method gets called. + * @param value the default value */ - protected def get[T](param: Param[T]): T = { - require(param.parent.eq(this)) - paramMap(param) + protected final def setDefault[T](param: Param[T], value: T): this.type = { + shouldOwn(param) + defaultParamMap.put(param, value) + this } /** - * Internal param map. + * Sets default values for a list of params. + * @param paramPairs a list of param pairs that specify params and their default values to set + * respectively. Make sure that the params are initialized before this method + * gets called. */ - protected val paramMap: ParamMap = ParamMap.empty + protected final def setDefault(paramPairs: ParamPair[_]*): this.type = { + paramPairs.foreach { p => + setDefault(p.param.asInstanceOf[Param[Any]], p.value) + } + this + } /** - * Check whether the given schema contains an input column. - * @param colName Input column name - * @param dataType Input column DataType + * Gets the default value of a parameter. */ - protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { - val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + final def getDefault[T](param: Param[T]): Option[T] = { + shouldOwn(param) + defaultParamMap.get(param) } /** - * Add an output column to the given schema. - * This fails if the given output column already exists. - * @param schema Initial schema (not modified) - * @param colName Output column name. If this column name is an empy String "", this method - * returns the initial schema, unchanged. This allows users to disable output - * columns. - * @param dataType Output column DataType - */ - protected def addOutputColumn( - schema: StructType, - colName: String, - dataType: DataType): StructType = { - if (colName.length == 0) return schema - val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Output column $colName already exists.") - val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) - StructType(outputFields) + * Tests whether the input param has a default value set. + */ + final def hasDefault[T](param: Param[T]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) + } + + /** + * Extracts the embedded default param values and user-supplied values, and then merges them with + * extra values from input into a flat param map, where the latter value is used if there exist + * conflicts, i.e., with ordering: default param values < user-supplied values < extraParamMap. + */ + protected final def extractParamMap(extraParamMap: ParamMap): ParamMap = { + defaultParamMap ++ paramMap ++ extraParamMap + } + + /** + * [[extractParamMap]] with no extra values. + */ + protected final def extractParamMap(): ParamMap = { + extractParamMap(ParamMap.empty) + } + + /** Internal param map for user-supplied values. */ + private val paramMap: ParamMap = ParamMap.empty + + /** Internal param map for default values. */ + private val defaultParamMap: ParamMap = ParamMap.empty + + /** Validates that the input param belongs to this instance. */ + private def shouldOwn(param: Param[_]): Unit = { + require(param.parent.eq(this), s"Param $param does not belong to $this.") } } @@ -261,12 +309,13 @@ private[spark] object Params { * A param to value map. */ @AlphaComponent -class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { +final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) + extends Serializable { /** * Creates an empty param map. */ - def this() = this(mutable.Map.empty[Param[Any], Any]) + def this() = this(mutable.Map.empty) /** * Puts a (param, value) pair (overwrites if the input param exists). @@ -288,12 +337,17 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Optionally returns the value associated with a param or its default. + * Optionally returns the value associated with a param. */ def get[T](param: Param[T]): Option[T] = { - map.get(param.asInstanceOf[Param[Any]]) - .orElse(param.defaultValue) - .asInstanceOf[Option[T]] + map.get(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + + /** + * Returns the value associated with a param or a default value. + */ + def getOrElse[T](param: Param[T], default: T): T = { + get(param).getOrElse(default) } /** @@ -301,10 +355,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Raises a NoSuchElementException if there is no value associated with the input param. */ def apply[T](param: Param[T]): T = { - val value = get(param) - if (value.isDefined) { - value.get - } else { + get(param).getOrElse { throw new NoSuchElementException(s"Cannot find param ${param.name}.") } } @@ -316,6 +367,13 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten map.contains(param.asInstanceOf[Param[Any]]) } + /** + * Removes a key from this map and returns its value associated previously as an option. + */ + def remove[T](param: Param[T]): Option[T] = { + map.remove(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + /** * Filters this param map for the given parent. */ @@ -325,7 +383,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Make a copy of this param map. + * Creates a copy of this param map. */ def copy: ParamMap = new ParamMap(map.clone()) @@ -337,7 +395,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Returns a new param map that contains parameters in this map and the given map, - * where the latter overwrites this if there exists conflicts. + * where the latter overwrites this if there exist conflicts. */ def ++(other: ParamMap): ParamMap = { // TODO: Provide a better method name for Java users. @@ -363,7 +421,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Number of param pairs in this set. + * Number of param pairs in this map. */ def size: Int = map.size } 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 new file mode 100644 index 0000000000000..95d7e64790c79 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -0,0 +1,169 @@ +/* + * 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 java.io.PrintWriter + +import scala.reflect.ClassTag + +/** + * Code generator for shared params (sharedParams.scala). Run under the Spark folder with + * {{{ + * build/sbt "mllib/runMain org.apache.spark.ml.param.shared.SharedParamsCodeGen" + * }}} + */ +private[shared] object SharedParamsCodeGen { + + def main(args: Array[String]): Unit = { + val params = Seq( + ParamDesc[Double]("regParam", "regularization parameter"), + ParamDesc[Int]("maxIter", "max number of iterations"), + ParamDesc[String]("featuresCol", "features column name", Some("\"features\"")), + ParamDesc[String]("labelCol", "label column name", Some("\"label\"")), + ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), + ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("\"rawPrediction\"")), + ParamDesc[String]("probabilityCol", + "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[Double]("threshold", "threshold in binary classification prediction"), + ParamDesc[String]("inputCol", "input column name"), + ParamDesc[Array[String]]("inputCols", "input column names"), + ParamDesc[String]("outputCol", "output column name"), + ParamDesc[Int]("checkpointInterval", "checkpoint interval"), + ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true"))) + + val code = genSharedParams(params) + val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" + val writer = new PrintWriter(file) + writer.write(code) + writer.close() + } + + /** Description of a param. */ + private case class ParamDesc[T: ClassTag]( + name: String, + doc: String, + defaultValueStr: Option[String] = None) { + + require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") + require(doc.nonEmpty) // TODO: more rigorous on doc + + def paramTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + c match { + case _ if c == classOf[Int] => "IntParam" + case _ if c == classOf[Long] => "LongParam" + case _ if c == classOf[Float] => "FloatParam" + case _ if c == classOf[Double] => "DoubleParam" + case _ if c == classOf[Boolean] => "BooleanParam" + case _ => s"Param[${getTypeString(c)}]" + } + } + + def valueTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + getTypeString(c) + } + + private def getTypeString(c: Class[_]): String = { + c match { + case _ if c == classOf[Int] => "Int" + case _ if c == classOf[Long] => "Long" + case _ if c == classOf[Float] => "Float" + case _ if c == classOf[Double] => "Double" + case _ if c == classOf[Boolean] => "Boolean" + case _ if c == classOf[String] => "String" + case _ if c.isArray => s"Array[${getTypeString(c.getComponentType)}]" + } + } + } + + /** Generates the HasParam trait code for the input param. */ + private def genHasParamTrait(param: ParamDesc[_]): String = { + val name = param.name + val Name = name(0).toUpper +: name.substring(1) + val Param = param.paramTypeName + val T = param.valueTypeName + val doc = param.doc + val defaultValue = param.defaultValueStr + val defaultValueDoc = defaultValue.map { v => + s" (default: $v)" + }.getOrElse("") + val setDefault = defaultValue.map { v => + s""" + | setDefault($name, $v) + |""".stripMargin + }.getOrElse("") + + s""" + |/** + | * :: DeveloperApi :: + | * Trait for shared param $name$defaultValueDoc. + | */ + |@DeveloperApi + |trait Has$Name extends Params { + | + | /** + | * Param for $doc. + | * @group param + | */ + | final val $name: $Param = new $Param(this, "$name", "$doc") + |$setDefault + | /** @group getParam */ + | final def get$Name: $T = getOrDefault($name) + |} + |""".stripMargin + } + + /** Generates Scala source code for the input params with header. */ + private def genSharedParams(params: Seq[ParamDesc[_]]): String = { + val header = + """/* + | * 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.apache.spark.annotation.DeveloperApi + |import org.apache.spark.ml.param._ + | + |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + | + |// scalastyle:off + |""".stripMargin + + val footer = "// scalastyle:on\n" + + val traits = params.map(genHasParamTrait).mkString + + header + traits + footer + } +} 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 new file mode 100644 index 0000000000000..72b08bf276483 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -0,0 +1,259 @@ +/* + * 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.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.param._ + +// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + +// scalastyle:off + +/** + * :: DeveloperApi :: + * Trait for shared param regParam. + */ +@DeveloperApi +trait HasRegParam extends Params { + + /** + * Param for regularization parameter. + * @group param + */ + final val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") + + /** @group getParam */ + final def getRegParam: Double = getOrDefault(regParam) +} + +/** + * :: DeveloperApi :: + * Trait for shared param maxIter. + */ +@DeveloperApi +trait HasMaxIter extends Params { + + /** + * Param for max number of iterations. + * @group param + */ + final val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") + + /** @group getParam */ + final def getMaxIter: Int = getOrDefault(maxIter) +} + +/** + * :: DeveloperApi :: + * Trait for shared param featuresCol (default: "features"). + */ +@DeveloperApi +trait HasFeaturesCol extends Params { + + /** + * Param for features column name. + * @group param + */ + final val featuresCol: Param[String] = new Param[String](this, "featuresCol", "features column name") + + setDefault(featuresCol, "features") + + /** @group getParam */ + final def getFeaturesCol: String = getOrDefault(featuresCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param labelCol (default: "label"). + */ +@DeveloperApi +trait HasLabelCol extends Params { + + /** + * Param for label column name. + * @group param + */ + final val labelCol: Param[String] = new Param[String](this, "labelCol", "label column name") + + setDefault(labelCol, "label") + + /** @group getParam */ + final def getLabelCol: String = getOrDefault(labelCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param predictionCol (default: "prediction"). + */ +@DeveloperApi +trait HasPredictionCol extends Params { + + /** + * Param for prediction column name. + * @group param + */ + final val predictionCol: Param[String] = new Param[String](this, "predictionCol", "prediction column name") + + setDefault(predictionCol, "prediction") + + /** @group getParam */ + final def getPredictionCol: String = getOrDefault(predictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param rawPredictionCol (default: "rawPrediction"). + */ +@DeveloperApi +trait HasRawPredictionCol extends Params { + + /** + * Param for raw prediction (a.k.a. confidence) column name. + * @group param + */ + final val rawPredictionCol: Param[String] = new Param[String](this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") + + setDefault(rawPredictionCol, "rawPrediction") + + /** @group getParam */ + final def getRawPredictionCol: String = getOrDefault(rawPredictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param probabilityCol (default: "probability"). + */ +@DeveloperApi +trait HasProbabilityCol extends Params { + + /** + * Param for column name for predicted class conditional probabilities. + * @group param + */ + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + + setDefault(probabilityCol, "probability") + + /** @group getParam */ + final def getProbabilityCol: String = getOrDefault(probabilityCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param threshold. + */ +@DeveloperApi +trait HasThreshold extends Params { + + /** + * Param for threshold in binary classification prediction. + * @group param + */ + final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction") + + /** @group getParam */ + final def getThreshold: Double = getOrDefault(threshold) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCol. + */ +@DeveloperApi +trait HasInputCol extends Params { + + /** + * Param for input column name. + * @group param + */ + final val inputCol: Param[String] = new Param[String](this, "inputCol", "input column name") + + /** @group getParam */ + final def getInputCol: String = getOrDefault(inputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCols. + */ +@DeveloperApi +trait HasInputCols extends Params { + + /** + * Param for input column names. + * @group param + */ + final val inputCols: Param[Array[String]] = new Param[Array[String]](this, "inputCols", "input column names") + + /** @group getParam */ + final def getInputCols: Array[String] = getOrDefault(inputCols) +} + +/** + * :: DeveloperApi :: + * Trait for shared param outputCol. + */ +@DeveloperApi +trait HasOutputCol extends Params { + + /** + * Param for output column name. + * @group param + */ + final val outputCol: Param[String] = new Param[String](this, "outputCol", "output column name") + + /** @group getParam */ + final def getOutputCol: String = getOrDefault(outputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param checkpointInterval. + */ +@DeveloperApi +trait HasCheckpointInterval extends Params { + + /** + * Param for checkpoint interval. + * @group param + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") + + /** @group getParam */ + final def getCheckpointInterval: Int = getOrDefault(checkpointInterval) +} + +/** + * :: DeveloperApi :: + * Trait for shared param fitIntercept (default: true). + */ +@DeveloperApi +trait HasFitIntercept extends Params { + + /** + * Param for whether to fit an intercept term. + * @group param + */ + final val fitIntercept: BooleanParam = new BooleanParam(this, "fitIntercept", "whether to fit an intercept term") + + setDefault(fitIntercept, true) + + /** @group getParam */ + final def getFitIntercept: Boolean = getOrDefault(fitIntercept) +} +// scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala deleted file mode 100644 index 07e6eb417763d..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ /dev/null @@ -1,173 +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.ml.param - -/* NOTE TO DEVELOPERS: - * If you mix these parameter traits into your algorithm, please add a setter method as well - * so that users may use a builder pattern: - * val myLearner = new MyLearner().setParam1(x).setParam2(y)... - */ - -private[ml] trait HasRegParam extends Params { - /** - * param for regularization parameter - * @group param - */ - val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") - - /** @group getParam */ - def getRegParam: Double = get(regParam) -} - -private[ml] trait HasMaxIter extends Params { - /** - * param for max number of iterations - * @group param - */ - val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - - /** @group getParam */ - def getMaxIter: Int = get(maxIter) -} - -private[ml] trait HasFeaturesCol extends Params { - /** - * param for features column name - * @group param - */ - val featuresCol: Param[String] = - new Param(this, "featuresCol", "features column name", Some("features")) - - /** @group getParam */ - def getFeaturesCol: String = get(featuresCol) -} - -private[ml] trait HasLabelCol extends Params { - /** - * param for label column name - * @group param - */ - val labelCol: Param[String] = new Param(this, "labelCol", "label column name", Some("label")) - - /** @group getParam */ - def getLabelCol: String = get(labelCol) -} - -private[ml] trait HasPredictionCol extends Params { - /** - * param for prediction column name - * @group param - */ - val predictionCol: Param[String] = - new Param(this, "predictionCol", "prediction column name", Some("prediction")) - - /** @group getParam */ - def getPredictionCol: String = get(predictionCol) -} - -private[ml] trait HasRawPredictionCol extends Params { - /** - * param for raw prediction column name - * @group param - */ - val rawPredictionCol: Param[String] = - new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", - Some("rawPrediction")) - - /** @group getParam */ - def getRawPredictionCol: String = get(rawPredictionCol) -} - -private[ml] trait HasProbabilityCol extends Params { - /** - * param for predicted class conditional probabilities column name - * @group param - */ - val probabilityCol: Param[String] = - new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", - Some("probability")) - - /** @group getParam */ - def getProbabilityCol: String = get(probabilityCol) -} - -private[ml] trait HasFitIntercept extends Params { - /** - * param for fitting the intercept term, defaults to true - * @group param - */ - val fitIntercept: BooleanParam = - new BooleanParam(this, "fitIntercept", "indicates whether to fit an intercept term", Some(true)) - - /** @group getParam */ - def getFitIntercept: Boolean = get(fitIntercept) -} - -private[ml] trait HasThreshold extends Params { - /** - * param for threshold in (binary) prediction - * @group param - */ - val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") - - /** @group getParam */ - def getThreshold: Double = get(threshold) -} - -private[ml] trait HasInputCol extends Params { - /** - * param for input column name - * @group param - */ - val inputCol: Param[String] = new Param(this, "inputCol", "input column name") - - /** @group getParam */ - def getInputCol: String = get(inputCol) -} - -private[ml] trait HasInputCols extends Params { - /** - * Param for input column names. - */ - val inputCols: Param[Array[String]] = new Param(this, "inputCols", "input column names") - - /** @group getParam */ - def getInputCols: Array[String] = get(inputCols) -} - -private[ml] trait HasOutputCol extends Params { - /** - * param for output column name - * @group param - */ - val outputCol: Param[String] = new Param(this, "outputCol", "output column name") - - /** @group getParam */ - def getOutputCol: String = get(outputCol) -} - -private[ml] trait HasCheckpointInterval extends Params { - /** - * param for checkpoint interval - * @group param - */ - val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") - - /** @group getParam */ - def getCheckpointInterval: Int = get(checkpointInterval) -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 52c9e95d6012f..bd793beba35b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -54,86 +55,88 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * Param for rank of the matrix factorization. * @group param */ - val rank = new IntParam(this, "rank", "rank of the factorization", Some(10)) + val rank = new IntParam(this, "rank", "rank of the factorization") /** @group getParam */ - def getRank: Int = get(rank) + def getRank: Int = getOrDefault(rank) /** * Param for number of user blocks. * @group param */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", Some(10)) + val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks") /** @group getParam */ - def getNumUserBlocks: Int = get(numUserBlocks) + def getNumUserBlocks: Int = getOrDefault(numUserBlocks) /** * Param for number of item blocks. * @group param */ val numItemBlocks = - new IntParam(this, "numItemBlocks", "number of item blocks", Some(10)) + new IntParam(this, "numItemBlocks", "number of item blocks") /** @group getParam */ - def getNumItemBlocks: Int = get(numItemBlocks) + def getNumItemBlocks: Int = getOrDefault(numItemBlocks) /** * Param to decide whether to use implicit preference. * @group param */ - val implicitPrefs = - new BooleanParam(this, "implicitPrefs", "whether to use implicit preference", Some(false)) + val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") /** @group getParam */ - def getImplicitPrefs: Boolean = get(implicitPrefs) + def getImplicitPrefs: Boolean = getOrDefault(implicitPrefs) /** * Param for the alpha parameter in the implicit preference formulation. * @group param */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", Some(1.0)) + val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference") /** @group getParam */ - def getAlpha: Double = get(alpha) + def getAlpha: Double = getOrDefault(alpha) /** * Param for the column name for user ids. * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids", Some("user")) + val userCol = new Param[String](this, "userCol", "column name for user ids") /** @group getParam */ - def getUserCol: String = get(userCol) + def getUserCol: String = getOrDefault(userCol) /** * Param for the column name for item ids. * @group param */ - val itemCol = - new Param[String](this, "itemCol", "column name for item ids", Some("item")) + val itemCol = new Param[String](this, "itemCol", "column name for item ids") /** @group getParam */ - def getItemCol: String = get(itemCol) + def getItemCol: String = getOrDefault(itemCol) /** * Param for the column name for ratings. * @group param */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) + val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") /** @group getParam */ - def getRatingCol: String = get(ratingCol) + def getRatingCol: String = getOrDefault(ratingCol) /** * Param for whether to apply nonnegativity constraints. * @group param */ val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + this, "nonnegative", "whether to use nonnegative constraint for least squares") /** @group getParam */ - val getNonnegative: Boolean = get(nonnegative) + def getNonnegative: Boolean = getOrDefault(nonnegative) + + setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, + implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", + ratingCol -> "rating", nonnegative -> false) /** * Validates and transforms the input schema. @@ -142,7 +145,7 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * @return output schema */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) assert(schema(map(userCol)).dataType == IntegerType) assert(schema(map(itemCol)).dataType== IntegerType) val ratingType = schema(map(ratingCol)).dataType @@ -171,7 +174,7 @@ class ALSModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { import dataset.sqlContext.implicits._ - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val users = userFactors.toDF("id", "features") val items = itemFactors.toDF("id", "features") @@ -283,7 +286,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { setCheckpointInterval(10) override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val ratings = dataset .select(col(map(userCol)), col(map(itemCol)), col(map(ratingCol)).cast(FloatType)) .map { row => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 65f6627a0c351..26ca7459c4fdf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -18,7 +18,8 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.sql.DataFrame @@ -41,8 +42,7 @@ private[regression] trait LinearRegressionParams extends RegressorParams class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams { - setRegParam(0.1) - setMaxIter(100) + setDefault(regParam -> 0.1, maxIter -> 100) /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -93,7 +93,7 @@ class LinearRegressionModel private[ml] ( override protected def copy(): LinearRegressionModel = { val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 2eb1dac56f1e9..4bb4ed813c006 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.types.StructType * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends Params { + /** * param for the estimator to be cross-validated * @group param @@ -38,7 +39,7 @@ private[ml] trait CrossValidatorParams extends Params { val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") /** @group getParam */ - def getEstimator: Estimator[_] = get(estimator) + def getEstimator: Estimator[_] = getOrDefault(estimator) /** * param for estimator param maps @@ -48,7 +49,7 @@ private[ml] trait CrossValidatorParams extends Params { new Param(this, "estimatorParamMaps", "param maps for the estimator") /** @group getParam */ - def getEstimatorParamMaps: Array[ParamMap] = get(estimatorParamMaps) + def getEstimatorParamMaps: Array[ParamMap] = getOrDefault(estimatorParamMaps) /** * param for the evaluator for selection @@ -57,17 +58,18 @@ private[ml] trait CrossValidatorParams extends Params { val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") /** @group getParam */ - def getEvaluator: Evaluator = get(evaluator) + def getEvaluator: Evaluator = getOrDefault(evaluator) /** * param for number of folds for cross validation * @group param */ - val numFolds: IntParam = - new IntParam(this, "numFolds", "number of folds for cross validation", Some(3)) + val numFolds: IntParam = new IntParam(this, "numFolds", "number of folds for cross validation") /** @group getParam */ - def getNumFolds: Int = get(numFolds) + def getNumFolds: Int = getOrDefault(numFolds) + + setDefault(numFolds -> 3) } /** @@ -92,7 +94,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP def setNumFolds(value: Int): this.type = set(numFolds, value) override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema transformSchema(dataset.schema, paramMap, logging = true) val sqlCtx = dataset.sqlContext @@ -130,7 +132,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) map(estimator).transformSchema(schema, paramMap) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala new file mode 100644 index 0000000000000..0383bf0b382b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.types.{DataType, StructField, StructType} + +/** + * :: DeveloperApi :: + * Utils for handling schemas. + */ +@DeveloperApi +object SchemaUtils { + + // TODO: Move the utility methods to SQL. + + /** + * Check whether the given schema contains a column of the required data type. + * @param colName column name + * @param dataType required column data type + */ + def checkColumnType(schema: StructType, colName: String, dataType: DataType): Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Column $colName must be of type $dataType but was actually $actualDataType.") + } + + /** + * Appends a new column to the input schema. This fails if the given output column already exists. + * @param schema input schema + * @param colName new column name. If this column name is an empty string "", this method returns + * the input schema unchanged. This allows users to disable output columns. + * @param dataType new column data type + * @return new schema with the input column appended + */ + def appendColumn( + schema: StructType, + colName: String, + dataType: DataType): StructType = { + if (colName.isEmpty) return schema + val fieldNames = schema.fieldNames + require(!fieldNames.contains(colName), s"Column $colName already exists.") + val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) + StructType(outputFields) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index 1ce2987612378..88ea679eeaad5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -21,19 +21,25 @@ import org.scalatest.FunSuite class ParamsSuite extends FunSuite { - val solver = new TestParams() - import solver.{inputCol, maxIter} - test("param") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + assert(maxIter.name === "maxIter") assert(maxIter.doc === "max number of iterations") - assert(maxIter.defaultValue.get === 100) assert(maxIter.parent.eq(solver)) - assert(maxIter.toString === "maxIter: max number of iterations (default: 100)") - assert(inputCol.defaultValue === None) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10)") + + solver.setMaxIter(5) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10, current: 5)") + + assert(inputCol.toString === "inputCol: input column name (undefined)") } test("param pair") { + val solver = new TestParams() + import solver.maxIter + val pair0 = maxIter -> 5 val pair1 = maxIter.w(5) val pair2 = ParamPair(maxIter, 5) @@ -44,10 +50,12 @@ class ParamsSuite extends FunSuite { } test("param map") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val map0 = ParamMap.empty assert(!map0.contains(maxIter)) - assert(map0(maxIter) === maxIter.defaultValue.get) map0.put(maxIter, 10) assert(map0.contains(maxIter)) assert(map0(maxIter) === 10) @@ -78,23 +86,39 @@ class ParamsSuite extends FunSuite { } test("params") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val params = solver.params - assert(params.size === 2) + assert(params.length === 2) assert(params(0).eq(inputCol), "params must be ordered by name") assert(params(1).eq(maxIter)) + + assert(!solver.isSet(maxIter)) + assert(solver.isDefined(maxIter)) + assert(solver.getMaxIter === 10) + solver.setMaxIter(100) + assert(solver.isSet(maxIter)) + assert(solver.getMaxIter === 100) + assert(!solver.isSet(inputCol)) + assert(!solver.isDefined(inputCol)) + intercept[NoSuchElementException](solver.getInputCol) + assert(solver.explainParams() === Seq(inputCol, maxIter).mkString("\n")) + assert(solver.getParam("inputCol").eq(inputCol)) assert(solver.getParam("maxIter").eq(maxIter)) - intercept[NoSuchMethodException] { + intercept[NoSuchElementException] { solver.getParam("abc") } - assert(!solver.isSet(inputCol)) + intercept[IllegalArgumentException] { solver.validate() } solver.validate(ParamMap(inputCol -> "input")) solver.setInputCol("input") assert(solver.isSet(inputCol)) + assert(solver.isDefined(inputCol)) assert(solver.getInputCol === "input") solver.validate() intercept[IllegalArgumentException] { @@ -104,5 +128,8 @@ class ParamsSuite extends FunSuite { intercept[IllegalArgumentException] { solver.validate() } + + solver.clearMaxIter() + assert(!solver.isSet(maxIter)) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index ce52f2f230085..8f9ab687c05cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -20,17 +20,21 @@ package org.apache.spark.ml.param /** A subclass of Params for testing. */ class TestParams extends Params { - val maxIter = new IntParam(this, "maxIter", "max number of iterations", Some(100)) + val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = get(inputCol) + def getInputCol: String = getOrDefault(inputCol) + + setDefault(maxIter -> 10) override def validate(paramMap: ParamMap): Unit = { - val m = this.paramMap ++ paramMap + val m = extractParamMap(paramMap) require(m(maxIter) >= 0) require(m.contains(inputCol)) } + + def clearMaxIter(): this.type = clear(maxIter) } From 77eeb10fd1f5beb558d6278d9443d267fc6c2f62 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 13 Apr 2015 22:31:44 -0700 Subject: [PATCH 753/817] [WIP][HOTFIX][SPARK-4123]: Fix bug in PR dependency (all deps. removed issue) We're seeing a bug sporadically in the new PR dependency comparison test whereby it notes that *all* dependencies are removed. This happens when the current PR is built, but the final, sorted, dependency file is left blank. I believe this is an error either in the way the `git checkout` calls have been or an error within the `mvn` build for that PR (again, likely related to the `git checkout`). As such I've set the checkouts to now force (with `-f` flag) which is more in line with what Jenkins currently does on the initial checkout. Setting this as a WIP for now to trigger the build process myriad times to see if the issue still arises. Author: Brennon York Closes #5443 from brennonyork/HOTFIX2-SPARK-4123 and squashes the following commits: f2186be [Brennon York] added output for the various git commit refs 3f073d6 [Brennon York] removed the git checkouts piping to dev null 07765a6 [Brennon York] updated the diff logic to reference the filenames rather than hardlink e3f63c7 [Brennon York] added '-f' to the checkout flags for git 710c8d1 [Brennon York] added 30 minutes to the test benchmark --- dev/run-tests-jenkins | 4 ++++ dev/tests/pr_new_dependencies.sh | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f6372835a6dbf..3c1c91a111357 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -161,6 +161,10 @@ pr_message="" # Ensure we save off the current HEAD to revert to current_pr_head="`git rev-parse HEAD`" +echo "HEAD: `git rev-parse HEAD`" +echo "GHPRB: $ghprbActualCommit" +echo "SHA1: $sha1" + # Run pull request tests for t in "${PR_TESTS[@]}"; do this_test="${FWDIR}/dev/tests/${t}.sh" diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh index 370c7cc737bbd..fdfb3c62aff58 100755 --- a/dev/tests/pr_new_dependencies.sh +++ b/dev/tests/pr_new_dependencies.sh @@ -39,12 +39,12 @@ CURR_CP_FILE="my-classpath.txt" MASTER_CP_FILE="master-classpath.txt" # First switch over to the master branch -git checkout master &>/dev/null +git checkout -f master # Find and copy all pom.xml files into a *.gate file that we can check # against through various `git` changes find -name "pom.xml" -exec cp {} {}.gate \; # Switch back to the current PR -git checkout "${current_pr_head}" &>/dev/null +git checkout -f "${current_pr_head}" # Check if any *.pom files from the current branch are different from the master difference_q="" @@ -71,7 +71,7 @@ else sort > ${CURR_CP_FILE} # Checkout the master branch to compare against - git checkout master &>/dev/null + git checkout -f master ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ sed -n -e '/Building Spark Project Assembly/,$p' | \ @@ -84,7 +84,7 @@ else rev | \ sort > ${MASTER_CP_FILE} - DIFF_RESULTS="`diff my-classpath.txt master-classpath.txt`" + DIFF_RESULTS="`diff ${CURR_CP_FILE} ${MASTER_CP_FILE}`" if [ -z "${DIFF_RESULTS}" ]; then echo " * This patch does not change any dependencies." From 628a72f70ed06b8d7aee81cfb16070eb2c87b9cd Mon Sep 17 00:00:00 2001 From: Punyashloka Biswal Date: Tue, 14 Apr 2015 11:43:06 +0100 Subject: [PATCH 754/817] [SPARK-6731] Bump version of apache commons-math3 Version 3.1.1 is two years old and the newer version includes approximate percentile statistics (among other things). Author: Punyashloka Biswal Closes #5380 from punya/patch-1 and squashes the following commits: 226622b [Punyashloka Biswal] Bump version of apache commons-math3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d8881c213bf07..0b8d664dbfe1b 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ 1.8.3 1.1.0 4.2.6 - 3.1.1 + 3.4.1 ${project.build.directory}/spark-test-classpath.txt 2.10.4 2.10 From 51b306b930cfe03ad21af72a3a6ef31e6e626235 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Tue, 14 Apr 2015 12:39:56 +0100 Subject: [PATCH 755/817] SPARK-6878 [CORE] Fix for sum on empty RDD fails with exception Author: Erik van Oosten Closes #5489 from erikvanoosten/master and squashes the following commits: 1c91954 [Erik van Oosten] Rewrote double range matcher to an exact equality assert (SPARK-6878) f1708c9 [Erik van Oosten] Fix for sum on empty RDD fails with exception (SPARK-6878) --- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 2 +- .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 29ca3e9c4bd04..843a893235e56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.StatCounter class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ def sum(): Double = { - self.reduce(_ + _) + self.fold(0.0)(_ + _) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 97079382c716f..01039b9449daf 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -22,6 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { + test("sum") { + assert(sc.parallelize(Seq.empty[Double]).sum() === 0.0) + assert(sc.parallelize(Seq(1.0)).sum() === 1.0) + assert(sc.parallelize(Seq(1.0, 2.0)).sum() === 3.0) + } + // Verify tests on the histogram functionality. We test with both evenly // and non-evenly spaced buckets as the bucket lookup function changes. test("WorksOnEmpty") { From 320bca4508e890b874c2eb7abb76a30ef14c932f Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 14 Apr 2015 11:48:12 -0700 Subject: [PATCH 756/817] [SPARK-6081] Support fetching http/https uris in driver runner. Currently if passed uris such as http/https, it won't able to fetch them as it only calls HadoopFs get. This fix utilizes the existing util method to fetch remote uris as well. Author: Timothy Chen Closes #4832 from tnachen/driver_remote and squashes the following commits: aa52cd6 [Timothy Chen] Support fetching remote uris in driver runner. --- .../spark/deploy/worker/DriverRunner.scala | 21 ++++++++++++------- .../apache/spark/deploy/worker/Worker.scala | 3 ++- .../spark/deploy/JsonProtocolSuite.scala | 7 ++++--- .../deploy/worker/DriverRunnerTest.scala | 7 ++++--- 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e0948e16ef354..ef7a703bffe67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -24,14 +24,14 @@ import scala.collection.JavaConversions._ import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.util.{Utils, Clock, SystemClock} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -44,7 +44,8 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String) + val workerUrl: String, + val securityManager: SecurityManager) extends Logging { @volatile private var process: Option[Process] = None @@ -136,12 +137,9 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val jarFileSystem = jarPath.getFileSystem(hadoopConf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) @@ -149,7 +147,14 @@ private[deploy] class DriverRunner( if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destPath, false, hadoopConf) + Utils.fetchFile( + driverDesc.jarUrl, + driverDir, + conf, + securityManager, + hadoopConf, + System.currentTimeMillis(), + useCache = false) } if (!localJarFile.exists()) { // Verify copy succeeded diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c4c24a7866aa3..3ee2eb69e8a4e 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -436,7 +436,8 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl) + akkaUrl, + securityMgr) drivers(driverId) = driver driver.start() diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2071701b313db..b58d62567afe1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} class JsonProtocolSuite extends FunSuite { @@ -124,8 +124,9 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) } def assertValidJson(json: JValue) { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index aa6e4874cecde..2159fd8c16c6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,8 +33,9 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { From f63b44a5c201d9678738a906462be9a6d7e3e8f8 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 14 Apr 2015 12:02:11 -0700 Subject: [PATCH 757/817] [SPARK-6894]spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath https://issues.apache.org/jira/browse/SPARK-6894 cc vanzin Author: WangTaoTheTonic Closes #5506 from WangTaoTheTonic/SPARK-6894 and squashes the following commits: 4b7ced7 [WangTaoTheTonic] spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath --- .../src/main/java/org/apache/spark/launcher/SparkLauncher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index b566507ee6061..d4cfeacb6ef18 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -52,7 +52,7 @@ public class SparkLauncher { /** Configuration key for the executor VM options. */ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ - public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryPath"; /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; From dcf8a9f331c6193a62bbc9282bdc99663e23ca19 Mon Sep 17 00:00:00 2001 From: pankaj arora Date: Tue, 14 Apr 2015 12:06:46 -0700 Subject: [PATCH 758/817] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure Fixed null check when all the dependent stages are cancelled due to previous stage failure. This happens when one of the executor node goes down and all the dependent stages are cancelled. Author: pankaj arora Closes #5494 from pankajarora12/NEWBRANCH and squashes the following commits: 55ba5e3 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure 4575720 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 508fe7b3303ca..4a32f8936fb0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -818,12 +818,7 @@ class DAGScheduler( } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } + val properties = jobIdToActiveJob.get(stage.jobId).map(_.properties).orNull runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are From 25998e4d73bcc95ac85d9af71adfdc726ec89568 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 14 Apr 2015 12:56:47 -0700 Subject: [PATCH 759/817] [SPARK-2033] Automatically cleanup checkpoint Author: GuoQiang Li Closes #855 from witgo/cleanup_checkpoint_date and squashes the following commits: 1649850 [GuoQiang Li] review commit c0087e0 [GuoQiang Li] Automatically cleanup checkpoint --- .../org/apache/spark/ContextCleaner.scala | 44 ++++++++++++----- .../apache/spark/rdd/RDDCheckpointData.scala | 27 ++++++++-- .../apache/spark/ContextCleanerSuite.scala | 49 ++++++++++++++++++- 3 files changed, 102 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 9b05c9623b704..715b259057569 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue, WeakReference} import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.util.Utils /** @@ -33,6 +33,7 @@ private case class CleanRDD(rddId: Int) extends CleanupTask private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask private case class CleanAccum(accId: Long) extends CleanupTask +private case class CleanCheckpoint(rddId: Int) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -94,12 +95,12 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ - def attachListener(listener: CleanerListener) { + def attachListener(listener: CleanerListener): Unit = { listeners += listener } /** Start the cleaner. */ - def start() { + def start(): Unit = { cleaningThread.setDaemon(true) cleaningThread.setName("Spark Context Cleaner") cleaningThread.start() @@ -108,7 +109,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Stop the cleaning thread and wait until the thread has finished running its current task. */ - def stop() { + def stop(): Unit = { stopped = true // Interrupt the cleaning thread, but wait until the current task has finished before // doing so. This guards against the race condition where a cleaning thread may @@ -121,7 +122,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a RDD for cleanup when it is garbage collected. */ - def registerRDDForCleanup(rdd: RDD[_]) { + def registerRDDForCleanup(rdd: RDD[_]): Unit = { registerForCleanup(rdd, CleanRDD(rdd.id)) } @@ -130,17 +131,22 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a ShuffleDependency for cleanup when it is garbage collected. */ - def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]): Unit = { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) } /** Register a Broadcast for cleanup when it is garbage collected. */ - def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]): Unit = { registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) } + /** Register a RDDCheckpointData for cleanup when it is garbage collected. */ + def registerRDDCheckpointDataForCleanup[T](rdd: RDD[_], parentId: Int): Unit = { + registerForCleanup(rdd, CleanCheckpoint(parentId)) + } + /** Register an object for cleanup. */ - private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) } @@ -164,6 +170,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) case CleanAccum(accId) => doCleanupAccum(accId, blocking = blockOnCleanupTasks) + case CleanCheckpoint(rddId) => + doCleanCheckpoint(rddId) } } } @@ -175,7 +183,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform RDD cleanup. */ - def doCleanupRDD(rddId: Int, blocking: Boolean) { + def doCleanupRDD(rddId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning RDD " + rddId) sc.unpersistRDD(rddId, blocking) @@ -187,7 +195,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform shuffle cleanup, asynchronously. */ - def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) mapOutputTrackerMaster.unregisterShuffle(shuffleId) @@ -200,7 +208,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform broadcast cleanup. */ - def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean): Unit = { try { logDebug(s"Cleaning broadcast $broadcastId") broadcastManager.unbroadcast(broadcastId, true, blocking) @@ -212,7 +220,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform accumulator cleanup. */ - def doCleanupAccum(accId: Long, blocking: Boolean) { + def doCleanupAccum(accId: Long, blocking: Boolean): Unit = { try { logDebug("Cleaning accumulator " + accId) Accumulators.remove(accId) @@ -223,6 +231,18 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } + /** Perform checkpoint cleanup. */ + def doCleanCheckpoint(rddId: Int): Unit = { + try { + logDebug("Cleaning rdd checkpoint data " + rddId) + RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + logInfo("Cleaned rdd checkpoint data " + rddId) + } + catch { + case e: Exception => logError("Error cleaning rdd checkpoint data " + rddId, e) + } + } + private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 6afd63d537d75..1722c27e55003 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, Partition, SerializableWritable, SparkException} +import org.apache.spark._ import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** @@ -83,7 +83,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } // Create the output path for the checkpoint - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val path = RDDCheckpointData.rddCheckpointDataPath(rdd.context, rdd.id).get val fs = path.getFileSystem(rdd.context.hadoopConfiguration) if (!fs.mkdirs(path)) { throw new SparkException("Failed to create checkpoint path " + path) @@ -92,8 +92,13 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) + if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { + rdd.context.cleaner.foreach { cleaner => + cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id) + } + } + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) if (newRDD.partitions.length != rdd.partitions.length) { throw new SparkException( "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.length + ") has different " + @@ -130,5 +135,17 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -// Used for synchronization -private[spark] object RDDCheckpointData +private[spark] object RDDCheckpointData { + def rddCheckpointDataPath(sc: SparkContext, rddId: Int): Option[Path] = { + sc.checkpointDir.map { dir => new Path(dir, "rdd-" + rddId) } + } + + def clearRDDCheckpointData(sc: SparkContext, rddId: Int): Unit = { + rddCheckpointDataPath(sc, rddId).foreach { path => + val fs = path.getFileSystem(sc.hadoopConfiguration) + if (fs.exists(path)) { + fs.delete(path, true) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 1de169d964d23..097e7076e5391 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager @@ -205,6 +206,52 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { postGCTester.assertCleanup() } + test("automatically cleanup checkpoint") { + val checkpointDir = java.io.File.createTempFile("temp", "") + checkpointDir.deleteOnExit() + checkpointDir.delete() + var rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).isDefined) + val path = RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + sc.stop() + val conf = new SparkConf().setMaster("local[2]").setAppName("cleanupCheckpoint"). + set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + sc = new SparkContext(conf) + rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + } + test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly From 8f8dc45f6d4c8d7b740eaa3d2ea09d0b531af9dd Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 14 Apr 2015 13:32:06 -0700 Subject: [PATCH 760/817] SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker --- .../spark/deploy/ApplicationDescription.scala | 9 +- .../apache/spark/deploy/JsonProtocol.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 + .../spark/deploy/SparkSubmitArguments.scala | 5 +- .../spark/deploy/master/ApplicationInfo.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 117 +++++++++--------- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 7 +- docs/configuration.md | 11 ++ 10 files changed, 96 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index b7ae9c1fc0a23..ae99432f5ce86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -22,12 +22,13 @@ import java.net.URI private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], - val memoryPerSlave: Int, + val memoryPerExecutorMB: Int, val command: Command, var appUiUrl: String, val eventLogDir: Option[URI] = None, // short name of compression codec used when writing event logs, if any (e.g. lzf) - val eventLogCodec: Option[String] = None) + val eventLogCodec: Option[String] = None, + val coresPerExecutor: Option[Int] = None) extends Serializable { val user = System.getProperty("user.name", "") @@ -35,13 +36,13 @@ private[spark] class ApplicationDescription( def copy( name: String = name, maxCores: Option[Int] = maxCores, - memoryPerSlave: Int = memoryPerSlave, + memoryPerExecutorMB: Int = memoryPerExecutorMB, command: Command = command, appUiUrl: String = appUiUrl, eventLogDir: Option[URI] = eventLogDir, eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = new ApplicationDescription( - name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) + name, maxCores, memoryPerExecutorMB, command, appUiUrl, eventLogDir, eventLogCodec) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index dfc5b97e6a6c8..2954f932b4f41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -46,7 +46,7 @@ private[deploy] object JsonProtocol { ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ - ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~ ("submitdate" -> obj.submitDate.toString) ~ ("state" -> obj.state.toString) ~ ("duration" -> obj.duration) @@ -55,7 +55,7 @@ private[deploy] object JsonProtocol { def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ - ("memoryperslave" -> obj.memoryPerSlave) ~ + ("memoryperslave" -> obj.memoryPerExecutorMB) ~ ("user" -> obj.user) ~ ("command" -> obj.command.toString) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 60bc243ebf40a..296a0764b8baf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -406,6 +406,8 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options + OptionAssigner(args.executorCores, STANDALONE, ALL_DEPLOY_MODES, + sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 03ecf3fd99ec5..faa8780288ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -482,10 +482,13 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | + | Spark standalone and YARN only: + | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, + | or all available cores on the worker in standalone mode) + | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). - | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index bc5b293379f2b..f59d550d4f3b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -75,9 +75,11 @@ private[deploy] class ApplicationInfo( } } - private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): - ExecutorDesc = { - val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + private[master] def addExecutor( + worker: WorkerInfo, + cores: Int, + useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerExecutorMB) executors(exec.id) = exec coresGranted += cores exec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9a5d5877da86d..c5a6b1beac9be 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -524,52 +524,28 @@ private[master] class Master( } /** - * Can an app use the given worker? True if the worker has enough memory and we haven't already - * launched an executor for the app on it (right now the standalone backend doesn't like having - * two executors on the same worker). - */ - private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { - worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) - } - - /** - * Schedule the currently available resources among waiting apps. This method will be called - * every time a new app joins or resource availability changes. + * Schedule executors to be launched on the workers. + * + * There are two modes of launching executors. The first attempts to spread out an application's + * executors on as many workers as possible, while the second does the opposite (i.e. launch them + * on as few workers as possible). The former is usually better for data locality purposes and is + * the default. + * + * The number of cores assigned to each executor is configurable. When this is explicitly set, + * multiple executors from the same application may be launched on the same worker if the worker + * has enough cores and memory. Otherwise, each executor grabs all the cores available on the + * worker by default, in which case only one executor may be launched on each worker. */ - private def schedule() { - if (state != RecoveryState.ALIVE) { return } - - // First schedule drivers, they take strict precedence over applications - // Randomization helps balance drivers - val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val numWorkersAlive = shuffledAliveWorkers.size - var curPos = 0 - - for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers - // We assign workers to each waiting driver in a round-robin fashion. For each driver, we - // start from the last worker that was assigned a driver, and continue onwards until we have - // explored all alive workers. - var launched = false - var numWorkersVisited = 0 - while (numWorkersVisited < numWorkersAlive && !launched) { - val worker = shuffledAliveWorkers(curPos) - numWorkersVisited += 1 - if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { - launchDriver(worker, driver) - waitingDrivers -= driver - launched = true - } - curPos = (curPos + 1) % numWorkersAlive - } - } - + private def startExecutorsOnWorkers(): Unit = { // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. if (spreadOutApps) { - // Try to spread out each app among all the nodes, until it has all its cores + // Try to spread out each app among all the workers, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && + worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) + .sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) @@ -582,32 +558,61 @@ private[master] class Master( pos = (pos + 1) % numUsable } // Now that we've decided how many cores to give on each node, let's actually give them - for (pos <- 0 until numUsable) { - if (assigned(pos) > 0) { - val exec = app.addExecutor(usableWorkers(pos), assigned(pos)) - launchExecutor(usableWorkers(pos), exec) - app.state = ApplicationState.RUNNING - } + for (pos <- 0 until numUsable if assigned(pos) > 0) { + allocateWorkerResourceToExecutors(app, assigned(pos), usableWorkers(pos)) } } } else { - // Pack each app into as few nodes as possible until we've assigned all its cores + // Pack each app into as few workers as possible until we've assigned all its cores for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) { for (app <- waitingApps if app.coresLeft > 0) { - if (canUse(app, worker)) { - val coresToUse = math.min(worker.coresFree, app.coresLeft) - if (coresToUse > 0) { - val exec = app.addExecutor(worker, coresToUse) - launchExecutor(worker, exec) - app.state = ApplicationState.RUNNING - } - } + allocateWorkerResourceToExecutors(app, app.coresLeft, worker) + } + } + } + } + + /** + * Allocate a worker's resources to one or more executors. + * @param app the info of the application which the executors belong to + * @param coresToAllocate cores on this worker to be allocated to this application + * @param worker the worker info + */ + private def allocateWorkerResourceToExecutors( + app: ApplicationInfo, + coresToAllocate: Int, + worker: WorkerInfo): Unit = { + val memoryPerExecutor = app.desc.memoryPerExecutorMB + val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(coresToAllocate) + var coresLeft = coresToAllocate + while (coresLeft >= coresPerExecutor && worker.memoryFree >= memoryPerExecutor) { + val exec = app.addExecutor(worker, coresPerExecutor) + coresLeft -= coresPerExecutor + launchExecutor(worker, exec) + app.state = ApplicationState.RUNNING + } + } + + /** + * Schedule the currently available resources among waiting apps. This method will be called + * every time a new app joins or resource availability changes. + */ + private def schedule(): Unit = { + if (state != RecoveryState.ALIVE) { return } + // Drivers take strict precedence over executors + val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers + for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { + for (driver <- waitingDrivers) { + if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { + launchDriver(worker, driver) + waitingDrivers -= driver } } } + startExecutorsOnWorkers() } - private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc): Unit = { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 761aa8f7b1ef6..273f077bd8f57 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -94,7 +94,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")

  • Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 45412a35e9a7d..399f07399a0aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -208,8 +208,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.coresGranted} - - {Utils.megabytesToString(app.desc.memoryPerSlave)} + + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)} {UIUtils.formatDate(app.submitDate)} {app.desc.user} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 7eb3fdc19b5b8..ed5b7c1088196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -82,12 +82,11 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, sc.eventLogDir, sc.eventLogCodec) - + val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, + command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() - waitForRegistration() } diff --git a/docs/configuration.md b/docs/configuration.md index 7169ec295ef7f..d9e9e67026cbb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -723,6 +723,17 @@ Apart from these, the following properties are also available, and may be useful this duration will be cleared as well. + + spark.executor.cores + 1 in YARN mode, all the available cores on the worker in standalone mode. + + The number of cores to use on each executor. For YARN and standalone mode only. + + In standalone mode, setting this parameter allows an application to run multiple executors on + the same worker, provided that there are enough cores on that worker. Otherwise, only one + executor per application will run on each worker. + + spark.default.parallelism From b075e4b720221a8204cae93468065a6708348830 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:34:44 -0700 Subject: [PATCH 761/817] [SPARK-6700] [yarn] Re-enable flaky test. Test runs have been successful on jenkins. So let's re-enable the test and look out for any failures, and fix things appropriately. Author: Marcelo Vanzin Closes #5459 from vanzin/SPARK-6700 and squashes the following commits: 2ead85b [Marcelo Vanzin] WIP: re-enable flaky test to catch failure in jenkins. --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index c06c0105670c0..76952e3341cc4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -144,7 +144,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } // Enable this once fix SPARK-6700 - ignore("run Python application in yarn-cluster mode") { + test("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) val pyFile = new File(tempDir, "test2.py") From 6adb8bcbf0a1a7bfe2990de18c59c66cd7a0aeb8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 13:40:07 -0700 Subject: [PATCH 762/817] [SPARK-6905] Upgrade to snappy-java 1.1.1.7 We should upgrade our snappy-java dependency to 1.1.1.7 in order to include a fix for a bug that results in worse compression in SnappyOutputStream (see https://github.com/xerial/snappy-java/issues/100). Author: Josh Rosen Closes #5512 from JoshRosen/snappy-1.1.1.7 and squashes the following commits: f1ac0f8 [Josh Rosen] Upgrade to snappy-java 1.1.1.7. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0b8d664dbfe1b..261292d5b6cde 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ 3.6.3 1.8.8 2.4.4 - 1.1.1.6 + 1.1.1.7 1.1.2 ${java.home} From 65774370a1275e25cd8a3357e397d116767793a9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:41:38 -0700 Subject: [PATCH 763/817] [SPARK-5808] [build] Package pyspark files in sbt assembly. This turned out to be more complicated than I wanted because the layout of python/ doesn't really follow the usual maven conventions. So some extra code is needed to copy just the right things. Author: Marcelo Vanzin Closes #5461 from vanzin/SPARK-5808 and squashes the following commits: 7153dac [Marcelo Vanzin] Only try to create resource dir if it doesn't already exist. ee90e84 [Marcelo Vanzin] [SPARK-5808] [build] Package pyspark files in sbt assembly. --- project/SparkBuild.scala | 60 +++++++++++++++++++++++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f51f4b58f97a..09b4976d10c26 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -import java.io.File +import java.io._ import scala.util.Properties import scala.collection.JavaConversions._ @@ -166,6 +166,9 @@ object SparkBuild extends PomBuild { /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) + /* Package pyspark artifacts in the main assembly. */ + enable(PySparkAssembly.settings)(assembly) + /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) @@ -316,6 +319,7 @@ object Hive { } object Assembly { + import sbtassembly.AssemblyUtils._ import sbtassembly.Plugin._ import AssemblyKeys._ @@ -347,6 +351,60 @@ object Assembly { ) } +object PySparkAssembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ + + lazy val settings = Seq( + unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, + // Use a resource generator to copy all .py files from python/pyspark into a managed directory + // to be included in the assembly. We can't just add "python/" to the assembly's resource dir + // list since that will copy unneeded / unwanted files. + resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val dst = new File(outDir, "pyspark") + if (!dst.isDirectory()) { + require(dst.mkdirs()) + } + + val src = new File(BuildCommons.sparkHome, "python/pyspark") + copy(src, dst) + } + ) + + private def copy(src: File, dst: File): Seq[File] = { + src.listFiles().flatMap { f => + val child = new File(dst, f.getName()) + if (f.isDirectory()) { + child.mkdir() + copy(f, child) + } else if (f.getName().endsWith(".py")) { + var in: Option[FileInputStream] = None + var out: Option[FileOutputStream] = None + try { + in = Some(new FileInputStream(f)) + out = Some(new FileOutputStream(child)) + + val bytes = new Array[Byte](1024) + var read = 0 + while (read >= 0) { + read = in.get.read(bytes) + if (read > 0) { + out.get.write(bytes, 0, read) + } + } + + Some(child) + } finally { + in.foreach(_.close()) + out.foreach(_.close()) + } + } else { + None + } + } + } +} + object Unidoc { import BuildCommons._ From 4d4b24927417b2c17810e94d6d46c37491c68869 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 14 Apr 2015 14:00:49 -0700 Subject: [PATCH 764/817] [SPARK-6769][YARN][TEST] Usage of the ListenerBus in YarnClusterSuite is wrong In YarnClusterSuite, a test case uses `SaveExecutorInfo` to handle ExecutorAddedEvent as follows. ``` private class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } ... listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { val data = sc.parallelize(1 to 4, 4).collect().toSet assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) data should be (Set(1, 2, 3, 4)) result = "success" } finally { sc.stop() Files.write(result, status, UTF_8) } ``` But, the usage is wrong because Executors will spawn during initializing SparkContext and SparkContext#addSparkListener should be invoked after the initialization, thus after Executors spawn, so SaveExecutorInfo cannot handle ExecutorAddedEvent. Following code refers the result of the handling ExecutorAddedEvent. Because of the reason above, we cannot reach the assertion. ``` // verify log urls are present listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) } ``` Author: Kousuke Saruta Closes #5417 from sarutak/SPARK-6769 and squashes the following commits: 8adc8ba [Kousuke Saruta] Fixed compile error e258530 [Kousuke Saruta] Fixed style 591cf3e [Kousuke Saruta] Fixed style 48ec89a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 860c965 [Kousuke Saruta] Simplified code 207d325 [Kousuke Saruta] Added findListenersByClass method to ListenerBus 2408c84 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 2d7e409 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 3874adf [Kousuke Saruta] Fixed the usage of listener bus in LogUrlsStandaloneSuite 153a91b [Kousuke Saruta] Fixed the usage of listener bus in YarnClusterSuite --- .../org/apache/spark/util/ListenerBus.scala | 8 ++++++++ .../spark/deploy/LogUrlsStandaloneSuite.scala | 20 ++++++++++--------- .../spark/deploy/yarn/YarnClusterSuite.scala | 17 +++++++++------- 3 files changed, 29 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index d60b8b9a31a9b..a725767d08cc2 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -19,9 +19,12 @@ package org.apache.spark.util import java.util.concurrent.CopyOnWriteArrayList +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.Logging +import org.apache.spark.scheduler.SparkListener /** * An event bus which posts events to its listeners. @@ -64,4 +67,9 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { */ def onPostEvent(listener: L, event: E): Unit + private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { + val c = implicitly[ClassTag[T]].runtimeClass + listeners.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 9cdb42814ca32..c93d16f8a1586 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.net.URL +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.io.Source @@ -65,16 +66,17 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { new MySparkConf().setAll(getAll) } } - val conf = new MySparkConf() + val conf = new MySparkConf().set( + "spark.extraListeners", classOf[SaveExecutorInfo].getName) sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - val listener = new SaveExecutorInfo - sc.addSparkListener(listener) - // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) info.logUrlMap.values.foreach { logUrl => @@ -82,12 +84,12 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { } } } +} - private class SaveExecutorInfo extends SparkListener { - val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() +private[spark] class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { - addedExecutorInfos(executor.executorId) = executor.executorInfo - } + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 76952e3341cc4..a18c94d4ab4a8 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** @@ -282,10 +282,10 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } -private class SaveExecutorInfo extends SparkListener { +private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor : SparkListenerExecutorAdded) { + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } @@ -293,7 +293,6 @@ private class SaveExecutorInfo extends SparkListener { private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 - var listener: SaveExecutorInfo = null def main(args: Array[String]): Unit = { if (args.length != 1) { @@ -306,10 +305,9 @@ private object YarnClusterDriver extends Logging with Matchers { System.exit(1) } - listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) - sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { @@ -323,7 +321,12 @@ private object YarnClusterDriver extends Logging with Matchers { } // verify log urls are present - listener.addedExecutorInfos.values.foreach { info => + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => assert(info.logUrlMap.nonEmpty) } } From a76b921a923ac37d3c73ee18d24df4bb611daba3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 14:07:25 -0700 Subject: [PATCH 765/817] Revert "[SPARK-6352] [SQL] Add DirectParquetOutputCommitter" This reverts commit b29663eeea440b1d1a288d41b5ddf67e77c5bd54. I'm reverting this because it broke test compilation for the Hadoop 1.x profiles. --- .../DirectParquetOutputCommitter.scala | 66 ------------------- .../sql/parquet/ParquetTableOperations.scala | 22 ------- .../spark/sql/parquet/ParquetIOSuite.scala | 21 ------ 3 files changed, 109 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala deleted file mode 100644 index 25a66cb488103..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala +++ /dev/null @@ -1,66 +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.parquet - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - -import parquet.Log -import parquet.hadoop.util.ContextUtil -import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter} - -private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) - extends ParquetOutputCommitter(outputPath, context) { - val LOG = Log.getLog(classOf[ParquetOutputCommitter]) - - override def getWorkPath(): Path = outputPath - override def abortTask(taskContext: TaskAttemptContext): Unit = {} - override def commitTask(taskContext: TaskAttemptContext): Unit = {} - override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true - override def setupJob(jobContext: JobContext): Unit = {} - override def setupTask(taskContext: TaskAttemptContext): Unit = {} - - override def commitJob(jobContext: JobContext) { - try { - val configuration = ContextUtil.getConfiguration(jobContext) - val fileSystem = outputPath.getFileSystem(configuration) - val outputStatus = fileSystem.getFileStatus(outputPath) - val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) - try { - ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) - if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { - val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) - fileSystem.create(successPath).close() - } - } catch { - case e: Exception => { - LOG.warn("could not write summary file for " + outputPath, e) - val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) - if (fileSystem.exists(metadataPath)) { - fileSystem.delete(metadataPath, true) - } - } - } - } catch { - case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) - } - } - -} - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 3724bda829d30..1c868da23e060 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -379,8 +379,6 @@ private[sql] case class InsertIntoParquetTable( */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { - var committer: OutputCommitter = null - // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} @@ -405,26 +403,6 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } - - // override to create output committer from configuration - override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { - if (committer == null) { - val output = getOutputPath(context) - val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", - classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) - val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] - } - committer - } - - // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 - private def getOutputPath(context: TaskAttemptContext): Path = { - context.getConfiguration().get("mapred.output.dir") match { - case null => null - case name => new Path(name) - } - } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 4d0bf7cf99cdf..97c0f439acf13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -381,27 +381,6 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } - - test("SPARK-6352 DirectParquetOutputCommitter") { - // Write to a parquet file and let it fail. - // _temporary should be missing if direct output committer works. - try { - configuration.set("spark.sql.parquet.output.committer.class", - "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") - sqlContext.udf.register("div0", (x: Int) => x / 0) - withTempPath { dir => - intercept[org.apache.spark.SparkException] { - sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) - } - val path = new Path(dir.getCanonicalPath, "_temporary") - val fs = path.getFileSystem(configuration) - assert(!fs.exists(path)) - } - } - finally { - configuration.unset("spark.sql.parquet.output.committer.class") - } - } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From 6de282e2de3cb69f9b746d03fde581429248824a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Apr 2015 16:51:36 -0700 Subject: [PATCH 766/817] [SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" lists to StreamingPage This PR adds two lists, `Active Batches` and `Completed Batches`. Here is the screenshot: ![batch_list](https://cloud.githubusercontent.com/assets/1000778/7060458/d8898572-deb3-11e4-938b-6f8602c71a9f.png) Due to [SPARK-6766](https://issues.apache.org/jira/browse/SPARK-6766), I need to merge #5414 in my local machine to get the above screenshot. Author: zsxwing Closes #5434 from zsxwing/SPARK-6796 and squashes the following commits: be50fc6 [zsxwing] Fix the code style 51b792e [zsxwing] Fix the unit test 6f3078e [zsxwing] Make 'startTime' readable f40e0a9 [zsxwing] Merge branch 'master' into SPARK-6796 2525336 [zsxwing] Rename 'Processed batches' and 'Waiting batches' and also add links a69c091 [zsxwing] Show the number of total completed batches too a12ad7b [zsxwing] Change 'records' to 'events' in the UI 86b5e7f [zsxwing] Make BatchTableBase abstract b248787 [zsxwing] Add tests to verify the new tables d18ab7d [zsxwing] Fix the code style 6ceffb3 [zsxwing] Add "Active Batches" and "Completed Batches" lists to StreamingPage --- .../spark/streaming/ui/AllBatchesTable.scala | 114 ++++++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 44 +++++-- .../spark/streaming/UISeleniumSuite.scala | 11 ++ 3 files changed, 159 insertions(+), 10 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala new file mode 100644 index 0000000000000..df1c0a10704c3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import scala.xml.Node + +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.ui.UIUtils + +private[ui] abstract class BatchTableBase(tableId: String) { + + protected def columns: Seq[Node] = { + Batch Time + Input Size + Scheduling Delay + Processing Time + } + + protected def baseRow(batch: BatchInfo): Seq[Node] = { + val batchTime = batch.batchTime.milliseconds + val formattedBatchTime = UIUtils.formatDate(batch.batchTime.milliseconds) + val eventCount = batch.receivedBlockInfo.values.map { + receivers => receivers.map(_.numRecords).sum + }.sum + val schedulingDelay = batch.schedulingDelay + val formattedSchedulingDelay = schedulingDelay.map(UIUtils.formatDuration).getOrElse("-") + val processingTime = batch.processingDelay + val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-") + + {formattedBatchTime} + {eventCount.toString} events + + {formattedSchedulingDelay} + + + {formattedProcessingTime} + + } + + private def batchTable: Seq[Node] = { + + + {columns} + + + {renderRows} + +
    + } + + def toNodeSeq: Seq[Node] = { + batchTable + } + + /** + * Return HTML for all rows of this table. + */ + protected def renderRows: Seq[Node] +} + +private[ui] class ActiveBatchTable(runningBatches: Seq[BatchInfo], waitingBatches: Seq[BatchInfo]) + extends BatchTableBase("active-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Status + + override protected def renderRows: Seq[Node] = { + // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display + // waiting batches before running batches + waitingBatches.flatMap(batch => {waitingBatchRow(batch)}) ++ + runningBatches.flatMap(batch => {runningBatchRow(batch)}) + } + + private def runningBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ processing + } + + private def waitingBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ queued + } +} + +private[ui] class CompletedBatchTable(batches: Seq[BatchInfo]) + extends BatchTableBase("completed-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Total Delay + + override protected def renderRows: Seq[Node] = { + batches.flatMap(batch => {completedBatchRow(batch)}) + } + + private def completedBatchRow(batch: BatchInfo): Seq[Node] = { + val totalDelay = batch.totalDelay + val formattedTotalDelay = totalDelay.map(UIUtils.formatDuration).getOrElse("-") + baseRow(batch) ++ + + {formattedTotalDelay} + + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index b6dcb62bfeec8..07fa285642eec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -41,7 +41,8 @@ private[ui] class StreamingPage(parent: StreamingTab) generateBasicStats() ++

    ++

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

    ++ generateReceiverStats() ++ - generateBatchStatsTable() + generateBatchStatsTable() ++ + generateBatchListTables() } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } @@ -49,9 +50,10 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime + // scalastyle:off
    + // scalastyle:on } /** Generate stats of data received by the receivers in the streaming program */ @@ -86,10 +89,10 @@ private[ui] class StreamingPage(parent: StreamingTab) "Receiver", "Status", "Location", - "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", - "Minimum rate\n[records/sec]", - "Median rate\n[records/sec]", - "Maximum rate\n[records/sec]", + "Events in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[events/sec]", + "Median rate\n[events/sec]", + "Maximum rate\n[events/sec]", "Last Error" ) val dataRows = (0 until listener.numReceivers).map { receiverId => @@ -190,5 +193,26 @@ private[ui] class StreamingPage(parent: StreamingTab) } UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } + + private def generateBatchListTables(): Seq[Node] = { + val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse + val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse + val completedBatches = listener.retainedCompletedBatches. + sortBy(_.batchTime.milliseconds).reverse + + val activeBatchesContent = { +

    Active Batches ({runningBatches.size + waitingBatches.size})

    ++ + new ActiveBatchTable(runningBatches, waitingBatches).toNodeSeq + } + + val completedBatchesContent = { +

    + Completed Batches (last {completedBatches.size} out of {listener.numTotalCompletedBatches}) +

    ++ + new CompletedBatchTable(completedBatches).toNodeSeq + } + + activeBatchesContent ++ completedBatchesContent + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 998426ebb82e5..205ddf6dbe9b0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -75,6 +75,17 @@ class UISeleniumSuite val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq statisticText should contain("Network receivers:") statisticText should contain("Batch interval:") + + val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq + h4Text should contain("Active Batches (0)") + h4Text should contain("Completed Batches (last 0 out of 0)") + + findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status") + } + findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay") + } } ssc.stop(false) From 9717389365772d218cd7c67f9a13c3440f3c6791 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:51:39 -0700 Subject: [PATCH 767/817] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. The fix for SPARK-6406 broke the case where sub-processes are launched when SPARK_PREPEND_CLASSES is set, because the code now would only add the launcher's build directory to the sub-process's classpath instead of the complete assembly. This patch fixes the problem by having the launch scripts stash the assembly's location in an environment variable. This is not the prettiest solution, but it avoids having to plumb that location all the way through the Worker code that launches executors. The env variable is always set by the launch scripts, so users cannot override it. Author: Marcelo Vanzin Closes #5504 from vanzin/SPARK-6890 and squashes the following commits: 7aec921 [Marcelo Vanzin] Fix tests. ff87a60 [Marcelo Vanzin] Merge branch 'master' into SPARK-6890 31d3ce8 [Marcelo Vanzin] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. --- bin/spark-class | 11 ++++- bin/spark-class2.cmd | 11 ++++- .../launcher/AbstractCommandBuilder.java | 44 +++++++++++++++++-- .../spark/launcher/CommandBuilderUtils.java | 1 + .../SparkSubmitCommandBuilderSuite.java | 15 ++++--- 5 files changed, 71 insertions(+), 11 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index c03946d92e2e4..c49d97ce5cf25 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -82,13 +82,22 @@ if [ $(command -v "$JAR_CMD") ] ; then fi fi +LAUNCH_CLASSPATH="$SPARK_ASSEMBLY_JAR" + +# Add the launcher build dir to the classpath if requested. +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + LAUNCH_CLASSPATH="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" +fi + +export _SPARK_ASSEMBLY="$SPARK_ASSEMBLY_JAR" + # The launcher library will print arguments separated by a NULL character, to allow arguments with # characters that would be otherwise interpreted by the shell. Read that in a while loop, populating # an array that will be used to exec the final command. CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$SPARK_ASSEMBLY_JAR" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4b3401d745f2a..3d068dd3a2739 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -46,13 +46,22 @@ if "%SPARK_ASSEMBLY_JAR%"=="0" ( exit /b 1 ) +set LAUNCH_CLASSPATH=%SPARK_ASSEMBLY_JAR% + +rem Add the launcher build dir to the classpath if requested. +if not "x%SPARK_PREPEND_CLASSES%"=="x" ( + set LAUNCH_CLASSPATH=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH% +) + +set _SPARK_ASSEMBLY=%SPARK_ASSEMBLY_JAR% + rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %SPARK_ASSEMBLY_JAR% org.apache.spark.launcher.Main %*"') do ( +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCH_CLASSPATH% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) %SPARK_CMD% diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index d8279145d8e90..b8f02b961113d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -186,12 +186,24 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - final String assembly = AbstractCommandBuilder.class.getProtectionDomain().getCodeSource(). - getLocation().getPath(); + // We can't rely on the ENV_SPARK_ASSEMBLY variable to be set. Certain situations, such as + // when running unit tests, or user code that embeds Spark and creates a SparkContext + // with a local or local-cluster master, will cause this code to be called from an + // environment where that env variable is not guaranteed to exist. + // + // For the testing case, we rely on the test code to set and propagate the test classpath + // appropriately. + // + // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. + // That duplicates some of the code in the shell scripts that look for the assembly, though. + String assembly = getenv(ENV_SPARK_ASSEMBLY); + if (assembly == null && isEmpty(getenv("SPARK_TESTING"))) { + assembly = findAssembly(); + } addToClassPath(cp, assembly); - // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only - // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate + // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only + // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive File libdir; if (new File(sparkHome, "RELEASE").isFile()) { @@ -299,6 +311,30 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } + private String findAssembly() { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index f4ebc25bdd32b..8028e42ffb483 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -30,6 +30,7 @@ class CommandBuilderUtils { static final String DEFAULT_MEM = "512m"; static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; static final String ENV_SPARK_HOME = "SPARK_HOME"; + static final String ENV_SPARK_ASSEMBLY = "_SPARK_ASSEMBLY"; /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 626116a9e7477..97043a76cc612 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -98,7 +98,7 @@ public void testShellCliParser() throws Exception { parser.NAME, "appName"); - List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List args = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); } @@ -110,7 +110,7 @@ public void testAlternateSyntaxParsing() throws Exception { parser.MASTER + "=foo", parser.DEPLOY_MODE + "=bar"); - List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); assertEquals("foo", findArgValue(cmd, parser.MASTER)); assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); @@ -153,7 +153,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - new SparkSubmitCommandBuilder(Collections.emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; @@ -273,10 +273,15 @@ private boolean findInStringList(String list, String sep, String needle) { return contains(needle, list.split(sep)); } - private List buildCommand(List args, Map env) throws Exception { + private SparkSubmitCommandBuilder newCommandBuilder(List args) { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); - return builder.buildCommand(env); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_ASSEMBLY, "dummy"); + return builder; + } + + private List buildCommand(List args, Map env) throws Exception { + return newCommandBuilder(args).buildCommand(env); } } From 30a6e0dcc0bd298731c1387546779cddcc16bc72 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:52:48 -0700 Subject: [PATCH 768/817] [SPARK-5634] [core] Show correct message in HS when no incomplete apps f... ...ound. Author: Marcelo Vanzin Closes #5515 from vanzin/SPARK-5634 and squashes the following commits: f74ecf1 [Marcelo Vanzin] [SPARK-5634] [core] Show correct message in HS when no incomplete apps found. --- .../scala/org/apache/spark/deploy/history/HistoryPage.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 6e432d63c6b5a..3781b4e8c12bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -90,6 +90,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") ++ appTable + } else if (requestedIncomplete) { +

    No incomplete applications found!

    } else {

    No completed applications found!

    ++

    Did you specify the correct logging directory? From 6be918942c4078692d169d72fa9c358f6e98e85e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 14 Apr 2015 23:47:16 -0700 Subject: [PATCH 769/817] [SPARK-6871][SQL] WITH clause in CTE can not following another WITH clause JIRA https://issues.apache.org/jira/browse/SPARK-6871 Author: Liang-Chi Hsieh Closes #5480 from viirya/no_cte_after_cte and squashes the following commits: 4da3712 [Liang-Chi Hsieh] Create new test. 40b38ed [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_cte_after_cte 0edf568 [Liang-Chi Hsieh] for comments. 6591b79 [Liang-Chi Hsieh] WITH clause in CTE can not following another WITH clause. --- .../apache/spark/sql/catalyst/SqlParser.scala | 18 +++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index bc8d3751f6616..9a3531ceb3343 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -121,14 +121,14 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val start: Parser[LogicalPlan] = - ( (select | ("(" ~> select <~ ")")) * - ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } - | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } - | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} - | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } - ) - | insert - | cte + start1 | insert | cte + + protected lazy val start1: Parser[LogicalPlan] = + (select | ("(" ~> select <~ ")")) * + ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } + | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } + | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} + | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) protected lazy val select: Parser[LogicalPlan] = @@ -159,7 +159,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val cte: Parser[LogicalPlan] = - WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start <~ ")"), ",") ~ start ^^ { + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start1 <~ ")"), ",") ~ (start1 | insert) ^^ { case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 73fb791c3ead7..0174aaee94246 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -431,6 +431,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("Allow only a single WITH clause per query") { + intercept[RuntimeException] { + sql("with q1 as (select * from testData) with q2 as (select * from q1) select * from q2") + } + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), From 29aabdd6c20197adb16706823a8c7f48a0074352 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 10:23:53 +0100 Subject: [PATCH 770/817] [HOTFIX] [SPARK-6896] [SQL] fix compile error in hive-thriftserver SPARK-6440 #5424 import guava but did not promote guava dependency to compile level. [INFO] compiler plugin: BasicArtifact(org.scalamacros,paradise_2.10.4,2.0.1,null) [info] Compiling 8 Scala sources to /root/projects/spark/sql/hive-thriftserver/target/scala-2.10/classes... [error] bad symbolic reference. A signature in Utils.class refers to term util [error] in package com.google.common which is not available. [error] It may be completely missing from the current classpath, or the version on [error] the classpath might be incompatible with the version used when compiling Utils.class. [error] [error] while compiling: /root/projects/spark/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala [error] during phase: erasure [error] library version: version 2.10.4 [error] compiler version: version 2.10.4 [error] reconstructed args: -deprecation -classpath Author: Daoyuan Wang Closes #5507 from adrian-wang/guava and squashes the following commits: c337dad [Daoyuan Wang] fix compile error --- sql/hive-thriftserver/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a96b1ffc26966..f38c796241df1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -44,7 +44,6 @@ com.google.guava guava - runtime ${hive.group} From 6c5ed8a6d552abd967d27cdb94b68d46ccb57221 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 15 Apr 2015 15:17:58 +0100 Subject: [PATCH 771/817] SPARK-6861 [BUILD] Scalastyle config prevents building Maven child modules alone Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 Author: Sean Owen Closes #5471 from srowen/SPARK-6861 and squashes the following commits: acac637 [Sean Owen] Oops, add back execution but leave it at the default verify phase 35a4fd2 [Sean Owen] Revert change to scalastyle-config.xml location, but return scalastyle Maven check to verify phase instead of package to get it farther out of the way, since the Maven invocation is optional c4fb42c [Sean Owen] Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 --- pom.xml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 261292d5b6cde..bcc2f57f1af5d 100644 --- a/pom.xml +++ b/pom.xml @@ -1447,7 +1447,7 @@ org.scalastyle scalastyle-maven-plugin - 0.4.0 + 0.7.0 false true @@ -1456,13 +1456,12 @@ ${basedir}/src/main/scala ${basedir}/src/test/scala scalastyle-config.xml - scalastyle-output.xml + ${basedir}/target/scalastyle-output.xml ${project.build.sourceEncoding} ${project.reporting.outputEncoding} - package check From f11288d5272bc18585b8cad4ee3bd59eade7c296 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 12:58:02 -0700 Subject: [PATCH 772/817] [SPARK-6886] [PySpark] fix big closure with shuffle Currently, the created broadcast object will have same life cycle as RDD in Python. For multistage jobs, an PythonRDD will be created in JVM and the RDD in Python may be GCed, then the broadcast will be destroyed in JVM before the PythonRDD. This PR change to use PythonRDD to track the lifecycle of the broadcast object. It also have a refactor about getNumPartitions() to avoid unnecessary creation of PythonRDD, which could be heavy. cc JoshRosen Author: Davies Liu Closes #5496 from davies/big_closure and squashes the following commits: 9a0ea4c [Davies Liu] fix big closure with shuffle --- python/pyspark/rdd.py | 15 +++++---------- python/pyspark/tests.py | 6 ++---- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c9ac95d117574..93e658eded9e2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1197,7 +1197,7 @@ def take(self, num): [91, 92, 93] """ items = [] - totalParts = self._jrdd.partitions().size() + totalParts = self.getNumPartitions() partsScanned = 0 while len(items) < num and partsScanned < totalParts: @@ -1260,7 +1260,7 @@ def isEmpty(self): >>> sc.parallelize([1]).isEmpty() False """ - return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0 + return self.getNumPartitions() == 0 or len(self.take(1)) == 0 def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): """ @@ -2235,11 +2235,9 @@ def _prepare_for_python_RDD(sc, command, obj=None): ser = CloudPickleSerializer() pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M + # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) - # tracking the life cycle by obj - if obj is not None: - obj._broadcast = broadcast broadcast_vars = ListConverter().convert( [x._jbroadcast for x in sc._pickled_broadcast_vars], sc._gateway._gateway_client) @@ -2294,12 +2292,9 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self.partitioner = prev.partitioner if self.preservesPartitioning else None - self._broadcast = None - def __del__(self): - if self._broadcast: - self._broadcast.unpersist() - self._broadcast = None + def getNumPartitions(self): + return self._prev_jrdd.partitions().size() @property def _jrdd(self): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b938b9ce12395..ee67e80d539f8 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -550,10 +550,8 @@ def test_large_closure(self): data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) self.assertEquals(N, rdd.first()) - self.assertTrue(rdd._broadcast is not None) - rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) - self.assertEqual(1, rdd.first()) - self.assertTrue(rdd._broadcast is None) + # regression test for SPARK-6886 + self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) From b75b3070740803480d235b0c9a86673721344f30 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:00:19 -0700 Subject: [PATCH 773/817] [SPARK-6730][SQL] Allow using keyword as identifier in OPTIONS JIRA: https://issues.apache.org/jira/browse/SPARK-6730 It is very possible that keyword will be used as identifier in `OPTIONS`, this pr makes it works. However, another approach is that we can request that `OPTIONS` can't include keywords and has to use alternative identifier (e.g. table -> cassandraTable) if needed. If so, please let me know to close this pr. Thanks. Author: Liang-Chi Hsieh Closes #5520 from viirya/relax_options and squashes the following commits: 339fd68 [Liang-Chi Hsieh] Use regex parser. 92be11c [Liang-Chi Hsieh] Allow using keyword as identifier in OPTIONS. --- .../scala/org/apache/spark/sql/sources/ddl.scala | 15 ++++++++++++++- .../apache/spark/sql/sources/DDLTestSuite.scala | 11 ++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 319de710fbc3e..2e861b84b7133 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import scala.util.matching.Regex import scala.language.implicitConversions import org.apache.spark.Logging @@ -155,7 +156,19 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} - protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) } + override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", { + case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str + case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str + } + ) + + protected lazy val optionName: Parser[String] = "[_a-zA-Z][a-zA-Z0-9]*".r ^^ { + case name => name + } + + protected lazy val pair: Parser[(String, String)] = + optionName ~ stringLit ^^ { case k ~ v => (k,v) } protected lazy val column: Parser[StructField] = ident ~ dataType ~ (COMMENT ~> stringLit).? ^^ { case columnName ~ typ ~ cm => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 3f24a497390c1..ca25751b9583d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -25,17 +25,17 @@ class DDLScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt, parameters("Table"))(sqlContext) } } -case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, - new MetadataBuilder().putString("comment", "test comment").build()), + new MetadataBuilder().putString("comment", s"test comment $table").build()), StructField("stringType", StringType, nullable = false), StructField("dateType", DateType, nullable = false), StructField("timestampType", TimestampType, nullable = false), @@ -73,7 +73,8 @@ class DDLTestSuite extends DataSourceTest { |USING org.apache.spark.sql.sources.DDLScanSource |OPTIONS ( | From '1', - | To '10' + | To '10', + | Table 'test1' |) """.stripMargin) } @@ -81,7 +82,7 @@ class DDLTestSuite extends DataSourceTest { sqlTest( "describe ddlPeople", Seq( - Row("intType", "int", "test comment"), + Row("intType", "int", "test comment test1"), Row("stringType", "string", ""), Row("dateType", "date", ""), Row("timestampType", "timestamp", ""), From e3e4e9a38b25174ed8bb460ba2b375813ebf3b4b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:01:29 -0700 Subject: [PATCH 774/817] [SPARK-6800][SQL] Update doc for JDBCRelation's columnPartition JIRA https://issues.apache.org/jira/browse/SPARK-6800 Author: Liang-Chi Hsieh Closes #5488 from viirya/fix_jdbc_where and squashes the following commits: 51386c8 [Liang-Chi Hsieh] Update code comment. 1dcc929 [Liang-Chi Hsieh] Update document. 3eb74d6 [Liang-Chi Hsieh] Revert and modify doc. df11783 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_jdbc_where 3e7db15 [Liang-Chi Hsieh] Fix wrong logic to generate WHERE clause for JDBC. --- docs/sql-programming-guide.md | 5 ++++- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 332618edf0c55..03500867df70f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1371,7 +1371,10 @@ the Data Sources API. The following options are supported: These options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple workers. - partitionColumn must be a numeric column from the table in question. + partitionColumn must be a numeric column from the table in question. Notice + that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c25ef58e6f62a..b237fe684cdc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -873,8 +873,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * passed to this function. * * @param columnName the name of a column of integral type that will be used for partitioning. - * @param lowerBound the minimum value of `columnName` to retrieve - * @param upperBound the maximum value of `columnName` to retrieve + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split * evenly into this many partitions * From 785f95586b951d7b05481ee925fb95c20c4d6b6f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 15 Apr 2015 13:04:03 -0700 Subject: [PATCH 775/817] [SPARK-6887][SQL] ColumnBuilder misses FloatType https://issues.apache.org/jira/browse/SPARK-6887 Author: Yin Huai Closes #5499 from yhuai/inMemFloat and squashes the following commits: 84cba38 [Yin Huai] Add test. 4b75ba6 [Yin Huai] Add FloatType back. --- .../spark/sql/columnar/ColumnBuilder.scala | 1 + .../org/apache/spark/sql/QueryTest.scala | 3 + .../columnar/InMemoryColumnarQuerySuite.scala | 59 ++++++++++++++++++- 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c881747751520..00ed70430b84d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -153,6 +153,7 @@ private[sql] object ColumnBuilder { val builder: ColumnBuilder = dataType match { case IntegerType => new IntColumnBuilder case LongType => new LongColumnBuilder + case FloatType => new FloatColumnBuilder case DoubleType => new DoubleColumnBuilder case BooleanType => new BooleanColumnBuilder case ByteType => new ByteColumnBuilder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9a81fc5d72819..59f9508444f25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -104,9 +104,12 @@ object QueryTest { // Converts data to types that we can do equality comparison using Scala collections. // For BigDecimal type, the Scala type has a better definition of equality test (similar to // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. val converted: Seq[Row] = answer.map { s => Row.fromSeq(s.toSeq.map { case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq case o => o }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 479210d1c9c43..56591d9dba29e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.columnar +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{DecimalType, Decimal} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -132,4 +134,59 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) } + + test("test different data types") { + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val dataTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), MapType(StringType, LongType), struct) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD for the schema + val rdd = + sparkContext.parallelize((1 to 100), 10).map { i => + Row( + s"str${i}: test cache.", + s"binary${i}: test cache.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i), + (1 to i).toSeq, + (0 to i).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, + Row((i - 0.25).toFloat, (1 to i).toSeq)) + } + createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + // Cache the table. + sql("cache table InMemoryCache_different_data_types") + // Make sure the table is indeed cached. + val tableScan = table("InMemoryCache_different_data_types").queryExecution.executedPlan + assert( + isCached("InMemoryCache_different_data_types"), + "InMemoryCache_different_data_types should be cached.") + // Issue a query and check the results. + checkAnswer( + sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), + table("InMemoryCache_different_data_types").collect()) + dropTempTable("InMemoryCache_different_data_types") + } } From 85842760dc4616577162f44cc0fa9db9bd23bd9c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 13:06:38 -0700 Subject: [PATCH 776/817] [SPARK-6638] [SQL] Improve performance of StringType in SQL This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte. This PR should not break any public API, Row.getString() will still return java.lang.String. This is the first step of improve the performance of String in SQL. cc rxin Author: Davies Liu Closes #5350 from davies/string and squashes the following commits: 3b7bfa8 [Davies Liu] fix schema of AddJar 2772f0d [Davies Liu] fix new test failure 6d776a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 59025c8 [Davies Liu] address comments from @marmbrus 341ec2c [Davies Liu] turn off scala style check in UTF8StringSuite 744788f [Davies Liu] Merge branch 'master' of github.com:apache/spark into string b04a19c [Davies Liu] add comment for getString/setString 08d897b [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 5116b43 [Davies Liu] rollback unrelated changes 1314a37 [Davies Liu] address comments from Yin 867bf50 [Davies Liu] fix String filter push down 13d9d42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 2089d24 [Davies Liu] add hashcode check back ac18ae6 [Davies Liu] address comment fd11364 [Davies Liu] optimize UTF8String 8d17f21 [Davies Liu] fix hive compatibility tests e5fa5b8 [Davies Liu] remove clone in UTF8String 28f3d81 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 28d6f32 [Davies Liu] refactor 537631c [Davies Liu] some comment about Date 9f4c194 [Davies Liu] convert data type for data source 956b0a4 [Davies Liu] fix hive tests 73e4363 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 9dc32d1 [Davies Liu] fix some hive tests 23a766c [Davies Liu] refactor 8b45864 [Davies Liu] fix codegen with UTF8String bb52e44 [Davies Liu] fix scala style c7dd4d2 [Davies Liu] fix some catalyst tests 38c303e [Davies Liu] fix python sql tests 5f9e120 [Davies Liu] fix sql tests 6b499ac [Davies Liu] fix style a85fb27 [Davies Liu] refactor d32abd1 [Davies Liu] fix utf8 for python api 4699c3a [Davies Liu] use Array[Byte] in UTF8String 21f67c6 [Davies Liu] cleanup 685fd07 [Davies Liu] use UTF8String instead of String for StringType --- python/pyspark/sql/dataframe.py | 10 +- .../main/scala/org/apache/spark/sql/Row.scala | 3 +- .../sql/catalyst/CatalystTypeConverters.scala | 37 +++ .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../catalyst/analysis/HiveTypeCoercion.scala | 6 +- .../spark/sql/catalyst/expressions/Cast.scala | 36 +-- .../expressions/SpecificMutableRow.scala | 12 +- .../expressions/codegen/CodeGenerator.scala | 32 ++- .../codegen/GenerateProjection.scala | 46 ++-- .../sql/catalyst/expressions/generators.scala | 7 +- .../sql/catalyst/expressions/literals.scala | 7 +- .../sql/catalyst/expressions/predicates.scala | 3 +- .../spark/sql/catalyst/expressions/rows.scala | 14 +- .../expressions/stringOperations.scala | 90 ++++---- .../sql/catalyst/optimizer/Optimizer.scala | 21 +- .../apache/spark/sql/types/DateUtils.scala | 1 + .../apache/spark/sql/types/UTF8String.scala | 214 ++++++++++++++++++ .../apache/spark/sql/types/dataTypes.scala | 6 +- .../ExpressionEvaluationSuite.scala | 90 ++++---- .../GeneratedMutableEvaluationSuite.scala | 4 +- .../spark/sql/types/UTF8StringSuite.scala | 70 ++++++ .../org/apache/spark/sql/SQLContext.scala | 1 + .../spark/sql/columnar/ColumnStats.scala | 6 +- .../spark/sql/columnar/ColumnType.scala | 20 +- .../spark/sql/execution/ExistingRDD.scala | 31 ++- .../apache/spark/sql/execution/commands.scala | 13 +- .../spark/sql/execution/debug/package.scala | 2 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 4 +- .../apache/spark/sql/jdbc/JDBCRelation.scala | 2 + .../org/apache/spark/sql/jdbc/jdbc.scala | 5 +- .../apache/spark/sql/json/JSONRelation.scala | 8 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 19 +- .../spark/sql/parquet/ParquetFilters.scala | 12 +- .../sql/parquet/ParquetTableSupport.scala | 7 +- .../apache/spark/sql/parquet/newParquet.scala | 11 +- .../sql/sources/DataSourceStrategy.scala | 37 +-- .../apache/spark/sql/sources/interfaces.scala | 10 + .../scala/org/apache/spark/sql/RowSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +- .../sql/columnar/ColumnarTestUtils.scala | 4 +- .../spark/sql/sources/TableScanSuite.scala | 10 +- .../spark/sql/hive/HiveInspectors.scala | 22 +- .../spark/sql/hive/HiveStrategies.scala | 13 +- .../hive/execution/ScriptTransformation.scala | 17 +- .../spark/sql/hive/execution/commands.scala | 10 +- .../org/apache/spark/sql/hive/Shim12.scala | 4 +- .../org/apache/spark/sql/hive/Shim13.scala | 36 ++- 50 files changed, 742 insertions(+), 298 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ef91a9c4f522d..f2c3b74a185cf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -456,7 +456,7 @@ def join(self, other, joinExprs=None, joinType=None): One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] """ if joinExprs is None: @@ -637,9 +637,9 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] """ jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) @@ -867,11 +867,11 @@ def agg(self, *exprs): >>> gdf = df.groupBy(df.name) >>> gdf.agg({"*": "count"}).collect() - [Row(name=u'Bob', COUNT(1)=1), Row(name=u'Alice', COUNT(1)=1)] + [Row(name=u'Alice', COUNT(1)=1), Row(name=u'Bob', COUNT(1)=1)] >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age)=5), Row(MIN(age)=2)] + [Row(MIN(age)=2), Row(MIN(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index d794f034f5578..ac8a782976465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.{StructType, DateUtils} +import org.apache.spark.sql.types.StructType object Row { /** @@ -257,6 +257,7 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ + // TODO(davies): This is not the right default implementation, we use Int as Date internally def getDate(i: Int): java.sql.Date = apply(i).asInstanceOf[java.sql.Date] /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 91976fef6dc0d..d4f9fdacda4fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -77,6 +77,9 @@ object CatalystTypeConverters { } new GenericRowWithSchema(ar, structType) + case (d: String, _) => + UTF8String(d) + case (d: BigDecimal, _) => Decimal(d) @@ -175,6 +178,11 @@ object CatalystTypeConverters { case other => other } + case dataType: StringType => (item: Any) => extractOption(item) match { + case s: String => UTF8String(s) + case other => other + } + case _ => (item: Any) => extractOption(item) match { case d: BigDecimal => Decimal(d) @@ -184,6 +192,26 @@ object CatalystTypeConverters { } } + /** + * Converts Scala objects to catalyst rows / types. + * + * Note: This should be called before do evaluation on Row + * (It does not support UDT) + * This is used to create an RDD or test results with correct types for Catalyst. + */ + def convertToCatalyst(a: Any): Any = a match { + case s: String => UTF8String(s) + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case seq: Seq[Any] => seq.map(convertToCatalyst) + case r: Row => Row(r.toSeq.map(convertToCatalyst): _*) + case arr: Array[Any] => arr.toSeq.map(convertToCatalyst).toArray + case m: Map[Any, Any] => + m.map { case (k, v) => (convertToCatalyst(k), convertToCatalyst(v)) }.toMap + case other => other + } + /** * Converts Catalyst types used internally in rows to standard Scala types * This method is slow, and for batch conversion you should be using converter @@ -211,6 +239,9 @@ object CatalystTypeConverters { case (i: Int, DateType) => DateUtils.toJavaDate(i) + case (s: UTF8String, StringType) => + s.toString() + case (other, _) => other } @@ -262,6 +293,12 @@ object CatalystTypeConverters { case other => other } + case StringType => + (item: Any) => item match { + case s: UTF8String => s.toString() + case other => other + } + case other => (item: Any) => item } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 01d5c1512201a..d9521953cad73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -138,6 +138,7 @@ trait ScalaReflection { // The data type can be determined without ambiguity. case obj: BooleanType.JvmType => BooleanType case obj: BinaryType.JvmType => BinaryType + case obj: String => StringType case obj: StringType.JvmType => StringType case obj: ByteType.JvmType => ByteType case obj: ShortType.JvmType => ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 3aeb964994d37..35c7f00d4e42a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -115,7 +115,7 @@ trait HiveTypeCoercion { * the appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal.create("NaN", StringType) + val stringNaN = Literal("NaN") def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { @@ -563,6 +563,10 @@ trait HiveTypeCoercion { case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType)) case Average(e @ TimestampType()) => Average(Cast(e, DoubleType)) + // Compatible with Hive + case Substring(e, start, len) if e.dataType != StringType => + Substring(Cast(e, StringType), start, len) + // Coalesce should return the first non-null value, which could be any column // from the list. So we need to make sure the return type is deterministic and // compatible with every child column. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 31f1a5fdc7e53..adf941ab2a45f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -21,7 +21,6 @@ import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ /** Cast the child expression to the target data type. */ @@ -112,21 +111,21 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { - case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) - case DateType => buildCast[Int](_, d => DateUtils.toString(d)) - case TimestampType => buildCast[Timestamp](_, timestampToString) - case _ => buildCast[Any](_, _.toString) + case BinaryType => buildCast[Array[Byte]](_, UTF8String(_)) + case DateType => buildCast[Int](_, d => UTF8String(DateUtils.toString(d))) + case TimestampType => buildCast[Timestamp](_, t => UTF8String(timestampToString(t))) + case _ => buildCast[Any](_, o => UTF8String(o.toString)) } // BinaryConverter private[this] def castToBinary(from: DataType): Any => Any = from match { - case StringType => buildCast[String](_, _.getBytes("UTF-8")) + case StringType => buildCast[UTF8String](_, _.getBytes) } // UDFToBoolean private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, _.length() != 0) + buildCast[UTF8String](_, _.length() != 0) case TimestampType => buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => @@ -151,8 +150,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => { + buildCast[UTF8String](_, utfs => { // Throw away extra if more than 9 decimal places + val s = utfs.toString val periodIdx = s.indexOf(".") var n = s if (periodIdx != -1 && n.length() - periodIdx > 9) { @@ -227,8 +227,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => - try DateUtils.fromJavaDate(Date.valueOf(s)) + buildCast[UTF8String](_, s => + try DateUtils.fromJavaDate(Date.valueOf(s.toString)) catch { case _: java.lang.IllegalArgumentException => null } ) case TimestampType => @@ -245,7 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toLong catch { + buildCast[UTF8String](_, s => try s.toString.toLong catch { case _: NumberFormatException => null }) case BooleanType => @@ -261,7 +261,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // IntConverter private[this] def castToInt(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toInt catch { + buildCast[UTF8String](_, s => try s.toString.toInt catch { case _: NumberFormatException => null }) case BooleanType => @@ -277,7 +277,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ShortConverter private[this] def castToShort(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toShort catch { + buildCast[UTF8String](_, s => try s.toString.toShort catch { case _: NumberFormatException => null }) case BooleanType => @@ -293,7 +293,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ByteConverter private[this] def castToByte(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toByte catch { + buildCast[UTF8String](_, s => try s.toString.toByte catch { case _: NumberFormatException => null }) case BooleanType => @@ -323,7 +323,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => - buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { + buildCast[UTF8String](_, s => try { + changePrecision(Decimal(s.toString.toDouble), target) + } catch { case _: NumberFormatException => null }) case BooleanType => @@ -348,7 +350,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DoubleConverter private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toDouble catch { + buildCast[UTF8String](_, s => try s.toString.toDouble catch { case _: NumberFormatException => null }) case BooleanType => @@ -364,7 +366,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // FloatConverter private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toFloat catch { + buildCast[UTF8String](_, s => try s.toString.toFloat catch { case _: NumberFormatException => null }) case BooleanType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 47b6f358ed1b1..3475ed05f4454 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -230,13 +230,17 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new GenericRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = { - if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + override def update(ordinal: Int, value: Any) { + if (value == null) { + setNullAt(ordinal) + } else { + values(ordinal).update(value) + } } - override def setString(ordinal: Int, value: String): Unit = update(ordinal, value) + override def setString(ordinal: Int, value: String): Unit = update(ordinal, UTF8String(value)) - override def getString(ordinal: Int): String = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int): String = apply(ordinal).toString override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index d141354a0f427..be2c101d63a63 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -216,10 +216,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val $primitiveTerm: ${termForType(dataType)} = $value """.children - case expressions.Literal(value: String, dataType) => + case expressions.Literal(value: UTF8String, dataType) => q""" val $nullTerm = ${value == null} - val $primitiveTerm: ${termForType(dataType)} = $value + val $primitiveTerm: ${termForType(dataType)} = + org.apache.spark.sql.types.UTF8String(${value.getBytes}) """.children case expressions.Literal(value: Int, dataType) => @@ -243,11 +244,14 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children case Cast(child @ DateType(), StringType) => - child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + child.castOrNull(c => + q"""org.apache.spark.sql.types.UTF8String( + org.apache.spark.sql.types.DateUtils.toString($c))""", + StringType) case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) @@ -272,9 +276,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - ${eval.primitiveTerm}.toString + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.toString) """.children + case EqualTo(e1: BinaryType, e2: BinaryType) => + (e1, e2).evaluateAs (BooleanType) { + case (eval1, eval2) => + q""" + java.util.Arrays.equals($eval1.asInstanceOf[Array[Byte]], + $eval2.asInstanceOf[Array[Byte]]) + """ + } + case EqualTo(e1, e2) => (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } @@ -597,7 +610,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val localLogger = log val localLoggerTree = reify { localLogger } q""" - $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + $localLoggerTree.debug( + ${e.toString} + ": " + (if ($nullTerm) "null" else $primitiveTerm.toString)) """ :: Nil } else { Nil @@ -608,6 +622,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { dataType match { + case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]" case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" } @@ -619,6 +634,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin ordinal: Int, value: TermName) = { dataType match { + case StringType => q"$destinationRow.update($ordinal, $value)" case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" case _ => q"$destinationRow.update($ordinal, $value)" } @@ -642,13 +658,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case DoubleType => "Double" case FloatType => "Float" case BooleanType => "Boolean" - case StringType => "String" + case StringType => "org.apache.spark.sql.types.UTF8String" } protected def defaultPrimitive(dt: DataType) = dt match { case BooleanType => ru.Literal(Constant(false)) case FloatType => ru.Literal(Constant(-1.0.toFloat)) - case StringType => ru.Literal(Constant("")) + case StringType => q"""org.apache.spark.sql.types.UTF8String("")""" case ShortType => ru.Literal(Constant(-1.toShort)) case LongType => ru.Literal(Constant(-1L)) case ByteType => ru.Literal(Constant(-1.toByte)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 69397a73a8880..6f572ff959fb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -111,36 +111,54 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val specificAccessorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // getString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) return $elementName" :: Nil case _ => Nil } - - q""" - override def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + // Row() need this interface to compile + case StringType => + q""" + override def getString(i: Int): String = { + $accessorFailure + }""" + case other => + q""" + override def ${accessorForType(dataType)}(i: Int): ${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } } val specificMutatorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // setString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } - - q""" - override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + case StringType => + // MutableRow() need this interface to compile + q""" + override def setString(i: Int, value: String) { + $accessorFailure + }""" + case other => + q""" + override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}) { + ..$ifStatements; + $accessorFailure + }""" + } } val hashValues = expressions.zipWithIndex.map { case (e,i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 860b72fad38b3..67caadb839ff9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.types._ /** @@ -85,8 +85,11 @@ case class UserDefinedGenerator( override protected def makeOutput(): Seq[Attribute] = schema override def eval(input: Row): TraversableOnce[Row] = { + // TODO(davies): improve this + // Convert the objects into Scala Type before calling function, we need schema to support UDT + val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true))) val inputRow = new InterpretedProjection(children) - function(inputRow(input)) + function(CatalystTypeConverters.convertToScala(inputRow(input), inputSchema).asInstanceOf[Row]) } override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 0e2d593e94124..18cba4cc46707 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types._ object Literal { @@ -29,7 +30,7 @@ object Literal { case f: Float => Literal(f, FloatType) case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) - case s: String => Literal(s, StringType) + case s: String => Literal(UTF8String(s), StringType) case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) @@ -42,7 +43,9 @@ object Literal { throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } - def create(v: Any, dataType: DataType): Literal = Literal(v, dataType) + def create(v: Any, dataType: DataType): Literal = { + Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 7e47cb3fffe12..fcd6352079b4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -179,8 +179,7 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison val r = right.eval(input) if (r == null) null else if (left.dataType != BinaryType) l == r - else BinaryType.ordering.compare( - l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) == 0 + else java.util.Arrays.equals(l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 0a275b84086cf..1b62e17ff47fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{StructType, NativeType} +import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} /** @@ -37,6 +37,7 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) + // TODO(davies): add setDate() and setDecimal() } /** @@ -114,9 +115,15 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { } override def getString(i: Int): String = { - values(i).asInstanceOf[String] + values(i) match { + case null => null + case s: String => s + case utf8: UTF8String => utf8.toString + } } + // TODO(davies): add getDate and getDecimal + // Custom hashCode function that matches the efficient code generated version. override def hashCode: Int = { var result: Int = 37 @@ -189,8 +196,7 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { override def setFloat(ordinal: Int, value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int, value: Int): Unit = { values(ordinal) = value } override def setLong(ordinal: Int, value: Long): Unit = { values(ordinal) = value } - override def setString(ordinal: Int, value: String): Unit = { values(ordinal) = value } - + override def setString(ordinal: Int, value: String) { values(ordinal) = UTF8String(value)} override def setNullAt(i: Int): Unit = { values(i) = null } override def setShort(ordinal: Int, value: Short): Unit = { values(ordinal) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index acfbbace608ef..d597bf7ce756a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,11 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import scala.collection.IndexedSeqOptimized - - import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType} +import org.apache.spark.sql.types._ trait StringRegexExpression { self: BinaryExpression => @@ -60,38 +57,17 @@ trait StringRegexExpression { if(r == null) { null } else { - val regex = pattern(r.asInstanceOf[String]) + val regex = pattern(r.asInstanceOf[UTF8String].toString) if(regex == null) { null } else { - matches(regex, l.asInstanceOf[String]) + matches(regex, l.asInstanceOf[UTF8String].toString) } } } } } -trait CaseConversionExpression { - self: UnaryExpression => - - type EvaluatedType = Any - - def convert(v: String): String - - override def foldable: Boolean = child.foldable - def nullable: Boolean = child.nullable - def dataType: DataType = StringType - - override def eval(input: Row): Any = { - val evaluated = child.eval(input) - if (evaluated == null) { - null - } else { - convert(evaluated.toString) - } - } -} - /** * Simple RegEx pattern matching function */ @@ -134,12 +110,33 @@ case class RLike(left: Expression, right: Expression) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } +trait CaseConversionExpression { + self: UnaryExpression => + + type EvaluatedType = Any + + def convert(v: UTF8String): UTF8String + + override def foldable: Boolean = child.foldable + def nullable: Boolean = child.nullable + def dataType: DataType = StringType + + override def eval(input: Row): Any = { + val evaluated = child.eval(input) + if (evaluated == null) { + null + } else { + convert(evaluated.asInstanceOf[UTF8String]) + } + } +} + /** * A function that converts the characters of a string to uppercase. */ case class Upper(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toUpperCase() + override def convert(v: UTF8String): UTF8String = v.toUpperCase override def toString: String = s"Upper($child)" } @@ -149,7 +146,7 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE */ case class Lower(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toLowerCase() + override def convert(v: UTF8String): UTF8String = v.toLowerCase override def toString: String = s"Lower($child)" } @@ -162,15 +159,16 @@ trait StringComparison { override def nullable: Boolean = left.nullable || right.nullable - def compare(l: String, r: String): Boolean + def compare(l: UTF8String, r: UTF8String): Boolean override def eval(input: Row): Any = { - val leftEval = left.eval(input).asInstanceOf[String] + val leftEval = left.eval(input) if(leftEval == null) { null } else { - val rightEval = right.eval(input).asInstanceOf[String] - if (rightEval == null) null else compare(leftEval, rightEval) + val rightEval = right.eval(input) + if (rightEval == null) null + else compare(leftEval.asInstanceOf[UTF8String], rightEval.asInstanceOf[UTF8String]) } } @@ -184,7 +182,7 @@ trait StringComparison { */ case class Contains(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.contains(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) } /** @@ -192,7 +190,7 @@ case class Contains(left: Expression, right: Expression) */ case class StartsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.startsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) } /** @@ -200,7 +198,7 @@ case class StartsWith(left: Expression, right: Expression) */ case class EndsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.endsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) } /** @@ -224,9 +222,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends override def children: Seq[Expression] = str :: pos :: len :: Nil @inline - def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) - (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { - val len = str.length + def slicePos(startPos: Int, sliceLen: Int, length: () => Int): (Int, Int) = { // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and // negative indices for start positions. If a start index i is greater than 0, it // refers to element i-1 in the sequence. If a start index i is less than 0, it refers @@ -235,7 +231,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends val start = startPos match { case pos if pos > 0 => pos - 1 - case neg if neg < 0 => len + neg + case neg if neg < 0 => length() + neg case _ => 0 } @@ -244,12 +240,11 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends case x => start + x } - str.slice(start, end) + (start, end) } override def eval(input: Row): Any = { val string = str.eval(input) - val po = pos.eval(input) val ln = len.eval(input) @@ -257,11 +252,14 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends null } else { val start = po.asInstanceOf[Int] - val length = ln.asInstanceOf[Int] - + val length = ln.asInstanceOf[Int] string match { - case ba: Array[Byte] => slice(ba, start, length) - case other => slice(other.toString, start, length) + case ba: Array[Byte] => + val (st, end) = slicePos(start, length, () => ba.length) + ba.slice(st, end) + case s: UTF8String => + val (st, end) = slicePos(start, length, () => s.length) + s.slice(st, end) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 93e69d409cb91..7c80634d2c852 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -198,14 +198,19 @@ object LikeSimplification extends Rule[LogicalPlan] { val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => - StartsWith(l, Literal(pattern)) - case Like(l, Literal(endsWith(pattern), StringType)) => - EndsWith(l, Literal(pattern)) - case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => - Contains(l, Literal(pattern)) - case Like(l, Literal(equalTo(pattern), StringType)) => - EqualTo(l, Literal(pattern)) + case Like(l, Literal(utf, StringType)) => + utf.toString match { + case startsWith(pattern) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case endsWith(pattern) => + EndsWith(l, Literal(pattern)) + case contains(pattern) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + case equalTo(pattern) => + EqualTo(l, Literal(pattern)) + case _ => + Like(l, Literal.create(utf, StringType)) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 504fb05842505..d36a49159b87f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -40,6 +40,7 @@ object DateUtils { millisToDays(d.getTime) } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisLocal: Long): Int = { ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala new file mode 100644 index 0000000000000..fc02ba6c9c43e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala @@ -0,0 +1,214 @@ +/* +* 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.types + +import java.util.Arrays + +/** + * A UTF-8 String, as internal representation of StringType in SparkSQL + * + * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + * + * Note: This is not designed for general use cases, should not be used outside SQL. + */ + +final class UTF8String extends Ordered[UTF8String] with Serializable { + + private[this] var bytes: Array[Byte] = _ + + /** + * Update the UTF8String with String. + */ + def set(str: String): UTF8String = { + bytes = str.getBytes("utf-8") + this + } + + /** + * Update the UTF8String with Array[Byte], which should be encoded in UTF-8 + */ + def set(bytes: Array[Byte]): UTF8String = { + this.bytes = bytes + this + } + + /** + * Return the number of bytes for a code point with the first byte as `b` + * @param b The first byte of a code point + */ + @inline + private[this] def numOfBytes(b: Byte): Int = { + val offset = (b & 0xFF) - 192 + if (offset >= 0) UTF8String.bytesOfCodePointInUTF8(offset) else 1 + } + + /** + * Return the number of code points in it. + * + * This is only used by Substring() when `start` is negative. + */ + def length(): Int = { + var len = 0 + var i: Int = 0 + while (i < bytes.length) { + i += numOfBytes(bytes(i)) + len += 1 + } + len + } + + def getBytes: Array[Byte] = { + bytes + } + + /** + * Return a substring of this, + * @param start the position of first code point + * @param until the position after last code point + */ + def slice(start: Int, until: Int): UTF8String = { + if (until <= start || start >= bytes.length || bytes == null) { + new UTF8String + } + + var c = 0 + var i: Int = 0 + while (c < start && i < bytes.length) { + i += numOfBytes(bytes(i)) + c += 1 + } + var j = i + while (c < until && j < bytes.length) { + j += numOfBytes(bytes(j)) + c += 1 + } + UTF8String(Arrays.copyOfRange(bytes, i, j)) + } + + def contains(sub: UTF8String): Boolean = { + val b = sub.getBytes + if (b.length == 0) { + return true + } + var i: Int = 0 + while (i <= bytes.length - b.length) { + // In worst case, it's O(N*K), but should works fine with SQL + if (bytes(i) == b(0) && Arrays.equals(Arrays.copyOfRange(bytes, i, i + b.length), b)) { + return true + } + i += 1 + } + false + } + + def startsWith(prefix: UTF8String): Boolean = { + val b = prefix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, 0, b.length), b) + } + + def endsWith(suffix: UTF8String): Boolean = { + val b = suffix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, bytes.length - b.length, bytes.length), b) + } + + def toUpperCase(): UTF8String = { + // upper case depends on locale, fallback to String. + UTF8String(toString().toUpperCase) + } + + def toLowerCase(): UTF8String = { + // lower case depends on locale, fallback to String. + UTF8String(toString().toLowerCase) + } + + override def toString(): String = { + new String(bytes, "utf-8") + } + + override def clone(): UTF8String = new UTF8String().set(this.bytes) + + override def compare(other: UTF8String): Int = { + var i: Int = 0 + val b = other.getBytes + while (i < bytes.length && i < b.length) { + val res = bytes(i).compareTo(b(i)) + if (res != 0) return res + i += 1 + } + bytes.length - b.length + } + + override def compareTo(other: UTF8String): Int = { + compare(other) + } + + override def equals(other: Any): Boolean = other match { + case s: UTF8String => + Arrays.equals(bytes, s.getBytes) + case s: String => + // This is only used for Catalyst unit tests + // fail fast + bytes.length >= s.length && length() == s.length && toString() == s + case _ => + false + } + + override def hashCode(): Int = { + Arrays.hashCode(bytes) + } +} + +object UTF8String { + // number of tailing bytes in a UTF8 sequence for a code point + // see http://en.wikipedia.org/wiki/UTF-8, 192-256 of Byte 1 + private[types] val bytesOfCodePointInUTF8: Array[Int] = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, + 6, 6, 6, 6) + + /** + * Create a UTF-8 String from String + */ + def apply(s: String): UTF8String = { + if (s != null) { + new UTF8String().set(s) + } else{ + null + } + } + + /** + * Create a UTF-8 String from Array[Byte], which should be encoded in UTF-8 + */ + def apply(bytes: Array[Byte]): UTF8String = { + if (bytes != null) { + new UTF8String().set(bytes) + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index cdf2bc68d9c5e..c6fb22c26bd3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -350,7 +350,7 @@ class StringType private() extends NativeType with PrimitiveType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = String + private[sql] type JvmType = UTF8String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] @@ -1196,8 +1196,8 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { /** * Convert the user type to a SQL datum * - * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, - * where we need to convert Any to UserType. + * TODO: Can we make this take obj: UserType? The issue is in + * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. */ def serialize(obj: Any): Any diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index d4362a91d992c..76298f03c94ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -25,8 +25,9 @@ import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types._ @@ -59,6 +60,10 @@ class ExpressionEvaluationBaseSuite extends FunSuite { class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { + def create_row(values: Any*): Row = { + new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) + } + test("literals") { checkEvaluation(Literal(1), 1) checkEvaluation(Literal(true), true) @@ -265,24 +270,23 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) - checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) - checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) - checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - - checkEvaluation(Literal.create(null, StringType) like regEx, null, - new GenericRow(Array[Any]("bc%"))) + checkEvaluation("abcd" like regEx, null, create_row(null)) + checkEvaluation("abdef" like regEx, true, create_row("abdef")) + checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) + checkEvaluation("addb" like regEx, true, create_row("a_%b")) + checkEvaluation("addb" like regEx, false, create_row("a\\__b")) + checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) + checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) + checkEvaluation("addb" like regEx, true, create_row("a%")) + checkEvaluation("addb" like regEx, false, create_row("**")) + checkEvaluation("abc" like regEx, true, create_row("a%")) + checkEvaluation("abc" like regEx, false, create_row("b%")) + checkEvaluation("abc" like regEx, false, create_row("bc%")) + checkEvaluation("a\nb" like regEx, true, create_row("a_b")) + checkEvaluation("ab" like regEx, true, create_row("a%b")) + checkEvaluation("a\nb" like regEx, true, create_row("a%b")) + + checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) } test("RLIKE literal Regular Expression") { @@ -313,14 +317,14 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("RLIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) - checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) - checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) - checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) + checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) + checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) + checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) + checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + evaluate("abbbbc" rlike regEx, create_row("**")) } } @@ -763,7 +767,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("null checking") { - val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val row = create_row("^Ba*n", null, true, null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) val c3 = 'a.boolean.at(2) @@ -803,7 +807,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("case when") { - val row = new GenericRow(Array[Any](null, false, true, "a", "b", "c")) + val row = create_row(null, false, true, "a", "b", "c") val c1 = 'a.boolean.at(0) val c2 = 'a.boolean.at(1) val c3 = 'a.boolean.at(2) @@ -846,13 +850,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("complex type") { - val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 - null.asInstanceOf[String], // 1 - new GenericRow(Array[Any]("aa", "bb")), // 2 - Map("aa"->"bb"), // 3 - Seq("aa", "bb") // 4 - )) + val row = create_row( + "^Ba*n", // 0 + null.asInstanceOf[UTF8String], // 1 + create_row("aa", "bb"), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + ) val typeS = StructType( StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil @@ -909,7 +913,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("arithmetic") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -934,7 +938,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("fractional arithmetic") { - val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val row = create_row(1.1, 2.0, 3.1, null) val c1 = 'a.double.at(0) val c2 = 'a.double.at(1) val c3 = 'a.double.at(2) @@ -958,7 +962,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) + val row = create_row(1, 2, 3, null, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -988,7 +992,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("StringComparison") { - val row = new GenericRow(Array[Any]("abc", null)) + val row = create_row("abc", null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) @@ -1009,7 +1013,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("Substring") { - val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + val row = create_row("example", "example".toArray.map(_.toByte)) val s = 'a.string.at(0) @@ -1053,7 +1057,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { // substring(null, _, _) -> null checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), - null, new GenericRow(Array[Any](null))) + null, create_row(null)) // substring(_, null, _) -> null checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), @@ -1102,20 +1106,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("SQRT") { val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) - val rowSequence = inputSequence.map(l => new GenericRow(Array[Any](l.toDouble))) + val rowSequence = inputSequence.map(l => create_row(l.toDouble)) val d = 'a.double.at(0) for ((row, expected) <- rowSequence zip expectedResults) { checkEvaluation(Sqrt(d), expected, row) } - checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) checkEvaluation(Sqrt(-1), null, EmptyRow) checkEvaluation(Sqrt(-1.5), null, EmptyRow) } test("Bitwise operations") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 275ea2627ebcd..bcc0c404d2cfb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.codegen._ /** @@ -43,7 +43,7 @@ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { } val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](expected)) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) if (actual.hashCode() != expectedRow.hashCode()) { fail( s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala new file mode 100644 index 0000000000000..a22aa6f244c48 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala @@ -0,0 +1,70 @@ +/* +* 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.types + +import org.scalatest.FunSuite + +// scalastyle:off +class UTF8StringSuite extends FunSuite { + test("basic") { + def check(str: String, len: Int) { + + assert(UTF8String(str).length == len) + assert(UTF8String(str.getBytes("utf8")).length() == len) + + assert(UTF8String(str) == str) + assert(UTF8String(str.getBytes("utf8")) == str) + assert(UTF8String(str).toString == str) + assert(UTF8String(str.getBytes("utf8")).toString == str) + assert(UTF8String(str.getBytes("utf8")) == UTF8String(str)) + + assert(UTF8String(str).hashCode() == UTF8String(str.getBytes("utf8")).hashCode()) + } + + check("hello", 5) + check("世 界", 3) + } + + test("contains") { + assert(UTF8String("hello").contains(UTF8String("ello"))) + assert(!UTF8String("hello").contains(UTF8String("vello"))) + assert(UTF8String("大千世界").contains(UTF8String("千世"))) + assert(!UTF8String("大千世界").contains(UTF8String("世千"))) + } + + test("prefix") { + assert(UTF8String("hello").startsWith(UTF8String("hell"))) + assert(!UTF8String("hello").startsWith(UTF8String("ell"))) + assert(UTF8String("大千世界").startsWith(UTF8String("大千"))) + assert(!UTF8String("大千世界").startsWith(UTF8String("千"))) + } + + test("suffix") { + assert(UTF8String("hello").endsWith(UTF8String("ello"))) + assert(!UTF8String("hello").endsWith(UTF8String("ellov"))) + assert(UTF8String("大千世界").endsWith(UTF8String("世界"))) + assert(!UTF8String("大千世界").endsWith(UTF8String("世"))) + } + + test("slice") { + assert(UTF8String("hello").slice(1, 3) == UTF8String("el")) + assert(UTF8String("大千世界").slice(0, 1) == UTF8String("大")) + assert(UTF8String("大千世界").slice(1, 3) == UTF8String("千世")) + assert(UTF8String("大千世界").slice(3, 5) == UTF8String("界")) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index b237fe684cdc1..89a4faf35e0d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1195,6 +1195,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case FloatType => true case DateType => true case TimestampType => true + case StringType => true case ArrayType(_, _) => true case MapType(_, _, _) => true case StructType(_) => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 87a6631da8300..b0f983c180673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -216,13 +216,13 @@ private[sql] class IntColumnStats extends ColumnStats { } private[sql] class StringColumnStats extends ColumnStats { - protected var upper: String = null - protected var lower: String = null + protected var upper: UTF8String = null + protected var lower: UTF8String = null override def gatherStats(row: Row, ordinal: Int): Unit = { super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { - val value = row.getString(ordinal) + val value = row(ordinal).asInstanceOf[UTF8String] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value sizeInBytes += STRING.actualSize(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index c47497e0662d9..1b9e0df2dcb5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag @@ -312,26 +312,28 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { row.getString(ordinal).getBytes("utf-8").length + 4 } - override def append(v: String, buffer: ByteBuffer): Unit = { - val stringBytes = v.getBytes("utf-8") + override def append(v: UTF8String, buffer: ByteBuffer): Unit = { + val stringBytes = v.getBytes buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } - override def extract(buffer: ByteBuffer): String = { + override def extract(buffer: ByteBuffer): UTF8String = { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes, "utf-8") + UTF8String(stringBytes) } - override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { - row.setString(ordinal, value) + override def setField(row: MutableRow, ordinal: Int, value: UTF8String): Unit = { + row.update(ordinal, value) } - override def getField(row: Row, ordinal: Int): String = row.getString(ordinal) + override def getField(row: Row, ordinal: Int): UTF8String = { + row(ordinal).asInstanceOf[UTF8String] + } override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { - to.setString(toOrdinal, from.getString(fromOrdinal)) + to.update(toOrdinal, from(fromOrdinal)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 656bdd7212f56..1fd387eec7e57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -54,6 +54,33 @@ object RDDConversions { } } } + + /** + * Convert the objects inside Row into the types Catalyst expected. + */ + def rowToRowRdd(data: RDD[Row], schema: StructType): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.toSeq.toArray) + val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = converters(i)(r(i)) + i += 1 + } + + mutableRow + } + } + } + } } /** Logical plan node for scanning data from an RDD. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index fad7a281dc1e2..99f24910fd61f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType} -import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -61,7 +62,11 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray - override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) + override def execute(): RDD[Row] = { + val converted = sideEffectResult.map(r => + CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[Row]) + sqlContext.sparkContext.parallelize(converted, 1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e916e68e58b5d..710787096e6cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -164,7 +164,7 @@ package object debug { case (_: Long, LongType) => case (_: Int, IntegerType) => - case (_: String, StringType) => + case (_: UTF8String, StringType) => case (_: Float, FloatType) => case (_: Byte, ByteType) => case (_: Short, ShortType) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 5b308d88d4cdf..7a43bfd8bc8d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -140,6 +140,7 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) case (date: Int, DateType) => DateUtils.toJavaDate(date) + case (s: UTF8String, StringType) => s.toString // Pyrolite can handle Timestamp and Decimal case (other, _) => other @@ -192,7 +193,8 @@ object EvaluatePython { case (c: Long, IntegerType) => c.toInt case (c: Int, LongType) => c.toLong case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString + case (c: String, StringType) => UTF8String(c) + case (c, StringType) if !c.isInstanceOf[String] => UTF8String(c.toString) case (c, _) => c } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 463e1dcc268bc..b9022fcd9e3ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -233,7 +233,7 @@ private[sql] class JDBCRDD( * Converts value to SQL expression. */ private def compileValue(value: Any): Any = value match { - case stringValue: String => s"'${escapeSql(stringValue)}'" + case stringValue: UTF8String => s"'${escapeSql(stringValue.toString)}'" case _ => value } @@ -349,12 +349,14 @@ private[sql] class JDBCRDD( val pos = i + 1 conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) + // TODO(davies): convert Date into Int case DateConversion => mutableRow.update(i, rs.getDate(pos)) case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) case LongConversion => mutableRow.setLong(i, rs.getLong(pos)) + // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 case StringConversion => mutableRow.setString(i, rs.getString(pos)) case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 4fa84dc076f7e..99b755c9f25d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -130,6 +130,8 @@ private[sql] case class JDBCRelation( extends BaseRelation with PrunedFilteredScan { + override val needConversion: Boolean = false + override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index 34f864f5fda7a..d4e0abc040bc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -18,11 +18,8 @@ package org.apache.spark.sql import java.sql.{Connection, DriverManager, PreparedStatement} -import org.apache.spark.{Logging, Partition} -import org.apache.spark.sql._ -import org.apache.spark.sql.sources.LogicalRelation -import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartition} +import org.apache.spark.Logging import org.apache.spark.sql.types._ package object jdbc { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f4c99b4b56606..e3352d02787fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row - -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] class DefaultSource @@ -113,6 +113,8 @@ private[sql] case class JSONRelation( // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) + override val needConversion: Boolean = false + override val schema = userSpecifiedSchema.getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index b1e8521383756..29de7401dda71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -409,7 +409,7 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case StringType => toString(value) + case StringType => UTF8String(toString(value)) case _ if value == null || value == "" => null // guard the non string type case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 43ca359b51735..bc108e37dfb0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -219,8 +219,8 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, value.getBytes) - protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - updateField(fieldIndex, value) + protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + updateField(fieldIndex, UTF8String(value)) protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, readTimestamp(value)) @@ -418,8 +418,8 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, value.getBytes) - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - current.setString(fieldIndex, value) + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + current.update(fieldIndex, UTF8String(value)) override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, readTimestamp(value)) @@ -475,19 +475,18 @@ private[parquet] class CatalystPrimitiveConverter( private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { - private[this] var dict: Array[String] = null + private[this] var dict: Array[Array[Byte]] = null override def hasDictionarySupport: Boolean = true override def setDictionary(dictionary: Dictionary):Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) {dictionary.decodeToBinary(_).toStringUsingUTF8} - + dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } override def addValueFromDictionary(dictionaryId: Int): Unit = parent.updateString(fieldIndex, dict(dictionaryId)) override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.toStringUsingUTF8) + parent.updateString(fieldIndex, value.getBytes) } private[parquet] object CatalystArrayConverter { @@ -714,9 +713,9 @@ private[parquet] class CatalystNativeArrayConverter( elements += 1 } - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = { + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] + buffer(elements) = UTF8String(value).asInstanceOf[NativeType] elements += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 0357dcc4688be..5eb1c6abc2432 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -55,7 +55,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), @@ -76,7 +76,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), @@ -94,7 +94,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -111,7 +111,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -128,7 +128,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -145,7 +145,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 5a1b15490d273..e05a4c20b0d41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -198,10 +198,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { if (value != null) { schema match { case StringType => writer.addBinary( - Binary.fromByteArray( - value.asInstanceOf[String].getBytes("utf-8") - ) - ) + Binary.fromByteArray(value.asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) @@ -349,7 +346,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { index: Int): Unit = { ctype match { case StringType => writer.addBinary( - Binary.fromByteArray(record(index).asInstanceOf[String].getBytes("utf-8"))) + Binary.fromByteArray(record(index).asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) 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 20fdf5e58ef82..af7b3c81ae7b2 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 @@ -33,7 +33,6 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} - import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.metadata.CompressionCodecName @@ -45,13 +44,13 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, expressions} import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode} -import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWritable, SparkException, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, SparkException, TaskContext, Partition => SparkPartition} /** * Allows creation of Parquet based tables using the syntax: @@ -409,6 +408,9 @@ private[sql] case class ParquetRelation2( file.getName == ParquetFileWriter.PARQUET_METADATA_FILE } + // Skip type conversion + override val needConversion: Boolean = false + // TODO Should calculate per scan size // It's common that a query only scans a fraction of a large Parquet file. Returning size of the // whole Parquet file disables some optimizations in this case (e.g. broadcast join). @@ -550,7 +552,8 @@ private[sql] case class ParquetRelation2( baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { - case p if split.getPath.getParent.toString == p.path => p.values + case p if split.getPath.getParent.toString == p.path => + CatalystTypeConverters.convertToCatalyst(p.values).asInstanceOf[Row] }.get val requiredPartOrdinal = partitionKeyLocations.keys.toSeq 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 34d048e426d10..b3d71f687a60a 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 @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.{UTF8String, StringType} import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -53,7 +54,7 @@ private[sql] object DataSourceStrategy extends Strategy { (a, _) => t.buildScan(a)) :: Nil case l @ LogicalRelation(t: TableScan) => - execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => @@ -102,20 +103,30 @@ private[sql] object DataSourceStrategy extends Strategy { projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. .map(relation.attributeMap) // Match original case of attributes. - val scan = - execution.PhysicalRDD( - projectList.map(_.toAttribute), + val scan = createPhysicalRDD(relation.relation, projectList.map(_.toAttribute), scanBuilder(requestedColumns, pushedFilters)) filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) } else { val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq - val scan = - execution.PhysicalRDD(requestedColumns, scanBuilder(requestedColumns, pushedFilters)) + val scan = createPhysicalRDD(relation.relation, requestedColumns, + scanBuilder(requestedColumns, pushedFilters)) execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) } } + private[this] def createPhysicalRDD( + relation: BaseRelation, + output: Seq[Attribute], + rdd: RDD[Row]): SparkPlan = { + val converted = if (relation.needConversion) { + execution.RDDConversions.rowToRowRdd(rdd, relation.schema) + } else { + rdd + } + execution.PhysicalRDD(output, converted) + } + /** * Selects Catalyst predicate [[Expression]]s which are convertible into data source [[Filter]]s, * and convert them. @@ -167,14 +178,14 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.Not(child) => translate(child).map(sources.Not) - case expressions.StartsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringStartsWith(a.name, v)) + case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringStartsWith(a.name, v.toString)) - case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringEndsWith(a.name, v)) + case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringEndsWith(a.name, v.toString)) - case expressions.Contains(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringContains(a.name, v)) + case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringContains(a.name, v.toString)) case _ => None } 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 8f9946a5a801e..ca53dcdb92c52 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 @@ -126,6 +126,16 @@ abstract class BaseRelation { * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). */ def sizeInBytes: Long = sqlContext.conf.defaultSizeInBytes + + /** + * Whether does it need to convert the objects in Row to internal representation, for example: + * java.lang.String -> UTF8String + * java.lang.Decimal -> Decimal + * + * Note: The internal representation is not stable across releases and thus data sources outside + * of Spark SQL should leave this as true. + */ + def needConversion: Boolean = true } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 36465cc2fa11a..bf6cf1321a056 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -30,7 +30,7 @@ class RowSuite extends FunSuite { test("create row") { val expected = new GenericMutableRow(4) expected.update(0, 2147483647) - expected.update(1, "this is a string") + expected.setString(1, "this is a string") expected.update(2, false) expected.update(3, null) val actual1 = Row(2147483647, "this is a string", false, null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0174aaee94246..4c48dca44498b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,18 +17,14 @@ package org.apache.spark.sql -import org.apache.spark.sql.execution.GeneratedAggregate -import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ -import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types._ - -import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} +import org.apache.spark.sql.types._ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 5f08834f73c6b..c86ef338fc644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -65,7 +65,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(STRING, UTF8String("hello"), 4 + "hello".getBytes("utf-8").length) checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) @@ -108,8 +108,8 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[StringType.type]( STRING, - (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") + (buffer: ByteBuffer, string: UTF8String) => { + val bytes = string.getBytes buffer.putInt(bytes.length) buffer.put(bytes) }, @@ -117,7 +117,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes) - new String(bytes, "utf-8") + UTF8String(bytes) }) testColumnType[BinaryType.type, Array[Byte]]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index b301818a008e7..f76314b9dab5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{Decimal, DataType, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int): GenericMutableRow = { @@ -48,7 +48,7 @@ object ColumnarTestUtils { case FLOAT => Random.nextFloat() case DOUBLE => Random.nextDouble() case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) - case STRING => Random.nextString(Random.nextInt(32)) + case STRING => UTF8String(Random.nextString(Random.nextInt(32))) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) case DATE => Random.nextInt() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 60c8c00bda4d5..3b47b8adf313b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -74,7 +74,7 @@ case class AllDataTypesScan( i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -82,7 +82,7 @@ case class AllDataTypesScan( Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) } } } @@ -103,7 +103,7 @@ class TableScanSuite extends DataSourceTest { i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -111,7 +111,7 @@ class TableScanSuite extends DataSourceTest { Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) }.toSeq before { @@ -266,7 +266,7 @@ class TableScanSuite extends DataSourceTest { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(new Date((i + 2) * 8640000)))).toSeq) + (1 to 10).map(i => Row(Seq(new Date(1970, 1, i + 1)))).toSeq) test("Caching") { // Cached Query Execution diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 921c6194c7b76..74ae984f34866 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -34,7 +34,7 @@ import scala.collection.JavaConversions._ * 1. The Underlying data type in catalyst and in Hive * In catalyst: * Primitive => - * java.lang.String + * UTF8String * int / scala.Int * boolean / scala.Boolean * float / scala.Float @@ -239,9 +239,10 @@ private[hive] trait HiveInspectors { */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null - case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString + case poi: WritableConstantStringObjectInspector => + UTF8String(poi.getWritableConstantValue.toString) case poi: WritableConstantHiveVarcharObjectInspector => - poi.getWritableConstantValue.getHiveVarchar.getValue + UTF8String(poi.getWritableConstantValue.getHiveVarchar.getValue) case poi: WritableConstantHiveDecimalObjectInspector => HiveShim.toCatalystDecimal( PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, @@ -284,10 +285,13 @@ private[hive] trait HiveInspectors { case pi: PrimitiveObjectInspector => pi match { // We think HiveVarchar is also a String case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => - hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue - case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue + UTF8String(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) + case hvoi: HiveVarcharObjectInspector => + UTF8String(hvoi.getPrimitiveJavaObject(data).getValue) case x: StringObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).toString + UTF8String(x.getPrimitiveWritableObject(data).toString) + case x: StringObjectInspector => + UTF8String(x.getPrimitiveJavaObject(data)) case x: IntObjectInspector if x.preferWritable() => x.get(data) case x: BooleanObjectInspector if x.preferWritable() => x.get(data) case x: FloatObjectInspector if x.preferWritable() => x.get(data) @@ -340,7 +344,9 @@ private[hive] trait HiveInspectors { */ protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { case _: JavaHiveVarcharObjectInspector => - (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + (o: Any) => + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.size) case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) @@ -409,7 +415,7 @@ private[hive] trait HiveInspectors { case x: PrimitiveObjectInspector => x match { // TODO we don't support the HiveVarcharObjectInspector yet. case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a) - case _: StringObjectInspector => a.asInstanceOf[java.lang.String] + case _: StringObjectInspector => a.asInstanceOf[UTF8String].toString() case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a) case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 1ccb0c279c60e..a6f4fbe8aba06 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,24 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.catalyst.expressions.Row - import scala.collection.JavaConversions._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.expressions.{Row, _} import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.sources.DescribeCommand -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, CreateTableUsing} +import org.apache.spark.sql.sources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} import org.apache.spark.sql.types.StringType @@ -131,7 +128,7 @@ private[hive] trait HiveStrategies { val partitionValues = part.getValues var i = 0 while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) + inputData(i) = CatalystTypeConverters.convertToCatalyst(partitionValues(i)) i += 1 } pruningCondition(inputData) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8efed7f0299bf..cab0fdd35723a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader} -import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.io.{BufferedReader, DataInputStream, DataOutputStream, EOFException, InputStreamReader} import java.util.Properties import scala.collection.JavaConversions._ @@ -28,12 +27,13 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils /** @@ -121,14 +121,13 @@ case class ScriptTransformation( if (outputSerde == null) { val prevLine = curLine curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"))) .asInstanceOf[Array[Any]]) } else { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2)) .asInstanceOf[Array[Any]]) } } else { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 902a12785e3e9..a40a1e53117cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -22,11 +22,11 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Analyzes the given table in the current database to generate statistics, which will be @@ -76,6 +76,12 @@ case class DropTable( private[hive] case class AddJar(path: String) extends RunnableCommand { + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("result", IntegerType, false) :: Nil) + schema.toAttributes + } + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 0ed93c2c5b1fa..33e96eaabfbf6 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -41,7 +41,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.types.{UTF8String, Decimal, DecimalType} private[hive] case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { @@ -135,7 +135,7 @@ private[hive] object HiveShim { PrimitiveCategory.VOID, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 7577309900209..d331c210e8939 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -17,37 +17,35 @@ package org.apache.spark.sql.hive -import java.util -import java.util.{ArrayList => JArrayList} -import java.util.Properties import java.rmi.server.UID +import java.util.{Properties, ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions +import com.esotericsoftware.kryo.Kryo import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{NullWritable, Writable} -import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, PrimitiveObjectInspector, ObjectInspector} -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfo, TypeInfoFactory} +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} +import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging -import org.apache.spark.sql.types.{Decimal, DecimalType} - +import org.apache.spark.sql.types.{Decimal, DecimalType, UTF8String} /** * This class provides the UDF creation and also the UDF instance serialization and @@ -63,18 +61,14 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) // for Serialization def this() = this(null) - import java.io.{OutputStream, InputStream} - import com.esotericsoftware.kryo.Kryo import org.apache.spark.util.Utils._ - import org.apache.hadoop.hive.ql.exec.Utilities - import org.apache.hadoop.hive.ql.exec.UDF @transient private val methodDeSerialize = { val method = classOf[Utilities].getDeclaredMethod( "deserializeObjectByKryo", classOf[Kryo], - classOf[InputStream], + classOf[java.io.InputStream], classOf[Class[_]]) method.setAccessible(true) @@ -87,7 +81,7 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) "serializeObjectByKryo", classOf[Kryo], classOf[Object], - classOf[OutputStream]) + classOf[java.io.OutputStream]) method.setAccessible(true) method @@ -224,7 +218,7 @@ private[hive] object HiveShim { TypeInfoFactory.voidTypeInfo, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) From cf38fe04f8782ff4573ae106ec0de8e8d183cb2b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:15:58 -0700 Subject: [PATCH 777/817] [SPARK-6844][SQL] Clean up accumulators used in InMemoryRelation when it is uncached JIRA: https://issues.apache.org/jira/browse/SPARK-6844 Author: Liang-Chi Hsieh Closes #5475 from viirya/cache_memory_leak and squashes the following commits: 0b41235 [Liang-Chi Hsieh] fix style. dc1d5d5 [Liang-Chi Hsieh] For comments. 78af229 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cache_memory_leak 26c9bb6 [Liang-Chi Hsieh] Add configuration to enable in-memory table scan accumulators. 1c3b06e [Liang-Chi Hsieh] Clean up accumulators used in InMemoryRelation when it is uncached. --- .../org/apache/spark/sql/CacheManager.scala | 2 +- .../columnar/InMemoryColumnarTableScan.scala | 47 ++++++++++++++----- .../apache/spark/sql/CachedTableSuite.scala | 18 +++++++ .../columnar/PartitionBatchPruningSuite.scala | 2 + 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index ca4a127120b37..18584c2dcf797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -112,7 +112,7 @@ private[sql] class CacheManager(sqlContext: SQLContext) extends Logging { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") - cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.uncache(blocking) cachedData.remove(dataIndex) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 6eee0c86d6a1c..d9b6fb43ab83d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import org.apache.spark.Accumulator +import org.apache.spark.{Accumulable, Accumulator, Accumulators} import org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -53,11 +55,16 @@ private[sql] case class InMemoryRelation( child: SparkPlan, tableName: Option[String])( private var _cachedColumnBuffers: RDD[CachedBatch] = null, - private var _statistics: Statistics = null) + private var _statistics: Statistics = null, + private var _batchStats: Accumulable[ArrayBuffer[Row], Row] = null) extends LogicalPlan with MultiInstanceRelation { - private val batchStats = - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + private val batchStats: Accumulable[ArrayBuffer[Row], Row] = + if (_batchStats == null) { + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + } else { + _batchStats + } val partitionStatistics = new PartitionStatistics(output) @@ -161,7 +168,7 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated) + _cachedColumnBuffers, statisticsToBePropagated, batchStats) } override def children: Seq[LogicalPlan] = Seq.empty @@ -175,13 +182,20 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - statisticsToBePropagated).asInstanceOf[this.type] + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated) + Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) + + private[sql] def uncache(blocking: Boolean): Unit = { + Accumulators.remove(batchStats.id) + cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } } private[sql] case class InMemoryColumnarTableScan( @@ -244,15 +258,20 @@ private[sql] case class InMemoryColumnarTableScan( } } + lazy val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + // Accumulators used for testing purposes - val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) - val readBatches: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0) private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning override def execute(): RDD[Row] = { - readPartitions.setValue(0) - readBatches.setValue(0) + if (enableAccumulators) { + readPartitions.setValue(0) + readBatches.setValue(0) + } relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( @@ -302,7 +321,7 @@ private[sql] case class InMemoryColumnarTableScan( } } - if (rows.hasNext) { + if (rows.hasNext && enableAccumulators) { readPartitions += 1 } @@ -321,7 +340,9 @@ private[sql] case class InMemoryColumnarTableScan( logInfo(s"Skipping partition based on stats $statsString") false } else { - readBatches += 1 + if (enableAccumulators) { + readBatches += 1 + } true } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index f7b5f08beb92f..01e3b8671071e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,6 +22,7 @@ import scala.language.{implicitConversions, postfixOps} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.Accumulators import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ @@ -297,4 +298,21 @@ class CachedTableSuite extends QueryTest { sql("Clear CACHE") assert(cacheManager.isEmpty) } + + test("Clear accumulators when uncacheTable to prevent memory leaking") { + val accsSize = Accumulators.originals.size + + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + cacheTable("t1") + cacheTable("t2") + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + uncacheTable("t1") + uncacheTable("t2") + + assert(accsSize >= Accumulators.originals.size) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index e57bb06e7263b..2a0b701cad7fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -39,6 +39,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + // Enable in-memory table scan accumulators + setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") } override protected def afterAll(): Unit = { From 557a797a273f1668065806cba53e19e6134a66d3 Mon Sep 17 00:00:00 2001 From: sboeschhuawei Date: Wed, 15 Apr 2015 13:28:10 -0700 Subject: [PATCH 778/817] [SPARK-6937][MLLIB] Fixed bug in PICExample in which the radius were not being accepted on c... Tiny bug in PowerIterationClusteringExample in which radius not accepted from command line Author: sboeschhuawei Closes #5531 from javadba/picsub and squashes the following commits: 2aab8cf [sboeschhuawei] Fixed bug in PICExample in which the radius were not being accepted on command line --- .../examples/mllib/PowerIterationClusteringExample.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 9f22d40c15f3f..6d8b806569dfd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -65,7 +65,7 @@ object PowerIterationClusteringExample { def main(args: Array[String]) { val defaultParams = Params() - val parser = new OptionParser[Params]("PIC Circles") { + val parser = new OptionParser[Params]("PowerIterationClusteringExample") { head("PowerIterationClusteringExample: an example PIC app using concentric circles.") opt[Int]('k', "k") .text(s"number of circles (/clusters), default: ${defaultParams.k}") @@ -76,9 +76,9 @@ object PowerIterationClusteringExample { opt[Int]("maxIterations") .text(s"number of iterations, default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Int]('r', "r") + opt[Double]('r', "r") .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") - .action((x, c) => c.copy(numPoints = x)) + .action((x, c) => c.copy(outerRadius = x)) } parser.parse(args, defaultParams).map { params => @@ -154,3 +154,4 @@ object PowerIterationClusteringExample { coeff * math.exp(expCoeff * ssquares) } } + From 4754e16f4746ebd882b2ce7f1efc6e4d4408922c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Apr 2015 13:39:12 -0700 Subject: [PATCH 779/817] [SPARK-6898][SQL] completely support special chars in column names Even if we wrap column names in backticks like `` `a#$b.c` ``, we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly? Author: Wenchen Fan This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5511 from cloud-fan/6898 and squashes the following commits: 48e3e57 [Wenchen Fan] more style fix 820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute d81ad43 [Wenchen Fan] fix style 11699d6 [Wenchen Fan] completely support special chars in column names --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +-- .../sql/catalyst/analysis/Analyzer.scala | 13 ++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 ++++- .../sql/catalyst/analysis/unresolved.scala | 14 ++++++++-- .../catalyst/plans/logical/LogicalPlan.scala | 27 +++++++++---------- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 -- .../org/apache/spark/sql/DataFrame.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 13 ++++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- 9 files changed, 52 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 9a3531ceb3343..0af969cc5cc67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -381,13 +381,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | "(" ~> expression <~ ")" | function | dotExpressionHeader - | ident ^^ UnresolvedAttribute + | ident ^^ {case i => UnresolvedAttribute.quoted(i)} | signedPrimary | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { - case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) + case i1 ~ i2 ~ rest => UnresolvedAttribute(Seq(i1, i2) ++ rest) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8b68b0df35f48..cb49e5ad5586f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -297,14 +297,15 @@ class Analyzer( case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressionsUp { - case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) && + case u @ UnresolvedAttribute(nameParts) if nameParts.length == 1 && + resolver(nameParts(0), VirtualColumn.groupingIdName) && q.isInstanceOf[GroupingAnalytics] => // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics q.asInstanceOf[GroupingAnalytics].gid - case u @ UnresolvedAttribute(name) => + case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = - withPosition(u) { q.resolveChildren(name, resolver).getOrElse(u) } + withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => @@ -383,12 +384,12 @@ class Analyzer( child: LogicalPlan, grandchild: LogicalPlan): (Seq[SortOrder], Seq[Attribute]) = { // Find any attributes that remain unresolved in the sort. - val unresolved: Seq[String] = - ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + val unresolved: Seq[Seq[String]] = + ordering.flatMap(_.collect { case UnresolvedAttribute(nameParts) => nameParts }) // Create a map from name, to resolved attributes, when the desired name can be found // prior to the projection. - val resolved: Map[String, NamedExpression] = + val resolved: Map[Seq[String], NamedExpression] = unresolved.flatMap(u => grandchild.resolve(u, resolver).map(a => u -> a)).toMap // Construct a set that contains all of the attributes that we need to evaluate the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fa02111385c06..1155dac28fc78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -46,8 +46,12 @@ trait CheckAnalysis { operator transformExpressionsUp { case a: Attribute if !a.resolved => if (operator.childrenResolved) { + val nameParts = a match { + case UnresolvedAttribute(nameParts) => nameParts + case _ => Seq(a.name) + } // Throw errors for specific problems with get field. - operator.resolveChildren(a.name, resolver, throwErrors = true) + operator.resolveChildren(nameParts, resolver, throwErrors = true) } val from = operator.inputSet.map(_.name).mkString(", ") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 300e9ba187bc5..3f567e3e8b2a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -49,7 +49,12 @@ case class UnresolvedRelation( /** * Holds the name of an attribute that has yet to be resolved. */ -case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { +case class UnresolvedAttribute(nameParts: Seq[String]) + extends Attribute with trees.LeafNode[Expression] { + + def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") @@ -59,7 +64,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance(): UnresolvedAttribute = this override def withNullability(newNullability: Boolean): UnresolvedAttribute = this override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this - override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute(name) + override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -68,6 +73,11 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } +object UnresolvedAttribute { + def apply(name: String): UnresolvedAttribute = new UnresolvedAttribute(name.split("\\.")) + def quoted(name: String): UnresolvedAttribute = new UnresolvedAttribute(Seq(name)) +} + case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 579a0fb8d3f93..ae4620a4e5abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, Resolver} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, EliminateSubQueries, Resolver} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.types.{ArrayType, StructType, StructField} abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { @@ -111,10 +110,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolveChildren( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, children.flatMap(_.output), resolver, throwErrors) + resolve(nameParts, children.flatMap(_.output), resolver, throwErrors) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this @@ -122,10 +121,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * `[scope].AttributeName.[nested].[fields]...`. */ def resolve( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, output, resolver, throwErrors) + resolve(nameParts, output, resolver, throwErrors) /** * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. @@ -135,7 +134,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsTableColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { assert(nameParts.length > 1) @@ -155,7 +154,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { if (resolver(attribute.name, nameParts.head)) { @@ -167,13 +166,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** Performs attribute resolution given a name and a sequence of possible attributes. */ protected def resolve( - name: String, + nameParts: Seq[String], input: Seq[Attribute], resolver: Resolver, throwErrors: Boolean): Option[NamedExpression] = { - val parts = name.split("\\.") - // A sequence of possible candidate matches. // Each candidate is a tuple. The first element is a resolved attribute, followed by a list // of parts that are to be resolved. @@ -182,9 +179,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // and the second element will be List("c"). var candidates: Seq[(Attribute, List[String])] = { // If the name has 2 or more parts, try to resolve it as `table.column` first. - if (parts.length > 1) { + if (nameParts.length > 1) { input.flatMap { option => - resolveAsTableColumn(parts, resolver, option) + resolveAsTableColumn(nameParts, resolver, option) } } else { Seq.empty @@ -194,10 +191,12 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // If none of attributes match `table.column` pattern, we try to resolve it as a column. if (candidates.isEmpty) { candidates = input.flatMap { candidate => - resolveAsColumn(parts, resolver, candidate) + resolveAsColumn(nameParts, resolver, candidate) } } + def name = UnresolvedAttribute(nameParts).name + candidates.distinct match { // One match, no nested fields, use it. case Seq((a, Nil)) => Some(a) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 6e3d6b9263e86..e10ddfdf5127c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -27,8 +27,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import scala.collection.immutable - class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) 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 94ae2d65fd0e4..3235f85d5bbd2 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 @@ -158,7 +158,7 @@ class DataFrame private[sql]( } protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + queryExecution.analyzed.resolve(colName.split("\\."), sqlContext.analyzer.resolver).getOrElse { throw new AnalysisException( s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") } @@ -166,7 +166,7 @@ class DataFrame private[sql]( protected[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + queryExecution.analyzed.resolve(n.name.split("\\."), sqlContext.analyzer.resolver).get } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c48dca44498b..d739e550f3e56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,14 +19,13 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} import org.apache.spark.sql.types._ - class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. TestData @@ -1125,7 +1124,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") - sql("SELECT `key?number1` FROM records") + sql("SELECT `key?number1`, `key.number2` FROM records") } test("SPARK-3814 Support Bitwise & operator") { @@ -1225,4 +1224,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY c0.a"), Row(1)) } + + test("SPARK-6898: complete support for special chars in column names") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) + .registerTempTable("t") + + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 53a204b8c2932..fd305eb480e63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1101,7 +1101,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => - UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + UnresolvedAttribute(qualifierName :+ cleanIdentifier(attr)) case other => UnresolvedGetField(other, attr) } From 585638e81ce09a72b9e7f95d38e0d432cfa02456 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 14:06:10 -0700 Subject: [PATCH 780/817] [SPARK-2213] [SQL] sort merge join for spark sql Thanks for the initial work from Ishiihara in #3173 This PR introduce a new join method of sort merge join, which firstly ensure that keys of same value are in the same partition, and inside each partition the Rows are sorted by key. Then we can run down both sides together, find matched rows using [sort merge join](http://en.wikipedia.org/wiki/Sort-merge_join). In this way, we don't have to store the whole hash table of one side as hash join, thus we have less memory usage. Also, this PR would benefit from #3438 , making the sorting phrase much more efficient. We introduced a new configuration of "spark.sql.planner.sortMergeJoin" to switch between this(`true`) and ShuffledHashJoin(`false`), probably we want the default value of it be `false` at first. Author: Daoyuan Wang Author: Michael Armbrust This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5208 from adrian-wang/smj and squashes the following commits: 2493b9f [Daoyuan Wang] fix style 5049d88 [Daoyuan Wang] propagate rowOrdering for RangePartitioning f91a2ae [Daoyuan Wang] yin's comment: use external sort if option is enabled, add comments f515cd2 [Daoyuan Wang] yin's comment: outputOrdering, join suite refine ec8061b [Daoyuan Wang] minor change 413fd24 [Daoyuan Wang] Merge pull request #3 from marmbrus/pr/5208 952168a [Michael Armbrust] add type 5492884 [Michael Armbrust] copy when ordering 7ddd656 [Michael Armbrust] Cleanup addition of ordering requirements b198278 [Daoyuan Wang] inherit ordering in project c8e82a3 [Daoyuan Wang] fix style 6e897dd [Daoyuan Wang] hide boundReference from manually construct RowOrdering for key compare in smj 8681d73 [Daoyuan Wang] refactor Exchange and fix copy for sorting 2875ef2 [Daoyuan Wang] fix changed configuration 61d7f49 [Daoyuan Wang] add omitted comment 00a4430 [Daoyuan Wang] fix bug 078d69b [Daoyuan Wang] address comments: add comments, do sort in shuffle, and others 3af6ba5 [Daoyuan Wang] use buffer for only one side 171001f [Daoyuan Wang] change default outputordering 47455c9 [Daoyuan Wang] add apache license ... a28277f [Daoyuan Wang] fix style 645c70b [Daoyuan Wang] address comments using sort 068c35d [Daoyuan Wang] fix new style and add some tests 925203b [Daoyuan Wang] address comments 07ce92f [Daoyuan Wang] fix ArrayIndexOutOfBound 42fca0e [Daoyuan Wang] code clean e3ec096 [Daoyuan Wang] fix comment style.. 2edd235 [Daoyuan Wang] fix outputpartitioning 57baa40 [Daoyuan Wang] fix sort eval bug 303b6da [Daoyuan Wang] fix several errors 95db7ad [Daoyuan Wang] fix brackets for if-statement 4464f16 [Daoyuan Wang] fix error 880d8e9 [Daoyuan Wang] sort merge join for spark sql --- .../spark/sql/catalyst/expressions/rows.scala | 10 +- .../plans/physical/partitioning.scala | 13 ++ .../scala/org/apache/spark/sql/SQLConf.scala | 8 + .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 148 ++++++++++++--- .../spark/sql/execution/SparkPlan.scala | 6 + .../spark/sql/execution/SparkStrategies.scala | 11 +- .../spark/sql/execution/basicOperators.scala | 10 ++ .../sql/execution/joins/SortMergeJoin.scala | 169 ++++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 28 ++- .../SortMergeCompatibilitySuite.scala | 162 +++++++++++++++++ 11 files changed, 534 insertions(+), 33 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala create mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 1b62e17ff47fd..b6ec7d3417ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} - +import org.apache.spark.sql.types.{UTF8String, DataType, StructType, NativeType} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -239,3 +238,10 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return 0 } } + +object RowOrdering { + def forSchema(dataTypes: Seq[DataType]): RowOrdering = + new RowOrdering(dataTypes.zipWithIndex.map { + case(dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + }) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 288c11f69fe22..fb4217a44807b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -94,6 +94,9 @@ sealed trait Partitioning { * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean + + /** Returns the expressions that are used to key the partitioning. */ + def keyExpressions: Seq[Expression] } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { @@ -106,6 +109,8 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case UnknownPartitioning(_) => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object SinglePartition extends Partitioning { @@ -117,6 +122,8 @@ case object SinglePartition extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object BroadcastPartitioning extends Partitioning { @@ -128,6 +135,8 @@ case object BroadcastPartitioning extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } /** @@ -158,6 +167,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = expressions + override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } @@ -200,6 +211,8 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = ordering.map(_.child) + override def eval(input: Row): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index ee641bdfeb2d7..5c65f04ee8497 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -47,6 +47,7 @@ private[spark] object SQLConf { // Options that control which operators can be chosen by the query planner. These should be // considered hints and may be ignored by future versions of Spark SQL. val EXTERNAL_SORT = "spark.sql.planner.externalSort" + val SORTMERGE_JOIN = "spark.sql.planner.sortMergeJoin" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -128,6 +129,13 @@ private[sql] class SQLConf extends Serializable { /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean + /** + * Sort merge join would sort the two side of join first, and then iterate both sides together + * only once to get all matches. Using sort merge join can save a lot of memory usage compared + * to HashJoin. + */ + private[spark] def sortMergeJoinEnabled: Boolean = getConf(SORTMERGE_JOIN, "false").toBoolean + /** * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 89a4faf35e0d2..f9f3eb2e03817 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1081,7 +1081,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: Nil + Batch("Add exchange", Once, EnsureRequirements(self)) :: Nil } protected[sql] def openSession(): SQLSession = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 437408d30bfd2..518fc9e57c708 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -19,24 +19,42 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +object Exchange { + /** + * Returns true when the ordering expressions are a subset of the key. + * if true, ShuffledRDD can use `setKeyOrdering(orderingKey)` to sort within [[Exchange]]. + */ + def canSortWithShuffle(partitioning: Partitioning, desiredOrdering: Seq[SortOrder]): Boolean = { + desiredOrdering.map(_.child).toSet.subsetOf(partitioning.keyExpressions.toSet) + } +} + /** * :: DeveloperApi :: + * Performs a shuffle that will result in the desired `newPartitioning`. Optionally sorts each + * resulting partition based on expressions from the partition key. It is invalid to construct an + * exchange operator with a `newOrdering` that cannot be calculated using the partitioning key. */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { +case class Exchange( + newPartitioning: Partitioning, + newOrdering: Seq[SortOrder], + child: SparkPlan) + extends UnaryNode { override def outputPartitioning: Partitioning = newPartitioning + override def outputOrdering: Seq[SortOrder] = newOrdering + override def output: Seq[Attribute] = child.output /** We must copy rows when sort based shuffle is on */ @@ -45,6 +63,20 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una private val bypassMergeThreshold = child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + private val keyOrdering = { + if (newOrdering.nonEmpty) { + val key = newPartitioning.keyExpressions + val boundOrdering = newOrdering.map { o => + val ordinal = key.indexOf(o.child) + if (ordinal == -1) sys.error(s"Invalid ordering on $o requested for $newPartitioning") + o.copy(child = BoundReference(ordinal, o.child.dataType, o.child.nullable)) + } + new RowOrdering(boundOrdering) + } else { + null // Ordering will not be used + } + } + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -56,7 +88,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una // we can avoid the defensive copies to improve performance. In the long run, we probably // want to include information in shuffle dependencies to indicate whether elements in the // source RDD should be copied. - val rdd = if (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) { + val willMergeSort = sortBasedShuffleOn && numPartitions > bypassMergeThreshold + + val rdd = if (willMergeSort || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -69,12 +103,17 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una } } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Row, Row](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Row, Row](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - val rdd = if (sortBasedShuffleOn) { + val rdd = if (sortBasedShuffleOn || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} } else { child.execute().mapPartitions { iter => @@ -87,7 +126,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Null, Null](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Null, Null](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -120,27 +164,34 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] * of input data meets the * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[Exchange]] Operators where required. + * each operator by inserting [[Exchange]] Operators where required. Also ensure that the + * required input partition ordering requirements are met. */ -private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { +private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. def numPartitions: Int = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => - // Check if every child's outputPartitioning satisfies the corresponding + // True iff every child's outputPartitioning satisfies the corresponding // required data distribution. def meetsRequirements: Boolean = - !operator.requiredChildDistribution.zip(operator.children).map { + operator.requiredChildDistribution.zip(operator.children).forall { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid - }.exists(!_) + } - // Check if outputPartitionings of children are compatible with each other. + // True iff any of the children are incorrectly sorted. + def needsAnySort: Boolean = + operator.requiredChildOrdering.zip(operator.children).exists { + case (required, child) => required.nonEmpty && required != child.outputOrdering + } + + // True iff outputPartitionings of children are compatible with each other. // It is possible that every child satisfies its required data distribution // but two children have incompatible outputPartitionings. For example, // A dataset is range partitioned by "a.asc" (RangePartitioning) and another @@ -157,28 +208,69 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl case Seq(a,b) => a compatibleWith b }.exists(!_) - // Check if the partitioning we want to ensure is the same as the child's output - // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan): SparkPlan = - if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child + // Adds Exchange or Sort operators as required + def addOperatorsIfNecessary( + partitioning: Partitioning, + rowOrdering: Seq[SortOrder], + child: SparkPlan): SparkPlan = { + val needSort = rowOrdering.nonEmpty && child.outputOrdering != rowOrdering + val needsShuffle = child.outputPartitioning != partitioning + val canSortWithShuffle = Exchange.canSortWithShuffle(partitioning, rowOrdering) + + if (needSort && needsShuffle && canSortWithShuffle) { + Exchange(partitioning, rowOrdering, child) + } else { + val withShuffle = if (needsShuffle) { + Exchange(partitioning, Nil, child) + } else { + child + } - if (meetsRequirements && compatible) { + val withSort = if (needSort) { + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, withShuffle) + } else { + Sort(rowOrdering, global = false, withShuffle) + } + } else { + withShuffle + } + + withSort + } + } + + if (meetsRequirements && compatible && !needsAnySort) { operator } else { // At least one child does not satisfies its required data distribution or // at least one child's outputPartitioning is not compatible with another child's // outputPartitioning. In this case, we need to add Exchange operators. - val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { - case (AllTuples, child) => - addExchangeIfNecessary(SinglePartition, child) - case (ClusteredDistribution(clustering), child) => - addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) - case (OrderedDistribution(ordering), child) => - addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) - case (UnspecifiedDistribution, child) => child - case (dist, _) => sys.error(s"Don't know how to ensure $dist") + val requirements = + (operator.requiredChildDistribution, operator.requiredChildOrdering, operator.children) + + val fixedChildren = requirements.zipped.map { + case (AllTuples, rowOrdering, child) => + addOperatorsIfNecessary(SinglePartition, rowOrdering, child) + case (ClusteredDistribution(clustering), rowOrdering, child) => + addOperatorsIfNecessary(HashPartitioning(clustering, numPartitions), rowOrdering, child) + case (OrderedDistribution(ordering), rowOrdering, child) => + addOperatorsIfNecessary(RangePartitioning(ordering, numPartitions), rowOrdering, child) + + case (UnspecifiedDistribution, Seq(), child) => + child + case (UnspecifiedDistribution, rowOrdering, child) => + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, child) + } else { + Sort(rowOrdering, global = false, child) + } + + case (dist, ordering, _) => + sys.error(s"Don't know how to ensure $dist with ordering $ordering") } - operator.withNewChildren(repartitionedChildren) + + operator.withNewChildren(fixedChildren) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index fabcf6b4a0570..e159ffe66cb24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -72,6 +72,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) + /** Specifies how data is ordered in each partition. */ + def outputOrdering: Seq[SortOrder] = Nil + + /** Specifies sort order for each partition requirements on the input data for this operator. */ + def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) + /** * Runs this query returning the result as an RDD. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5b99e40c2f491..e687d01f57520 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -90,6 +90,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { left.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) + // If the sort merge join option is set, we want to use sort merge join prior to hashjoin + // for now let's support inner join first, then add outer join + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.conf.sortMergeJoinEnabled => + val mergeJoin = + joins.SortMergeJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, mergeJoin)).getOrElse(mergeJoin) :: Nil + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { @@ -309,7 +317,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => - execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + execution.Exchange( + HashPartitioning(expressions, numPartitions), Nil, planLater(child)) :: Nil case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f8221f41bc6c3..308dae236a5ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -41,6 +41,8 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends val resuableProjection = buildProjection() iter.map(resuableProjection) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -55,6 +57,8 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def execute(): RDD[Row] = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -147,6 +151,8 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. override def execute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1) + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -172,6 +178,8 @@ case class Sort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -202,6 +210,8 @@ case class ExternalSort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala new file mode 100644 index 0000000000000..b5123668ba11e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -0,0 +1,169 @@ +/* + * 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.execution.joins + +import java.util.NoSuchElementException + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * Performs an sort merge join of two child relations. + */ +@DeveloperApi +case class SortMergeJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def output: Seq[Attribute] = left.output ++ right.output + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + // this is to manually construct an ordering that can be used to compare keys from both sides + private val keyOrdering: RowOrdering = RowOrdering.forSchema(leftKeys.map(_.dataType)) + + override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys) + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil + + @transient protected lazy val leftKeyGenerator = newProjection(leftKeys, left.output) + @transient protected lazy val rightKeyGenerator = newProjection(rightKeys, right.output) + + private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = + keys.map(SortOrder(_, Ascending)) + + override def execute(): RDD[Row] = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.zipPartitions(rightResults) { (leftIter, rightIter) => + new Iterator[Row] { + // Mutable per row objects. + private[this] val joinRow = new JoinedRow5 + private[this] var leftElement: Row = _ + private[this] var rightElement: Row = _ + private[this] var leftKey: Row = _ + private[this] var rightKey: Row = _ + private[this] var rightMatches: CompactBuffer[Row] = _ + private[this] var rightPosition: Int = -1 + private[this] var stop: Boolean = false + private[this] var matchKey: Row = _ + + // initialize iterator + initialize() + + override final def hasNext: Boolean = nextMatchingPair() + + override final def next(): Row = { + if (hasNext) { + // we are using the buffered right rows and run down left iterator + val joinedRow = joinRow(leftElement, rightMatches(rightPosition)) + rightPosition += 1 + if (rightPosition >= rightMatches.size) { + rightPosition = 0 + fetchLeft() + if (leftElement == null || keyOrdering.compare(leftKey, matchKey) != 0) { + stop = false + rightMatches = null + } + } + joinedRow + } else { + // no more result + throw new NoSuchElementException + } + } + + private def fetchLeft() = { + if (leftIter.hasNext) { + leftElement = leftIter.next() + leftKey = leftKeyGenerator(leftElement) + } else { + leftElement = null + } + } + + private def fetchRight() = { + if (rightIter.hasNext) { + rightElement = rightIter.next() + rightKey = rightKeyGenerator(rightElement) + } else { + rightElement = null + } + } + + private def initialize() = { + fetchLeft() + fetchRight() + } + + /** + * Searches the right iterator for the next rows that have matches in left side, and store + * them in a buffer. + * + * @return true if the search is successful, and false if the right iterator runs out of + * tuples. + */ + private def nextMatchingPair(): Boolean = { + if (!stop && rightElement != null) { + // run both side to get the first match pair + while (!stop && leftElement != null && rightElement != null) { + val comparing = keyOrdering.compare(leftKey, rightKey) + // for inner join, we need to filter those null keys + stop = comparing == 0 && !leftKey.anyNull + if (comparing > 0 || rightKey.anyNull) { + fetchRight() + } else if (comparing < 0 || leftKey.anyNull) { + fetchLeft() + } + } + rightMatches = new CompactBuffer[Row]() + if (stop) { + stop = false + // iterate the right side to buffer all rows that matches + // as the records should be ordered, exit when we meet the first that not match + while (!stop && rightElement != null) { + rightMatches += rightElement + fetchRight() + stop = keyOrdering.compare(leftKey, rightKey) != 0 + } + if (rightMatches.size > 0) { + rightPosition = 0 + matchKey = leftKey + } + } + } + rightMatches != null && rightMatches.size > 0 + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index e4dee87849fd4..037d392c1f929 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -51,6 +51,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j case j: BroadcastLeftSemiJoinHash => j + case j: SortMergeJoin => j } assert(operators.size === 1) @@ -62,6 +63,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("join operator selection") { cacheManager.clearCache() + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), @@ -91,17 +93,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", classOf[BroadcastNestedLoopJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } } test("broadcasted hash join operator selection") { cacheManager.clearCache() sql("CACHE TABLE testData") + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } sql("UNCACHE TABLE testData") } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala new file mode 100644 index 0000000000000..65d070bd3cbde --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala @@ -0,0 +1,162 @@ +/* + * 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.execution + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.test.TestHive + +/** + * Runs the test cases that are included in the hive distribution with sort merge join is true. + */ +class SortMergeCompatibilitySuite extends HiveCompatibilitySuite { + override def beforeAll() { + super.beforeAll() + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "true") + } + + override def afterAll() { + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "false") + super.afterAll() + } + + override def whiteList = Seq( + "auto_join0", + "auto_join1", + "auto_join10", + "auto_join11", + "auto_join12", + "auto_join13", + "auto_join14", + "auto_join14_hadoop20", + "auto_join15", + "auto_join17", + "auto_join18", + "auto_join19", + "auto_join2", + "auto_join20", + "auto_join21", + "auto_join22", + "auto_join23", + "auto_join24", + "auto_join25", + "auto_join26", + "auto_join27", + "auto_join28", + "auto_join3", + "auto_join30", + "auto_join31", + "auto_join32", + "auto_join4", + "auto_join5", + "auto_join6", + "auto_join7", + "auto_join8", + "auto_join9", + "auto_join_filters", + "auto_join_nulls", + "auto_join_reordering_values", + "auto_smb_mapjoin_14", + "auto_sortmerge_join_1", + "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", + "auto_sortmerge_join_15", + "auto_sortmerge_join_16", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", + "auto_sortmerge_join_9", + "correlationoptimizer1", + "correlationoptimizer10", + "correlationoptimizer11", + "correlationoptimizer13", + "correlationoptimizer14", + "correlationoptimizer15", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", + "correlationoptimizer8", + "correlationoptimizer9", + "join0", + "join1", + "join10", + "join11", + "join12", + "join13", + "join14", + "join14_hadoop20", + "join15", + "join16", + "join17", + "join18", + "join19", + "join2", + "join20", + "join21", + "join22", + "join23", + "join24", + "join25", + "join26", + "join27", + "join28", + "join29", + "join3", + "join30", + "join31", + "join32", + "join32_lessSize", + "join33", + "join34", + "join35", + "join36", + "join37", + "join38", + "join39", + "join4", + "join40", + "join41", + "join5", + "join6", + "join7", + "join8", + "join9", + "join_1to1", + "join_array", + "join_casesensitive", + "join_empty", + "join_filters", + "join_hive_626", + "join_map_ppr", + "join_nulls", + "join_nullsafe", + "join_rc", + "join_reorder2", + "join_reorder3", + "join_reorder4", + "join_star" + ) +} From d5f1b9650b6e46cf6a9d61f01cda0df0cda5b1c9 Mon Sep 17 00:00:00 2001 From: Isaias Barroso Date: Wed, 15 Apr 2015 22:40:52 +0100 Subject: [PATCH 781/817] [SPARK-2312] Logging Unhandled messages The previous solution has changed based on https://github.com/apache/spark/pull/2048 discussions. Author: Isaias Barroso Closes #2055 from isaias/SPARK-2312 and squashes the following commits: f61d9e6 [Isaias Barroso] Change Log level for unhandled message to debug f341777 [Isaias Barroso] [SPARK-2312] Logging Unhandled messages --- .../scala/org/apache/spark/util/ActorLogReceive.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala index 332d0cbb2dc0c..81a7cbde01ce5 100644 --- a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala +++ b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala @@ -43,7 +43,13 @@ private[spark] trait ActorLogReceive { private val _receiveWithLogging = receiveWithLogging - override def isDefinedAt(o: Any): Boolean = _receiveWithLogging.isDefinedAt(o) + override def isDefinedAt(o: Any): Boolean = { + val handled = _receiveWithLogging.isDefinedAt(o) + if (!handled) { + log.debug(s"Received unexpected actor system event: $o") + } + handled + } override def apply(o: Any): Unit = { if (log.isDebugEnabled) { From 8a53de16fc8208358b76d0f3d45538e0304bcc8e Mon Sep 17 00:00:00 2001 From: Max Seiden Date: Wed, 15 Apr 2015 16:15:11 -0700 Subject: [PATCH 782/817] [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators There were a few places where new SparkSqlSerializer instances were created with new, empty SparkConfs resulting in user specified registrators sometimes not getting initialized. The fix is to try and pull a conf from the SparkEnv, and construct a new conf (that loads defaults) if one cannot be found. The changes touched: 1) SparkSqlSerializer's resource pool (this appears to fix the issue in the comment) 2) execution.Exchange (for all of the partitioners) 3) execution.Limit (for the HashPartitioner) A few tests were added to ColumnTypeSuite, ensuring that a custom registrator and serde is initialized and used when in-memory columns are written. Author: Max Seiden This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5237 from mhseiden/sql_udt_kryo and squashes the following commits: 3175c2f [Max Seiden] [SPARK-5277][SQL] - address code review comments e5011fb [Max Seiden] [SPARK-5277][SQL] - SparkSqlSerializer does not register user specified KryoRegistrators --- .../apache/spark/sql/execution/Exchange.scala | 9 +-- .../sql/execution/SparkSqlSerializer.scala | 7 +-- .../spark/sql/execution/basicOperators.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 62 ++++++++++++++++++- 4 files changed, 68 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 518fc9e57c708..69a620e1ec929 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -78,6 +78,8 @@ case class Exchange( } override def execute(): RDD[Row] = attachTree(this , "execute") { + lazy val sparkConf = child.sqlContext.sparkContext.getConf + newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. @@ -109,7 +111,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Row, Row](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => @@ -132,8 +134,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Null, Null](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) - + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._1) case SinglePartition => @@ -151,7 +152,7 @@ case class Exchange( } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 914f387dec78f..eea15aff5dbcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -65,12 +65,9 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co private[execution] class KryoResourcePool(size: Int) extends ResourcePool[SerializerInstance](size) { - val ser: KryoSerializer = { + val ser: SparkSqlSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - new KryoSerializer(sparkConf) + new SparkSqlSerializer(sparkConf) } def newInstance(): SerializerInstance = ser.newInstance() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 308dae236a5ed..d286fe81bee5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -121,7 +121,7 @@ case class Limit(limit: Int, child: SparkPlan) } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(child.sqlContext.sparkContext.getConf)) shuffled.mapPartitions(_.take(limit).map(_._2)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index c86ef338fc644..b48bed1871c50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import java.sql.Timestamp +import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.spark.serializer.KryoRegistrator import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -73,7 +76,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(BINARY, binary, 4 + 4) val generic = Map(1 -> "a") - checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 8) } testNativeColumnType[BooleanType.type]( @@ -158,6 +161,41 @@ class ColumnTypeSuite extends FunSuite with Logging { } } + test("CUSTOM") { + val conf = new SparkConf() + conf.set("spark.kryo.registrator", "org.apache.spark.sql.columnar.Registrator") + val serializer = new SparkSqlSerializer(conf).newInstance() + + val buffer = ByteBuffer.allocate(512) + val obj = CustomClass(Int.MaxValue,Long.MaxValue) + val serializedObj = serializer.serialize(obj).array() + + GENERIC.append(serializer.serialize(obj).array(), buffer) + buffer.rewind() + + val length = buffer.getInt + assert(length === serializedObj.length) + assert(13 == length) // id (1) + int (4) + long (8) + + val genericSerializedObj = SparkSqlSerializer.serialize(obj) + assert(length != genericSerializedObj.length) + assert(length < genericSerializedObj.length) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + serializer.deserialize(ByteBuffer.wrap(bytes)) + } + + buffer.rewind() + buffer.putInt(serializedObj.length).put(serializedObj) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + buffer.rewind() + serializer.deserialize(ByteBuffer.wrap(GENERIC.extract(buffer))) + } + } + def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, @@ -229,3 +267,23 @@ class ColumnTypeSuite extends FunSuite with Logging { } } } + +private[columnar] final case class CustomClass(a: Int, b: Long) + +private[columnar] object CustomerSerializer extends Serializer[CustomClass] { + override def write(kryo: Kryo, output: Output, t: CustomClass) { + output.writeInt(t.a) + output.writeLong(t.b) + } + override def read(kryo: Kryo, input: Input, aClass: Class[CustomClass]): CustomClass = { + val a = input.readInt() + val b = input.readLong() + CustomClass(a,b) + } +} + +private[columnar] final class Registrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[CustomClass], CustomerSerializer) + } +} From 52c3439a8a107ce1fc10e4f0b59fd7881e851622 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Wed, 15 Apr 2015 21:52:25 -0700 Subject: [PATCH 783/817] SPARK-6938: All require statements now have an informative error message. This pr adds informative error messages to all require statements in the Vectors class that did not previously have them. This references [SPARK-6938](https://issues.apache.org/jira/browse/SPARK-6938). Author: Juliet Hougland Closes #5532 from jhlch/SPARK-6938 and squashes the following commits: ab321bb [Juliet Hougland] Remove braces from string interpolation when not required. 1221f94 [Juliet Hougland] All require statements now have an informative error message. --- .../org/apache/spark/mllib/linalg/Vectors.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 328dbe2ce11fa..4ef171f4f0419 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -227,7 +227,7 @@ object Vectors { * @param elements vector elements in (index, value) pairs. */ def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { - require(size > 0) + require(size > 0, "The size of the requested sparse vector must be greater than 0.") val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 @@ -235,7 +235,8 @@ object Vectors { require(prev < i, s"Found duplicate indices: $i.") prev = i } - require(prev < size) + require(prev < size, s"You may not write an element to index $prev because the declared " + + s"size of your vector is $size") new SparseVector(size, indices.toArray, values.toArray) } @@ -309,7 +310,8 @@ object Vectors { * @return norm in L^p^ space. */ def norm(vector: Vector, p: Double): Double = { - require(p >= 1.0) + require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + + s"You specified p=$p.") val values = vector match { case DenseVector(vs) => vs case SparseVector(n, ids, vs) => vs @@ -360,7 +362,8 @@ object Vectors { * @return squared distance between two Vectors. */ def sqdist(v1: Vector, v2: Vector): Double = { - require(v1.size == v2.size, "vector dimension mismatch") + require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + + s"=${v2.size}.") var squaredDistance = 0.0 (v1, v2) match { case (v1: SparseVector, v2: SparseVector) => @@ -518,7 +521,9 @@ class SparseVector( val indices: Array[Int], val values: Array[Double]) extends Vector { - require(indices.length == values.length) + require(indices.length == values.length, "Sparse vectors require that the dimension of the" + + s" indices match the dimension of the values. You provided ${indices.size} indices and " + + s" ${values.size} values.") override def toString: String = "(%s,%s,%s)".format(size, indices.mkString("[", ",", "]"), values.mkString("[", ",", "]")) From 57cd1e86d1d450f85fc9e296aff498a940452113 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 15 Apr 2015 23:49:42 -0700 Subject: [PATCH 784/817] [SPARK-6893][ML] default pipeline parameter handling in python Same as #5431 but for Python. jkbradley Author: Xiangrui Meng Closes #5534 from mengxr/SPARK-6893 and squashes the following commits: d3b519b [Xiangrui Meng] address comments ebaccc6 [Xiangrui Meng] style update fce244e [Xiangrui Meng] update explainParams with test 4d6b07a [Xiangrui Meng] add tests 5294500 [Xiangrui Meng] update default param handling in python --- .../org/apache/spark/ml/Identifiable.scala | 2 +- .../apache/spark/ml/param/TestParams.scala | 9 +- python/pyspark/ml/classification.py | 3 +- python/pyspark/ml/feature.py | 19 +-- python/pyspark/ml/param/__init__.py | 146 +++++++++++++++--- ...d_params.py => _shared_params_code_gen.py} | 42 ++--- python/pyspark/ml/param/shared.py | 106 ++++++------- python/pyspark/ml/pipeline.py | 6 +- python/pyspark/ml/tests.py | 52 ++++++- python/pyspark/ml/util.py | 4 +- python/pyspark/ml/wrapper.py | 2 +- 11 files changed, 270 insertions(+), 121 deletions(-) rename python/pyspark/ml/param/{_gen_shared_params.py => _shared_params_code_gen.py} (70%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala index a50090671ae48..a1d49095c24ac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala @@ -25,7 +25,7 @@ import java.util.UUID private[ml] trait Identifiable extends Serializable { /** - * A unique id for the object. The default implementation concatenates the class name, "-", and 8 + * A unique id for the object. The default implementation concatenates the class name, "_", and 8 * random hex chars. */ private[ml] val uid: String = diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 8f9ab687c05cb..641b64b42a5e7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml.param +import org.apache.spark.ml.param.shared.{HasInputCol, HasMaxIter} + /** A subclass of Params for testing. */ -class TestParams extends Params { +class TestParams extends Params with HasMaxIter with HasInputCol { - val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = getOrDefault(maxIter) - - val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = getOrDefault(inputCol) setDefault(maxIter -> 10) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 7f42de531f3b4..d7bc09fd77adb 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -59,6 +59,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxIter=100, regParam=0.1) """ super(LogisticRegression, self).__init__() + self._setDefault(maxIter=100, regParam=0.1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -71,7 +72,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre Sets params for logistic regression. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def _create_model(self, java_model): return LogisticRegressionModel(java_model) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1cfcd019dfb18..263fe2a5bcc41 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -52,22 +52,22 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): _java_class = "org.apache.spark.ml.feature.Tokenizer" @keyword_only - def __init__(self, inputCol="input", outputCol="output"): + def __init__(self, inputCol=None, outputCol=None): """ - __init__(self, inputCol="input", outputCol="output") + __init__(self, inputCol=None, outputCol=None) """ super(Tokenizer, self).__init__() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, inputCol="input", outputCol="output"): + def setParams(self, inputCol=None, outputCol=None): """ setParams(self, inputCol="input", outputCol="output") Sets params for this Tokenizer. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) @inherit_doc @@ -91,22 +91,23 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): _java_class = "org.apache.spark.ml.feature.HashingTF" @keyword_only - def __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) """ super(HashingTF, self).__init__() + self._setDefault(numFeatures=1 << 18) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) Sets params for this HashingTF. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) if __name__ == "__main__": diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index e3a53dd780c4c..5c62620562a84 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -25,23 +25,21 @@ class Param(object): """ - A param with self-contained documentation and optionally default value. + A param with self-contained documentation. """ - def __init__(self, parent, name, doc, defaultValue=None): - if not isinstance(parent, Identifiable): - raise ValueError("Parent must be identifiable but got type %s." % type(parent).__name__) + def __init__(self, parent, name, doc): + if not isinstance(parent, Params): + raise ValueError("Parent must be a Params but got type %s." % type(parent).__name__) self.parent = parent self.name = str(name) self.doc = str(doc) - self.defaultValue = defaultValue def __str__(self): - return str(self.parent) + "-" + self.name + return str(self.parent) + "__" + self.name def __repr__(self): - return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ - (self.parent, self.name, self.doc, self.defaultValue) + return "Param(parent=%r, name=%r, doc=%r)" % (self.parent, self.name, self.doc) class Params(Identifiable): @@ -52,26 +50,128 @@ class Params(Identifiable): __metaclass__ = ABCMeta - def __init__(self): - super(Params, self).__init__() - #: embedded param map - self.paramMap = {} + #: internal param map for user-supplied values param map + paramMap = {} + + #: internal param map for default values + defaultParamMap = {} @property def params(self): """ - Returns all params. The default implementation uses - :py:func:`dir` to get all attributes of type + Returns all params ordered by name. The default implementation + uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ return filter(lambda attr: isinstance(attr, Param), [getattr(self, x) for x in dir(self) if x != "params"]) - def _merge_params(self, params): - paramMap = self.paramMap.copy() - paramMap.update(params) + def _explain(self, param): + """ + Explains a single param and returns its name, doc, and optional + default value and user-supplied value in a string. + """ + param = self._resolveParam(param) + values = [] + if self.isDefined(param): + if param in self.defaultParamMap: + values.append("default: %s" % self.defaultParamMap[param]) + if param in self.paramMap: + values.append("current: %s" % self.paramMap[param]) + else: + values.append("undefined") + valueStr = "(" + ", ".join(values) + ")" + return "%s: %s %s" % (param.name, param.doc, valueStr) + + def explainParams(self): + """ + Returns the documentation of all params with their optionally + default values and user-supplied values. + """ + return "\n".join([self._explain(param) for param in self.params]) + + def getParam(self, paramName): + """ + Gets a param by its name. + """ + param = getattr(self, paramName) + if isinstance(param, Param): + return param + else: + raise ValueError("Cannot find param with name %s." % paramName) + + def isSet(self, param): + """ + Checks whether a param is explicitly set by user. + """ + param = self._resolveParam(param) + return param in self.paramMap + + def hasDefault(self, param): + """ + Checks whether a param has a default value. + """ + param = self._resolveParam(param) + return param in self.defaultParamMap + + def isDefined(self, param): + """ + Checks whether a param is explicitly set by user or has a default value. + """ + return self.isSet(param) or self.hasDefault(param) + + 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. + """ + if isinstance(param, Param): + if param in self.paramMap: + return self.paramMap[param] + else: + return self.defaultParamMap[param] + elif isinstance(param, str): + return self.getOrDefault(self.getParam(param)) + else: + raise KeyError("Cannot recognize %r as a param." % param) + + def extractParamMap(self, extraParamMap={}): + """ + Extracts the embedded default param values and user-supplied + values, and then merges them with extra values from input into + a flat param map, where the latter value is used if there exist + conflicts, i.e., with ordering: default param values < + user-supplied values < extraParamMap. + :param extraParamMap: extra param values + :return: merged param map + """ + paramMap = self.defaultParamMap.copy() + paramMap.update(self.paramMap) + paramMap.update(extraParamMap) return paramMap + def _shouldOwn(self, param): + """ + Validates that the input param belongs to this Params instance. + """ + if param.parent is not self: + raise ValueError("Param %r does not belong to %r." % (param, self)) + + def _resolveParam(self, param): + """ + Resolves a param and validates the ownership. + :param param: param name or the param instance, which must + belong to this Params instance + :return: resolved param instance + """ + if isinstance(param, Param): + self._shouldOwn(param) + return param + elif isinstance(param, str): + return self.getParam(param) + else: + raise ValueError("Cannot resolve %r as a param." % param) + @staticmethod def _dummy(): """ @@ -81,10 +181,18 @@ def _dummy(): dummy.uid = "undefined" return dummy - def _set_params(self, **kwargs): + def _set(self, **kwargs): """ - Sets params. + Sets user-supplied params. """ for param, value in kwargs.iteritems(): self.paramMap[getattr(self, param)] = value return self + + def _setDefault(self, **kwargs): + """ + Sets default params. + """ + for param, value in kwargs.iteritems(): + self.defaultParamMap[getattr(self, param)] = value + return self diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_shared_params_code_gen.py similarity index 70% rename from python/pyspark/ml/param/_gen_shared_params.py rename to python/pyspark/ml/param/_shared_params_code_gen.py index 5eb81106f116c..55f422497672f 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -32,29 +32,34 @@ # limitations under the License. #""" +# Code generator for shared params (shared.py). Run under this folder with: +# python _shared_params_code_gen.py > shared.py -def _gen_param_code(name, doc, defaultValue): + +def _gen_param_code(name, doc, defaultValueStr): """ Generates Python code for a shared param class. :param name: param name :param doc: param doc - :param defaultValue: string representation of the param + :param defaultValueStr: string representation of the default value :return: code string """ # TODO: How to correctly inherit instance attributes? template = '''class Has$Name(Params): """ - Params with $name. + Mixin for param $name: $doc. """ # a placeholder to make it appear in the generated doc - $name = Param(Params._dummy(), "$name", "$doc", $defaultValue) + $name = Param(Params._dummy(), "$name", "$doc") def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc", $defaultValue) + self.$name = Param(self, "$name", "$doc") + if $defaultValueStr is not None: + self._setDefault($name=$defaultValueStr) def set$Name(self, value): """ @@ -67,32 +72,29 @@ def get$Name(self): """ Gets the value of $name or its default value. """ - if self.$name in self.paramMap: - return self.paramMap[self.$name] - else: - return self.$name.defaultValue''' + return self.getOrDefault(self.$name)''' - upperCamelName = name[0].upper() + name[1:] + Name = name[0].upper() + name[1:] return template \ .replace("$name", name) \ - .replace("$Name", upperCamelName) \ + .replace("$Name", Name) \ .replace("$doc", doc) \ - .replace("$defaultValue", defaultValue) + .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": print header - print "\n# DO NOT MODIFY. The code is generated by _gen_shared_params.py.\n" + print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" print "from pyspark.ml.param import Param, Params\n\n" shared = [ - ("maxIter", "max number of iterations", "100"), - ("regParam", "regularization constant", "0.1"), + ("maxIter", "max number of iterations", None), + ("regParam", "regularization constant", None), ("featuresCol", "features column name", "'features'"), ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), - ("inputCol", "input column name", "'input'"), - ("outputCol", "output column name", "'output'"), - ("numFeatures", "number of features", "1 << 18")] + ("inputCol", "input column name", None), + ("outputCol", "output column name", None), + ("numFeatures", "number of features", None)] code = [] - for name, doc, defaultValue in shared: - code.append(_gen_param_code(name, doc, defaultValue)) + for name, doc, defaultValueStr in shared: + code.append(_gen_param_code(name, doc, defaultValueStr)) print "\n\n\n".join(code) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 586822f2de423..13b6749998ad0 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -15,23 +15,25 @@ # limitations under the License. # -# DO NOT MODIFY. The code is generated by _gen_shared_params.py. +# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py. from pyspark.ml.param import Param, Params class HasMaxIter(Params): """ - Params with maxIter. + Mixin for param maxIter: max number of iterations. """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations", 100) + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations") def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations - self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + self.maxIter = Param(self, "maxIter", "max number of iterations") + if None is not None: + self._setDefault(maxIter=None) def setMaxIter(self, value): """ @@ -44,24 +46,23 @@ def getMaxIter(self): """ Gets the value of maxIter or its default value. """ - if self.maxIter in self.paramMap: - return self.paramMap[self.maxIter] - else: - return self.maxIter.defaultValue + return self.getOrDefault(self.maxIter) class HasRegParam(Params): """ - Params with regParam. + Mixin for param regParam: regularization constant. """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization constant", 0.1) + regParam = Param(Params._dummy(), "regParam", "regularization constant") def __init__(self): super(HasRegParam, self).__init__() #: param for regularization constant - self.regParam = Param(self, "regParam", "regularization constant", 0.1) + self.regParam = Param(self, "regParam", "regularization constant") + if None is not None: + self._setDefault(regParam=None) def setRegParam(self, value): """ @@ -74,24 +75,23 @@ def getRegParam(self): """ Gets the value of regParam or its default value. """ - if self.regParam in self.paramMap: - return self.paramMap[self.regParam] - else: - return self.regParam.defaultValue + return self.getOrDefault(self.regParam) class HasFeaturesCol(Params): """ - Params with featuresCol. + Mixin for param featuresCol: features column name. """ # a placeholder to make it appear in the generated doc - featuresCol = Param(Params._dummy(), "featuresCol", "features column name", 'features') + featuresCol = Param(Params._dummy(), "featuresCol", "features column name") def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name - self.featuresCol = Param(self, "featuresCol", "features column name", 'features') + self.featuresCol = Param(self, "featuresCol", "features column name") + if 'features' is not None: + self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ @@ -104,24 +104,23 @@ def getFeaturesCol(self): """ Gets the value of featuresCol or its default value. """ - if self.featuresCol in self.paramMap: - return self.paramMap[self.featuresCol] - else: - return self.featuresCol.defaultValue + return self.getOrDefault(self.featuresCol) class HasLabelCol(Params): """ - Params with labelCol. + Mixin for param labelCol: label column name. """ # a placeholder to make it appear in the generated doc - labelCol = Param(Params._dummy(), "labelCol", "label column name", 'label') + labelCol = Param(Params._dummy(), "labelCol", "label column name") def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name - self.labelCol = Param(self, "labelCol", "label column name", 'label') + self.labelCol = Param(self, "labelCol", "label column name") + if 'label' is not None: + self._setDefault(labelCol='label') def setLabelCol(self, value): """ @@ -134,24 +133,23 @@ def getLabelCol(self): """ Gets the value of labelCol or its default value. """ - if self.labelCol in self.paramMap: - return self.paramMap[self.labelCol] - else: - return self.labelCol.defaultValue + return self.getOrDefault(self.labelCol) class HasPredictionCol(Params): """ - Params with predictionCol. + Mixin for param predictionCol: prediction column name. """ # a placeholder to make it appear in the generated doc - predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name", 'prediction') + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name") def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name - self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') + self.predictionCol = Param(self, "predictionCol", "prediction column name") + if 'prediction' is not None: + self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ @@ -164,24 +162,23 @@ def getPredictionCol(self): """ Gets the value of predictionCol or its default value. """ - if self.predictionCol in self.paramMap: - return self.paramMap[self.predictionCol] - else: - return self.predictionCol.defaultValue + return self.getOrDefault(self.predictionCol) class HasInputCol(Params): """ - Params with inputCol. + Mixin for param inputCol: input column name. """ # a placeholder to make it appear in the generated doc - inputCol = Param(Params._dummy(), "inputCol", "input column name", 'input') + inputCol = Param(Params._dummy(), "inputCol", "input column name") def __init__(self): super(HasInputCol, self).__init__() #: param for input column name - self.inputCol = Param(self, "inputCol", "input column name", 'input') + self.inputCol = Param(self, "inputCol", "input column name") + if None is not None: + self._setDefault(inputCol=None) def setInputCol(self, value): """ @@ -194,24 +191,23 @@ def getInputCol(self): """ Gets the value of inputCol or its default value. """ - if self.inputCol in self.paramMap: - return self.paramMap[self.inputCol] - else: - return self.inputCol.defaultValue + return self.getOrDefault(self.inputCol) class HasOutputCol(Params): """ - Params with outputCol. + Mixin for param outputCol: output column name. """ # a placeholder to make it appear in the generated doc - outputCol = Param(Params._dummy(), "outputCol", "output column name", 'output') + outputCol = Param(Params._dummy(), "outputCol", "output column name") def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name - self.outputCol = Param(self, "outputCol", "output column name", 'output') + self.outputCol = Param(self, "outputCol", "output column name") + if None is not None: + self._setDefault(outputCol=None) def setOutputCol(self, value): """ @@ -224,24 +220,23 @@ def getOutputCol(self): """ Gets the value of outputCol or its default value. """ - if self.outputCol in self.paramMap: - return self.paramMap[self.outputCol] - else: - return self.outputCol.defaultValue + return self.getOrDefault(self.outputCol) class HasNumFeatures(Params): """ - Params with numFeatures. + Mixin for param numFeatures: number of features. """ # a placeholder to make it appear in the generated doc - numFeatures = Param(Params._dummy(), "numFeatures", "number of features", 1 << 18) + numFeatures = Param(Params._dummy(), "numFeatures", "number of features") def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features - self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + self.numFeatures = Param(self, "numFeatures", "number of features") + if None is not None: + self._setDefault(numFeatures=None) def setNumFeatures(self, value): """ @@ -254,7 +249,4 @@ def getNumFeatures(self): """ Gets the value of numFeatures or its default value. """ - if self.numFeatures in self.paramMap: - return self.paramMap[self.numFeatures] - else: - return self.numFeatures.defaultValue + return self.getOrDefault(self.numFeatures) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 83880a5afcd1d..d94ecfff09f66 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -124,10 +124,10 @@ def setParams(self, stages=[]): Sets params for Pipeline. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def fit(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): @@ -164,7 +164,7 @@ def __init__(self, transformers): self.transformers = transformers def transform(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for t in self.transformers: dataset = t.transform(dataset, paramMap) return dataset diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index b627c2b4e930b..3a42bcf723894 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -33,6 +33,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasMaxIter, HasInputCol from pyspark.ml.pipeline import Transformer, Estimator, Pipeline @@ -46,7 +47,7 @@ class MockTransformer(Transformer): def __init__(self): super(MockTransformer, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None @@ -62,7 +63,7 @@ class MockEstimator(Estimator): def __init__(self): super(MockEstimator, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None self.model = None @@ -111,5 +112,52 @@ def test_pipeline(self): self.assertEqual(6, dataset.index) +class TestParams(HasMaxIter, HasInputCol): + """ + A subclass of Params mixed with HasMaxIter and HasInputCol. + """ + + def __init__(self): + super(TestParams, self).__init__() + self._setDefault(maxIter=10) + + +class ParamTests(PySparkTestCase): + + def test_param(self): + testParams = TestParams() + maxIter = testParams.maxIter + self.assertEqual(maxIter.name, "maxIter") + self.assertEqual(maxIter.doc, "max number of iterations") + self.assertTrue(maxIter.parent is testParams) + + def test_params(self): + testParams = TestParams() + maxIter = testParams.maxIter + inputCol = testParams.inputCol + + params = testParams.params + self.assertEqual(params, [inputCol, maxIter]) + + self.assertTrue(testParams.hasDefault(maxIter)) + self.assertFalse(testParams.isSet(maxIter)) + self.assertTrue(testParams.isDefined(maxIter)) + self.assertEqual(testParams.getMaxIter(), 10) + testParams.setMaxIter(100) + self.assertTrue(testParams.isSet(maxIter)) + self.assertEquals(testParams.getMaxIter(), 100) + + self.assertFalse(testParams.hasDefault(inputCol)) + self.assertFalse(testParams.isSet(inputCol)) + self.assertFalse(testParams.isDefined(inputCol)) + with self.assertRaises(KeyError): + testParams.getInputCol() + + self.assertEquals( + testParams.explainParams(), + "\n".join(["inputCol: input column name (undefined)", + "maxIter: max number of iterations (default: 10, current: 100)"])) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 6f7f39c40eb5a..d3cb100a9efa5 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -40,8 +40,8 @@ class Identifiable(object): def __init__(self): #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + #: concatenates the class name, "_", and 8 random hex chars. + self.uid = type(self).__name__ + "_" + uuid.uuid4().hex[:8] def __repr__(self): return self.uid diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 31a66b3d2f730..394f23c5e9b12 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -64,7 +64,7 @@ def _transfer_params_to_java(self, params, java_obj): :param params: additional params (overwriting embedded values) :param java_obj: Java object to receive the params """ - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for param in self.params: if param in paramMap: java_obj.set(param.name, paramMap[param]) From 8370550593f3549e90ace446961281dad0cd7498 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 16 Apr 2015 10:39:02 +0100 Subject: [PATCH 785/817] [Streaming][minor] Remove additional quote and unneeded imports Author: jerryshao Closes #5540 from jerryshao/minor-fix and squashes the following commits: ebaa646 [jerryshao] Minor fix --- .../apache/spark/examples/streaming/DirectKafkaWordCount.scala | 2 +- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index 1c8a20bf8f1ae..11a8cf09533ce 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -41,7 +41,7 @@ object DirectKafkaWordCount { | is a list of one or more Kafka brokers | is a list of one or more kafka topics to consume from | - """".stripMargin) + """.stripMargin) System.exit(1) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index a0b8a0c565210..a1b4a12e5d6a0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -23,10 +23,9 @@ import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskC import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator -import java.util.Properties import kafka.api.{FetchRequestBuilder, FetchResponse} import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import kafka.consumer.SimpleConsumer import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties From 6179a948371897cecb7322ebda366c2de8ecaedd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 16 Apr 2015 10:45:32 +0100 Subject: [PATCH 786/817] SPARK-4783 [CORE] System.exit() calls in SparkContext disrupt applications embedding Spark Avoid `System.exit(1)` in `TaskSchedulerImpl` and convert to `SparkException`; ensure scheduler calls `sc.stop()` even when this exception is thrown. CC mateiz aarondav as those who may have last touched this code. Author: Sean Owen Closes #5492 from srowen/SPARK-4783 and squashes the following commits: 60dc682 [Sean Owen] Avoid System.exit(1) in TaskSchedulerImpl and convert to SparkException; ensure scheduler calls sc.stop() even when this exception is thrown --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 5 ++--- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 9 ++++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2362cc7240039..ecc8bf189986d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -394,7 +394,7 @@ private[spark] class TaskSchedulerImpl( def error(message: String) { synchronized { - if (activeTaskSets.size > 0) { + if (activeTaskSets.nonEmpty) { // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { @@ -407,8 +407,7 @@ private[spark] class TaskSchedulerImpl( // No task sets are active but we still got an error. Just exit since this // must mean the error is during registration. // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) + throw new SparkException(s"Exiting due to error from cluster scheduler: $message") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ed5b7c1088196..ccf1dc5af6120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -118,9 +118,12 @@ private[spark] class SparkDeploySchedulerBackend( notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) - scheduler.error(reason) - // Ensure the application terminates, as we can no longer run jobs. - sc.stop() + try { + scheduler.error(reason) + } finally { + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() + } } } From de4fa6b6d12e2bee0307ffba2abfca0c33f15e45 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 16 Apr 2015 10:48:31 +0100 Subject: [PATCH 787/817] [SPARK-4194] [core] Make SparkContext initialization exception-safe. SparkContext has a very long constructor, where multiple things are initialized, multiple threads are spawned, and multiple opportunities for exceptions to be thrown exist. If one of these happens at an innoportune time, lots of garbage tends to stick around. This patch re-organizes SparkContext so that its internal state is initialized in a big "try" block. The fields keeping state are now completely private to SparkContext, and are "vars", because Scala doesn't allow you to initialize a val later. The existing API interface is kept by turning vals into defs (which works because Scala guarantees the same binary interface for those). On top of that, a few things in other areas were changed to avoid more things leaking: - Executor was changed to explicitly wait for the heartbeat thread to stop. LocalBackend was changed to wait for the "StopExecutor" message to be received, since otherwise there could be a race between that message arriving and the actor system being shut down. - ConnectionManager could possibly hang during shutdown, because an interrupt at the wrong moment could cause the selector thread to still call select and then wait forever. So also wake up the selector so that this situation is avoided. Author: Marcelo Vanzin Closes #5335 from vanzin/SPARK-4194 and squashes the following commits: 746b661 [Marcelo Vanzin] Fix borked merge. 80fc00e [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 408dada [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 2621609 [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 6b73fcb [Marcelo Vanzin] Scalastyle. c671c46 [Marcelo Vanzin] Fix merge. 3979aad [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 8caa8b3 [Marcelo Vanzin] [SPARK-4194] [core] Make SparkContext initialization exception-safe. 071f16e [Marcelo Vanzin] Nits. 27456b9 [Marcelo Vanzin] More exception safety. a0b0881 [Marcelo Vanzin] Stop alloc manager before scheduler. 5545d83 [Marcelo Vanzin] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops. --- .../scala/org/apache/spark/SparkContext.scala | 505 ++++++++++-------- .../org/apache/spark/executor/Executor.scala | 33 +- .../spark/network/nio/ConnectionManager.scala | 7 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/scheduler/local/LocalBackend.scala | 19 +- .../ExecutorAllocationManagerSuite.scala | 6 - 6 files changed, 329 insertions(+), 244 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f1a7dd99d635..e106c5c4bef60 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -31,6 +31,7 @@ import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -50,9 +51,10 @@ import org.apache.spark.executor.{ExecutorEndpoint, TriggerThreadDump} import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.io.CompressionCodec +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ -import org.apache.spark.rpc.RpcAddress +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} @@ -192,8 +194,42 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - private[spark] val conf = config.clone() - conf.validateSettings() + /* ------------------------------------------------------------------------------------- * + | Private variables. These variables keep the internal state of the context, and are | + | not accessible by the outside world. They're mutable since we want to initialize all | + | of them to some neutral value ahead of time, so that calling "stop()" while the | + | constructor is still running is safe. | + * ------------------------------------------------------------------------------------- */ + + private var _conf: SparkConf = _ + private var _eventLogDir: Option[URI] = None + private var _eventLogCodec: Option[String] = None + private var _env: SparkEnv = _ + private var _metadataCleaner: MetadataCleaner = _ + private var _jobProgressListener: JobProgressListener = _ + private var _statusTracker: SparkStatusTracker = _ + private var _progressBar: Option[ConsoleProgressBar] = None + private var _ui: Option[SparkUI] = None + private var _hadoopConfiguration: Configuration = _ + private var _executorMemory: Int = _ + private var _schedulerBackend: SchedulerBackend = _ + private var _taskScheduler: TaskScheduler = _ + private var _heartbeatReceiver: RpcEndpointRef = _ + @volatile private var _dagScheduler: DAGScheduler = _ + private var _applicationId: String = _ + private var _eventLogger: Option[EventLoggingListener] = None + private var _executorAllocationManager: Option[ExecutorAllocationManager] = None + private var _cleaner: Option[ContextCleaner] = None + private var _listenerBusStarted: Boolean = false + private var _jars: Seq[String] = _ + private var _files: Seq[String] = _ + + /* ------------------------------------------------------------------------------------- * + | Accessors and public fields. These provide access to the internal state of the | + | context. | + * ------------------------------------------------------------------------------------- */ + + private[spark] def conf: SparkConf = _conf /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -201,65 +237,24 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def getConf: SparkConf = conf.clone() - if (!conf.contains("spark.master")) { - throw new SparkException("A master URL must be set in your configuration") - } - if (!conf.contains("spark.app.name")) { - throw new SparkException("An application name must be set in your configuration") - } - - if (conf.getBoolean("spark.logConf", false)) { - logInfo("Spark configuration:\n" + conf.toDebugString) - } - - // Set Spark driver host and port system properties - conf.setIfMissing("spark.driver.host", Utils.localHostName()) - conf.setIfMissing("spark.driver.port", "0") - - val jars: Seq[String] = - conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val files: Seq[String] = - conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val master = conf.get("spark.master") - val appName = conf.get("spark.app.name") + def jars: Seq[String] = _jars + def files: Seq[String] = _files + def master: String = _conf.get("spark.master") + def appName: String = _conf.get("spark.app.name") - private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - private[spark] val eventLogDir: Option[URI] = { - if (isEventLogEnabled) { - val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) - .stripSuffix("/") - Some(Utils.resolveURI(unresolvedDir)) - } else { - None - } - } - private[spark] val eventLogCodec: Option[String] = { - val compress = conf.getBoolean("spark.eventLog.compress", false) - if (compress && isEventLogEnabled) { - Some(CompressionCodec.getCodecName(conf)).map(CompressionCodec.getShortName) - } else { - None - } - } + private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def eventLogDir: Option[URI] = _eventLogDir + private[spark] def eventLogCodec: Option[String] = _eventLogCodec // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() - conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val isLocal = (master == "local" || master.startsWith("local[")) - - if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + def isLocal: Boolean = (master == "local" || master.startsWith("local[")) // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - - // Create the Spark execution environment (cache, map output tracker, etc) - // This function allows components created by SparkEnv to be mocked in unit tests: private[spark] def createSparkEnv( conf: SparkConf, @@ -268,8 +263,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli SparkEnv.createDriverEnv(conf, isLocal, listenerBus) } - private[spark] val env = createSparkEnv(conf, isLocal, listenerBus) - SparkEnv.set(env) + private[spark] def env: SparkEnv = _env // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() @@ -277,35 +271,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - + private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner + private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener - private[spark] val jobProgressListener = new JobProgressListener(conf) - listenerBus.addListener(jobProgressListener) + def statusTracker: SparkStatusTracker = _statusTracker - val statusTracker = new SparkStatusTracker(this) + private[spark] def progressBar: Option[ConsoleProgressBar] = _progressBar - private[spark] val progressBar: Option[ConsoleProgressBar] = - if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { - Some(new ConsoleProgressBar(this)) - } else { - None - } - - // Initialize the Spark UI - private[spark] val ui: Option[SparkUI] = - if (conf.getBoolean("spark.ui.enabled", true)) { - Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, - env.securityManager,appName)) - } else { - // For tests, do not enable the UI - None - } - - // Bind the UI before starting the task scheduler to communicate - // the bound port to the cluster manager properly - ui.foreach(_.bind()) + private[spark] def ui: Option[SparkUI] = _ui /** * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. @@ -313,134 +286,248 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you * plan to set some global configurations for all Hadoop RDDs. */ - val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) + def hadoopConfiguration: Configuration = _hadoopConfiguration + + private[spark] def executorMemory: Int = _executorMemory + + // Environment variables to pass to our executors. + private[spark] val executorEnvs = HashMap[String, String]() + + // Set SPARK_USER for user who is running SparkContext. + val sparkUser = Utils.getCurrentUserName() - // Add each JAR given through the constructor - if (jars != null) { - jars.foreach(addJar) + private[spark] def schedulerBackend: SchedulerBackend = _schedulerBackend + private[spark] def schedulerBackend_=(sb: SchedulerBackend): Unit = { + _schedulerBackend = sb } - if (files != null) { - files.foreach(addFile) + private[spark] def taskScheduler: TaskScheduler = _taskScheduler + private[spark] def taskScheduler_=(ts: TaskScheduler): Unit = { + _taskScheduler = ts } + private[spark] def dagScheduler: DAGScheduler = _dagScheduler + private[spark] def dagScheduler_=(ds: DAGScheduler): Unit = { + _dagScheduler = ds + } + + def applicationId: String = _applicationId + + def metricsSystem: MetricsSystem = if (_env != null) _env.metricsSystem else null + + private[spark] def eventLogger: Option[EventLoggingListener] = _eventLogger + + private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = + _executorAllocationManager + + private[spark] def cleaner: Option[ContextCleaner] = _cleaner + + private[spark] var checkpointDir: Option[String] = None + + // Thread Local variable that can be used by users to pass information down the stack + private val localProperties = new InheritableThreadLocal[Properties] { + override protected def childValue(parent: Properties): Properties = new Properties(parent) + override protected def initialValue(): Properties = new Properties() + } + + /* ------------------------------------------------------------------------------------- * + | Initialization. This code initializes the context in a manner that is exception-safe. | + | All internal fields holding state are initialized here, and any error prompts the | + | stop() method to be called. | + * ------------------------------------------------------------------------------------- */ + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value } - private[spark] val executorMemory = conf.getOption("spark.executor.memory") - .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) - .orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem)) - .map(Utils.memoryStringToMb) - .getOrElse(512) + try { + _conf = config.clone() + _conf.validateSettings() - // Environment variables to pass to our executors. - private[spark] val executorEnvs = HashMap[String, String]() + if (!_conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!_conf.contains("spark.app.name")) { + throw new SparkException("An application name must be set in your configuration") + } - // Convert java options to env vars as a work around - // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) - value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value - } - Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => - executorEnvs("SPARK_PREPEND_CLASSES") = v - } - // The Mesos scheduler backend relies on this environment variable to set executor memory. - // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + if (_conf.getBoolean("spark.logConf", false)) { + logInfo("Spark configuration:\n" + _conf.toDebugString) + } - // Set SPARK_USER for user who is running SparkContext. - val sparkUser = Utils.getCurrentUserName() - executorEnvs("SPARK_USER") = sparkUser + // Set Spark driver host and port system properties + _conf.setIfMissing("spark.driver.host", Utils.localHostName()) + _conf.setIfMissing("spark.driver.port", "0") - // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will - // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) - private val heartbeatReceiver = env.rpcEnv.setupEndpoint( - HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - // Create and start the scheduler - private[spark] var (schedulerBackend, taskScheduler) = - SparkContext.createTaskScheduler(this, master) + _jars =_conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)) + .toSeq.flatten - heartbeatReceiver.send(TaskSchedulerIsSet) + _eventLogDir = + if (isEventLogEnabled) { + val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) + .stripSuffix("/") + Some(Utils.resolveURI(unresolvedDir)) + } else { + None + } - @volatile private[spark] var dagScheduler: DAGScheduler = _ - try { - dagScheduler = new DAGScheduler(this) - } catch { - case e: Exception => { - try { - stop() - } finally { - throw new SparkException("Error while constructing DAGScheduler", e) + _eventLogCodec = { + val compress = _conf.getBoolean("spark.eventLog.compress", false) + if (compress && isEventLogEnabled) { + Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) + } else { + None } } - } - // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's - // constructor - taskScheduler.start() + _conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val applicationId: String = taskScheduler.applicationId() - conf.set("spark.app.id", applicationId) + if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - env.blockManager.initialize(applicationId) + // Create the Spark execution environment (cache, map output tracker, etc) + _env = createSparkEnv(_conf, isLocal, listenerBus) + SparkEnv.set(_env) - val metricsSystem = env.metricsSystem + _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - // The metrics system for Driver need to be set spark.app.id to app ID. - // So it should start after we get app ID from the task scheduler and set spark.app.id. - metricsSystem.start() - // Attach the driver metrics servlet handler to the web ui after the metrics system is started. - metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (isEventLogEnabled) { - val logger = - new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } + _statusTracker = new SparkStatusTracker(this) - // Optionally scale number of executors dynamically based on workload. Exposed for testing. - private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) - private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) - private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (dynamicAllocationEnabled) { - assert(supportDynamicAllocation, - "Dynamic allocation of executors is currently only supported in YARN mode") - Some(new ExecutorAllocationManager(this, listenerBus, conf)) - } else { - None + _progressBar = + if (_conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { + Some(new ConsoleProgressBar(this)) + } else { + None + } + + _ui = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, + _env.securityManager,appName)) + } else { + // For tests, do not enable the UI + None + } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + _ui.foreach(_.bind()) + + _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) + + // Add each JAR given through the constructor + if (jars != null) { + jars.foreach(addJar) } - executorAllocationManager.foreach(_.start()) - private[spark] val cleaner: Option[ContextCleaner] = { - if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { - Some(new ContextCleaner(this)) - } else { - None + if (files != null) { + files.foreach(addFile) } - } - cleaner.foreach(_.start()) - setupAndStartListenerBus() - postEnvironmentUpdate() - postApplicationStart() + _executorMemory = _conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) + .orElse(Option(System.getenv("SPARK_MEM")) + .map(warnSparkMem)) + .map(Utils.memoryStringToMb) + .getOrElse(512) + + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + executorEnvs("SPARK_PREPEND_CLASSES") = v + } + // The Mesos scheduler backend relies on this environment variable to set executor memory. + // TODO: Set this only in the Mesos scheduler. + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= _conf.getExecutorEnv + executorEnvs("SPARK_USER") = sparkUser + + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will + // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) + _heartbeatReceiver = env.rpcEnv.setupEndpoint( + HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + + // Create and start the scheduler + val (sched, ts) = SparkContext.createTaskScheduler(this, master) + _schedulerBackend = sched + _taskScheduler = ts + _dagScheduler = new DAGScheduler(this) + _heartbeatReceiver.send(TaskSchedulerIsSet) + + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + _taskScheduler.start() + + _applicationId = _taskScheduler.applicationId() + _conf.set("spark.app.id", _applicationId) + _env.blockManager.initialize(_applicationId) + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. + metricsSystem.start() + // Attach the driver metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + + _eventLogger = + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(_applicationId, _eventLogDir.get, _conf, _hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else { + None + } - private[spark] var checkpointDir: Option[String] = None + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + val dynamicAllocationEnabled = _conf.getBoolean("spark.dynamicAllocation.enabled", false) + _executorAllocationManager = + if (dynamicAllocationEnabled) { + assert(supportDynamicAllocation, + "Dynamic allocation of executors is currently only supported in YARN mode") + Some(new ExecutorAllocationManager(this, listenerBus, _conf)) + } else { + None + } + _executorAllocationManager.foreach(_.start()) - // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { - override protected def childValue(parent: Properties): Properties = new Properties(parent) - override protected def initialValue(): Properties = new Properties() + _cleaner = + if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + + setupAndStartListenerBus() + postEnvironmentUpdate() + postApplicationStart() + + // Post init + _taskScheduler.postStartHook() + _env.metricsSystem.registerSource(new DAGSchedulerSource(dagScheduler)) + _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) + } catch { + case NonFatal(e) => + logError("Error initializing SparkContext.", e) + try { + stop() + } catch { + case NonFatal(inner) => + logError("Error stopping SparkContext after init error.", inner) + } finally { + throw e + } } /** @@ -544,19 +631,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } - // Post init - taskScheduler.postStartHook() - - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - - private def initDriverMetrics() { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) - } - - initDriverMetrics() - // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. @@ -1146,7 +1220,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * this application is supported. This is currently only available for YARN. */ private[spark] def supportDynamicAllocation = - master.contains("yarn") || dynamicAllocationTesting + master.contains("yarn") || _conf.getBoolean("spark.dynamicAllocation.testing", false) /** * :: DeveloperApi :: @@ -1163,7 +1237,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * This is currently only supported in YARN mode. Return whether the request is received. */ private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => @@ -1403,28 +1477,40 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def stop() { // Use the stopping variable to ensure no contention for the stop scenario. // Still track the stopped variable for use elsewhere in the code. - if (!stopped.compareAndSet(false, true)) { logInfo("SparkContext already stopped.") return } - + postApplicationEnd() - ui.foreach(_.stop()) - env.metricsSystem.report() - metadataCleaner.cancel() - cleaner.foreach(_.stop()) - executorAllocationManager.foreach(_.stop()) - dagScheduler.stop() - dagScheduler = null - listenerBus.stop() - eventLogger.foreach(_.stop()) - env.rpcEnv.stop(heartbeatReceiver) - progressBar.foreach(_.stop()) - taskScheduler = null + _ui.foreach(_.stop()) + if (env != null) { + env.metricsSystem.report() + } + if (metadataCleaner != null) { + metadataCleaner.cancel() + } + _cleaner.foreach(_.stop()) + _executorAllocationManager.foreach(_.stop()) + if (_dagScheduler != null) { + _dagScheduler.stop() + _dagScheduler = null + } + if (_listenerBusStarted) { + listenerBus.stop() + _listenerBusStarted = false + } + _eventLogger.foreach(_.stop()) + if (env != null && _heartbeatReceiver != null) { + env.rpcEnv.stop(_heartbeatReceiver) + } + _progressBar.foreach(_.stop()) + _taskScheduler = null // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) + if (_env != null) { + _env.stop() + SparkEnv.set(null) + } SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") } @@ -1749,6 +1835,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } listenerBus.start(this) + _listenerBusStarted = true } /** Post the application start event */ @@ -2152,7 +2239,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, 1) + val backend = new LocalBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2164,7 +2251,7 @@ object SparkContext extends Logging { throw new SparkException(s"Asked to run locally with $threadCount threads") } val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2174,7 +2261,7 @@ object SparkContext extends Logging { // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 516f619529c48..1b5fdeba28ee2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, Executors, TimeUnit} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -60,8 +60,6 @@ private[spark] class Executor( private val conf = env.conf - @volatile private var isStopped = false - // No ip or host:port - just hostname Utils.checkHost(executorHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -114,6 +112,10 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + // Executor for the heartbeat task. + private val heartbeater = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("driver-heartbeater")) + startDriverHeartbeater() def launchTask( @@ -138,7 +140,8 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() env.rpcEnv.stop(executorEndpoint) - isStopped = true + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() if (!isLocal) { env.stop() @@ -432,23 +435,17 @@ private[spark] class Executor( } /** - * Starts a thread to report heartbeat and partial metrics for active tasks to driver. - * This thread stops running when the executor is stopped. + * Schedules a task to report heartbeat and partial metrics for active tasks to driver. */ private def startDriverHeartbeater(): Unit = { val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") - val thread = new Thread() { - override def run() { - // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) - while (!isStopped) { - reportHeartBeat() - Thread.sleep(intervalMs) - } - } + + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) } - thread.setDaemon(true) - thread.setName("driver-heartbeater") - thread.start() + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 8e3c30fc3d781..5a74c13b38bf7 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -86,11 +86,11 @@ private[nio] class ConnectionManager( conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. - // + // // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, // we only query for the minimum value because we are using LinkedBlockingDeque. - // - // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" // parameter is necessary. private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) @@ -989,6 +989,7 @@ private[nio] class ConnectionManager( def stop() { ackTimeoutMonitor.stop() + selector.wakeup() selectorThread.interrupt() selectorThread.join() selector.close() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ecc8bf189986d..13a52d836f32f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -142,11 +142,10 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") - import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } - } + }(sc.env.actorSystem.dispatcher) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 70a477a6895cc..50ba0b9d5a612 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -20,12 +20,12 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer import java.util.concurrent.{Executors, TimeUnit} -import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.util.Utils private case class ReviveOffers() @@ -71,11 +71,15 @@ private[spark] class LocalEndpoint( case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case StopExecutor => executor.stop() + context.reply(true) } + def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) val tasks = scheduler.resourceOffers(offers).flatten @@ -104,8 +108,11 @@ private[spark] class LocalEndpoint( * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ -private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) - extends SchedulerBackend with ExecutorBackend { +private[spark] class LocalBackend( + conf: SparkConf, + scheduler: TaskSchedulerImpl, + val totalCores: Int) + extends SchedulerBackend with ExecutorBackend with Logging { private val appId = "local-" + System.currentTimeMillis var localEndpoint: RpcEndpointRef = null @@ -116,7 +123,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def stop() { - localEndpoint.send(StopExecutor) + localEndpoint.sendWithReply(StopExecutor) } override def reviveOffers() { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 6b3049b28cd5e..22acc270b983e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -56,19 +56,13 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit // Min < 0 val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Max < 0 val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf2) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) From 3ae37b93a7c299bd8b22a36248035bca5de3422f Mon Sep 17 00:00:00 2001 From: Jin Adachi Date: Thu, 16 Apr 2015 23:41:04 +0800 Subject: [PATCH 788/817] [SPARK-6694][SQL]SparkSQL CLI must be able to specify an option --database on the command line. SparkSQL CLI has an option --database as follows. But, the option --database is ignored. ``` $ spark-sql --help : CLI options: : --database Specify the database to use ``` Author: Jin Adachi Author: adachij Closes #5345 from adachij2002/SPARK-6694 and squashes the following commits: 8659084 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 0301eb9 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 df81086 [Jin Adachi] Modify code style. 846f83e [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 dbe8c63 [Jin Adachi] Change file permission to 644. 7b58f42 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 c581d06 [Jin Adachi] Add an option --database test db56122 [Jin Adachi] Merge branch 'SPARK-6694' of https://github.com/adachij2002/spark into SPARK-6694 ee09fa5 [adachij] Merge branch 'master' into SPARK-6694 c804c03 [adachij] SparkSQL CLI must be able to specify an option --database on the command line. --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 3 ++ .../sql/hive/thriftserver/CliSuite.scala | 45 +++++++++++++++---- 2 files changed, 39 insertions(+), 9 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 62c061bef690a..85281c6d73a3b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -145,6 +145,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } + // use the specified database if specified + cli.processSelectDatabase(sessionState); + // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6d1d7c3a4e698..b070fa8eaa469 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -25,22 +25,31 @@ import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging import org.apache.spark.util.Utils -class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { +class CliSuite extends FunSuite with BeforeAndAfter with Logging { + val warehousePath = Utils.createTempDir() + val metastorePath = Utils.createTempDir() + + before { + warehousePath.delete() + metastorePath.delete() + } + + after { + warehousePath.delete() + metastorePath.delete() + } + def runCliWithin( timeout: FiniteDuration, extraArgs: Seq[String] = Seq.empty)( - queriesAndExpectedAnswers: (String, String)*) { + queriesAndExpectedAnswers: (String, String)*): Unit = { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip - val warehousePath = Utils.createTempDir() - warehousePath.delete() - val metastorePath = Utils.createTempDir() - metastorePath.delete() val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val command = { @@ -95,8 +104,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin, cause) throw cause } finally { - warehousePath.delete() - metastorePath.delete() process.destroy() } } @@ -124,4 +131,24 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { test("Single command with -e") { runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") } + + test("Single command with --database") { + runCliWithin(1.minute)( + "CREATE DATABASE hive_test_db;" + -> "OK", + "USE hive_test_db;" + -> "OK", + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "Time taken: " + ) + + runCliWithin(1.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( + "" + -> "OK", + "" + -> "hive_test" + ) + } } From ef3fb801ae971656ed9cd1b0ab95bc5a1548adbd Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Apr 2015 13:45:55 -0500 Subject: [PATCH 789/817] [SPARK-6934][Core] Use 'spark.akka.askTimeout' for the ask timeout Fixed my mistake in #4588 Author: zsxwing Closes #5529 from zsxwing/SPARK-6934 and squashes the following commits: 9890b2d [zsxwing] Use 'spark.akka.askTimeout' for the ask timeout --- core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index e259867c14040..f2c1c86af767e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -284,7 +284,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) private[this] val maxRetries = conf.getInt("spark.akka.num.retries", 3) private[this] val retryWaitMs = conf.getLong("spark.akka.retry.wait", 3000) - private[this] val defaultTimeout = conf.getLong("spark.akka.lookupTimeout", 30) seconds + private[this] val defaultAskTimeout = conf.getLong("spark.akka.askTimeout", 30) seconds /** * return the address for the [[RpcEndpointRef]] @@ -304,7 +304,8 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * * This method only sends the message once and never retries. */ - def sendWithReply[T: ClassTag](message: Any): Future[T] = sendWithReply(message, defaultTimeout) + def sendWithReply[T: ClassTag](message: Any): Future[T] = + sendWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a `Future` to @@ -327,7 +328,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * @tparam T type of the reply message * @return the reply message from the corresponding [[RpcEndpoint]] */ - def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultTimeout) + def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a From 55f553a979db925aa0c3559f7e80b99d2bf3feb4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 16 Apr 2015 13:06:34 -0700 Subject: [PATCH 790/817] [SPARK-6855] [SPARKR] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders Author: Shivaram Venkataraman Closes #5462 from shivaram/collate-order and squashes the following commits: f3db562 [Shivaram Venkataraman] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders --- R/pkg/DESCRIPTION | 6 +++--- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/column.R | 2 +- R/pkg/R/group.R | 3 +++ R/pkg/R/jobj.R | 3 +++ R/pkg/R/pairRDD.R | 2 ++ 6 files changed, 13 insertions(+), 5 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1842b97d43651..052f68c6c24e2 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -17,19 +17,19 @@ License: Apache License (== 2.0) Collate: 'generics.R' 'jobj.R' - 'SQLTypes.R' 'RDD.R' 'pairRDD.R' + 'SQLTypes.R' 'column.R' 'group.R' 'DataFrame.R' 'SQLContext.R' + 'backend.R' 'broadcast.R' + 'client.R' 'context.R' 'deserialize.R' 'serialize.R' 'sparkR.R' - 'backend.R' - 'client.R' 'utils.R' 'zzz.R' diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index feafd56909a67..044fdb4d01223 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index e196305186b9a..b282001d8b6b5 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R +#' @include generics.R jobj.R SQLTypes.R NULL setOldClass("jobj") diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 09fc0a7abe48a..855fbdfc7c4ca 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,6 +17,9 @@ # group.R - GroupedData class and methods implemented in S4 OO classes +#' @include generics.R jobj.R SQLTypes.R column.R +NULL + setOldClass("jobj") #' @title S4 class that represents a GroupedData diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 4180f146b7fbc..a8a25230b636d 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -18,6 +18,9 @@ # References to objects that exist on the JVM backend # are maintained using the jobj. +#' @include generics.R +NULL + # Maintain a reference count of Java object references # This allows us to GC the java object when it is safe .validJobjs <- new.env(parent = emptyenv()) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 739d399f0820f..5d64822859d1f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -16,6 +16,8 @@ # # Operations supported on RDDs contains pairs (i.e key, value) +#' @include generics.R jobj.R RDD.R +NULL ############ Actions and Transformations ############ From 04e44b37cc04f62fbf9e08c7076349e0a4d12ea8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 16:20:57 -0700 Subject: [PATCH 791/817] [SPARK-4897] [PySpark] Python 3 support This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu Author: twneale Author: Josh Rosen Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py. --- bin/pyspark | 1 + bin/spark-submit | 3 + bin/spark-submit2.cmd | 3 + dev/run-tests | 2 + dev/run-tests-jenkins | 2 +- ec2/spark_ec2.py | 262 ++++---- examples/src/main/python/als.py | 15 +- examples/src/main/python/avro_inputformat.py | 9 +- .../src/main/python/cassandra_inputformat.py | 8 +- .../src/main/python/cassandra_outputformat.py | 6 +- examples/src/main/python/hbase_inputformat.py | 8 +- .../src/main/python/hbase_outputformat.py | 6 +- examples/src/main/python/kmeans.py | 11 +- .../src/main/python/logistic_regression.py | 20 +- .../ml/simple_text_classification_pipeline.py | 20 +- .../src/main/python/mllib/correlations.py | 19 +- .../src/main/python/mllib/dataset_example.py | 13 +- .../main/python/mllib/decision_tree_runner.py | 29 +- .../python/mllib/gaussian_mixture_model.py | 9 +- .../python/mllib/gradient_boosted_trees.py | 7 +- examples/src/main/python/mllib/kmeans.py | 5 +- .../main/python/mllib/logistic_regression.py | 9 +- .../python/mllib/random_forest_example.py | 9 +- .../python/mllib/random_rdd_generation.py | 21 +- .../src/main/python/mllib/sampled_rdds.py | 29 +- examples/src/main/python/mllib/word2vec.py | 5 +- examples/src/main/python/pagerank.py | 16 +- .../src/main/python/parquet_inputformat.py | 7 +- examples/src/main/python/pi.py | 5 +- examples/src/main/python/sort.py | 6 +- examples/src/main/python/sql.py | 4 +- examples/src/main/python/status_api_demo.py | 10 +- .../main/python/streaming/hdfs_wordcount.py | 3 +- .../main/python/streaming/kafka_wordcount.py | 3 +- .../python/streaming/network_wordcount.py | 3 +- .../recoverable_network_wordcount.py | 11 +- .../python/streaming/sql_network_wordcount.py | 5 +- .../streaming/stateful_network_wordcount.py | 3 +- .../src/main/python/transitive_closure.py | 10 +- examples/src/main/python/wordcount.py | 6 +- .../MatrixFactorizationModelWrapper.scala | 9 +- .../mllib/api/python/PythonMLLibAPI.scala | 39 +- python/pyspark/accumulators.py | 9 +- python/pyspark/broadcast.py | 37 +- python/pyspark/cloudpickle.py | 577 +++++------------- python/pyspark/conf.py | 9 +- python/pyspark/context.py | 42 +- python/pyspark/daemon.py | 36 +- python/pyspark/heapq3.py | 24 +- python/pyspark/java_gateway.py | 2 +- python/pyspark/join.py | 1 + python/pyspark/ml/classification.py | 4 +- python/pyspark/ml/feature.py | 22 +- python/pyspark/ml/param/__init__.py | 8 +- .../ml/param/_shared_params_code_gen.py | 10 +- python/pyspark/mllib/__init__.py | 11 +- python/pyspark/mllib/classification.py | 7 +- python/pyspark/mllib/clustering.py | 18 +- python/pyspark/mllib/common.py | 19 +- python/pyspark/mllib/feature.py | 18 +- python/pyspark/mllib/fpm.py | 2 + python/pyspark/mllib/linalg.py | 48 +- python/pyspark/mllib/rand.py | 33 +- python/pyspark/mllib/recommendation.py | 7 +- python/pyspark/mllib/stat/_statistics.py | 25 +- python/pyspark/mllib/tests.py | 20 +- python/pyspark/mllib/tree.py | 15 +- python/pyspark/mllib/util.py | 26 +- python/pyspark/profiler.py | 10 +- python/pyspark/rdd.py | 189 +++--- python/pyspark/rddsampler.py | 4 +- python/pyspark/serializers.py | 101 ++- python/pyspark/shell.py | 16 +- python/pyspark/shuffle.py | 126 ++-- python/pyspark/sql/__init__.py | 15 +- python/pyspark/sql/{types.py => _types.py} | 49 +- python/pyspark/sql/context.py | 32 +- python/pyspark/sql/dataframe.py | 63 +- python/pyspark/sql/functions.py | 6 +- python/pyspark/sql/tests.py | 11 +- python/pyspark/statcounter.py | 4 +- python/pyspark/streaming/context.py | 5 +- python/pyspark/streaming/dstream.py | 51 +- python/pyspark/streaming/kafka.py | 8 +- python/pyspark/streaming/tests.py | 39 +- python/pyspark/streaming/util.py | 6 +- python/pyspark/tests.py | 327 +++++----- python/pyspark/worker.py | 16 +- python/run-tests | 15 +- python/test_support/userlib-0.1-py2.7.egg | Bin 1945 -> 0 bytes python/test_support/userlib-0.1.zip | Bin 0 -> 668 bytes 91 files changed, 1398 insertions(+), 1396 deletions(-) rename python/pyspark/sql/{types.py => _types.py} (97%) delete mode 100644 python/test_support/userlib-0.1-py2.7.egg create mode 100644 python/test_support/userlib-0.1.zip diff --git a/bin/pyspark b/bin/pyspark index 776b28dc41099..8acad6113797d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,6 +89,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYTHONHASHSEED=0 if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/bin/spark-submit b/bin/spark-submit index bcff78edd51ca..0e0afe71a0f05 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,6 +19,9 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +# disable randomized hash for string in Python 3.3+ +export PYTHONHASHSEED=0 + # Only define a usage function if an upstream script hasn't done so. if ! type -t usage >/dev/null 2>&1; then usage() { diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 08ddb185742d2..d3fc4a5cc3f6e 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -20,6 +20,9 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. +rem disable randomized hash for string in Python 3.3+ +set PYTHONHASHSEED=0 + set CLASS=org.apache.spark.deploy.SparkSubmit call %~dp0spark-class2.cmd %CLASS% %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% diff --git a/dev/run-tests b/dev/run-tests index bb21ab6c9aa04..861d1671182c2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -235,6 +235,8 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS +# add path for python 3 in jenkins +export PATH="${PATH}:/home/anaonda/envs/py3k/bin" ./python/run-tests echo "" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 3c1c91a111357..030f2cdddb350 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -47,7 +47,7 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" -TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +TESTS_TIMEOUT="150m" # format: http://linux.die.net/man/1/timeout # Array to capture all tests to run on the pull request. These tests are held under the #+ dev/tests/ directory. diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0c1f24761d0de..87c0818279713 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -19,7 +19,7 @@ # limitations under the License. # -from __future__ import with_statement +from __future__ import with_statement, print_function import hashlib import itertools @@ -37,12 +37,17 @@ import tempfile import textwrap import time -import urllib2 import warnings from datetime import datetime from optparse import OptionParser from sys import stderr +if sys.version < "3": + from urllib2 import urlopen, Request, HTTPError +else: + from urllib.request import urlopen, Request + from urllib.error import HTTPError + SPARK_EC2_VERSION = "1.2.1" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -88,10 +93,10 @@ def setup_external_libs(libs): SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") if not os.path.exists(SPARK_EC2_LIB_DIR): - print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( + print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( path=SPARK_EC2_LIB_DIR - ) - print "This should be a one-time operation." + )) + print("This should be a one-time operation.") os.mkdir(SPARK_EC2_LIB_DIR) for lib in libs: @@ -100,8 +105,8 @@ def setup_external_libs(libs): if not os.path.isdir(lib_dir): tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print " - Downloading {lib}...".format(lib=lib["name"]) - download_stream = urllib2.urlopen( + print(" - Downloading {lib}...".format(lib=lib["name"])) + download_stream = urlopen( "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( prefix=PYPI_URL_PREFIX, first_letter=lib["name"][:1], @@ -113,13 +118,13 @@ def setup_external_libs(libs): tgz_file.write(download_stream.read()) with open(tgz_file_path) as tar: if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]) + print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) sys.exit(1) tar = tarfile.open(tgz_file_path) tar.extractall(path=SPARK_EC2_LIB_DIR) tar.close() os.remove(tgz_file_path) - print " - Finished downloading {lib}.".format(lib=lib["name"]) + print(" - Finished downloading {lib}.".format(lib=lib["name"])) sys.path.insert(1, lib_dir) @@ -299,12 +304,12 @@ def parse_args(): if home_dir is None or not os.path.isfile(home_dir + '/.boto'): if not os.path.isfile('/etc/boto.cfg'): if os.getenv('AWS_ACCESS_KEY_ID') is None: - print >> stderr, ("ERROR: The environment variable AWS_ACCESS_KEY_ID " + - "must be set") + print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", + file=stderr) sys.exit(1) if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print >> stderr, ("ERROR: The environment variable AWS_SECRET_ACCESS_KEY " + - "must be set") + print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", + file=stderr) sys.exit(1) return (opts, action, cluster_name) @@ -316,7 +321,7 @@ def get_or_make_group(conn, name, vpc_id): if len(group) > 0: return group[0] else: - print "Creating security group " + name + print("Creating security group " + name) return conn.create_security_group(name, "Spark EC2 group", vpc_id) @@ -324,18 +329,19 @@ def get_validate_spark_version(version, repo): if "." in version: version = version.replace("v", "") if version not in VALID_SPARK_VERSIONS: - print >> stderr, "Don't know about Spark version: {v}".format(v=version) + print("Don't know about Spark version: {v}".format(v=version), file=stderr) sys.exit(1) return version else: github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) - request = urllib2.Request(github_commit_url) + request = Request(github_commit_url) request.get_method = lambda: 'HEAD' try: - response = urllib2.urlopen(request) - except urllib2.HTTPError, e: - print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url) - print >> stderr, "Received HTTP response code of {code}.".format(code=e.code) + response = urlopen(request) + except HTTPError as e: + print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), + file=stderr) + print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) sys.exit(1) return version @@ -394,8 +400,7 @@ def get_spark_ami(opts): instance_type = EC2_INSTANCE_TYPES[opts.instance_type] else: instance_type = "pvm" - print >> stderr,\ - "Don't recognize %s, assuming type is pvm" % opts.instance_type + print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) # URL prefix from which to fetch AMI information ami_prefix = "{r}/{b}/ami-list".format( @@ -404,10 +409,10 @@ def get_spark_ami(opts): ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) try: - ami = urllib2.urlopen(ami_path).read().strip() - print "Spark AMI: " + ami + ami = urlopen(ami_path).read().strip() + print("Spark AMI: " + ami) except: - print >> stderr, "Could not resolve AMI at: " + ami_path + print("Could not resolve AMI at: " + ami_path, file=stderr) sys.exit(1) return ami @@ -419,11 +424,11 @@ def get_spark_ami(opts): # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): if opts.identity_file is None: - print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections." + print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) sys.exit(1) if opts.key_pair is None: - print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." + print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) sys.exit(1) user_data_content = None @@ -431,7 +436,7 @@ def launch_cluster(conn, opts, cluster_name): with open(opts.user_data) as user_data_file: user_data_content = user_data_file.read() - print "Setting up security groups..." + print("Setting up security groups...") master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) authorized_address = opts.authorized_address @@ -497,8 +502,8 @@ def launch_cluster(conn, opts, cluster_name): existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print("ERROR: There are already instances running in group %s or %s" % + (master_group.name, slave_group.name), file=stderr) sys.exit(1) # Figure out Spark AMI @@ -511,12 +516,12 @@ def launch_cluster(conn, opts, cluster_name): additional_group_ids = [sg.id for sg in conn.get_all_security_groups() if opts.additional_security_group in (sg.name, sg.id)] - print "Launching instances..." + print("Launching instances...") try: image = conn.get_all_images(image_ids=[opts.ami])[0] except: - print >> stderr, "Could not find AMI " + opts.ami + print("Could not find AMI " + opts.ami, file=stderr) sys.exit(1) # Create block device mapping so that we can add EBS volumes if asked to. @@ -542,8 +547,8 @@ def launch_cluster(conn, opts, cluster_name): # Launch slaves if opts.spot_price is not None: # Launch spot instances with the requested price - print ("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) + print("Requesting %d slaves as spot instances with price $%.3f" % + (opts.slaves, opts.spot_price)) zones = get_zones(conn, opts) num_zones = len(zones) i = 0 @@ -566,7 +571,7 @@ def launch_cluster(conn, opts, cluster_name): my_req_ids += [req.id for req in slave_reqs] i += 1 - print "Waiting for spot instances to be granted..." + print("Waiting for spot instances to be granted...") try: while True: time.sleep(10) @@ -579,24 +584,24 @@ def launch_cluster(conn, opts, cluster_name): if i in id_to_req and id_to_req[i].state == "active": active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: - print "All %d slaves granted" % opts.slaves + print("All %d slaves granted" % opts.slaves) reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print("%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves)) except: - print "Canceling spot instance requests" + print("Canceling spot instance requests") conn.cancel_spot_instance_requests(my_req_ids) # Log a warning if any of these requests actually launched instances: (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) running = len(master_nodes) + len(slave_nodes) if running: - print >> stderr, ("WARNING: %d instances are still running" % running) + print(("WARNING: %d instances are still running" % running), file=stderr) sys.exit(0) else: # Launch non-spot instances @@ -618,16 +623,16 @@ def launch_cluster(conn, opts, cluster_name): placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances - print "Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id) + print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( + s=num_slaves_this_zone, + plural_s=('' if num_slaves_this_zone == 1 else 's'), + z=zone, + r=slave_res.id)) i += 1 # Launch or resume masters if existing_masters: - print "Starting master..." + print("Starting master...") for inst in existing_masters: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -650,10 +655,10 @@ def launch_cluster(conn, opts, cluster_name): user_data=user_data_content) master_nodes = master_res.instances - print "Launched master in %s, regid = %s" % (zone, master_res.id) + print("Launched master in %s, regid = %s" % (zone, master_res.id)) # This wait time corresponds to SPARK-4983 - print "Waiting for AWS to propagate instance metadata..." + print("Waiting for AWS to propagate instance metadata...") time.sleep(5) # Give the instances descriptive names for master in master_nodes: @@ -674,8 +679,8 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): Get the EC2 instances in an existing cluster if available. Returns a tuple of lists of EC2 instance objects for the masters and slaves. """ - print "Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region) + print("Searching for existing cluster {c} in region {r}...".format( + c=cluster_name, r=opts.region)) def get_instances(group_names): """ @@ -693,16 +698,15 @@ def get_instances(group_names): slave_instances = get_instances([cluster_name + "-slaves"]) if any((master_instances, slave_instances)): - print "Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's')) + print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( + m=len(master_instances), + plural_m=('' if len(master_instances) == 1 else 's'), + s=len(slave_instances), + plural_s=('' if len(slave_instances) == 1 else 's'))) if not master_instances and die_on_error: - print >> sys.stderr, \ - "ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region) + print("ERROR: Could not find a master for cluster {c} in region {r}.".format( + c=cluster_name, r=opts.region), file=sys.stderr) sys.exit(1) return (master_instances, slave_instances) @@ -713,7 +717,7 @@ def get_instances(group_names): def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: - print "Generating cluster's SSH key on master..." + print("Generating cluster's SSH key on master...") key_setup = """ [ -f ~/.ssh/id_rsa ] || (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && @@ -721,10 +725,10 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): """ ssh(master, opts, key_setup) dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print "Transferring cluster's SSH key to slaves..." + print("Transferring cluster's SSH key to slaves...") for slave in slave_nodes: slave_address = get_dns_name(slave, opts.private_ips) - print slave_address + print(slave_address) ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', @@ -738,8 +742,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch) + print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( + r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) ssh( host=master, opts=opts, @@ -749,7 +753,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): b=opts.spark_ec2_git_branch) ) - print "Deploying files to master..." + print("Deploying files to master...") deploy_files( conn=conn, root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", @@ -760,25 +764,25 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ) if opts.deploy_root_dir is not None: - print "Deploying {s} to master...".format(s=opts.deploy_root_dir) + print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) deploy_user_files( root_dir=opts.deploy_root_dir, opts=opts, master_nodes=master_nodes ) - print "Running setup on master..." + print("Running setup on master...") setup_spark_cluster(master, opts) - print "Done!" + print("Done!") def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") - print "Spark standalone cluster started at http://%s:8080" % master + print("Spark standalone cluster started at http://%s:8080" % master) if opts.ganglia: - print "Ganglia started at http://%s:5080/ganglia" % master + print("Ganglia started at http://%s:5080/ganglia" % master) def is_ssh_available(host, opts, print_ssh_output=True): @@ -795,7 +799,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): if s.returncode != 0 and print_ssh_output: # extra leading newline is for spacing in wait_for_cluster_state() - print textwrap.dedent("""\n + print(textwrap.dedent("""\n Warning: SSH connection error. (This could be temporary.) Host: {h} SSH return code: {r} @@ -804,7 +808,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): h=host, r=s.returncode, o=cmd_output.strip() - ) + )) return s.returncode == 0 @@ -865,10 +869,10 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.write("\n") end_time = datetime.now() - print "Cluster is now in '{s}' state. Waited {t} seconds.".format( + print("Cluster is now in '{s}' state. Waited {t} seconds.".format( s=cluster_state, t=(end_time - start_time).seconds - ) + )) # Get number of local disks available for a given EC2 instance type. @@ -916,8 +920,8 @@ def get_num_disks(instance_type): if instance_type in disks_by_instance: return disks_by_instance[instance_type] else: - print >> stderr, ("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type) + print("WARNING: Don't know number of disks on instance type %s; assuming 1" + % instance_type, file=stderr) return 1 @@ -951,7 +955,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) tachyon_v = "" - print "Deploying Spark via git hash; Tachyon won't be set up" + print("Deploying Spark via git hash; Tachyon won't be set up") modules = filter(lambda x: x != "tachyon", modules) master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] @@ -1067,8 +1071,8 @@ def ssh(host, opts, command): "--key-pair parameters and try again.".format(host)) else: raise e - print >> stderr, \ - "Error executing remote command, retrying after 30 seconds: {0}".format(e) + print("Error executing remote command, retrying after 30 seconds: {0}".format(e), + file=stderr) time.sleep(30) tries = tries + 1 @@ -1107,8 +1111,8 @@ def ssh_write(host, opts, command, arguments): elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: - print >> stderr, \ - "Error {0} while executing remote command, retrying after 30 seconds".format(status) + print("Error {0} while executing remote command, retrying after 30 seconds". + format(status), file=stderr) time.sleep(30) tries = tries + 1 @@ -1162,42 +1166,41 @@ def real_main(): if opts.identity_file is not None: if not os.path.exists(opts.identity_file): - print >> stderr,\ - "ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file) + print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), + file=stderr) sys.exit(1) file_mode = os.stat(opts.identity_file).st_mode if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print >> stderr, "ERROR: The identity file must be accessible only by you." - print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file) + print("ERROR: The identity file must be accessible only by you.", file=stderr) + print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), + file=stderr) sys.exit(1) if opts.instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type) + print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( + t=opts.instance_type), file=stderr) if opts.master_instance_type != "": if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, \ - "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type) + print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( + t=opts.master_instance_type), file=stderr) # Since we try instance types even if we can't resolve them, we check if they resolve first # and, if they do, see if they resolve to the same virtualization type. if opts.instance_type in EC2_INSTANCE_TYPES and \ opts.master_instance_type in EC2_INSTANCE_TYPES: if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: - print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves " + \ - "with different AMI virtualization types." - print >> stderr, "master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]) - print >> stderr, "slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]) + print("Error: spark-ec2 currently does not support having a master and slaves " + "with different AMI virtualization types.", file=stderr) + print("master instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) + print("slave instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) sys.exit(1) if opts.ebs_vol_num > 8: - print >> stderr, "ebs-vol-num cannot be greater than 8" + print("ebs-vol-num cannot be greater than 8", file=stderr) sys.exit(1) # Prevent breaking ami_prefix (/, .git and startswith checks) @@ -1206,23 +1209,22 @@ def real_main(): opts.spark_ec2_git_repo.endswith(".git") or \ not opts.spark_ec2_git_repo.startswith("https://github.com") or \ not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \ - "trailing / or .git. " \ - "Furthermore, we currently only support forks named spark-ec2." + print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " + "Furthermore, we currently only support forks named spark-ec2.", file=stderr) sys.exit(1) if not (opts.deploy_root_dir is None or (os.path.isabs(opts.deploy_root_dir) and os.path.isdir(opts.deploy_root_dir) and os.path.exists(opts.deploy_root_dir))): - print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \ - "on the local file system" + print("--deploy-root-dir must be an absolute path to a directory that exists " + "on the local file system", file=stderr) sys.exit(1) try: conn = ec2.connect_to_region(opts.region) except Exception as e: - print >> stderr, (e) + print((e), file=stderr) sys.exit(1) # Select an AZ at random if it was not specified. @@ -1231,7 +1233,7 @@ def real_main(): if action == "launch": if opts.slaves <= 0: - print >> sys.stderr, "ERROR: You have to start at least 1 slave" + print("ERROR: You have to start at least 1 slave", file=sys.stderr) sys.exit(1) if opts.resume: (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) @@ -1250,18 +1252,18 @@ def real_main(): conn, opts, cluster_name, die_on_error=False) if any(master_nodes + slave_nodes): - print "The following instances will be terminated:" + print("The following instances will be terminated:") for inst in master_nodes + slave_nodes: - print "> %s" % get_dns_name(inst, opts.private_ips) - print "ALL DATA ON ALL NODES WILL BE LOST!!" + print("> %s" % get_dns_name(inst, opts.private_ips)) + print("ALL DATA ON ALL NODES WILL BE LOST!!") msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) response = raw_input(msg) if response == "y": - print "Terminating master..." + print("Terminating master...") for inst in master_nodes: inst.terminate() - print "Terminating slaves..." + print("Terminating slaves...") for inst in slave_nodes: inst.terminate() @@ -1274,16 +1276,16 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated' ) - print "Deleting security groups (this will take some time)..." + print("Deleting security groups (this will take some time)...") attempt = 1 while attempt <= 3: - print "Attempt %d" % attempt + print("Attempt %d" % attempt) groups = [g for g in conn.get_all_security_groups() if g.name in group_names] success = True # Delete individual rules in all groups before deleting groups to # remove dependencies between them for group in groups: - print "Deleting rules in security group " + group.name + print("Deleting rules in security group " + group.name) for rule in group.rules: for grant in rule.grants: success &= group.revoke(ip_protocol=rule.ip_protocol, @@ -1298,10 +1300,10 @@ def real_main(): try: # It is needed to use group_id to make it work with VPC conn.delete_security_group(group_id=group.id) - print "Deleted security group %s" % group.name + print("Deleted security group %s" % group.name) except boto.exception.EC2ResponseError: success = False - print "Failed to delete security group %s" % group.name + print("Failed to delete security group %s" % group.name) # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails @@ -1311,17 +1313,16 @@ def real_main(): attempt += 1 if not success: - print "Failed to delete all security groups after 3 tries." - print "Try re-running in a few minutes." + print("Failed to delete all security groups after 3 tries.") + print("Try re-running in a few minutes.") elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: master = get_dns_name(master_nodes[0], opts.private_ips) - print "Logging into master " + master + "..." + print("Logging into master " + master + "...") proxy_opt = [] if opts.proxy_port is not None: proxy_opt = ['-D', opts.proxy_port] @@ -1336,19 +1337,18 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Rebooting slaves..." + print("Rebooting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: - print "Rebooting " + inst.id + print("Rebooting " + inst.id) inst.reboot() elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: - print get_dns_name(master_nodes[0], opts.private_ips) + print(get_dns_name(master_nodes[0], opts.private_ips)) elif action == "stop": response = raw_input( @@ -1361,11 +1361,11 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Stopping master..." + print("Stopping master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.stop() - print "Stopping slaves..." + print("Stopping slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: if inst.spot_instance_request_id: @@ -1375,11 +1375,11 @@ def real_main(): elif action == "start": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print "Starting slaves..." + print("Starting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - print "Starting master..." + print("Starting master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -1403,15 +1403,15 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: - print >> stderr, "Invalid action: %s" % action + print("Invalid action: %s" % action, file=stderr) sys.exit(1) def main(): try: real_main() - except UsageError, e: - print >> stderr, "\nError:\n", e + except UsageError as e: + print("\nError:\n", e, file=stderr) sys.exit(1) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 70b6146e39a87..1c3a787bd0e94 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -21,7 +21,8 @@ This example requires numpy (http://www.numpy.org/) """ -from os.path import realpath +from __future__ import print_function + import sys import numpy as np @@ -57,9 +58,9 @@ def update(i, vec, mat, ratings): Usage: als [M] [U] [F] [iterations] [partitions]" """ - print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + print("""WARN: This is a naive implementation of ALS and is given as an example. Please use the ALS method found in pyspark.mllib.recommendation for more - conventional use.""" + conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 @@ -68,8 +69,8 @@ def update(i, vec, mat, ratings): ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ - (M, U, F, ITERATIONS, partitions) + print("Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % + (M, U, F, ITERATIONS, partitions)) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -95,7 +96,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i - print "\nRMSE: %5.4f\n" % error + print("Iteration %d:" % i) + print("\nRMSE: %5.4f\n" % error) sc.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 4626bbb7e3b02..da368ac628a49 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -15,9 +15,12 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext +from functools import reduce """ Read data file users.avro in local Spark distro: @@ -49,7 +52,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 2 and len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: avro_inputformat [reader_schema_file] Run with example jar: @@ -57,7 +60,7 @@ /path/to/examples/avro_inputformat.py [reader_schema_file] Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -77,6 +80,6 @@ conf=conf) output = avro_rdd.map(lambda x: x[0]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 05f34b74df45a..93ca0cfcc9302 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, """ + print(""" Usage: cassandra_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -77,6 +79,6 @@ conf=conf) output = cass_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index d144539e58b8f..5d643eac92f94 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -46,7 +48,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: cassandra_outputformat Run with example jar: @@ -60,7 +62,7 @@ ... fname text, ... lname text ... ); - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3b16010f1cb97..e17819d5feb76 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: hbase_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/hbase_inputformat.py
    Assumes you have some data in HBase already, running on , in
    - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -74,6 +76,6 @@ conf=conf) output = hbase_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index abb425b1f886a..9e5641789a976 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -40,7 +42,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: hbase_outputformat
    Run with example jar: @@ -48,7 +50,7 @@ /path/to/examples/hbase_outputformat.py Assumes you have created
    with column family in HBase running on already - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 86ef6f32c84e8..19391506463f0 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -22,6 +22,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -47,12 +48,12 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + print("""WARN: This is a naive implementation of KMeans Clustering and is given as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""" + how to use MLlib's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) @@ -69,13 +70,13 @@ def closestPoint(p, centers): pointStats = closest.reduceByKey( lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) newPoints = pointStats.map( - lambda (x, (y, z)): (x, y / z)).collect() + lambda xy: (xy[0], xy[1][0] / xy[1][1])).collect() tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) for (x, y) in newPoints: kPoints[x] = y - print "Final centers: " + str(kPoints) + print("Final centers: " + str(kPoints)) sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 3aa56b0528168..b318b7d87bfdc 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -22,10 +22,8 @@ In practice, one may prefer to use the LogisticRegression algorithm in MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ +from __future__ import print_function -from collections import namedtuple -from math import exp -from os.path import realpath import sys import numpy as np @@ -42,19 +40,19 @@ def readPointBatch(iterator): strs = list(iterator) matrix = np.zeros((len(strs), D + 1)) - for i in xrange(len(strs)): - matrix[i] = np.fromstring(strs[i].replace(',', ' '), dtype=np.float32, sep=' ') + for i, s in enumerate(strs): + matrix[i] = np.fromstring(s.replace(',', ' '), dtype=np.float32, sep=' ') return [matrix] if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + print("""WARN: This is a naive implementation of Logistic Regression and is given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""" + to see how MLlib's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() @@ -62,7 +60,7 @@ def readPointBatch(iterator): # Initialize w to a random value w = 2 * np.random.ranf(size=D) - 1 - print "Initial w: " + str(w) + print("Initial w: " + str(w)) # Compute logistic regression gradient for a matrix of data points def gradient(matrix, w): @@ -76,9 +74,9 @@ def add(x, y): return x for i in range(iterations): - print "On iteration %i" % (i + 1) + print("On iteration %i" % (i + 1)) w -= points.map(lambda m: gradient(m, w)).reduce(add) - print "Final w: " + str(w) + print("Final w: " + str(w)) sc.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index c73edb7fd6b20..fab21f003b233 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression @@ -37,10 +39,10 @@ # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ + training = sc.parallelize([(0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0)]) \ .map(lambda x: LabeledDocument(*x)).toDF() # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. @@ -54,16 +56,16 @@ # Prepare test documents, which are unlabeled. Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ + test = sc.parallelize([(4, "spark i j k"), + (5, "l m n"), + (6, "mapreduce spark"), + (7, "apache hadoop")]) \ .map(lambda x: Document(*x)).toDF() # Make predictions on test documents and print columns of interest. prediction = model.transform(test) selected = prediction.select("id", "text", "prediction") for row in selected.collect(): - print row + print(row) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 4218eca822a99..0e13546b88e67 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -18,6 +18,7 @@ """ Correlations using MLlib. """ +from __future__ import print_function import sys @@ -29,7 +30,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: correlations ()" + print("Usage: correlations ()", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonCorrelations") if len(sys.argv) == 2: @@ -41,20 +42,20 @@ points = MLUtils.loadLibSVMFile(sc, filepath)\ .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) - print - print 'Summary of data file: ' + filepath - print '%d data points' % points.count() + print() + print('Summary of data file: ' + filepath) + print('%d data points' % points.count()) # Statistics (correlations) - print - print 'Correlation (%s) between label and each feature' % corrType - print 'Feature\tCorrelation' + print() + print('Correlation (%s) between label and each feature' % corrType) + print('Feature\tCorrelation') numFeatures = points.take(1)[0].features.size labelRDD = points.map(lambda lp: lp.label) for i in range(numFeatures): featureRDD = points.map(lambda lp: lp.features[i]) corr = Statistics.corr(labelRDD, featureRDD, corrType) - print '%d\t%g' % (i, corr) - print + print('%d\t%g' % (i, corr)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index fcbf56cbf0c52..e23ecc0c5d302 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -19,6 +19,7 @@ An example of how to use DataFrame as a dataset for ML. Run with:: bin/spark-submit examples/src/main/python/mllib/dataset_example.py """ +from __future__ import print_function import os import sys @@ -32,16 +33,16 @@ def summarize(dataset): - print "schema: %s" % dataset.schema().json() + print("schema: %s" % dataset.schema().json()) labels = dataset.map(lambda r: r.label) - print "label average: %f" % labels.mean() + print("label average: %f" % labels.mean()) features = dataset.map(lambda r: r.features) summary = Statistics.colStats(features) - print "features average: %r" % summary.mean() + print("features average: %r" % summary.mean()) if __name__ == "__main__": if len(sys.argv) > 2: - print >> sys.stderr, "Usage: dataset_example.py " + print("Usage: dataset_example.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="DatasetExample") sqlContext = SQLContext(sc) @@ -54,9 +55,9 @@ def summarize(dataset): summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) - print "Save dataset as a Parquet file to %s." % tempdir + print("Save dataset as a Parquet file to %s." % tempdir) dataset0.saveAsParquetFile(tempdir) - print "Load it back and summarize it again." + print("Load it back and summarize it again.") dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index fccabd841b139..513ed8fd51450 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import numpy import os @@ -83,18 +84,17 @@ def reindexClassLabels(data): numClasses = len(classCounts) # origToNewLabels: class --> index in 0,...,numClasses-1 if (numClasses < 2): - print >> sys.stderr, \ - "Dataset for classification should have at least 2 classes." + \ - " The given dataset had only %d classes." % numClasses + print("Dataset for classification should have at least 2 classes." + " The given dataset had only %d classes." % numClasses, file=sys.stderr) exit(1) origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) - print "numClasses = %d" % numClasses - print "Per-class example fractions, counts:" - print "Class\tFrac\tCount" + print("numClasses = %d" % numClasses) + print("Per-class example fractions, counts:") + print("Class\tFrac\tCount") for c in sortedClasses: frac = classCounts[c] / (numExamples + 0.0) - print "%g\t%g\t%d" % (c, frac, classCounts[c]) + print("%g\t%g\t%d" % (c, frac, classCounts[c])) if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): return (data, origToNewLabels) @@ -105,8 +105,7 @@ def reindexClassLabels(data): def usage(): - print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]" + print("Usage: decision_tree_runner [libsvm format data filepath]", file=sys.stderr) exit(1) @@ -133,13 +132,13 @@ def usage(): model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. - print "Trained DecisionTree for classification:" - print " Model numNodes: %d" % model.numNodes() - print " Model depth: %d" % model.depth() - print " Training accuracy: %g" % getAccuracy(model, reindexedData) + print("Trained DecisionTree for classification:") + print(" Model numNodes: %d" % model.numNodes()) + print(" Model depth: %d" % model.depth()) + print(" Training accuracy: %g" % getAccuracy(model, reindexedData)) if model.numNodes() < 20: - print model.toDebugString() + print(model.toDebugString()) else: - print model + print(model) sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index a2cd626c9f19d..2cb8010cdc07f 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -18,7 +18,8 @@ """ A Gaussian Mixture Model clustering program using MLlib. """ -import sys +from __future__ import print_function + import random import argparse import numpy as np @@ -59,7 +60,7 @@ def parseVector(line): model = GaussianMixture.train(data, args.k, args.convergenceTol, args.maxIterations, args.seed) for i in range(args.k): - print ("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, - "sigma = ", model.gaussians[i].sigma.toArray()) - print ("Cluster labels (first 100): ", model.predict(data).take(100)) + print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, + "sigma = ", model.gaussians[i].sigma.toArray())) + print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosted_trees.py b/examples/src/main/python/mllib/gradient_boosted_trees.py index e647773ad9060..781bd61c9d2b5 100644 --- a/examples/src/main/python/mllib/gradient_boosted_trees.py +++ b/examples/src/main/python/mllib/gradient_boosted_trees.py @@ -18,6 +18,7 @@ """ Gradient boosted Trees classification and regression using MLlib. """ +from __future__ import print_function import sys @@ -34,7 +35,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() \ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count() \ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification ensemble model:') @@ -49,7 +50,7 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() \ + testMSE = labelsAndPredictions.map(lambda vp: (vp[0] - vp[1]) * (vp[0] - vp[1])).sum() \ / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression ensemble model:') @@ -58,7 +59,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: gradient_boosted_trees" + print("Usage: gradient_boosted_trees", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonGradientBoostedTrees") diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index 2eeb1abeeb12b..f901a87fa63ac 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -34,12 +35,12 @@ def parseVector(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) sc = SparkContext(appName="KMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector) k = int(sys.argv[2]) model = KMeans.train(data, k) - print "Final centers: " + str(model.clusterCenters) + print("Final centers: " + str(model.clusterCenters)) sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 8cae27fc4a52d..d4f1d34e2d8cf 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -20,11 +20,10 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function -from math import exp import sys -import numpy as np from pyspark import SparkContext from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import LogisticRegressionWithSGD @@ -42,12 +41,12 @@ def parsePoint(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).map(parsePoint) iterations = int(sys.argv[2]) model = LogisticRegressionWithSGD.train(points, iterations) - print "Final weights: " + str(model.weights) - print "Final intercept: " + str(model.intercept) + print("Final weights: " + str(model.weights)) + print("Final intercept: " + str(model.intercept)) sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py index d3c24f7664329..4cfdad868c66e 100755 --- a/examples/src/main/python/mllib/random_forest_example.py +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -22,6 +22,7 @@ For information on multiclass classification, please refer to the decision_tree_runner.py example. """ +from __future__ import print_function import sys @@ -43,7 +44,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count()\ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count()\ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification forest model:') @@ -62,8 +63,8 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum()\ - / float(testData.count()) + testMSE = labelsAndPredictions.map(lambda v_p1: (v_p1[0] - v_p1[1]) * (v_p1[0] - v_p1[1]))\ + .sum() / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression forest model:') print(model.toDebugString()) @@ -71,7 +72,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: random_forest_example" + print("Usage: random_forest_example", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonRandomForestExample") diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index 1e8892741e714..729bae30b152c 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -18,6 +18,7 @@ """ Randomly generated RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: random_rdd_generation" + print("Usage: random_rdd_generation", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonRandomRDDGeneration") @@ -37,19 +38,19 @@ # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) - print 'Generated RDD of %d examples sampled from the standard normal distribution'\ - % normalRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples sampled from the standard normal distribution' + % normalRDD.count()) + print(' First 5 samples:') for sample in normalRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() # Example: RandomRDDs.normalVectorRDD normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) - print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count()) + print(' First 5 samples:') for sample in normalVectorRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index 92af3af5ebd1e..b7033ab7daeb3 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -18,6 +18,7 @@ """ Randomly sampled RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: sampled_rdds " + print("Usage: sampled_rdds ", file=sys.stderr) exit(-1) if len(sys.argv) == 2: datapath = sys.argv[1] @@ -41,24 +42,24 @@ examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() if numExamples == 0: - print >> sys.stderr, "Error: Data file had no samples to load." + print("Error: Data file had no samples to load.", file=sys.stderr) exit(1) - print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + print('Loaded data with %d examples from file: %s' % (numExamples, datapath)) # Example: RDD.sample() and RDD.takeSample() expectedSampleSize = int(numExamples * fraction) - print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ - % (fraction, expectedSampleSize) + print('Sampling RDD using fraction %g. Expected sample size = %d.' + % (fraction, expectedSampleSize)) sampledRDD = examples.sample(withReplacement=True, fraction=fraction) - print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + print(' RDD.sample(): sample has %d examples' % sampledRDD.count()) sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) - print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + print(' RDD.takeSample(): sample has %d examples' % len(sampledArray)) - print + print() # Example: RDD.sampleByKey() keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) - print ' Keyed data using label (Int) as key ==> Orig' + print(' Keyed data using label (Int) as key ==> Orig') # Count examples per label in original data. keyCountsA = keyedRDD.countByKey() @@ -69,18 +70,18 @@ sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) - print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ - % sizeB + print(' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' + % sizeB) # Compare samples - print ' \tFractions of examples with key' - print 'Key\tOrig\tSample' + print(' \tFractions of examples with key') + print('Key\tOrig\tSample') for k in sorted(keyCountsA.keys()): fracA = keyCountsA[k] / float(numExamples) if sizeB != 0: fracB = keyCountsB.get(k, 0) / float(sizeB) else: fracB = 0 - print '%d\t%g\t%g' % (k, fracA, fracB) + print('%d\t%g\t%g' % (k, fracA, fracB)) sc.stop() diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 99fef4276a369..40d1b887927e0 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -23,6 +23,7 @@ # grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines # This was done so that the example can be run in local mode +from __future__ import print_function import sys @@ -34,7 +35,7 @@ if __name__ == "__main__": if len(sys.argv) < 2: - print USAGE + print(USAGE) sys.exit("Argument for file not provided") file_path = sys.argv[1] sc = SparkContext(appName='Word2Vec') @@ -46,5 +47,5 @@ synonyms = model.findSynonyms('china', 40) for word, cosine_distance in synonyms: - print "{}: {}".format(word, cosine_distance) + print("{}: {}".format(word, cosine_distance)) sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index a5f25d78c1146..2fdc9773d4eb1 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -19,6 +19,7 @@ This is an example implementation of PageRank. For more conventional use, Please refer to PageRank implementation provided by graphx """ +from __future__ import print_function import re import sys @@ -42,11 +43,12 @@ def parseNeighbors(urls): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: pagerank " + print("Usage: pagerank ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of PageRank and is - given as an example! Please refer to PageRank implementation provided by graphx""" + print("""WARN: This is a naive implementation of PageRank and is + given as an example! Please refer to PageRank implementation provided by graphx""", + file=sys.stderr) # Initialize the spark context. sc = SparkContext(appName="PythonPageRank") @@ -62,19 +64,19 @@ def parseNeighbors(urls): links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() # Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - ranks = links.map(lambda (url, neighbors): (url, 1.0)) + ranks = links.map(lambda url_neighbors: (url_neighbors[0], 1.0)) # Calculates and updates URL ranks continuously using PageRank algorithm. - for iteration in xrange(int(sys.argv[2])): + for iteration in range(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. contribs = links.join(ranks).flatMap( - lambda (url, (urls, rank)): computeContribs(urls, rank)) + lambda url_urls_rank: computeContribs(url_urls_rank[1][0], url_urls_rank[1][1])) # Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(add).mapValues(lambda rank: rank * 0.85 + 0.15) # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): - print "%s has rank: %s." % (link, rank) + print("%s has rank: %s." % (link, rank)) sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index fa4c20ab20281..96ddac761d698 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,14 +36,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, """ + print(""" Usage: parquet_inputformat.py Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \\ /path/to/examples/parquet_inputformat.py Assumes you have Parquet data stored in . - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -56,6 +57,6 @@ valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') output = parquet_rdd.map(lambda x: x[1]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index a7c74e969cdb9..92e5cf45abc8b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,7 +36,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) - print "Pi is roughly %f" % (4.0 * count / n) + count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index bb686f17518a0..f6b0ecb02c100 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -22,7 +24,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: sort " + print("Usage: sort ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonSort") lines = sc.textFile(sys.argv[1], 1) @@ -33,6 +35,6 @@ # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() for (num, unitcount) in output: - print num + print(num) sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index d89361f324917..87d7b088f077b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os from pyspark import SparkContext @@ -68,6 +70,6 @@ teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): - print each[0] + print(each[0]) sc.stop() diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index a33bdc475a06d..49b7902185aaa 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import time import threading import Queue @@ -52,15 +54,15 @@ def run(): ids = status.getJobIdsForGroup() for id in ids: job = status.getJobInfo(id) - print "Job", id, "status: ", job.status + print("Job", id, "status: ", job.status) for sid in job.stageIds: info = status.getStageInfo(sid) if info: - print "Stage %d: %d tasks total (%d active, %d complete)" % \ - (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks) + print("Stage %d: %d tasks total (%d active, %d complete)" % + (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks)) time.sleep(1) - print "Job results are:", result.get() + print("Job results are:", result.get()) sc.stop() if __name__ == "__main__": diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index f7ffb5379681e..f815dd26823d1 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -25,6 +25,7 @@ Then create a text file in `localdir` and the words in the file will get counted. """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: hdfs_wordcount.py " + print("Usage: hdfs_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingHDFSWordCount") diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index 51e1ff822fc55..b178e7899b5e1 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -27,6 +27,7 @@ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ +from __future__ import print_function import sys @@ -36,7 +37,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kafka_wordcount.py " + print("Usage: kafka_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingKafkaWordCount") diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py index cfa9c1ff5bfbc..2b48bcfd55db0 100644 --- a/examples/src/main/python/streaming/network_wordcount.py +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -25,6 +25,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: network_wordcount.py " + print("Usage: network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index fc6827c82bf9b..ac91f0a06b172 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -35,6 +35,7 @@ checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from the checkpoint data. """ +from __future__ import print_function import os import sys @@ -46,7 +47,7 @@ def createContext(host, port, outputPath): # If you do not see this printed, that means the StreamingContext has been loaded # from the new checkpoint - print "Creating new context" + print("Creating new context") if os.path.exists(outputPath): os.remove(outputPath) sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") @@ -60,8 +61,8 @@ def createContext(host, port, outputPath): def echo(time, rdd): counts = "Counts at time %s %s" % (time, rdd.collect()) - print counts - print "Appending to " + os.path.abspath(outputPath) + print(counts) + print("Appending to " + os.path.abspath(outputPath)) with open(outputPath, 'a') as f: f.write(counts + "\n") @@ -70,8 +71,8 @@ def echo(time, rdd): if __name__ == "__main__": if len(sys.argv) != 5: - print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ - " " + print("Usage: recoverable_network_wordcount.py " + " ", file=sys.stderr) exit(-1) host, port, checkpoint, output = sys.argv[1:] ssc = StreamingContext.getOrCreate(checkpoint, diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index f89bc562d856b..da90c07dbd82f 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -27,6 +27,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` """ +from __future__ import print_function import os import sys @@ -44,7 +45,7 @@ def getSqlContextInstance(sparkContext): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: sql_network_wordcount.py " + print("Usage: sql_network_wordcount.py ", file=sys.stderr) exit(-1) host, port = sys.argv[1:] sc = SparkContext(appName="PythonSqlNetworkWordCount") @@ -57,7 +58,7 @@ def getSqlContextInstance(sparkContext): # Convert RDDs of the words DStream to DataFrame and run SQL query def process(time, rdd): - print "========= %s =========" % str(time) + print("========= %s =========" % str(time)) try: # Get the singleton instance of SQLContext diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index 18a9a5a452ffb..16ef646b7c42e 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -29,6 +29,7 @@ `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ localhost 9999` """ +from __future__ import print_function import sys @@ -37,7 +38,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: stateful_network_wordcount.py " + print("Usage: stateful_network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingStatefulNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 00a281bfb6506..7bf5fb6ddfe29 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from random import Random @@ -49,20 +51,20 @@ def generateGraph(): # the graph to obtain the path (x, z). # Because join() joins on keys, the edges are stored in reversed order. - edges = tc.map(lambda (x, y): (y, x)) + edges = tc.map(lambda x_y: (x_y[1], x_y[0])) - oldCount = 0L + oldCount = 0 nextCount = tc.count() while True: oldCount = nextCount # Perform the join, obtaining an RDD of (y, (z, x)) pairs, # then project the result to obtain the new (x, z) paths. - new_edges = tc.join(edges).map(lambda (_, (a, b)): (b, a)) + new_edges = tc.join(edges).map(lambda __a_b: (__a_b[1][1], __a_b[1][0])) tc = tc.union(new_edges).distinct().cache() nextCount = tc.count() if nextCount == oldCount: break - print "TC has %i edges" % tc.count() + print("TC has %i edges" % tc.count()) sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index ae6cd13b83d92..7c0143607b61d 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from operator import add @@ -23,7 +25,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: wordcount " + print("Usage: wordcount ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonWordCount") lines = sc.textFile(sys.argv[1], 1) @@ -32,6 +34,6 @@ .reduceByKey(add) output = counts.collect() for (word, count) in output: - print "%s: %i" % (word, count) + print("%s: %i" % (word, count)) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala index ecd3b16598438..534edac56bc5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.recommendation.{MatrixFactorizationModel, Rating} import org.apache.spark.rdd.RDD @@ -31,10 +32,14 @@ private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorization predict(SerDe.asTupleRDD(userAndProducts.rdd)) def getUserFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(userFeatures.map { + case (user, feature) => (user, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } def getProductFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(productFeatures.map { + case (product, feature) => (product, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ab15f0f36a14b..f976d2f97b043 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -28,7 +28,6 @@ import scala.reflect.ClassTag import net.razorvine.pickle._ -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ @@ -40,15 +39,15 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.stat.test.ChiSqTestResult -import org.apache.spark.mllib.tree.{GradientBoostedTrees, RandomForest, DecisionTree} -import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo, Strategy} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel, RandomForestModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, RandomForestModel} +import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -279,7 +278,7 @@ private[python] class PythonMLLibAPI extends Serializable { data: JavaRDD[LabeledPoint], lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta.map(Vectors.dense)). map(_.asInstanceOf[Object]).asJava } @@ -335,7 +334,7 @@ private[python] class PythonMLLibAPI extends Serializable { mu += model.gaussians(i).mu sigma += model.gaussians(i).sigma } - List(wt.toArray, mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + List(Vectors.dense(wt.toArray), mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava } finally { data.rdd.unpersist(blocking = false) } @@ -346,20 +345,20 @@ private[python] class PythonMLLibAPI extends Serializable { */ def predictSoftGMM( data: JavaRDD[Vector], - wt: Object, + wt: Vector, mu: Array[Object], - si: Array[Object]): RDD[Array[Double]] = { + si: Array[Object]): RDD[Vector] = { - val weight = wt.asInstanceOf[Array[Double]] + val weight = wt.toArray val mean = mu.map(_.asInstanceOf[DenseVector]) val sigma = si.map(_.asInstanceOf[DenseMatrix]) val gaussians = Array.tabulate(weight.length){ i => new MultivariateGaussian(mean(i), sigma(i)) } val model = new GaussianMixtureModel(weight, gaussians) - model.predictSoft(data) + model.predictSoft(data).map(Vectors.dense) } - + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -936,6 +935,14 @@ private[spark] object SerDe extends Serializable { out.write(code) } + protected def getBytes(obj: Object): Array[Byte] = { + if (obj.getClass.isArray) { + obj.asInstanceOf[Array[Byte]] + } else { + obj.asInstanceOf[String].getBytes(LATIN1) + } + } + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } @@ -961,7 +968,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(0)) val bb = ByteBuffer.wrap(bytes, 0, bytes.length) bb.order(ByteOrder.nativeOrder()) val db = bb.asDoubleBuffer() @@ -994,7 +1001,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 3) { throw new PickleException("should be 3") } - val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(2)) val n = bytes.length / 8 val values = new Array[Double](n) val order = ByteOrder.nativeOrder() @@ -1031,8 +1038,8 @@ private[spark] object SerDe extends Serializable { throw new PickleException("should be 3") } val size = args(0).asInstanceOf[Int] - val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) - val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) val n = indiceBytes.length / 4 val indices = new Array[Int](n) val values = new Array[Double](n) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..7271809e43880 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -54,7 +54,7 @@ ... def zero(self, value): ... return [0.0] * len(value) ... def addInPlace(self, val1, val2): -... for i in xrange(len(val1)): +... for i in range(len(val1)): ... val1[i] += val2[i] ... return val1 >>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam()) @@ -86,9 +86,13 @@ Exception:... """ +import sys import select import struct -import SocketServer +if sys.version < '3': + import SocketServer +else: + import socketserver as SocketServer import threading from pyspark.cloudpickle import CloudPickler from pyspark.serializers import read_int, PickleSerializer @@ -247,6 +251,7 @@ class AccumulatorServer(SocketServer.TCPServer): def shutdown(self): self.server_shutdown = True SocketServer.TCPServer.shutdown(self) + self.server_close() def _start_update_server(): diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 6b8a8b256a891..3de4615428bb6 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -16,10 +16,15 @@ # import os -import cPickle +import sys import gc from tempfile import NamedTemporaryFile +if sys.version < '3': + import cPickle as pickle +else: + import pickle + unicode = str __all__ = ['Broadcast'] @@ -70,33 +75,19 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None): self._path = path def dump(self, value, f): - if isinstance(value, basestring): - if isinstance(value, unicode): - f.write('U') - value = value.encode('utf8') - else: - f.write('S') - f.write(value) - else: - f.write('P') - cPickle.dump(value, f, 2) + pickle.dump(value, f, 2) f.close() return f.name def load(self, path): with open(path, 'rb', 1 << 20) as f: - flag = f.read(1) - data = f.read() - if flag == 'P': - # cPickle.loads() may create lots of objects, disable GC - # temporary for better performance - gc.disable() - try: - return cPickle.loads(data) - finally: - gc.enable() - else: - return data.decode('utf8') if flag == 'U' else data + # pickle.load() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return pickle.load(f) + finally: + gc.enable() @property def value(self): diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index bb0783555aa77..9ef93071d2e77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -40,164 +40,126 @@ NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ - +from __future__ import print_function import operator import os +import io import pickle import struct import sys import types from functools import partial import itertools -from copy_reg import _extension_registry, _inverted_registry, _extension_cache -import new import dis import traceback -import platform - -PyImp = platform.python_implementation() - -import logging -cloudLog = logging.getLogger("Cloud.Transport") +if sys.version < '3': + from pickle import Pickler + try: + from cStringIO import StringIO + except ImportError: + from StringIO import StringIO + PY3 = False +else: + types.ClassType = type + from pickle import _Pickler as Pickler + from io import BytesIO as StringIO + PY3 = True #relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +STORE_GLOBAL = dis.opname.index('STORE_GLOBAL') +DELETE_GLOBAL = dis.opname.index('DELETE_GLOBAL') +LOAD_GLOBAL = dis.opname.index('LOAD_GLOBAL') GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) - -if PyImp == "PyPy": - # register builtin type in `new` - new.method = types.MethodType - -try: - from cStringIO import StringIO -except ImportError: - from StringIO import StringIO -# These helper functions were copied from PiCloud's util module. def islambda(func): - return getattr(func,'func_name') == '' + return getattr(func,'__name__') == '' -def xrange_params(xrangeobj): - """Returns a 3 element tuple describing the xrange start, step, and len - respectively - Note: Only guarentees that elements of xrange are the same. parameters may - be different. - e.g. xrange(1,1) is interpretted as xrange(0,0); both behave the same - though w/ iteration - """ - - xrange_len = len(xrangeobj) - if not xrange_len: #empty - return (0,1,0) - start = xrangeobj[0] - if xrange_len == 1: #one element - return start, 1, 1 - return (start, xrangeobj[1] - xrangeobj[0], xrange_len) - -#debug variables intended for developer use: -printSerialization = False -printMemoization = False +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k -useForcedImports = True #Should I use forced imports for tracking? +def _builtin_type(name): + return getattr(types, name) -class CloudPickler(pickle.Pickler): +class CloudPickler(Pickler): - dispatch = pickle.Pickler.dispatch.copy() - savedForceImports = False - savedDjangoEnv = False #hack tro transport django environment + dispatch = Pickler.dispatch.copy() - def __init__(self, file, protocol=None, min_size_to_save= 0): - pickle.Pickler.__init__(self,file,protocol) - self.modules = set() #set of modules needed to depickle - self.globals_ref = {} # map ids to dictionary. used to ensure that functions can share global env + def __init__(self, file, protocol=None): + Pickler.__init__(self, file, protocol) + # set of modules to unpickle + self.modules = set() + # map ids to dictionary. used to ensure that functions can share global env + self.globals_ref = {} def dump(self, obj): - # note: not thread safe - # minimal side-effects, so not fixing - recurse_limit = 3000 - base_recurse = sys.getrecursionlimit() - if base_recurse < recurse_limit: - sys.setrecursionlimit(recurse_limit) self.inject_addons() try: - return pickle.Pickler.dump(self, obj) - except RuntimeError, e: + return Pickler.dump(self, obj) + except RuntimeError as e: if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required. - Try _fast_serialization=2 or contact PiCloud support""" + msg = """Could not pickle object as excessively deep recursion required.""" raise pickle.PicklingError(msg) - finally: - new_recurse = sys.getrecursionlimit() - if new_recurse == recurse_limit: - sys.setrecursionlimit(base_recurse) + + def save_memoryview(self, obj): + """Fallback to save_string""" + Pickler.save_string(self, str(obj)) def save_buffer(self, obj): """Fallback to save_string""" - pickle.Pickler.save_string(self,str(obj)) - dispatch[buffer] = save_buffer + Pickler.save_string(self,str(obj)) + if PY3: + dispatch[memoryview] = save_memoryview + else: + dispatch[buffer] = save_buffer - #block broken objects - def save_unsupported(self, obj, pack=None): + def save_unsupported(self, obj): raise pickle.PicklingError("Cannot pickle objects of type %s" % type(obj)) dispatch[types.GeneratorType] = save_unsupported - #python2.6+ supports slice pickling. some py2.5 extensions might as well. We just test it - try: - slice(0,1).__reduce__() - except TypeError: #can't pickle - - dispatch[slice] = save_unsupported - - #itertools objects do not pickle! + # itertools objects do not pickle! for v in itertools.__dict__.values(): if type(v) is type: dispatch[v] = save_unsupported - - def save_dict(self, obj): - """hack fix - If the dict is a global, deal with it in a special way - """ - #print 'saving', obj - if obj is __builtins__: - self.save_reduce(_get_module_builtins, (), obj=obj) - else: - pickle.Pickler.save_dict(self, obj) - dispatch[pickle.DictionaryType] = save_dict - - - def save_module(self, obj, pack=struct.pack): + def save_module(self, obj): """ Save a module as an import """ - #print 'try save import', obj.__name__ self.modules.add(obj) - self.save_reduce(subimport,(obj.__name__,), obj=obj) - dispatch[types.ModuleType] = save_module #new type + self.save_reduce(subimport, (obj.__name__,), obj=obj) + dispatch[types.ModuleType] = save_module - def save_codeobject(self, obj, pack=struct.pack): + def save_codeobject(self, obj): """ Save a code object """ - #print 'try to save codeobj: ', obj - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) + if PY3: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, obj.co_varnames, + obj.co_filename, obj.co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, + obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars + ) self.save_reduce(types.CodeType, args, obj=obj) - dispatch[types.CodeType] = save_codeobject #new type + dispatch[types.CodeType] = save_codeobject - def save_function(self, obj, name=None, pack=struct.pack): + def save_function(self, obj, name=None): """ Registered with the dispatch to handle all function types. Determines what kind of function obj is (e.g. lambda, defined at @@ -205,12 +167,14 @@ def save_function(self, obj, name=None, pack=struct.pack): """ write = self.write - name = obj.__name__ + if name is None: + name = obj.__name__ modname = pickle.whichmodule(obj, name) - #print 'which gives %s %s %s' % (modname, obj, name) + # print('which gives %s %s %s' % (modname, obj, name)) try: themodule = sys.modules[modname] - except KeyError: # eval'd items such as namedtuple give invalid items for their function __module__ + except KeyError: + # eval'd items such as namedtuple give invalid items for their function __module__ modname = '__main__' if modname == '__main__': @@ -221,37 +185,18 @@ def save_function(self, obj, name=None, pack=struct.pack): if getattr(themodule, name, None) is obj: return self.save_global(obj, name) - if not self.savedDjangoEnv: - #hack for django - if we detect the settings module, we transport it - django_settings = os.environ.get('DJANGO_SETTINGS_MODULE', '') - if django_settings: - django_mod = sys.modules.get(django_settings) - if django_mod: - cloudLog.debug('Transporting django settings %s during save of %s', django_mod, name) - self.savedDjangoEnv = True - self.modules.add(django_mod) - write(pickle.MARK) - self.save_reduce(django_settings_load, (django_mod.__name__,), obj=django_mod) - write(pickle.POP_MARK) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: - #Force server to import modules that have been imported in main - modList = None - if themodule is None and not self.savedForceImports: - mainmod = sys.modules['__main__'] - if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): - modList = list(mainmod.___pyc_forcedImports__) - self.savedForceImports = True - self.save_function_tuple(obj, modList) + if islambda(obj) or obj.__code__.co_filename == '' or themodule is None: + #print("save global", islambda(obj), obj.__code__.co_filename, modname, themodule) + self.save_function_tuple(obj) return - else: # func is nested + else: + # func is nested klass = getattr(themodule, name, None) if klass is None or klass is not obj: - self.save_function_tuple(obj, [themodule]) + self.save_function_tuple(obj) return if obj.__dict__: @@ -266,7 +211,7 @@ def save_function(self, obj, name=None, pack=struct.pack): self.memoize(obj) dispatch[types.FunctionType] = save_function - def save_function_tuple(self, func, forced_imports): + def save_function_tuple(self, func): """ Pickles an actual func object. A func comprises: code, globals, defaults, closure, and dict. We @@ -281,19 +226,6 @@ def save_function_tuple(self, func, forced_imports): save = self.save write = self.write - # save the modules (if any) - if forced_imports: - write(pickle.MARK) - save(_modules_to_main) - #print 'forced imports are', forced_imports - - forced_names = map(lambda m: m.__name__, forced_imports) - save((forced_names,)) - - #save((forced_imports,)) - write(pickle.REDUCE) - write(pickle.POP_MARK) - code, f_globals, defaults, closure, dct, base_globals = self.extract_func_data(func) save(_fill_function) # skeleton function updater @@ -318,6 +250,8 @@ def extract_code_globals(co): Find all globals names read or written to by codeblock co """ code = co.co_code + if not PY3: + code = [ord(c) for c in code] names = co.co_names out_names = set() @@ -327,18 +261,18 @@ def extract_code_globals(co): while i < n: op = code[i] - i = i+1 + i += 1 if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + oparg = code[i] + code[i+1] * 256 + extended_arg extended_arg = 0 - i = i+2 + i += 2 if op == EXTENDED_ARG: - extended_arg = oparg*65536L + extended_arg = oparg*65536 if op in GLOBAL_OPS: out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - if co.co_consts: # see if nested function have any global refs + # see if nested function have any global refs + if co.co_consts: for const in co.co_consts: if type(const) is types.CodeType: out_names |= CloudPickler.extract_code_globals(const) @@ -350,46 +284,28 @@ def extract_func_data(self, func): Turn the function into a tuple of data necessary to recreate it: code, globals, defaults, closure, dict """ - code = func.func_code + code = func.__code__ # extract all global ref's - func_global_refs = CloudPickler.extract_code_globals(code) + func_global_refs = self.extract_code_globals(code) # process all variables referenced by global environment f_globals = {} for var in func_global_refs: - #Some names, such as class functions are not global - we don't need them - if func.func_globals.has_key(var): - f_globals[var] = func.func_globals[var] + if var in func.__globals__: + f_globals[var] = func.__globals__[var] # defaults requires no processing - defaults = func.func_defaults - - def get_contents(cell): - try: - return cell.cell_contents - except ValueError, e: #cell is empty error on not yet assigned - raise pickle.PicklingError('Function to be pickled has free variables that are referenced before assignment in enclosing scope') - + defaults = func.__defaults__ # process closure - if func.func_closure: - closure = map(get_contents, func.func_closure) - else: - closure = [] + closure = [c.cell_contents for c in func.__closure__] if func.__closure__ else [] # save the dict - dct = func.func_dict - - if printSerialization: - outvars = ['code: ' + str(code) ] - outvars.append('globals: ' + str(f_globals)) - outvars.append('defaults: ' + str(defaults)) - outvars.append('closure: ' + str(closure)) - print 'function ', func, 'is extracted to: ', ', '.join(outvars) + dct = func.__dict__ - base_globals = self.globals_ref.get(id(func.func_globals), {}) - self.globals_ref[id(func.func_globals)] = base_globals + base_globals = self.globals_ref.get(id(func.__globals__), {}) + self.globals_ref[id(func.__globals__)] = base_globals return (code, f_globals, defaults, closure, dct, base_globals) @@ -400,8 +316,9 @@ def save_builtin_function(self, obj): dispatch[types.BuiltinFunctionType] = save_builtin_function def save_global(self, obj, name=None, pack=struct.pack): - write = self.write - memo = self.memo + if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": + if obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce(_builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) if name is None: name = obj.__name__ @@ -410,98 +327,57 @@ def save_global(self, obj, name=None, pack=struct.pack): if modname is None: modname = pickle.whichmodule(obj, name) - try: - __import__(modname) - themodule = sys.modules[modname] - except (ImportError, KeyError, AttributeError): #should never occur - raise pickle.PicklingError( - "Can't pickle %r: Module %s cannot be found" % - (obj, modname)) - if modname == '__main__': themodule = None - - if themodule: + else: + __import__(modname) + themodule = sys.modules[modname] self.modules.add(themodule) - sendRef = True - typ = type(obj) - #print 'saving', obj, typ - try: - try: #Deal with case when getattribute fails with exceptions - klass = getattr(themodule, name) - except (AttributeError): - if modname == '__builtin__': #new.* are misrepeported - modname = 'new' - __import__(modname) - themodule = sys.modules[modname] - try: - klass = getattr(themodule, name) - except AttributeError, a: - # print themodule, name, obj, type(obj) - raise pickle.PicklingError("Can't pickle builtin %s" % obj) - else: - raise + if hasattr(themodule, name) and getattr(themodule, name) is obj: + return Pickler.save_global(self, obj, name) - except (ImportError, KeyError, AttributeError): - if typ == types.TypeType or typ == types.ClassType: - sendRef = False - else: #we can't deal with this - raise - else: - if klass is not obj and (typ == types.TypeType or typ == types.ClassType): - sendRef = False - if not sendRef: - #note: Third party types might crash this - add better checks! - d = dict(obj.__dict__) #copy dict proxy to a dict - if not isinstance(d.get('__dict__', None), property): # don't extract dict that are properties - d.pop('__dict__',None) - d.pop('__weakref__',None) + typ = type(obj) + if typ is not obj and isinstance(obj, (type, types.ClassType)): + d = dict(obj.__dict__) # copy dict proxy to a dict + if not isinstance(d.get('__dict__', None), property): + # don't extract dict that are properties + d.pop('__dict__', None) + d.pop('__weakref__', None) # hack as __new__ is stored differently in the __dict__ new_override = d.get('__new__', None) if new_override: d['__new__'] = obj.__new__ - self.save_reduce(type(obj),(obj.__name__,obj.__bases__, - d),obj=obj) - #print 'internal reduce dask %s %s' % (obj, d) - return - - if self.proto >= 2: - code = _extension_registry.get((modname, name)) - if code: - assert code > 0 - if code <= 0xff: - write(pickle.EXT1 + chr(code)) - elif code <= 0xffff: - write("%c%c%c" % (pickle.EXT2, code&0xff, code>>8)) - else: - write(pickle.EXT4 + pack("> sys.stderr, 'Cloud not import django settings %s:' % (name) - print_exec(sys.stderr) - if modified_env: - del os.environ['DJANGO_SETTINGS_MODULE'] - else: - #add project directory to sys,path: - if hasattr(module,'__file__'): - dirname = os.path.split(module.__file__)[0] + '/' - sys.path.append(dirname) # restores function attributes def _restore_attr(obj, attr): @@ -851,13 +636,16 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj + def _get_module_builtins(): return pickle.__builtins__ + def print_exec(stream): ei = sys.exc_info() traceback.print_exception(ei[0], ei[1], ei[2], None, stream) + def _modules_to_main(modList): """Force every module in modList to be placed into main""" if not modList: @@ -868,22 +656,16 @@ def _modules_to_main(modList): if type(modname) is str: try: mod = __import__(modname) - except Exception, i: #catch all... - sys.stderr.write('warning: could not import %s\n. Your function may unexpectedly error due to this import failing; \ -A version mismatch is likely. Specific error was:\n' % modname) + except Exception as e: + sys.stderr.write('warning: could not import %s\n. ' + 'Your function may unexpectedly error due to this import failing;' + 'A version mismatch is likely. Specific error was:\n' % modname) print_exec(sys.stderr) else: - setattr(main,mod.__name__, mod) - else: - #REVERSE COMPATIBILITY FOR CLOUD CLIENT 1.5 (WITH EPD) - #In old version actual module was sent - setattr(main,modname.__name__, modname) + setattr(main, mod.__name__, mod) -#object generators: -def _build_xrange(start, step, len): - """Built xrange explicitly""" - return xrange(start, start + step*len, step) +#object generators: def _genpartial(func, args, kwds): if not args: args = () @@ -891,22 +673,26 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) + def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ - func.func_globals.update(globals) - func.func_defaults = defaults - func.func_dict = dict + func.__globals__.update(globals) + func.__defaults__ = defaults + func.__dict__ = dict return func + def _make_cell(value): - return (lambda: value).func_closure[0] + return (lambda: value).__closure__[0] + def _reconstruct_closure(values): return tuple([_make_cell(v) for v in values]) + def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other @@ -928,40 +714,3 @@ def _make_skel_func(code, closures, base_globals = None): def _getobject(modname, attribute): mod = __import__(modname, fromlist=[attribute]) return mod.__dict__[attribute] - -def _generateImage(size, mode, str_rep): - """Generate image from string representation""" - import Image - i = Image.new(mode, size) - i.fromstring(str_rep) - return i - -def _lazyloadImage(fp): - import Image - fp.seek(0) #works in almost any case - return Image.open(fp) - -"""Timeseries""" -def _genTimeSeries(reduce_args, state): - import scikits.timeseries.tseries as ts - from numpy import ndarray - from numpy.ma import MaskedArray - - - time_series = ts._tsreconstruct(*reduce_args) - - #from setstate modified - (ver, shp, typ, isf, raw, msk, flv, dsh, dtm, dtyp, frq, infodict) = state - #print 'regenerating %s' % dtyp - - MaskedArray.__setstate__(time_series, (ver, shp, typ, isf, raw, msk, flv)) - _dates = time_series._dates - #_dates.__setstate__((ver, dsh, typ, isf, dtm, frq)) #use remote typ - ndarray.__setstate__(_dates,(dsh,dtyp, isf, dtm)) - _dates.freq = frq - _dates._cachedinfo.update(dict(full=None, hasdups=None, steps=None, - toobj=None, toord=None, tostr=None)) - # Update the _optinfo dictionary - time_series._optinfo.update(infodict) - return time_series - diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index dc7cd0bce56f3..924da3eecf214 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -44,7 +44,7 @@ >>> conf.get("spark.executorEnv.VAR1") u'value1' ->>> print conf.toDebugString() +>>> print(conf.toDebugString()) spark.executorEnv.VAR1=value1 spark.executorEnv.VAR3=value3 spark.executorEnv.VAR4=value4 @@ -56,6 +56,13 @@ __all__ = ['SparkConf'] +import sys +import re + +if sys.version > '3': + unicode = str + __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__) + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 78dccc40470e3..1dc2fec0ae5c8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os import shutil import sys @@ -32,11 +34,14 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +if sys.version > '3': + xrange = range + __all__ = ['SparkContext'] @@ -133,7 +138,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if sparkHome: self._conf.setSparkHome(sparkHome) if environment: - for key, value in environment.iteritems(): + for key, value in environment.items(): self._conf.setExecutorEnv(key, value) for key, value in DEFAULT_CONFIGS.items(): self._conf.setIfMissing(key, value) @@ -153,6 +158,10 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] self.environment[varName] = v + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + # disable randomness of hash of string in worker, if this is not + # launched by spark-submit + self.environment["PYTHONHASHSEED"] = "0" # Create the Java SparkContext through Py4J self._jsc = jsc or self._initialize_context(self._conf._jconf) @@ -323,7 +332,7 @@ def parallelize(self, c, numSlices=None): start0 = c[0] def getStart(split): - return start0 + (split * size / numSlices) * step + return start0 + int((split * size / numSlices)) * step def f(split, iterator): return xrange(getStart(split), getStart(split + 1), step) @@ -357,6 +366,7 @@ def pickleFile(self, name, minPartitions=None): minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.objectFile(name, minPartitions), self) + @ignore_unicode_prefix def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all @@ -369,7 +379,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello world!") + ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello world!'] @@ -378,6 +388,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer(use_unicode)) + @ignore_unicode_prefix def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system @@ -411,9 +422,9 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): >>> dirPath = os.path.join(tempdir, "files") >>> os.mkdir(dirPath) >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: - ... file1.write("1") + ... _ = file1.write("1") >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: - ... file2.write("2") + ... _ = file2.write("2") >>> textFiles = sc.wholeTextFiles(dirPath) >>> sorted(textFiles.collect()) [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] @@ -456,7 +467,7 @@ def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() if not d: d = {} - for k, v in d.iteritems(): + for k, v in d.items(): jm[k] = v return jm @@ -608,6 +619,7 @@ def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) + @ignore_unicode_prefix def union(self, rdds): """ Build the union of a list of RDDs. @@ -618,7 +630,7 @@ def union(self, rdds): >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello") + ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] @@ -677,7 +689,7 @@ def addFile(self, path): >>> from pyspark import SparkFiles >>> path = os.path.join(tempdir, "test.txt") >>> with open(path, "w") as testFile: - ... testFile.write("100") + ... _ = testFile.write("100") >>> sc.addFile(path) >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: @@ -705,11 +717,13 @@ def addPyFile(self, path): """ self.addFile(path) (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix - if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() def setCheckpointDir(self, dirName): """ @@ -744,7 +758,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): The application can use L{SparkContext.cancelJobGroup} to cancel all running jobs in this group. - >>> import thread, threading + >>> import threading >>> from time import sleep >>> result = "Not Set" >>> lock = threading.Lock() @@ -763,10 +777,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ... sleep(5) ... sc.cancelJobGroup("job_to_cancel") >>> supress = lock.acquire() - >>> supress = thread.start_new_thread(start_job, (10,)) - >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = threading.Thread(target=start_job, args=(10,)).start() + >>> supress = threading.Thread(target=stop_job).start() >>> supress = lock.acquire() - >>> print result + >>> print(result) Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 93885985fe377..7f06d4288c872 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -24,9 +24,10 @@ import traceback import time import gc -from errno import EINTR, ECHILD, EAGAIN +from errno import EINTR, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT + from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -53,8 +54,8 @@ def worker(sock): # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536) exit_code = 0 try: worker_main(infile, outfile) @@ -68,17 +69,6 @@ def worker(sock): return exit_code -# Cleanup zombie children -def cleanup_dead_children(): - try: - while True: - pid, _ = os.waitpid(0, os.WNOHANG) - if not pid: - break - except: - pass - - def manager(): # Create a new process group to corral our children os.setpgid(0, 0) @@ -88,8 +78,12 @@ def manager(): listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() - write_int(listen_port, sys.stdout) - sys.stdout.flush() + + # re-open stdin/stdout in 'wb' mode + stdin_bin = os.fdopen(sys.stdin.fileno(), 'rb', 4) + stdout_bin = os.fdopen(sys.stdout.fileno(), 'wb', 4) + write_int(listen_port, stdout_bin) + stdout_bin.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -101,6 +95,7 @@ def handle_sigterm(*args): shutdown(1) signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + signal.signal(SIGCHLD, SIG_IGN) reuse = os.environ.get("SPARK_REUSE_WORKER") @@ -115,12 +110,9 @@ def handle_sigterm(*args): else: raise - # cleanup in signal handler will cause deadlock - cleanup_dead_children() - if 0 in ready_fds: try: - worker_pid = read_int(sys.stdin) + worker_pid = read_int(stdin_bin) except EOFError: # Spark told us to exit by closing stdin shutdown(0) @@ -145,7 +137,7 @@ def handle_sigterm(*args): time.sleep(1) pid = os.fork() # error here will shutdown daemon else: - outfile = sock.makefile('w') + outfile = sock.makefile(mode='wb') write_int(e.errno, outfile) # Signal that the fork failed outfile.flush() outfile.close() @@ -157,7 +149,7 @@ def handle_sigterm(*args): listen_sock.close() try: # Acknowledge that the fork was successful - outfile = sock.makefile("w") + outfile = sock.makefile(mode="wb") write_int(os.getpid(), outfile) outfile.flush() outfile.close() diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py index bc441f138f7fc..4ef2afe03544f 100644 --- a/python/pyspark/heapq3.py +++ b/python/pyspark/heapq3.py @@ -627,51 +627,49 @@ def merge(iterables, key=None, reverse=False): if key is None: for order, it in enumerate(map(iter, iterables)): try: - next = it.next - h_append([next(), order * direction, next]) + h_append([next(it), order * direction, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - value, order, next = s = h[0] + value, order, it = s = h[0] yield value - s[0] = next() # raises StopIteration when exhausted + s[0] = next(it) # raises StopIteration when exhausted _heapreplace(h, s) # restore heap condition except StopIteration: _heappop(h) # remove empty iterator if h: # fast case when only a single iterator remains - value, order, next = h[0] + value, order, it = h[0] yield value - for value in next.__self__: + for value in it: yield value return for order, it in enumerate(map(iter, iterables)): try: - next = it.next - value = next() - h_append([key(value), order * direction, value, next]) + value = next(it) + h_append([key(value), order * direction, value, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - key_value, order, value, next = s = h[0] + key_value, order, value, it = s = h[0] yield value - value = next() + value = next(it) s[0] = key(value) s[2] = value _heapreplace(h, s) except StopIteration: _heappop(h) if h: - key_value, order, value, next = h[0] + key_value, order, value, it = h[0] yield value - for value in next.__self__: + for value in it: yield value diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2a5e84a7dfdb4..45bc38f7e61f8 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -69,7 +69,7 @@ def preexec_func(): if callback_socket in readable: gateway_connection = callback_socket.accept()[0] # Determine which ephemeral port the server started on: - gateway_port = read_int(gateway_connection.makefile()) + gateway_port = read_int(gateway_connection.makefile(mode="rb")) gateway_connection.close() callback_socket.close() if gateway_port is None: diff --git a/python/pyspark/join.py b/python/pyspark/join.py index c3491defb2b29..94df3990164d6 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -32,6 +32,7 @@ """ from pyspark.resultiterable import ResultIterable +from functools import reduce def _do_python_join(rdd, other, numPartitions, dispatch): diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d7bc09fd77adb..45754bc9d4b10 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -39,10 +39,10 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> lr = LogisticRegression(maxIter=5, regParam=0.01) >>> model = lr.fit(df) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0))]).toDF() - >>> print model.transform(test0).head().prediction + >>> model.transform(test0).head().prediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))]).toDF() - >>> print model.transform(test1).head().prediction + >>> model.transform(test1).head().prediction 1.0 >>> lr.setParams("vector") Traceback (most recent call last): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 263fe2a5bcc41..4e4614b859ac6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,6 +15,7 @@ # limitations under the License. # +from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaTransformer @@ -24,6 +25,7 @@ @inherit_doc +@ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ A tokenizer that converts the input string to lowercase and then @@ -32,15 +34,15 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(text="a b c")]).toDF() >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) >>> # Change a parameter. - >>> print tokenizer.setParams(outputCol="tokens").transform(df).head() + >>> tokenizer.setParams(outputCol="tokens").transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Temporarily modify a parameter. - >>> print tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Must use keyword arguments to specify params. >>> tokenizer.setParams("text") @@ -79,13 +81,13 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") - >>> print hashingTF.transform(df).head().features - (10,[7,8,9],[1.0,1.0,1.0]) - >>> print hashingTF.setParams(outputCol="freqs").transform(df).head().freqs - (10,[7,8,9],[1.0,1.0,1.0]) + >>> hashingTF.transform(df).head().features + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(df, params).head().vector - (5,[2,3,4],[1.0,1.0,1.0]) + >>> hashingTF.transform(df, params).head().vector + SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) """ _java_class = "org.apache.spark.ml.feature.HashingTF" diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 5c62620562a84..9fccb65675185 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -63,8 +63,8 @@ def params(self): uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return filter(lambda attr: isinstance(attr, Param), - [getattr(self, x) for x in dir(self) if x != "params"]) + return list(filter(lambda attr: isinstance(attr, Param), + [getattr(self, x) for x in dir(self) if x != "params"])) def _explain(self, param): """ @@ -185,7 +185,7 @@ def _set(self, **kwargs): """ Sets user-supplied params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.paramMap[getattr(self, param)] = value return self @@ -193,6 +193,6 @@ def _setDefault(self, **kwargs): """ Sets default params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.defaultParamMap[getattr(self, param)] = value return self diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 55f422497672f..6a3192465d66d 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + header = """# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -82,9 +84,9 @@ def get$Name(self): .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": - print header - print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" - print "from pyspark.ml.param import Param, Params\n\n" + print(header) + print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") + print("from pyspark.ml.param import Param, Params\n\n") shared = [ ("maxIter", "max number of iterations", None), ("regParam", "regularization constant", None), @@ -97,4 +99,4 @@ def get$Name(self): code = [] for name, doc, defaultValueStr in shared: code.append(_gen_param_code(name, doc, defaultValueStr)) - print "\n\n\n".join(code) + print("\n\n\n".join(code)) diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index f2ef573fe9f6f..07507b2ad0d05 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -18,6 +18,7 @@ """ Python bindings for MLlib. """ +from __future__ import absolute_import # MLlib currently needs NumPy 1.4+, so complain if lower @@ -29,7 +30,9 @@ 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys -import rand as random -random.__name__ = 'random' -random.RandomRDDs.__module__ = __name__ + '.random' -sys.modules[__name__ + '.random'] = random +from . import rand as random +modname = __name__ + '.random' +random.__name__ = modname +random.RandomRDDs.__module__ = modname +sys.modules[modname] = random +del modname, sys diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 2466e8ac43458..eda0b60f8b1e7 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -510,9 +510,10 @@ def save(self, sc, path): def load(cls, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel.load( sc._jsc.sc(), path) - py_labels = _java2py(sc, java_model.labels()) - py_pi = _java2py(sc, java_model.pi()) - py_theta = _java2py(sc, java_model.theta()) + # Can not unpickle array.array from Pyrolite in Python3 with "bytes" + py_labels = _java2py(sc, java_model.labels(), "latin1") + py_pi = _java2py(sc, java_model.pi(), "latin1") + py_theta = _java2py(sc, java_model.theta(), "latin1") return NaiveBayesModel(py_labels, py_pi, numpy.array(py_theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 464f49aeee3cd..abbb7cf60eece 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,6 +15,12 @@ # limitations under the License. # +import sys +import array as pyarray + +if sys.version > '3': + xrange = range + from numpy import array from pyspark import RDD @@ -55,8 +61,8 @@ class KMeansModel(Saveable, Loader): True >>> model.predict(sparse_data[2]) == model.predict(sparse_data[3]) True - >>> type(model.clusterCenters) - + >>> isinstance(model.clusterCenters, list) + True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -90,7 +96,7 @@ def predict(self, x): return best def save(self, sc, path): - java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_centers = _py2java(sc, [_convert_to_vector(c) for c in self.centers]) java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) java_model.save(sc._jsc.sc(), path) @@ -133,7 +139,7 @@ class GaussianMixtureModel(object): ... 5.7048, 4.6567, 5.5026, ... 4.5605, 5.2043, 6.2734]).reshape(5, 3)) >>> model = GaussianMixture.train(clusterdata_2, 2, convergenceTol=0.0001, - ... maxIterations=150, seed=10) + ... maxIterations=150, seed=10) >>> labels = model.predict(clusterdata_2).collect() >>> labels[0]==labels[1]==labels[2] True @@ -168,8 +174,8 @@ def predictSoft(self, x): if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) membership_matrix = callMLlibFunc("predictSoftGMM", x.map(_convert_to_vector), - self.weights, means, sigmas) - return membership_matrix + _convert_to_vector(self.weights), means, sigmas) + return membership_matrix.map(lambda x: pyarray.array('d', x)) class GaussianMixture(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index a539d2f2846f9..ba6058978880a 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -15,6 +15,11 @@ # limitations under the License. # +import sys +if sys.version >= '3': + long = int + unicode = str + import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -36,7 +41,7 @@ def _new_smart_decode(obj): if isinstance(obj, float): - s = unicode(obj) + s = str(obj) return _float_str_mapping.get(s, s) return _old_smart_decode(obj) @@ -74,15 +79,15 @@ def _py2java(sc, obj): obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, basestring)): + elif isinstance(obj, (int, long, float, bool, bytes, unicode)): pass else: - bytes = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.SerDe.loads(bytes) + data = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(data) return obj -def _java2py(sc, r): +def _java2py(sc, r, encoding="bytes"): if isinstance(r, JavaObject): clsName = r.getClass().getSimpleName() # convert RDD into JavaRDD @@ -102,8 +107,8 @@ def _java2py(sc, r): except Py4JJavaError: pass # not pickable - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) + if isinstance(r, (bytearray, bytes)): + r = PickleSerializer().loads(bytes(r), encoding=encoding) return r diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 8be819aceec24..1140539a24e95 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -23,12 +23,17 @@ import sys import warnings import random +import binascii +if sys.version >= '3': + basestring = str + unicode = str from py4j.protocol import Py4JJavaError -from pyspark import RDD, SparkContext +from pyspark import SparkContext +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector +from pyspark.mllib.linalg import Vectors, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -206,7 +211,7 @@ class HashingTF(object): >>> htf = HashingTF(100) >>> doc = "a a b b c d".split(" ") >>> htf.transform(doc) - SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + SparseVector(100, {...}) """ def __init__(self, numFeatures=1 << 20): """ @@ -360,6 +365,7 @@ def getVectors(self): return self.call("getVectors") +@ignore_unicode_prefix class Word2Vec(object): """ Word2Vec creates vector representation of words in a text corpus. @@ -382,7 +388,7 @@ class Word2Vec(object): >>> sentence = "a b " * 100 + "a c " * 10 >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) - >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] @@ -400,7 +406,7 @@ def __init__(self): self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = random.randint(0, sys.maxint) + self.seed = random.randint(0, sys.maxsize) self.minCount = 5 def setVectorSize(self, vectorSize): @@ -459,7 +465,7 @@ def fit(self, data): raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed), + int(self.numIterations), int(self.seed), int(self.minCount)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 3aa6d79d7093c..628ccc01cf3cc 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -16,12 +16,14 @@ # from pyspark import SparkContext +from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc __all__ = ['FPGrowth', 'FPGrowthModel'] @inherit_doc +@ignore_unicode_prefix class FPGrowthModel(JavaModelWrapper): """ diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index a80320c52d1d0..38b3aa3ad460e 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -25,7 +25,13 @@ import sys import array -import copy_reg + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg +else: + import copy_reg import numpy as np @@ -57,7 +63,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -88,7 +94,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple): + elif type(v) in (array.array, list, tuple, xrange): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -193,7 +199,7 @@ class DenseVector(Vector): DenseVector([1.0, 0.0]) """ def __init__(self, ar): - if isinstance(ar, basestring): + if isinstance(ar, bytes): ar = np.frombuffer(ar, dtype=np.float64) elif not isinstance(ar, np.ndarray): ar = np.array(ar, dtype=np.float64) @@ -321,11 +327,13 @@ def func(self, other): __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") __div__ = _delegate("__div__") + __truediv__ = _delegate("__truediv__") __mod__ = _delegate("__mod__") __radd__ = _delegate("__radd__") __rsub__ = _delegate("__rsub__") __rmul__ = _delegate("__rmul__") __rdiv__ = _delegate("__rdiv__") + __rtruediv__ = _delegate("__rtruediv__") __rmod__ = _delegate("__rmod__") @@ -344,12 +352,12 @@ def __init__(self, size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print SparseVector(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> SparseVector(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" @@ -361,8 +369,8 @@ def __init__(self, size, *args): self.indices = np.array([p[0] for p in pairs], dtype=np.int32) self.values = np.array([p[1] for p in pairs], dtype=np.float64) else: - if isinstance(args[0], basestring): - assert isinstance(args[1], str), "values should be string too" + if isinstance(args[0], bytes): + assert isinstance(args[1], bytes), "values should be string too" if args[0]: self.indices = np.frombuffer(args[0], np.int32) self.values = np.frombuffer(args[1], np.float64) @@ -591,12 +599,12 @@ def sparse(size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> Vectors.sparse(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ return SparseVector(size, *args) @@ -645,7 +653,7 @@ def _convert_to_array(array_like, dtype): """ Convert Matrix attributes which are array-like or buffer to array. """ - if isinstance(array_like, basestring): + if isinstance(array_like, bytes): return np.frombuffer(array_like, dtype=dtype) return np.asarray(array_like, dtype=dtype) @@ -677,7 +685,7 @@ def toArray(self): def toSparse(self): """Convert to SparseMatrix""" indices = np.nonzero(self.values)[0] - colCounts = np.bincount(indices / self.numRows) + colCounts = np.bincount(indices // self.numRows) colPtrs = np.cumsum(np.hstack( (0, colCounts, np.zeros(self.numCols - colCounts.size)))) values = self.values[indices] diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py index 20ee9d78bf5b0..06fbc0eb6aef0 100644 --- a/python/pyspark/mllib/rand.py +++ b/python/pyspark/mllib/rand.py @@ -88,10 +88,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): :param seed: Random seed (default: a random long integer). :return: RDD of float comprised of i.i.d. samples ~ N(0.0, 1.0). - >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - 0.0) < 0.1 True >>> abs(stats.stdev() - 1.0) < 0.1 @@ -118,10 +118,10 @@ def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None): >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L) + >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> from math import sqrt @@ -145,10 +145,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Pois(mean). >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -171,10 +171,10 @@ def exponentialRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Exp(mean). >>> mean = 2.0 - >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -202,10 +202,10 @@ def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None): >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L) + >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> abs(stats.stdev() - expStd) < 0.5 @@ -254,7 +254,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): :return: RDD of Vector with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. >>> import numpy as np - >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -286,8 +286,8 @@ def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \ - 100, 100, seed=1L).collect()) + >>> m = RandomRDDs.logNormalVectorRDD(sc, mean, std, 100, 100, seed=1).collect() + >>> mat = np.matrix(m) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 @@ -315,7 +315,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -345,7 +345,7 @@ def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=No >>> import numpy as np >>> mean = 0.5 - >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -380,8 +380,7 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \ - 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index c5c4c13dae105..80e0a356bb78a 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -15,6 +15,7 @@ # limitations under the License. # +import array from collections import namedtuple from pyspark import SparkContext @@ -104,14 +105,14 @@ def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() assert len(first) == 2, "user_product should be RDD of (user, product)" - user_product = user_product.map(lambda (u, p): (int(u), int(p))) + user_product = user_product.map(lambda u_p: (int(u_p[0]), int(u_p[1]))) return self.call("predict", user_product) def userFeatures(self): - return self.call("getUserFeatures") + return self.call("getUserFeatures").mapValues(lambda v: array.array('d', v)) def productFeatures(self): - return self.call("getProductFeatures") + return self.call("getProductFeatures").mapValues(lambda v: array.array('d', v)) @classmethod def load(cls, sc, path): diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 1d83e9d483f8e..b475be4b4d953 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import RDD +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -38,7 +38,7 @@ def variance(self): return self.call("variance").toArray() def count(self): - return self.call("count") + return int(self.call("count")) def numNonzeros(self): return self.call("numNonzeros").toArray() @@ -78,7 +78,7 @@ def colStats(rdd): >>> cStats.variance() array([ 4., 13., 0., 25.]) >>> cStats.count() - 3L + 3 >>> cStats.numNonzeros() array([ 3., 2., 0., 3.]) >>> cStats.max() @@ -124,20 +124,20 @@ def corr(x, y=None, method=None): >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) - >>> print str(pearsonCorr).replace('nan', 'NaN') + >>> print(str(pearsonCorr).replace('nan', 'NaN')) [[ 1. 0.05564149 NaN 0.40047142] [ 0.05564149 1. NaN 0.91359586] [ NaN NaN 1. NaN] [ 0.40047142 0.91359586 NaN 1. ]] >>> spearmanCorr = Statistics.corr(rdd, method="spearman") - >>> print str(spearmanCorr).replace('nan', 'NaN') + >>> print(str(spearmanCorr).replace('nan', 'NaN')) [[ 1. 0.10540926 NaN 0.4 ] [ 0.10540926 1. NaN 0.9486833 ] [ NaN NaN 1. NaN] [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") - ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... print("Method name as second argument without 'method=' shouldn't be allowed.") ... except TypeError: ... pass """ @@ -153,6 +153,7 @@ def corr(x, y=None, method=None): return callMLlibFunc("corr", x.map(float), y.map(float), method) @staticmethod + @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ .. note:: Experimental @@ -188,11 +189,11 @@ def chiSqTest(observed, expected=None): >>> from pyspark.mllib.linalg import Vectors, Matrices >>> observed = Vectors.dense([4, 6, 5]) >>> pearson = Statistics.chiSqTest(observed) - >>> print pearson.statistic + >>> print(pearson.statistic) 0.4 >>> pearson.degreesOfFreedom 2 - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.8187 >>> pearson.method u'pearson' @@ -202,12 +203,12 @@ def chiSqTest(observed, expected=None): >>> observed = Vectors.dense([21, 38, 43, 80]) >>> expected = Vectors.dense([3, 5, 7, 20]) >>> pearson = Statistics.chiSqTest(observed, expected) - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.0027 >>> data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0] >>> chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) - >>> print round(chi.statistic, 4) + >>> print(round(chi.statistic, 4)) 21.9958 >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), @@ -218,9 +219,9 @@ def chiSqTest(observed, expected=None): ... LabeledPoint(1.0, Vectors.dense([3.5, 40.0])),] >>> rdd = sc.parallelize(data, 4) >>> chi = Statistics.chiSqTest(rdd) - >>> print chi[0].statistic + >>> print(chi[0].statistic) 0.75 - >>> print chi[1].statistic + >>> print(chi[1].statistic) 1.5 """ if isinstance(observed, RDD): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8eaddcf8b9b5e..c6ed5acd1770e 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -72,11 +72,11 @@ class VectorTests(PySparkTestCase): def _test_serialize(self, v): self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) - nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + nv = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): @@ -412,11 +412,11 @@ def test_col_norms(self): self.assertEqual(10, len(summary.normL1())) self.assertEqual(10, len(summary.normL2())) - data2 = self.sc.parallelize(xrange(10)).map(lambda x: Vectors.dense(x)) + data2 = self.sc.parallelize(range(10)).map(lambda x: Vectors.dense(x)) summary2 = Statistics.colStats(data2) self.assertEqual(array([45.0]), summary2.normL1()) import math - expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, xrange(10)))) + expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, range(10)))) self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) @@ -438,11 +438,11 @@ def test_serialization(self): def test_infer_schema(self): sqlCtx = SQLContext(self.sc) rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) - srdd = sqlCtx.inferSchema(rdd) - schema = srdd.schema + df = rdd.toDF() + schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] self.assertEqual(field.dataType, self.udt) - vectors = srdd.map(lambda p: p.features).collect() + vectors = df.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: if isinstance(v, SparseVector): @@ -695,7 +695,7 @@ def test_right_number_of_results(self): class SerDeTest(PySparkTestCase): def test_to_java_object_rdd(self): # SPARK-6660 - data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0) self.assertEqual(_to_java_object_rdd(data).count(), 10) @@ -771,7 +771,7 @@ def test_model_transform(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a7a4d2aaf855b..0fe6e4fabe43a 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -163,14 +163,16 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) - >>> print model, # it already has newline + >>> print(model) DecisionTreeModel classifier of depth 1 with 3 nodes - >>> print model.toDebugString(), # it already has newline + + >>> print(model.toDebugString()) DecisionTreeModel classifier of depth 1 with 3 nodes If (feature 0 <= 0.0) Predict: 0.0 Else (feature 0 > 0.0) Predict: 1.0 + >>> model.predict(array([1.0])) 1.0 >>> model.predict(array([0.0])) @@ -318,9 +320,10 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, 3 >>> model.totalNumNodes() 7 - >>> print model, + >>> print(model) TreeEnsembleModel classifier with 3 trees - >>> print model.toDebugString(), + + >>> print(model.toDebugString()) TreeEnsembleModel classifier with 3 trees Tree 0: @@ -335,6 +338,7 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, Predict: 0.0 Else (feature 0 > 1.0) Predict: 1.0 + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) @@ -483,8 +487,9 @@ def trainClassifier(cls, data, categoricalFeaturesInfo, 100 >>> model.totalNumNodes() 300 - >>> print model, # it already has newline + >>> print(model) # it already has newline TreeEnsembleModel classifier with 100 trees + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index c5c3468eb95e9..16a90db146ef0 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -15,10 +15,14 @@ # limitations under the License. # +import sys import numpy as np import warnings -from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc +if sys.version > '3': + xrange = range + +from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector @@ -94,22 +98,16 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> from pyspark.mllib.util import MLUtils >>> from pyspark.mllib.regression import LabeledPoint >>> tempFile = NamedTemporaryFile(delete=True) - >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") + >>> _ = tempFile.write(b"+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() - >>> type(examples[0]) == LabeledPoint - True - >>> print examples[0] - (1.0,(6,[0,2,4],[1.0,2.0,3.0])) - >>> type(examples[1]) == LabeledPoint - True - >>> print examples[1] - (-1.0,(6,[],[])) - >>> type(examples[2]) == LabeledPoint - True - >>> print examples[2] - (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) + >>> examples[0] + LabeledPoint(1.0, (6,[0,2,4],[1.0,2.0,3.0])) + >>> examples[1] + LabeledPoint(-1.0, (6,[],[])) + >>> examples[2] + LabeledPoint(-1.0, (6,[1,3,5],[4.0,5.0,6.0])) """ from pyspark.mllib.regression import LabeledPoint if multiclass is not None: diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 4408996db0790..d18daaabfcb3c 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -84,11 +84,11 @@ class Profiler(object): >>> from pyspark import BasicProfiler >>> class MyCustomProfiler(BasicProfiler): ... def show(self, id): - ... print "My custom profiles for RDD:%s" % id + ... print("My custom profiles for RDD:%s" % id) ... >>> conf = SparkConf().set("spark.python.profile", "true") >>> sc = SparkContext('local', 'test', conf=conf, profiler_cls=MyCustomProfiler) - >>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10) + >>> sc.parallelize(range(1000)).map(lambda x: 2 * x).take(10) [0, 2, 4, 6, 8, 10, 12, 14, 16, 18] >>> sc.show_profiles() My custom profiles for RDD:1 @@ -111,9 +111,9 @@ def show(self, id): """ Print the profile stats to stdout, id is the RDD id """ stats = self.stats() if stats: - print "=" * 60 - print "Profile of RDD" % id - print "=" * 60 + print("=" * 60) + print("Profile of RDD" % id) + print("=" * 60) stats.sort_stats("time", "cumulative").print_stats() def dump(self, id, path): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 93e658eded9e2..d9cdbb666f92a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -16,21 +16,29 @@ # import copy -from collections import defaultdict -from itertools import chain, ifilter, imap -import operator import sys +import os +import re +import operator import shlex -from subprocess import Popen, PIPE -from tempfile import NamedTemporaryFile -from threading import Thread import warnings import heapq import bisect import random import socket +from subprocess import Popen, PIPE +from tempfile import NamedTemporaryFile +from threading import Thread +from collections import defaultdict +from itertools import chain +from functools import reduce from math import sqrt, log, isinf, isnan, pow, ceil +if sys.version > '3': + basestring = unicode = str +else: + from itertools import imap as map, ifilter as filter + from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -50,20 +58,21 @@ __all__ = ["RDD"] -# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized -# hash for string def portable_hash(x): """ - This function returns consistant hash code for builtin types, especially + This function returns consistent hash code for builtin types, especially for None and tuple with None. - The algrithm is similar to that one used by CPython 2.7 + The algorithm is similar to that one used by CPython 2.7 >>> portable_hash(None) 0 >>> portable_hash((None, 1)) & 0xffffffff 219750521 """ + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED") + if x is None: return 0 if isinstance(x, tuple): @@ -71,7 +80,7 @@ def portable_hash(x): for i in x: h ^= portable_hash(i) h *= 1000003 - h &= sys.maxint + h &= sys.maxsize h ^= len(x) if h == -1: h = -2 @@ -123,6 +132,19 @@ def _load_from_socket(port, serializer): sock.close() +def ignore_unicode_prefix(f): + """ + Ignore the 'u' prefix of string in doc tests, to make it works + in both python 2 and 3 + """ + if sys.version >= '3': + # the representation of unicode string in Python 3 does not have prefix 'u', + # so remove the prefix 'u' for doc tests + literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) + f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) + return f + + class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -251,7 +273,7 @@ def map(self, f, preservesPartitioning=False): [('a', 1), ('b', 1), ('c', 1)] """ def func(_, iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -266,7 +288,7 @@ def flatMap(self, f, preservesPartitioning=False): [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -329,7 +351,7 @@ def filter(self, f): [2, 4] """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def distinct(self, numPartitions=None): @@ -341,7 +363,7 @@ def distinct(self, numPartitions=None): """ return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x, numPartitions) \ - .map(lambda (x, _): x) + .map(lambda x: x[0]) def sample(self, withReplacement, fraction, seed=None): """ @@ -354,8 +376,8 @@ def sample(self, withReplacement, fraction, seed=None): :param seed: seed for the random number generator >>> rdd = sc.parallelize(range(100), 4) - >>> rdd.sample(False, 0.1, 81).count() - 10 + >>> 6 <= rdd.sample(False, 0.1, 81).count() <= 14 + True """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) @@ -368,12 +390,14 @@ def randomSplit(self, weights, seed=None): :param seed: random seed :return: split RDDs in a list - >>> rdd = sc.parallelize(range(5), 1) + >>> rdd = sc.parallelize(range(500), 1) >>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17) - >>> rdd1.collect() - [1, 3] - >>> rdd2.collect() - [0, 2, 4] + >>> len(rdd1.collect() + rdd2.collect()) + 500 + >>> 150 < rdd1.count() < 250 + True + >>> 250 < rdd2.count() < 350 + True """ s = float(sum(weights)) cweights = [0.0] @@ -416,7 +440,7 @@ def takeSample(self, withReplacement, num, seed=None): rand.shuffle(samples) return samples - maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) + maxSampleSize = sys.maxsize - int(numStDev * sqrt(sys.maxsize)) if num > maxSampleSize: raise ValueError( "Sample size cannot be greater than %d." % maxSampleSize) @@ -430,7 +454,7 @@ def takeSample(self, withReplacement, num, seed=None): # See: scala/spark/RDD.scala while len(samples) < num: # TODO: add log warning for when more than one iteration was run - seed = rand.randint(0, sys.maxint) + seed = rand.randint(0, sys.maxsize) samples = self.sample(withReplacement, fraction, seed).collect() rand.shuffle(samples) @@ -507,7 +531,7 @@ def intersection(self, other): """ return self.map(lambda v: (v, None)) \ .cogroup(other.map(lambda v: (v, None))) \ - .filter(lambda (k, vs): all(vs)) \ + .filter(lambda k_vs: all(k_vs[1])) \ .keys() def _reserialize(self, serializer=None): @@ -549,7 +573,7 @@ def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=p def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda k_v: keyfunc(k_v[0]), reverse=(not ascending))) return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) @@ -579,7 +603,7 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda kv: keyfunc(kv[0]), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: @@ -594,12 +618,12 @@ def sortPartition(iterator): return self # empty RDD maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() + samples = self.sample(False, fraction, 1).map(lambda kv: kv[0]).collect() samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary - bounds = [samples[len(samples) * (i + 1) / numPartitions] + bounds = [samples[int(len(samples) * (i + 1) / numPartitions)] for i in range(0, numPartitions - 1)] def rangePartitioner(k): @@ -662,12 +686,13 @@ def groupBy(self, f, numPartitions=None): """ return self.map(lambda x: (f(x), x)).groupByKey(numPartitions) + @ignore_unicode_prefix def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() - ['1', '2', '', '3'] + [u'1', u'2', u'', u'3'] """ def func(iterator): pipe = Popen( @@ -675,17 +700,18 @@ def func(iterator): def pipe_objs(out): for obj in iterator: - out.write(str(obj).rstrip('\n') + '\n') + s = str(obj).rstrip('\n') + '\n' + out.write(s.encode('utf-8')) out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) + return (x.rstrip(b'\n').decode('utf-8') for x in iter(pipe.stdout.readline, b'')) return self.mapPartitions(func) def foreach(self, f): """ Applies a function to all elements of this RDD. - >>> def f(x): print x + >>> def f(x): print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ def processPartition(iterator): @@ -700,7 +726,7 @@ def foreachPartition(self, f): >>> def f(iterator): ... for x in iterator: - ... print x + ... print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f) """ def func(it): @@ -874,7 +900,7 @@ def aggregatePartition(iterator): # aggregation. while numPartitions > scale + numPartitions / scale: numPartitions /= scale - curNumPartitions = numPartitions + curNumPartitions = int(numPartitions) def mapPartition(i, iterator): for obj in iterator: @@ -984,7 +1010,7 @@ def histogram(self, buckets): (('a', 'b', 'c'), [2, 2]) """ - if isinstance(buckets, (int, long)): + if isinstance(buckets, int): if buckets < 1: raise ValueError("number of buckets must be >= 1") @@ -1020,6 +1046,7 @@ def minmax(a, b): raise ValueError("Can not generate buckets with infinite value") # keep them as integer if possible + inc = int(inc) if inc * buckets != maxv - minv: inc = (maxv - minv) * 1.0 / buckets @@ -1137,7 +1164,7 @@ def countPartition(iterator): yield counts def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) @@ -1378,8 +1405,8 @@ def saveAsPickleFile(self, path, batchSize=10): >>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile.close() >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) - >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) - [1, 2, 'rdd', 'spark'] + >>> sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect()) + ['1', '2', 'rdd', 'spark'] """ if batchSize == 0: ser = AutoBatchedSerializer(PickleSerializer()) @@ -1387,6 +1414,7 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) + @ignore_unicode_prefix def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. @@ -1418,12 +1446,13 @@ def saveAsTextFile(self, path, compressionCodecClass=None): >>> codec = "org.apache.hadoop.io.compress.GzipCodec" >>> sc.parallelize(['foo', 'bar']).saveAsTextFile(tempFile3.name, codec) >>> from fileinput import input, hook_compressed - >>> ''.join(sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed))) - 'bar\\nfoo\\n' + >>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed)) + >>> b''.join(result).decode('utf-8') + u'bar\\nfoo\\n' """ def func(split, iterator): for x in iterator: - if not isinstance(x, basestring): + if not isinstance(x, (unicode, bytes)): x = unicode(x) if isinstance(x, unicode): x = x.encode("utf-8") @@ -1458,7 +1487,7 @@ def keys(self): >>> m.collect() [1, 3] """ - return self.map(lambda (k, v): k) + return self.map(lambda x: x[0]) def values(self): """ @@ -1468,7 +1497,7 @@ def values(self): >>> m.collect() [2, 4] """ - return self.map(lambda (k, v): v) + return self.map(lambda x: x[1]) def reduceByKey(self, func, numPartitions=None): """ @@ -1507,7 +1536,7 @@ def reducePartition(iterator): yield m def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] = func(m1[k], v) if k in m1 else v return m1 return self.mapPartitions(reducePartition).reduce(mergeMaps) @@ -1604,8 +1633,8 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x)) >>> sets = pairs.partitionBy(2).glom().collect() - >>> set(sets[0]).intersection(set(sets[1])) - set([]) + >>> len(set(sets[0]).intersection(set(sets[1]))) + 0 """ if numPartitions is None: numPartitions = self._defaultReducePartitions() @@ -1637,22 +1666,22 @@ def add_shuffle_key(split, iterator): if (c % 1000 == 0 and get_used_memory() > limit or c > batch): n, size = len(buckets), 0 - for split in buckets.keys(): + for split in list(buckets.keys()): yield pack_long(split) d = outputSerializer.dumps(buckets[split]) del buckets[split] yield d size += len(d) - avg = (size / n) >> 20 + avg = int(size / n) >> 20 # let 1M < avg < 10M if avg < 1: batch *= 1.5 elif avg > 10: - batch = max(batch / 1.5, 1) + batch = max(int(batch / 1.5), 1) c = 0 - for split, items in buckets.iteritems(): + for split, items in buckets.items(): yield pack_long(split) yield outputSerializer.dumps(items) @@ -1707,7 +1736,7 @@ def combineLocally(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1716,7 +1745,7 @@ def _mergeCombiners(iterator): merger = ExternalMerger(agg, memory, serializer) \ if spill else InMemoryMerger(agg) merger.mergeCombiners(iterator) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True) @@ -1745,7 +1774,7 @@ def foldByKey(self, zeroValue, func, numPartitions=None): >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> from operator import add - >>> rdd.foldByKey(0, add).collect() + >>> sorted(rdd.foldByKey(0, add).collect()) [('a', 2), ('b', 1)] """ def createZero(): @@ -1769,10 +1798,10 @@ def groupByKey(self, numPartitions=None): sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. - >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().mapValues(len).collect()) + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.groupByKey().mapValues(len).collect()) [('a', 2), ('b', 1)] - >>> sorted(x.groupByKey().mapValues(list).collect()) + >>> sorted(rdd.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ def createCombiner(x): @@ -1795,7 +1824,7 @@ def combine(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combine, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1804,7 +1833,7 @@ def groupByKey(it): merger = ExternalGroupBy(agg, memory, serializer)\ if spill else InMemoryMerger(agg) merger.mergeCombiners(it) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) @@ -1819,7 +1848,7 @@ def flatMapValues(self, f): >>> x.flatMapValues(f).collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')] """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def mapValues(self, f): @@ -1833,7 +1862,7 @@ def mapValues(self, f): >>> x.mapValues(f).collect() [('a', 3), ('b', 1)] """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def groupWith(self, other, *others): @@ -1844,8 +1873,7 @@ def groupWith(self, other, *others): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) >>> z = sc.parallelize([("b", 42)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1]), list(y[2]), list(y[3])))), \ - sorted(list(w.groupWith(x, y, z).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))] [('a', ([5], [1], [2], [])), ('b', ([6], [4], [], [42]))] """ @@ -1860,7 +1888,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))] [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup((self, other), numPartitions) @@ -1896,8 +1924,9 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - def filter_func((key, vals)): - return vals[0] and not vals[1] + def filter_func(pair): + key, (val1, val2) = pair + return val1 and not val2 return self.cogroup(other, numPartitions).filter(filter_func).flatMapValues(lambda x: x[0]) def subtract(self, other, numPartitions=None): @@ -1919,8 +1948,8 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) - [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] + >>> [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())] + [(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])] """ return self.map(lambda x: (f(x), x)) @@ -2049,17 +2078,18 @@ def name(self): """ Return the name of this RDD. """ - name_ = self._jrdd.name() - if name_: - return name_.encode('utf-8') + n = self._jrdd.name() + if n: + return n + @ignore_unicode_prefix def setName(self, name): """ Assign a name to this RDD. - >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() - 'RDD1' + u'RDD1' """ self._jrdd.setName(name) return self @@ -2121,7 +2151,7 @@ def lookup(self, key): >>> sorted.lookup(1024) [] """ - values = self.filter(lambda (k, v): k == key).values() + values = self.filter(lambda kv: kv[0] == key).values() if self.partitioner is not None: return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)], False) @@ -2159,7 +2189,7 @@ def sumApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) + >>> r = sum(range(1000)) >>> (rdd.sumApprox(1000) - r) / r < 0.05 True """ @@ -2176,7 +2206,7 @@ def meanApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) / 1000.0 + >>> r = sum(range(1000)) / 1000.0 >>> (rdd.meanApprox(1000) - r) / r < 0.05 True """ @@ -2201,10 +2231,10 @@ def countApproxDistinct(self, relativeSD=0.05): It must be greater than 0.000017. >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() - >>> 950 < n < 1050 + >>> 900 < n < 1100 True >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() - >>> 18 < n < 22 + >>> 16 < n < 24 True """ if relativeSD < 0.000017: @@ -2223,8 +2253,7 @@ def toLocalIterator(self): >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ - partitions = xrange(self.getNumPartitions()) - for partition in partitions: + for partition in range(self.getNumPartitions()): rows = self.context.runJob(self, lambda x: x, [partition]) for row in rows: yield row diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 459e1427803cb..fe8f87324804b 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -23,7 +23,7 @@ class RDDSamplerBase(object): def __init__(self, withReplacement, seed=None): - self._seed = seed if seed is not None else random.randint(0, sys.maxint) + self._seed = seed if seed is not None else random.randint(0, sys.maxsize) self._withReplacement = withReplacement self._random = None @@ -31,7 +31,7 @@ def initRandomGenerator(self, split): self._random = random.Random(self._seed ^ split) # mixing because the initial seeds are close to each other - for _ in xrange(10): + for _ in range(10): self._random.randint(0, 1) def getUniformSample(self): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 4afa82f4b2973..d8cdcda3a3783 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -49,16 +49,24 @@ >>> sc.stop() """ -import cPickle -from itertools import chain, izip, product +import sys +from itertools import chain, product import marshal import struct -import sys import types import collections import zlib import itertools +if sys.version < '3': + import cPickle as pickle + protocol = 2 + from itertools import izip as zip +else: + import pickle + protocol = 3 + xrange = range + from pyspark import cloudpickle @@ -97,7 +105,7 @@ def _load_stream_without_unbatching(self, stream): # subclasses should override __eq__ as appropriate. def __eq__(self, other): - return isinstance(other, self.__class__) + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not self.__eq__(other) @@ -212,10 +220,6 @@ def load_stream(self, stream): def _load_stream_without_unbatching(self, stream): return self.serializer.load_stream(stream) - def __eq__(self, other): - return (isinstance(other, BatchedSerializer) and - other.serializer == self.serializer and other.batchSize == self.batchSize) - def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) @@ -233,14 +237,14 @@ def __init__(self, serializer, batchSize=10): def _batched(self, iterator): n = self.batchSize for key, values in iterator: - for i in xrange(0, len(values), n): + for i in range(0, len(values), n): yield key, values[i:i + n] def load_stream(self, stream): return self.serializer.load_stream(stream) def __repr__(self): - return "FlattenedValuesSerializer(%d)" % self.batchSize + return "FlattenedValuesSerializer(%s, %d)" % (self.serializer, self.batchSize) class AutoBatchedSerializer(BatchedSerializer): @@ -270,12 +274,8 @@ def dump_stream(self, iterator, stream): elif size > best * 10 and batch > 1: batch /= 2 - def __eq__(self, other): - return (isinstance(other, AutoBatchedSerializer) and - other.serializer == self.serializer and other.bestSize == self.bestSize) - def __repr__(self): - return "AutoBatchedSerializer(%s)" % str(self.serializer) + return "AutoBatchedSerializer(%s)" % self.serializer class CartesianDeserializer(FramedSerializer): @@ -285,6 +285,7 @@ class CartesianDeserializer(FramedSerializer): """ def __init__(self, key_ser, val_ser): + FramedSerializer.__init__(self) self.key_ser = key_ser self.val_ser = val_ser @@ -293,7 +294,7 @@ def prepare_keys_values(self, stream): val_stream = self.val_ser._load_stream_without_unbatching(stream) key_is_batched = isinstance(self.key_ser, BatchedSerializer) val_is_batched = isinstance(self.val_ser, BatchedSerializer) - for (keys, vals) in izip(key_stream, val_stream): + for (keys, vals) in zip(key_stream, val_stream): keys = keys if key_is_batched else [keys] vals = vals if val_is_batched else [vals] yield (keys, vals) @@ -303,10 +304,6 @@ def load_stream(self, stream): for pair in product(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, CartesianDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "CartesianDeserializer(%s, %s)" % \ (str(self.key_ser), str(self.val_ser)) @@ -318,22 +315,14 @@ class PairDeserializer(CartesianDeserializer): Deserializes the JavaRDD zip() of two PythonRDDs. """ - def __init__(self, key_ser, val_ser): - self.key_ser = key_ser - self.val_ser = val_ser - def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): if len(keys) != len(vals): raise ValueError("Can not deserialize RDD with different number of items" " in pair: (%d, %d)" % (len(keys), len(vals))) - for pair in izip(keys, vals): + for pair in zip(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, PairDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "PairDeserializer(%s, %s)" % (str(self.key_ser), str(self.val_ser)) @@ -382,8 +371,8 @@ def _hijack_namedtuple(): global _old_namedtuple # or it will put in closure def _copy_func(f): - return types.FunctionType(f.func_code, f.func_globals, f.func_name, - f.func_defaults, f.func_closure) + return types.FunctionType(f.__code__, f.__globals__, f.__name__, + f.__defaults__, f.__closure__) _old_namedtuple = _copy_func(collections.namedtuple) @@ -392,15 +381,15 @@ def namedtuple(*args, **kwargs): return _hack_namedtuple(cls) # replace namedtuple with new one - collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple - collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple - collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__globals__["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.__globals__["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.__code__ = namedtuple.__code__ collections.namedtuple.__hijack = 1 # hack the cls already generated by namedtuple # those created in other module can be pickled as normal, # so only hack those in __main__ module - for n, o in sys.modules["__main__"].__dict__.iteritems(): + for n, o in sys.modules["__main__"].__dict__.items(): if (type(o) is type and o.__base__ is tuple and hasattr(o, "_fields") and "__reduce__" not in o.__dict__): @@ -413,7 +402,7 @@ def namedtuple(*args, **kwargs): class PickleSerializer(FramedSerializer): """ - Serializes objects using Python's cPickle serializer: + Serializes objects using Python's pickle serializer: http://docs.python.org/2/library/pickle.html @@ -422,10 +411,14 @@ class PickleSerializer(FramedSerializer): """ def dumps(self, obj): - return cPickle.dumps(obj, 2) + return pickle.dumps(obj, protocol) - def loads(self, obj): - return cPickle.loads(obj) + if sys.version >= '3': + def loads(self, obj, encoding="bytes"): + return pickle.loads(obj, encoding=encoding) + else: + def loads(self, obj, encoding=None): + return pickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -454,7 +447,7 @@ def loads(self, obj): class AutoSerializer(FramedSerializer): """ - Choose marshal or cPickle as serialization protocol automatically + Choose marshal or pickle as serialization protocol automatically """ def __init__(self): @@ -463,19 +456,19 @@ def __init__(self): def dumps(self, obj): if self._type is not None: - return 'P' + cPickle.dumps(obj, -1) + return b'P' + pickle.dumps(obj, -1) try: - return 'M' + marshal.dumps(obj) + return b'M' + marshal.dumps(obj) except Exception: - self._type = 'P' - return 'P' + cPickle.dumps(obj, -1) + self._type = b'P' + return b'P' + pickle.dumps(obj, -1) def loads(self, obj): _type = obj[0] - if _type == 'M': + if _type == b'M': return marshal.loads(obj[1:]) - elif _type == 'P': - return cPickle.loads(obj[1:]) + elif _type == b'P': + return pickle.loads(obj[1:]) else: raise ValueError("invalid sevialization type: %s" % _type) @@ -495,8 +488,8 @@ def dumps(self, obj): def loads(self, obj): return self.serializer.loads(zlib.decompress(obj)) - def __eq__(self, other): - return isinstance(other, CompressedSerializer) and self.serializer == other.serializer + def __repr__(self): + return "CompressedSerializer(%s)" % self.serializer class UTF8Deserializer(Serializer): @@ -505,7 +498,7 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ - def __init__(self, use_unicode=False): + def __init__(self, use_unicode=True): self.use_unicode = use_unicode def loads(self, stream): @@ -526,13 +519,13 @@ def load_stream(self, stream): except EOFError: return - def __eq__(self, other): - return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode + def __repr__(self): + return "UTF8Deserializer(%s)" % self.use_unicode def read_long(stream): length = stream.read(8) - if length == "": + if not length: raise EOFError return struct.unpack("!q", length)[0] @@ -547,7 +540,7 @@ def pack_long(value): def read_int(stream): length = stream.read(4) - if length == "": + if not length: raise EOFError return struct.unpack("!i", length)[0] diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 81aa970a32f76..144cdf0b0cdd5 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -21,13 +21,6 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ -import sys -if sys.version_info[0] != 2: - print("Error: Default Python used is Python%s" % sys.version_info.major) - print("\tSet env variable PYSPARK_PYTHON to Python2 binary and re-run it.") - sys.exit(1) - - import atexit import os import platform @@ -53,9 +46,14 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = sqlContext = HiveContext(sc) + sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = sqlContext = SQLContext(sc) + sqlContext = SQLContext(sc) +except TypeError: + sqlContext = SQLContext(sc) + +# for compatibility +sqlCtx = sqlContext print("""Welcome to ____ __ diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 8a6fc627eb383..b54baa57ec28a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -78,8 +78,8 @@ def _get_local_dirs(sub): # global stats -MemoryBytesSpilled = 0L -DiskBytesSpilled = 0L +MemoryBytesSpilled = 0 +DiskBytesSpilled = 0 class Aggregator(object): @@ -126,7 +126,7 @@ def mergeCombiners(self, iterator): """ Merge the combined items by mergeCombiner """ raise NotImplementedError - def iteritems(self): + def items(self): """ Return the merged items ad iterator """ raise NotImplementedError @@ -156,9 +156,9 @@ def mergeCombiners(self, iterator): for k, v in iterator: d[k] = comb(d[k], v) if k in d else v - def iteritems(self): - """ Return the merged items as iterator """ - return self.data.iteritems() + def items(self): + """ Return the merged items ad iterator """ + return iter(self.data.items()) def _compressed_serializer(self, serializer=None): @@ -208,15 +208,15 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N))) + >>> merger.mergeValues(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N))) + >>> merger.mergeCombiners(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 """ @@ -335,10 +335,10 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch @@ -354,9 +354,9 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(iter(self.pdata[i].items()), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -364,10 +364,10 @@ def _spill(self): gc.collect() # release the memory as much as possible MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 - def iteritems(self): + def items(self): """ Return all merged items as iterator """ if not self.pdata and not self.spills: - return self.data.iteritems() + return iter(self.data.items()) return self._external_items() def _external_items(self): @@ -398,7 +398,8 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) # limit the total partitions if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS @@ -408,7 +409,7 @@ def _merged_items(self, index): gc.collect() # release the memory as much as possible return self._recursive_merged_items(index) - return self.data.iteritems() + return self.data.items() def _recursive_merged_items(self, index): """ @@ -426,7 +427,8 @@ def _recursive_merged_items(self, index): for j in range(self.spills): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - m.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, 'rb') as f: + m.mergeCombiners(self.serializer.load_stream(f), 0) if get_used_memory() > limit: m._spill() @@ -451,7 +453,7 @@ class ExternalSorter(object): >>> sorter = ExternalSorter(1) # 1M >>> import random - >>> l = range(1024) + >>> l = list(range(1024)) >>> random.shuffle(l) >>> sorted(l) == list(sorter.sorted(l)) True @@ -499,9 +501,16 @@ def sorted(self, iterator, key=None, reverse=False): # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) - with open(path, 'w') as f: + with open(path, 'wb') as f: self.serializer.dump_stream(current_chunk, f) - chunks.append(self.serializer.load_stream(open(path))) + + def load(f): + for v in self.serializer.load_stream(f): + yield v + # close the file explicit once we consume all the items + # to avoid ResourceWarning in Python3 + f.close() + chunks.append(load(open(path, 'rb'))) current_chunk = [] gc.collect() limit = self._next_limit() @@ -527,7 +536,7 @@ class ExternalList(object): ExternalList can have many items which cannot be hold in memory in the same time. - >>> l = ExternalList(range(100)) + >>> l = ExternalList(list(range(100))) >>> len(l) 100 >>> l.append(10) @@ -555,11 +564,11 @@ def __init__(self, values): def __getstate__(self): if self._file is not None: self._file.flush() - f = os.fdopen(os.dup(self._file.fileno())) - f.seek(0) - serialized = f.read() + with os.fdopen(os.dup(self._file.fileno()), "rb") as f: + f.seek(0) + serialized = f.read() else: - serialized = '' + serialized = b'' return self.values, self.count, serialized def __setstate__(self, item): @@ -575,7 +584,7 @@ def __iter__(self): if self._file is not None: self._file.flush() # read all items from disks first - with os.fdopen(os.dup(self._file.fileno()), 'r') as f: + with os.fdopen(os.dup(self._file.fileno()), 'rb') as f: f.seek(0) for v in self._ser.load_stream(f): yield v @@ -598,11 +607,16 @@ def _open_file(self): d = dirs[id(self) % len(dirs)] if not os.path.exists(d): os.makedirs(d) - p = os.path.join(d, str(id)) - self._file = open(p, "w+", 65536) + p = os.path.join(d, str(id(self))) + self._file = open(p, "wb+", 65536) self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) os.unlink(p) + def __del__(self): + if self._file: + self._file.close() + self._file = None + def _spill(self): """ dump the values into disk """ global MemoryBytesSpilled, DiskBytesSpilled @@ -651,33 +665,28 @@ class GroupByKey(object): """ Group a sorted iterator as [(k1, it1), (k2, it2), ...] - >>> k = [i/3 for i in range(6)] + >>> k = [i // 3 for i in range(6)] >>> v = [[i] for i in range(6)] - >>> g = GroupByKey(iter(zip(k, v))) + >>> g = GroupByKey(zip(k, v)) >>> [(k, list(it)) for k, it in g] [(0, [0, 1, 2]), (1, [3, 4, 5])] """ def __init__(self, iterator): - self.iterator = iter(iterator) - self.next_item = None + self.iterator = iterator def __iter__(self): - return self - - def next(self): - key, value = self.next_item if self.next_item else next(self.iterator) - values = ExternalListOfList([value]) - try: - while True: - k, v = next(self.iterator) - if k != key: - self.next_item = (k, v) - break + key, values = None, None + for k, v in self.iterator: + if values is not None and k == key: values.append(v) - except StopIteration: - self.next_item = None - return key, values + else: + if values is not None: + yield (key, values) + key = k + values = ExternalListOfList([v]) + if values is not None: + yield (key, values) class ExternalGroupBy(ExternalMerger): @@ -744,7 +753,7 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] # If the number of keys is small, then the overhead of sort is small @@ -756,7 +765,7 @@ def _spill(self): h = self._partition(k) self.serializer.dump_stream([(k, self.data[k])], streams[h]) else: - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) self.serializer.dump_stream([(k, v)], streams[h]) @@ -771,14 +780,14 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch if self._sorted: # sort by key only (stable) - sorted_items = sorted(self.pdata[i].iteritems(), key=operator.itemgetter(0)) + sorted_items = sorted(self.pdata[i].items(), key=operator.itemgetter(0)) self.serializer.dump_stream(sorted_items, f) else: - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(self.pdata[i].items(), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -792,7 +801,7 @@ def _merged_items(self, index): # if the memory can not hold all the partition, # then use sort based merge. Because of compression, # the data on disks will be much smaller than needed memory - if (size >> 20) >= self.memory_limit / 10: + if size >= self.memory_limit << 17: # * 1M / 8 return self._merge_sorted_items(index) self.data = {} @@ -800,15 +809,18 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) - return self.data.iteritems() + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) + return self.data.items() def _merge_sorted_items(self, index): """ load a partition from disk, then sort and group by key """ def load_partition(j): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - return self.serializer.load_stream(open(p, 'r', 65536)) + with open(p, 'rb', 65536) as f: + for v in self.serializer.load_stream(f): + yield v disk_items = [load_partition(j) for j in range(self.spills)] diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 65abb24eed823..6d54b9e49ed10 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -37,9 +37,22 @@ - L{types} List of data types available. """ +from __future__ import absolute_import + +# fix the module name conflict for Python 3+ +import sys +from . import _types as types +modname = __name__ + '.types' +types.__name__ = modname +# update the __module__ for all objects, make them picklable +for v in types.__dict__.values(): + if hasattr(v, "__module__") and v.__module__.endswith('._types'): + v.__module__ = modname +sys.modules[modname] = types +del modname, sys -from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row +from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD, DataFrameNaFunctions __all__ = [ diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/_types.py similarity index 97% rename from python/pyspark/sql/types.py rename to python/pyspark/sql/_types.py index ef76d84c00481..492c0cbdcf693 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/_types.py @@ -15,6 +15,7 @@ # limitations under the License. # +import sys import decimal import datetime import keyword @@ -25,6 +26,9 @@ from array import array from operator import itemgetter +if sys.version >= "3": + long = int + unicode = str __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", @@ -410,7 +414,7 @@ def fromJson(cls, json): split = pyUDT.rfind(".") pyModule = pyUDT[:split] pyClass = pyUDT[split+1:] - m = __import__(pyModule, globals(), locals(), [pyClass], -1) + m = __import__(pyModule, globals(), locals(), [pyClass]) UDT = getattr(m, pyClass) return UDT() @@ -419,10 +423,9 @@ def __eq__(self, other): _all_primitive_types = dict((v.typeName(), v) - for v in globals().itervalues() - if type(v) is PrimitiveTypeSingleton and - v.__base__ == PrimitiveType) - + for v in list(globals().values()) + if (type(v) is type or type(v) is PrimitiveTypeSingleton) + and v.__base__ == PrimitiveType) _all_complex_types = dict((v.typeName(), v) for v in [ArrayType, MapType, StructType]) @@ -486,10 +489,10 @@ def _parse_datatype_json_string(json_string): def _parse_datatype_json_value(json_value): - if type(json_value) is unicode: + if not isinstance(json_value, dict): if json_value in _all_primitive_types.keys(): return _all_primitive_types[json_value]() - elif json_value == u'decimal': + elif json_value == 'decimal': return DecimalType() elif _FIXED_DECIMAL.match(json_value): m = _FIXED_DECIMAL.match(json_value) @@ -511,10 +514,8 @@ def _parse_datatype_json_value(json_value): type(None): NullType, bool: BooleanType, int: LongType, - long: LongType, float: DoubleType, str: StringType, - unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.date: DateType, @@ -522,6 +523,12 @@ def _parse_datatype_json_value(json_value): datetime.time: TimestampType, } +if sys.version < "3": + _type_mappings.update({ + unicode: StringType, + long: LongType, + }) + def _infer_type(obj): """Infer the DataType from obj @@ -541,7 +548,7 @@ def _infer_type(obj): return dataType() if isinstance(obj, dict): - for key, value in obj.iteritems(): + for key, value in obj.items(): if key is not None and value is not None: return MapType(_infer_type(key), _infer_type(value), True) else: @@ -565,10 +572,10 @@ def _infer_schema(row): items = sorted(row.items()) elif isinstance(row, (tuple, list)): - if hasattr(row, "_fields"): # namedtuple - items = zip(row._fields, tuple(row)) - elif hasattr(row, "__fields__"): # Row + if hasattr(row, "__fields__"): # Row items = zip(row.__fields__, tuple(row)) + elif hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -647,7 +654,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__fields__"): + if hasattr(obj, "__fields__") or hasattr(obj, "_fields"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -733,12 +740,12 @@ def _create_converter(dataType): if isinstance(dataType, ArrayType): conv = _create_converter(dataType.elementType) - return lambda row: map(conv, row) + return lambda row: [conv(v) for v in row] elif isinstance(dataType, MapType): kconv = _create_converter(dataType.keyType) vconv = _create_converter(dataType.valueType) - return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.items()) elif isinstance(dataType, NullType): return lambda x: None @@ -881,7 +888,7 @@ def _infer_schema_type(obj, dataType): >>> _infer_schema_type(row, schema) StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ - if dataType is NullType(): + if isinstance(dataType, NullType): return _infer_type(obj) if not obj: @@ -892,7 +899,7 @@ def _infer_schema_type(obj, dataType): return ArrayType(eType, True) elif isinstance(dataType, MapType): - k, v = obj.iteritems().next() + k, v = next(iter(obj.items())) return MapType(_infer_schema_type(k, dataType.keyType), _infer_schema_type(v, dataType.valueType)) @@ -935,7 +942,7 @@ def _verify_type(obj, dataType): >>> _verify_type(None, StructType([])) >>> _verify_type("", StringType()) >>> _verify_type(0, LongType()) - >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(list(range(3)), ArrayType(ShortType())) >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... @@ -976,7 +983,7 @@ def _verify_type(obj, dataType): _verify_type(i, dataType.elementType) elif isinstance(dataType, MapType): - for k, v in obj.iteritems(): + for k, v in obj.items(): _verify_type(k, dataType.keyType) _verify_type(v, dataType.valueType) @@ -1213,6 +1220,8 @@ def __getattr__(self, item): return self[idx] except IndexError: raise AttributeError(item) + except ValueError: + raise AttributeError(item) def __reduce__(self): if hasattr(self, "__fields__"): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8529a8f8e3a4..c90afc326ca0e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -15,14 +15,19 @@ # limitations under the License. # +import sys import warnings import json -from itertools import imap + +if sys.version >= '3': + basestring = unicode = str +else: + from itertools import imap as map from py4j.protocol import Py4JError from py4j.java_collections import MapConverter -from pyspark.rdd import RDD, _prepare_for_python_RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.sql.types import Row, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter @@ -62,31 +67,27 @@ class SQLContext(object): A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. - When created, :class:`SQLContext` adds a method called ``toDF`` to :class:`RDD`, - which could be used to convert an RDD into a DataFrame, it's a shorthand for - :func:`SQLContext.createDataFrame`. - :param sparkContext: The :class:`SparkContext` backing this SQLContext. :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instantiate a new SQLContext in the JVM, instead we make all calls to this object. """ + @ignore_unicode_prefix def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. >>> from datetime import datetime >>> sqlContext = SQLContext(sc) - >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() - [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] - >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, - ... x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] + >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc @@ -122,6 +123,7 @@ def udf(self): """Returns a :class:`UDFRegistration` for UDF registration.""" return UDFRegistration(self) + @ignore_unicode_prefix def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -147,7 +149,7 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) @@ -185,6 +187,7 @@ def _inferSchema(self, rdd, samplingRatio=None): schema = rdd.map(_infer_schema).reduce(_merge_type) return schema + @ignore_unicode_prefix def inferSchema(self, rdd, samplingRatio=None): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -195,6 +198,7 @@ def inferSchema(self, rdd, samplingRatio=None): return self.createDataFrame(rdd, None, samplingRatio) + @ignore_unicode_prefix def applySchema(self, rdd, schema): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -208,6 +212,7 @@ def applySchema(self, rdd, schema): return self.createDataFrame(rdd, schema) + @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None): """ Creates a :class:`DataFrame` from an :class:`RDD` of :class:`tuple`/:class:`list`, @@ -380,6 +385,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): df = self._ssql_ctx.jsonFile(path, scala_datatype) return DataFrame(df, self) + @ignore_unicode_prefix def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): """Loads an RDD storing one JSON object per string as a :class:`DataFrame`. @@ -477,6 +483,7 @@ def createExternalTable(self, tableName, path=None, source=None, joptions) return DataFrame(df, self) + @ignore_unicode_prefix def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. @@ -497,6 +504,7 @@ def table(self, tableName): """ return DataFrame(self._ssql_ctx.table(tableName), self) + @ignore_unicode_prefix def tables(self, dbName=None): """Returns a :class:`DataFrame` containing names of tables in the given database. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f2c3b74a185cf..d76504f986270 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -16,14 +16,19 @@ # import sys -import itertools import warnings import random +if sys.version >= '3': + basestring = unicode = str + long = int +else: + from itertools import imap as map + from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -65,19 +70,20 @@ def __init__(self, jdf, sql_ctx): self._sc = sql_ctx and sql_ctx._sc self.is_cached = False self._schema = None # initialized lazily + self._lazy_rdd = None @property def rdd(self): """Returns the content as an :class:`pyspark.RDD` of :class:`Row`. """ - if not hasattr(self, '_lazy_rdd'): + if self._lazy_rdd is None: jrdd = self._jdf.javaToPython() rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) schema = self.schema def applySchema(it): cls = _create_cls(schema) - return itertools.imap(cls, it) + return map(cls, it) self._lazy_rdd = rdd.mapPartitions(applySchema) @@ -89,13 +95,14 @@ def na(self): """ return DataFrameNaFunctions(self) - def toJSON(self, use_unicode=False): + @ignore_unicode_prefix + def toJSON(self, use_unicode=True): """Converts a :class:`DataFrame` into a :class:`RDD` of string. Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() - '{"age":2,"name":"Alice"}' + u'{"age":2,"name":"Alice"}' """ rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) @@ -228,7 +235,7 @@ def printSchema(self): |-- name: string (nullable = true) """ - print (self._jdf.schema().treeString()) + print(self._jdf.schema().treeString()) def explain(self, extended=False): """Prints the (logical and physical) plans to the console for debugging purpose. @@ -250,9 +257,9 @@ def explain(self, extended=False): == RDD == """ if extended: - print self._jdf.queryExecution().toString() + print(self._jdf.queryExecution().toString()) else: - print self._jdf.queryExecution().executedPlan().toString() + print(self._jdf.queryExecution().executedPlan().toString()) def isLocal(self): """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally @@ -270,7 +277,7 @@ def show(self, n=20): 2 Alice 5 Bob """ - print self._jdf.showString(n).encode('utf8', 'ignore') + print(self._jdf.showString(n)) def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) @@ -279,10 +286,11 @@ def count(self): """Returns the number of rows in this :class:`DataFrame`. >>> df.count() - 2L + 2 """ - return self._jdf.count() + return int(self._jdf.count()) + @ignore_unicode_prefix def collect(self): """Returns all the records as a list of :class:`Row`. @@ -295,6 +303,7 @@ def collect(self): cls = _create_cls(self.schema) return [cls(r) for r in rs] + @ignore_unicode_prefix def limit(self, num): """Limits the result count to the number specified. @@ -306,6 +315,7 @@ def limit(self, num): jdf = self._jdf.limit(num) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def take(self, num): """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. @@ -314,6 +324,7 @@ def take(self, num): """ return self.limit(num).collect() + @ignore_unicode_prefix def map(self, f): """ Returns a new :class:`RDD` by applying a the ``f`` function to each :class:`Row`. @@ -324,6 +335,7 @@ def map(self, f): """ return self.rdd.map(f) + @ignore_unicode_prefix def flatMap(self, f): """ Returns a new :class:`RDD` by first applying the ``f`` function to each :class:`Row`, and then flattening the results. @@ -353,7 +365,7 @@ def foreach(self, f): This is a shorthand for ``df.rdd.foreach()``. >>> def f(person): - ... print person.name + ... print(person.name) >>> df.foreach(f) """ return self.rdd.foreach(f) @@ -365,7 +377,7 @@ def foreachPartition(self, f): >>> def f(people): ... for person in people: - ... print person.name + ... print(person.name) >>> df.foreachPartition(f) """ return self.rdd.foreachPartition(f) @@ -412,7 +424,7 @@ def distinct(self): """Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`. >>> df.distinct().count() - 2L + 2 """ return DataFrame(self._jdf.distinct(), self.sql_ctx) @@ -420,10 +432,10 @@ def sample(self, withReplacement, fraction, seed=None): """Returns a sampled subset of this :class:`DataFrame`. >>> df.sample(False, 0.5, 97).count() - 1L + 1 """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction - seed = seed if seed is not None else random.randint(0, sys.maxint) + seed = seed if seed is not None else random.randint(0, sys.maxsize) rdd = self._jdf.sample(withReplacement, fraction, long(seed)) return DataFrame(rdd, self.sql_ctx) @@ -437,6 +449,7 @@ def dtypes(self): return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] @property + @ignore_unicode_prefix def columns(self): """Returns all column names as a list. @@ -445,6 +458,7 @@ def columns(self): """ return [f.name for f in self.schema.fields] + @ignore_unicode_prefix def join(self, other, joinExprs=None, joinType=None): """Joins with another :class:`DataFrame`, using the given join expression. @@ -470,6 +484,7 @@ def join(self, other, joinExprs=None, joinType=None): jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def sort(self, *cols): """Returns a new :class:`DataFrame` sorted by the specified column(s). @@ -513,6 +528,7 @@ def describe(self, *cols): jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def head(self, n=None): """ Returns the first ``n`` rows as a list of :class:`Row`, @@ -528,6 +544,7 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + @ignore_unicode_prefix def first(self): """Returns the first row as a :class:`Row`. @@ -536,6 +553,7 @@ def first(self): """ return self.head() + @ignore_unicode_prefix def __getitem__(self, item): """Returns the column as a :class:`Column`. @@ -567,6 +585,7 @@ def __getattr__(self, name): jc = self._jdf.apply(name) return Column(jc) + @ignore_unicode_prefix def select(self, *cols): """Projects a set of expressions and returns a new :class:`DataFrame`. @@ -598,6 +617,7 @@ def selectExpr(self, *expr): jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def filter(self, condition): """Filters rows using the given condition. @@ -626,6 +646,7 @@ def filter(self, condition): where = filter + @ignore_unicode_prefix def groupBy(self, *cols): """Groups the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedData` @@ -775,6 +796,7 @@ def fillna(self, value, subset=None): cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + @ignore_unicode_prefix def withColumn(self, colName, col): """Returns a new :class:`DataFrame` by adding a column. @@ -786,6 +808,7 @@ def withColumn(self, colName, col): """ return self.select('*', col.alias(colName)) + @ignore_unicode_prefix def withColumnRenamed(self, existing, new): """REturns a new :class:`DataFrame` by renaming an existing column. @@ -852,6 +875,7 @@ def __init__(self, jdf, sql_ctx): self._jdf = jdf self.sql_ctx = sql_ctx + @ignore_unicode_prefix def agg(self, *exprs): """Compute aggregates and returns the result as a :class:`DataFrame`. @@ -1041,11 +1065,13 @@ def __init__(self, jc): __sub__ = _bin_op("minus") __mul__ = _bin_op("multiply") __div__ = _bin_op("divide") + __truediv__ = _bin_op("divide") __mod__ = _bin_op("mod") __radd__ = _bin_op("plus") __rsub__ = _reverse_op("minus") __rmul__ = _bin_op("multiply") __rdiv__ = _reverse_op("divide") + __rtruediv__ = _reverse_op("divide") __rmod__ = _reverse_op("mod") # logistic operators @@ -1075,6 +1101,7 @@ def __init__(self, jc): startswith = _bin_op("startsWith") endswith = _bin_op("endsWith") + @ignore_unicode_prefix def substr(self, startPos, length): """ Return a :class:`Column` which is a substring of the column @@ -1097,6 +1124,7 @@ def substr(self, startPos, length): __getslice__ = substr + @ignore_unicode_prefix def inSet(self, *cols): """ A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments. @@ -1131,6 +1159,7 @@ def alias(self, alias): """ return Column(getattr(self._jc, "as")(alias)) + @ignore_unicode_prefix def cast(self, dataType): """ Convert the column into type `dataType` diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index daeb6916b58bc..1d6536952810f 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -18,8 +18,10 @@ """ A collections of builtin functions """ +import sys -from itertools import imap +if sys.version < "3": + from itertools import imap as map from py4j.java_collections import ListConverter @@ -116,7 +118,7 @@ def __init__(self, func, returnType): def _create_judf(self): f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) sc = SparkContext._active_spark_context diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b3a6a2c6a9229..7c09a0cfe30ab 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -157,13 +157,13 @@ def test_udf2(self): self.assertEqual(4, res[0]) def test_udf_with_array_type(self): - d = [Row(l=range(3), d={"key": range(5)})] + d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) self.sqlCtx.createDataFrame(rdd).registerTempTable("test") self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() - self.assertEqual(range(3), l1) + self.assertEqual(list(range(3)), l1) self.assertEqual(1, l2) def test_broadcast_in_udf(self): @@ -266,7 +266,7 @@ def test_infer_nested_schema(self): def test_apply_schema(self): from datetime import date, datetime - rdd = self.sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + rdd = self.sc.parallelize([(127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, (2,), [1, 2, 3], None)]) schema = StructType([ @@ -309,7 +309,7 @@ def test_apply_schema(self): def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] df = self.sc.parallelize(d).toDF() - k, v = df.head().m.items()[0] + k, v = list(df.head().m.items())[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -554,6 +554,9 @@ def setUpClass(cls): except py4j.protocol.Py4JError: cls.sqlCtx = None return + except TypeError: + cls.sqlCtx = None + return os.unlink(cls.tempdir.name) _scala_HiveContext =\ cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 1e597d64e03fe..944fa414b0c0e 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -31,7 +31,7 @@ class StatCounter(object): def __init__(self, values=[]): - self.n = 0L # Running count of our values + self.n = 0 # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") @@ -87,7 +87,7 @@ def copy(self): return copy.deepcopy(self) def count(self): - return self.n + return int(self.n) def mean(self): return self.mu diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 2c73083c9f9a8..4590c58839266 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -14,6 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +from __future__ import print_function + import os import sys @@ -157,7 +160,7 @@ def getOrCreate(cls, checkpointPath, setupFunc): try: jssc = gw.jvm.JavaStreamingContext(checkpointPath) except Exception: - print >>sys.stderr, "failed to load StreamingContext from checkpoint" + print("failed to load StreamingContext from checkpoint", file=sys.stderr) raise jsc = jssc.sparkContext() diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 3fa42444239f7..ff097985fae3e 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -15,11 +15,15 @@ # limitations under the License. # -from itertools import chain, ifilter, imap +import sys import operator import time +from itertools import chain from datetime import datetime +if sys.version < "3": + from itertools import imap as map, ifilter as filter + from py4j.protocol import Py4JJavaError from pyspark import RDD @@ -76,7 +80,7 @@ def filter(self, f): Return a new DStream containing only the elements that satisfy predicate. """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def flatMap(self, f, preservesPartitioning=False): @@ -85,7 +89,7 @@ def flatMap(self, f, preservesPartitioning=False): this DStream, and then flattening the results """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def map(self, f, preservesPartitioning=False): @@ -93,7 +97,7 @@ def map(self, f, preservesPartitioning=False): Return a new DStream by applying a function to each element of DStream. """ def func(iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitions(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -150,7 +154,7 @@ def foreachRDD(self, func): """ Apply a function to each RDD in this DStream. """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: old_func = func func = lambda t, rdd: old_func(rdd) jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) @@ -165,14 +169,14 @@ def pprint(self, num=10): """ def takeAndPrint(time, rdd): taken = rdd.take(num + 1) - print "-------------------------------------------" - print "Time: %s" % time - print "-------------------------------------------" + print("-------------------------------------------") + print("Time: %s" % time) + print("-------------------------------------------") for record in taken[:num]: - print record + print(record) if len(taken) > num: - print "..." - print + print("...") + print() self.foreachRDD(takeAndPrint) @@ -181,7 +185,7 @@ def mapValues(self, f): Return a new DStream by applying a map function to the value of each key-value pairs in this DStream without changing the key. """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def flatMapValues(self, f): @@ -189,7 +193,7 @@ def flatMapValues(self, f): Return a new DStream by applying a flatmap function to the value of each key-value pairs in this DStream without changing the key. """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def glom(self): @@ -286,10 +290,10 @@ def transform(self, func): `func` can have one argument of `rdd`, or have two arguments of (`time`, `rdd`) """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: oldfunc = func func = lambda t, rdd: oldfunc(rdd) - assert func.func_code.co_argcount == 2, "func should take one or two arguments" + assert func.__code__.co_argcount == 2, "func should take one or two arguments" return TransformedDStream(self, func) def transformWith(self, func, other, keepSerializer=False): @@ -300,10 +304,10 @@ def transformWith(self, func, other, keepSerializer=False): `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three arguments of (`time`, `rdd_a`, `rdd_b`) """ - if func.func_code.co_argcount == 2: + if func.__code__.co_argcount == 2: oldfunc = func func = lambda t, a, b: oldfunc(a, b) - assert func.func_code.co_argcount == 3, "func should take two or three arguments" + assert func.__code__.co_argcount == 3, "func should take two or three arguments" jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer, other._jrdd_deserializer) dstream = self._sc._jvm.PythonTransformed2DStream(self._jdstream.dstream(), other._jdstream.dstream(), jfunc) @@ -460,7 +464,7 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio keyed = self.map(lambda x: (1, x)) reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1) - return reduced.map(lambda (k, v): v) + return reduced.map(lambda kv: kv[1]) def countByWindow(self, windowDuration, slideDuration): """ @@ -489,7 +493,7 @@ def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=Non keyed = self.map(lambda x: (x, 1)) counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub, windowDuration, slideDuration, numPartitions) - return counted.filter(lambda (k, v): v > 0).count() + return counted.filter(lambda kv: kv[1] > 0).count() def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None): """ @@ -548,7 +552,8 @@ def reduceFunc(t, a, b): def invReduceFunc(t, a, b): b = b.reduceByKey(func, numPartitions) joined = a.leftOuterJoin(b, numPartitions) - return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1) + return joined.mapValues(lambda kv: invFunc(kv[0], kv[1]) + if kv[1] is not None else kv[0]) jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer) if invReduceFunc: @@ -579,9 +584,9 @@ def reduceFunc(t, a, b): g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) else: g = a.cogroup(b.partitionBy(numPartitions), numPartitions) - g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) - state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) - return state.filter(lambda (k, v): v is not None) + g = g.mapValues(lambda ab: (list(ab[1]), list(ab[0])[0] if len(ab[0]) else None)) + state = g.mapValues(lambda vs_s: updateFunc(vs_s[0], vs_s[1])) + return state.filter(lambda k_v: k_v[1] is not None) jreduceFunc = TransformFunction(self._sc, reduceFunc, self._sc.serializer, self._jrdd_deserializer) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index f083ed149effb..7a7b6e1d9a527 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -67,10 +67,10 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") helper = helperClass.newInstance() jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) - except Py4JJavaError, e: + except Py4JJavaError as e: # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): - print """ + print(""" ________________________________________________________________________________________________ Spark Streaming's Kafka libraries not found in class path. Try one of the following. @@ -88,8 +88,8 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, ________________________________________________________________________________________________ -""" % (ssc.sparkContext.version, ssc.sparkContext.version) +""" % (ssc.sparkContext.version, ssc.sparkContext.version)) raise e ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) stream = DStream(jstream, ssc, ser) - return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 9b4635e49020b..06d22154373bc 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -22,6 +22,7 @@ import unittest import tempfile import struct +from functools import reduce from py4j.java_collections import MapConverter @@ -51,7 +52,7 @@ def wait_for(self, result, n): while len(result) < n and time.time() - start_time < self.timeout: time.sleep(0.01) if len(result) < n: - print "timeout after", self.timeout + print("timeout after", self.timeout) def _take(self, dstream, n): """ @@ -131,7 +132,7 @@ def test_map(self): def func(dstream): return dstream.map(str) - expected = map(lambda x: map(str, x), input) + expected = [list(map(str, x)) for x in input] self._test_func(input, func, expected) def test_flatMap(self): @@ -140,8 +141,8 @@ def test_flatMap(self): def func(dstream): return dstream.flatMap(lambda x: (x, x * 2)) - expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), - input) + expected = [list(chain.from_iterable((map(lambda y: [y, y * 2], x)))) + for x in input] self._test_func(input, func, expected) def test_filter(self): @@ -150,7 +151,7 @@ def test_filter(self): def func(dstream): return dstream.filter(lambda x: x % 2 == 0) - expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input) + expected = [[y for y in x if y % 2 == 0] for x in input] self._test_func(input, func, expected) def test_count(self): @@ -159,7 +160,7 @@ def test_count(self): def func(dstream): return dstream.count() - expected = map(lambda x: [len(x)], input) + expected = [[len(x)] for x in input] self._test_func(input, func, expected) def test_reduce(self): @@ -168,7 +169,7 @@ def test_reduce(self): def func(dstream): return dstream.reduce(operator.add) - expected = map(lambda x: [reduce(operator.add, x)], input) + expected = [[reduce(operator.add, x)] for x in input] self._test_func(input, func, expected) def test_reduceByKey(self): @@ -185,27 +186,27 @@ def func(dstream): def test_mapValues(self): """Basic operation test for DStream.mapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 2), (3, 3)], + [(0, 4), (1, 1), (2, 2), (3, 3)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.mapValues(lambda x: x + 10) expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], - [("", 14), (1, 11), (2, 12), (3, 13)], + [(0, 14), (1, 11), (2, 12), (3, 13)], [(1, 11), (2, 11), (3, 11), (4, 11)]] self._test_func(input, func, expected, sort=True) def test_flatMapValues(self): """Basic operation test for DStream.flatMapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 1), (3, 1)], + [(0, 4), (1, 1), (2, 1), (3, 1)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.flatMapValues(lambda x: (x, x + 10)) expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), ("c", 1), ("c", 11), ("d", 1), ("d", 11)], - [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(0, 4), (0, 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] self._test_func(input, func, expected) @@ -233,7 +234,7 @@ def f(iterator): def test_countByValue(self): """Basic operation test for DStream.countByValue.""" - input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + input = [list(range(1, 5)) * 2, list(range(5, 7)) + list(range(5, 9)), ["a", "a", "b", ""]] def func(dstream): return dstream.countByValue() @@ -285,7 +286,7 @@ def test_union(self): def func(d1, d2): return d1.union(d2) - expected = [range(6), range(6), range(6)] + expected = [list(range(6)), list(range(6)), list(range(6))] self._test_func(input1, func, expected, input2=input2) def test_cogroup(self): @@ -424,7 +425,7 @@ class StreamingContextTests(PySparkStreamingTestCase): duration = 0.1 def _add_input_stream(self): - inputs = map(lambda x: range(1, x), range(101)) + inputs = [range(1, x) for x in range(101)] stream = self.ssc.queueStream(inputs) self._collect(stream, 1, block=False) @@ -441,7 +442,7 @@ def test_stop_multiple_times(self): self.ssc.stop() def test_queue_stream(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) result = self._collect(dstream, 3) self.assertEqual(input, result) @@ -457,13 +458,13 @@ def test_text_file_stream(self): with open(os.path.join(d, name), "w") as f: f.writelines(["%d\n" % i for i in range(10)]) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], result) + self.assertEqual([list(range(10)), list(range(10))], result) def test_binary_records_stream(self): d = tempfile.mkdtemp() self.ssc = StreamingContext(self.sc, self.duration) dstream = self.ssc.binaryRecordsStream(d, 10).map( - lambda v: struct.unpack("10b", str(v))) + lambda v: struct.unpack("10b", bytes(v))) result = self._collect(dstream, 2, block=False) self.ssc.start() for name in ('a', 'b'): @@ -471,10 +472,10 @@ def test_binary_records_stream(self): with open(os.path.join(d, name), "wb") as f: f.write(bytearray(range(10))) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], map(lambda v: list(v[0]), result)) + self.assertEqual([list(range(10)), list(range(10))], [list(v[0]) for v in result]) def test_union(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) dstream2 = self.ssc.queueStream(input) dstream3 = self.ssc.union(dstream, dstream2) diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index 86ee5aa04f252..34291f30a5652 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -91,9 +91,9 @@ def dumps(self, id): except Exception: traceback.print_exc() - def loads(self, bytes): + def loads(self, data): try: - f, deserializers = self.serializer.loads(str(bytes)) + f, deserializers = self.serializer.loads(bytes(data)) return TransformFunction(self.ctx, f, *deserializers) except Exception: traceback.print_exc() @@ -116,7 +116,7 @@ def rddToFileName(prefix, suffix, timestamp): """ if isinstance(timestamp, datetime): seconds = time.mktime(timestamp.timetuple()) - timestamp = long(seconds * 1000) + timestamp.microsecond / 1000 + timestamp = int(seconds * 1000) + timestamp.microsecond // 1000 if suffix is None: return prefix + "-" + str(timestamp) else: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ee67e80d539f8..75f39d9e75f38 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,8 +19,8 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ + from array import array -from fileinput import input from glob import glob import os import re @@ -45,6 +45,9 @@ sys.exit(1) else: import unittest + if sys.version_info[0] >= 3: + xrange = range + basestring = str from pyspark.conf import SparkConf @@ -52,7 +55,9 @@ from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer, \ + PairDeserializer, CartesianDeserializer, AutoBatchedSerializer, AutoSerializer, \ + FlattenedValuesSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark import shuffle from pyspark.profiler import BasicProfiler @@ -81,7 +86,7 @@ class MergerTests(unittest.TestCase): def setUp(self): self.N = 1 << 12 self.l = [i for i in xrange(self.N)] - self.data = zip(self.l, self.l) + self.data = list(zip(self.l, self.l)) self.agg = Aggregator(lambda x: [x], lambda x, y: x.append(y) or x, lambda x, y: x.extend(y) or x) @@ -89,45 +94,45 @@ def setUp(self): def test_in_memory(self): m = InMemoryMerger(self.agg) m.mergeValues(self.data) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = InMemoryMerger(self.agg) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + m.mergeCombiners(map(lambda x_y: (x_y[0], [x_y[1]]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_small_dataset(self): m = ExternalMerger(self.agg, 1000) m.mergeValues(self.data) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 1000) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data)) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 30) + m = ExternalMerger(self.agg, 20) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 10) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10, partitions=3) - m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + m = ExternalMerger(self.agg, 5, partitions=3) + m.mergeCombiners(map(lambda k_v: (k_v[0], [str(k_v[1])]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m.iteritems()), + self.assertEqual(sum(len(v) for k, v in m.items()), self.N * 10) m._cleanup() @@ -144,55 +149,55 @@ def gen_gs(N, step=1): self.assertEqual(1, len(list(gen_gs(1)))) self.assertEqual(2, len(list(gen_gs(2)))) self.assertEqual(100, len(list(gen_gs(100)))) - self.assertEqual(range(1, 101), [k for k, _ in gen_gs(100)]) - self.assertTrue(all(range(k) == list(vs) for k, vs in gen_gs(100))) + self.assertEqual(list(range(1, 101)), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(list(range(k)) == list(vs) for k, vs in gen_gs(100))) for k, vs in gen_gs(50002, 10000): self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) ser = PickleSerializer() l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) for k, vs in l: self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) class SorterTests(unittest.TestCase): def test_in_memory_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1024) - self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) def test_external_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1) - self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) self.assertGreater(shuffle.DiskBytesSpilled, 0) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") sc = SparkContext(conf=conf) - l = range(10240) + l = list(range(10240)) random.shuffle(l) - rdd = sc.parallelize(l, 10) - self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + rdd = sc.parallelize(l, 2) + self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect()) sc.stop() @@ -200,11 +205,11 @@ class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): from collections import namedtuple - from cPickle import dumps, loads + from pickle import dumps, loads P = namedtuple("P", "x y") p1 = P(1, 3) p2 = loads(dumps(p1, 2)) - self.assertEquals(p1, p2) + self.assertEqual(p1, p2) def test_itemgetter(self): from operator import itemgetter @@ -246,7 +251,7 @@ def test_pickling_file_handles(self): ser = CloudPickleSerializer() out1 = sys.stderr out2 = ser.loads(ser.dumps(out1)) - self.assertEquals(out1, out2) + self.assertEqual(out1, out2) def test_func_globals(self): @@ -263,19 +268,36 @@ def __reduce__(self): def foo(): sys.exit(0) - self.assertTrue("exit" in foo.func_code.co_names) + self.assertTrue("exit" in foo.__code__.co_names) ser.dumps(foo) def test_compressed_serializer(self): ser = CompressedSerializer(PickleSerializer()) - from StringIO import StringIO + try: + from StringIO import StringIO + except ImportError: + from io import BytesIO as StringIO io = StringIO() ser.dump_stream(["abc", u"123", range(5)], io) io.seek(0) self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io))) ser.dump_stream(range(1000), io) io.seek(0) - self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) + self.assertEqual(["abc", u"123", range(5)] + list(range(1000)), list(ser.load_stream(io))) + io.close() + + def test_hash_serializer(self): + hash(NoOpSerializer()) + hash(UTF8Deserializer()) + hash(PickleSerializer()) + hash(MarshalSerializer()) + hash(AutoSerializer()) + hash(BatchedSerializer(PickleSerializer())) + hash(AutoBatchedSerializer(MarshalSerializer())) + hash(PairDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CartesianDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CompressedSerializer(PickleSerializer())) + hash(FlattenedValuesSerializer(PickleSerializer())) class PySparkTestCase(unittest.TestCase): @@ -340,7 +362,7 @@ def test_checkpoint_and_restore(self): self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(), flatMappedRDD._jrdd_deserializer) - self.assertEquals([1, 2, 3, 4], recovered.collect()) + self.assertEqual([1, 2, 3, 4], recovered.collect()) class AddFileTests(PySparkTestCase): @@ -356,8 +378,7 @@ def test_add_py_file(self): def func(x): from userlibrary import UserClass return UserClass().hello() - self.assertRaises(Exception, - self.sc.parallelize(range(2)).map(func).first) + self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) log4j.LogManager.getRootLogger().setLevel(old_level) # Add the file, so the job should now succeed: @@ -372,7 +393,7 @@ def test_add_file_locally(self): download_path = SparkFiles.get("hello.txt") self.assertNotEqual(path, download_path) with open(download_path) as test_file: - self.assertEquals("Hello World!\n", test_file.readline()) + self.assertEqual("Hello World!\n", test_file.readline()) def test_add_py_file_locally(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -381,7 +402,7 @@ def func(): from userlibrary import UserClass self.assertRaises(ImportError, func) path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") - self.sc.addFile(path) + self.sc.addPyFile(path) from userlibrary import UserClass self.assertEqual("Hello World!", UserClass().hello()) @@ -391,7 +412,7 @@ def test_add_egg_file_locally(self): def func(): from userlib import UserClass self.assertRaises(ImportError, func) - path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg") + path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip") self.sc.addPyFile(path) from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) @@ -427,8 +448,9 @@ def test_save_as_textfile_with_unicode(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode("utf-8")) def test_save_as_textfile_with_utf8(self): x = u"\u00A1Hola, mundo!" @@ -436,19 +458,20 @@ def test_save_as_textfile_with_utf8(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode('utf8')) def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) rdd2 = self.sc.parallelize([3, 4]) cart = rdd1.cartesian(rdd2) - result = cart.map(lambda (x, y): x + y).collect() + result = cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect() def test_transforming_pickle_file(self): # Regression test for SPARK-2601 - data = self.sc.parallelize(["Hello", "World!"]) + data = self.sc.parallelize([u"Hello", u"World!"]) tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsPickleFile(tempFile.name) @@ -461,13 +484,13 @@ def test_cartesian_on_textfile(self): a = self.sc.textFile(path) result = a.cartesian(a).collect() (x, y) = result[0] - self.assertEqual("Hello World!", x.strip()) - self.assertEqual("Hello World!", y.strip()) + self.assertEqual(u"Hello World!", x.strip()) + self.assertEqual(u"Hello World!", y.strip()) def test_deleting_input_files(self): # Regression test for SPARK-1025 tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) filtered_data = data.filter(lambda x: True) @@ -510,21 +533,21 @@ def test_namedtuple_in_rdd(self): jon = Person(1, "Jon", "Doe") jane = Person(2, "Jane", "Doe") theDoes = self.sc.parallelize([jon, jane]) - self.assertEquals([jon, jane], theDoes.collect()) + self.assertEqual([jon, jane], theDoes.collect()) def test_large_broadcast(self): N = 100000 data = [[float(i) for i in range(300)] for i in range(N)] bdata = self.sc.broadcast(data) # 270MB m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() - self.assertEquals(N, m) + self.assertEqual(N, m) def test_multiple_broadcasts(self): N = 1 << 21 b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM - r = range(1 << 15) + r = list(range(1 << 15)) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -535,7 +558,7 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -549,7 +572,7 @@ def test_large_closure(self): N = 1000000 data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) - self.assertEquals(N, rdd.first()) + self.assertEqual(N, rdd.first()) # regression test for SPARK-6886 self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) @@ -590,15 +613,15 @@ def test_zip_with_different_number_of_items(self): # same total number of items, but different distributions a = self.sc.parallelize([2, 3], 2).flatMap(range) b = self.sc.parallelize([3, 2], 2).flatMap(range) - self.assertEquals(a.count(), b.count()) + self.assertEqual(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): rdd = self.sc.parallelize(range(1000)) - self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050) rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7) self.assertTrue(18 < rdd.countApproxDistinct() < 22) @@ -612,59 +635,59 @@ def test_count_approx_distinct(self): def test_histogram(self): # empty rdd = self.sc.parallelize([]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) self.assertRaises(ValueError, lambda: rdd.histogram(1)) # out of range rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1]) # in range with one bucket rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals([4], rdd.histogram([0, 10])[1]) - self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([4], rdd.histogram([0, 10])[1]) + self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1]) # in range with one bucket exact match - self.assertEquals([4], rdd.histogram([1, 4])[1]) + self.assertEqual([4], rdd.histogram([1, 4])[1]) # out of range with two buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1]) # out of range with two uneven buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) # in range with two buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two bucket and None rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two uneven buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1]) # mixed range with two uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) - self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1]) # mixed range with four uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # mixed range with uneven buckets and NaN rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, None, float('nan')]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # out of range with infinite buckets rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) - self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + self.assertEqual([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) # invalid buckets self.assertRaises(ValueError, lambda: rdd.histogram([])) @@ -674,25 +697,25 @@ def test_histogram(self): # without buckets rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + self.assertEqual(([1, 4], [4]), rdd.histogram(1)) # without buckets single element rdd = self.sc.parallelize([1]) - self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + self.assertEqual(([1, 1], [1]), rdd.histogram(1)) # without bucket no range rdd = self.sc.parallelize([1] * 4) - self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + self.assertEqual(([1, 1], [4]), rdd.histogram(1)) # without buckets basic two rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) # without buckets with more requested than elements rdd = self.sc.parallelize([1, 2]) buckets = [1 + 0.2 * i for i in range(6)] hist = [1, 0, 0, 0, 1] - self.assertEquals((buckets, hist), rdd.histogram(5)) + self.assertEqual((buckets, hist), rdd.histogram(5)) # invalid RDDs rdd = self.sc.parallelize([1, float('inf')]) @@ -702,15 +725,8 @@ def test_histogram(self): # string rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) - self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) - self.assertRaises(TypeError, lambda: rdd.histogram(2)) - - # mixed RDD - rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) - self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) - self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) def test_repartitionAndSortWithinPartitions(self): @@ -718,31 +734,31 @@ def test_repartitionAndSortWithinPartitions(self): repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) partitions = repartitioned.glom().collect() - self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) - self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)]) def test_distinct(self): rdd = self.sc.parallelize((1, 2, 3)*10, 10) - self.assertEquals(rdd.getNumPartitions(), 10) - self.assertEquals(rdd.distinct().count(), 3) + self.assertEqual(rdd.getNumPartitions(), 10) + self.assertEqual(rdd.distinct().count(), 3) result = rdd.distinct(5) - self.assertEquals(result.getNumPartitions(), 5) - self.assertEquals(result.count(), 3) + self.assertEqual(result.getNumPartitions(), 5) + self.assertEqual(result.count(), 3) def test_external_group_by_key(self): - self.sc._conf.set("spark.python.worker.memory", "5m") + self.sc._conf.set("spark.python.worker.memory", "1m") N = 200001 kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) gkv = kv.groupByKey().cache() self.assertEqual(3, gkv.count()) - filtered = gkv.filter(lambda (k, vs): k == 1) + filtered = gkv.filter(lambda kv: kv[0] == 1) self.assertEqual(1, filtered.count()) - self.assertEqual([(1, N/3)], filtered.mapValues(len).collect()) - self.assertEqual([(N/3, N/3)], + self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect()) + self.assertEqual([(N // 3, N // 3)], filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) result = filtered.collect()[0][1] - self.assertEqual(N/3, len(result)) - self.assertTrue(isinstance(result.data, shuffle.ExternalList)) + self.assertEqual(N // 3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList)) def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -767,7 +783,7 @@ def test_null_in_rdd(self): rdd = RDD(jrdd, self.sc, UTF8Deserializer()) self.assertEqual([u"a", None, u"b"], rdd.collect()) rdd = RDD(jrdd, self.sc, NoOpSerializer()) - self.assertEqual(["a", None, "b"], rdd.collect()) + self.assertEqual([b"a", None, b"b"], rdd.collect()) def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 @@ -813,14 +829,14 @@ def test_narrow_dependency_in_join(self): self.sc.setJobGroup("test3", "test", True) d = sorted(parted.cogroup(parted).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test3")[0] self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) self.sc.setJobGroup("test4", "test", True) d = sorted(parted.cogroup(rdd).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) @@ -906,6 +922,7 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/", @@ -954,15 +971,16 @@ def test_sequencefiles(self): en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] self.assertEqual(nulls, en) - maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) + maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect() em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] - self.assertEqual(maps, em) + for v in maps: + self.assertTrue(v in em) # arrays get pickled to tuples by default tuples = sorted(self.sc.sequenceFile( @@ -1089,8 +1107,8 @@ def test_converters(self): def test_binary_files(self): path = os.path.join(self.tempdir.name, "binaryfiles") os.mkdir(path) - data = "short binary data" - with open(os.path.join(path, "part-0000"), 'w') as f: + data = b"short binary data" + with open(os.path.join(path, "part-0000"), 'wb') as f: f.write(data) [(p, d)] = self.sc.binaryFiles(path).collect() self.assertTrue(p.endswith("part-0000")) @@ -1103,7 +1121,7 @@ def test_binary_records(self): for i in range(100): f.write('%04d' % i) result = self.sc.binaryRecords(path, 4).map(int).collect() - self.assertEqual(range(100), result) + self.assertEqual(list(range(100)), result) class OutputFormatTests(ReusedPySparkTestCase): @@ -1115,6 +1133,7 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.tempdir.name, ignore_errors=True) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] @@ -1155,8 +1174,9 @@ def test_sequencefiles(self): (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") - maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) - self.assertEqual(maps, em) + maps = self.sc.sequenceFile(basepath + "/sfmap/").collect() + for v in maps: + self.assertTrue(v, em) def test_oldhadoop(self): basepath = self.tempdir.name @@ -1168,12 +1188,13 @@ def test_oldhadoop(self): "org.apache.hadoop.mapred.SequenceFileOutputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable") - result = sorted(self.sc.hadoopFile( + result = self.sc.hadoopFile( basepath + "/oldhadoop/", "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) - self.assertEqual(result, dict_data) + "org.apache.hadoop.io.MapWritable").collect() + for v in result: + self.assertTrue(v, dict_data) conf = { "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", @@ -1183,12 +1204,13 @@ def test_oldhadoop(self): } self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) input_conf = {"mapred.input.dir": basepath + "/olddataset/"} - old_dataset = sorted(self.sc.hadoopRDD( + result = self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - conf=input_conf).collect()) - self.assertEqual(old_dataset, dict_data) + conf=input_conf).collect() + for v in result: + self.assertTrue(v, dict_data) def test_newhadoop(self): basepath = self.tempdir.name @@ -1223,6 +1245,7 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) + @unittest.skipIf(sys.version >= "3", "serialize of array") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays @@ -1303,7 +1326,7 @@ def test_reserialization(self): basepath = self.tempdir.name x = range(1, 5) y = range(1001, 1005) - data = zip(x, y) + data = list(zip(x, y)) rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) @@ -1354,7 +1377,7 @@ def connect(self, port): sock = socket(AF_INET, SOCK_STREAM) sock.connect(('127.0.0.1', port)) # send a split index of -1 to shutdown the worker - sock.send("\xFF\xFF\xFF\xFF") + sock.send(b"\xFF\xFF\xFF\xFF") sock.close() return True @@ -1395,7 +1418,6 @@ def test_termination_sigterm(self): class WorkerTests(PySparkTestCase): - def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() @@ -1410,7 +1432,7 @@ def sleep(x): # start job in background thread def run(): - self.sc.parallelize(range(1)).foreach(sleep) + self.sc.parallelize(range(1), 1).foreach(sleep) import threading t = threading.Thread(target=run) t.daemon = True @@ -1419,7 +1441,8 @@ def run(): daemon_pid, worker_pid = 0, 0 while True: if os.path.exists(path): - data = open(path).read().split(' ') + with open(path) as f: + data = f.read().split(' ') daemon_pid, worker_pid = map(int, data) break time.sleep(0.1) @@ -1455,7 +1478,7 @@ def raise_exception(_): def test_after_jvm_exception(self): tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name, 1) filtered_data = data.filter(lambda x: True) @@ -1577,12 +1600,12 @@ def test_single_script(self): |from pyspark import SparkContext | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + |print(sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) def test_script_with_local_functions(self): """Submit and test a single script file calling a global function""" @@ -1593,12 +1616,12 @@ def test_script_with_local_functions(self): | return x * 3 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[3, 6, 9]", out) + self.assertIn("[3, 6, 9]", out.decode('utf-8')) def test_module_dependency(self): """Submit and test a script with a dependency on another module""" @@ -1607,7 +1630,7 @@ def test_module_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1617,7 +1640,7 @@ def test_module_dependency(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_module_dependency_on_cluster(self): """Submit and test a script with a dependency on another module on a cluster""" @@ -1626,7 +1649,7 @@ def test_module_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1637,7 +1660,7 @@ def test_module_dependency_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency(self): """Submit and test a script with a dependency on a Spark Package""" @@ -1646,14 +1669,14 @@ def test_package_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", "file:" + self.programDir, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency_on_cluster(self): """Submit and test a script with a dependency on a Spark Package on a cluster""" @@ -1662,7 +1685,7 @@ def test_package_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", @@ -1670,7 +1693,7 @@ def test_package_dependency_on_cluster(self): "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_single_script_on_cluster(self): """Submit and test a single script on a cluster""" @@ -1681,7 +1704,7 @@ def test_single_script_on_cluster(self): | return x * 2 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) # this will fail if you have different spark.executor.memory # in conf/spark-defaults.conf @@ -1690,7 +1713,7 @@ def test_single_script_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) class ContextTests(unittest.TestCase): @@ -1765,7 +1788,7 @@ class SciPyTests(PySparkTestCase): def test_serialize(self): from scipy.special import gammaln x = range(1, 5) - expected = map(gammaln, x) + expected = list(map(gammaln, x)) observed = self.sc.parallelize(x).map(gammaln).collect() self.assertEqual(expected, observed) @@ -1786,11 +1809,11 @@ def test_statcounter_array(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") if not _have_numpy: - print "NOTE: Skipping NumPy tests as it does not seem to be installed" + print("NOTE: Skipping NumPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") if not _have_numpy: - print "NOTE: NumPy tests were skipped as it does not seem to be installed" + print("NOTE: NumPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 452d6fabdcc17..fbdaf3a5814cd 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -18,6 +18,7 @@ """ Worker that receives input from Piped RDD. """ +from __future__ import print_function import os import sys import time @@ -37,9 +38,9 @@ def report_times(outfile, boot, init, finish): write_int(SpecialLengths.TIMING_DATA, outfile) - write_long(1000 * boot, outfile) - write_long(1000 * init, outfile) - write_long(1000 * finish, outfile) + write_long(int(1000 * boot), outfile) + write_long(int(1000 * init), outfile) + write_long(int(1000 * finish), outfile) def add_path(path): @@ -72,6 +73,9 @@ def main(infile, outfile): for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) add_path(os.path.join(spark_files_dir, filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) @@ -106,14 +110,14 @@ def process(): except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) + write_with_length(traceback.format_exc().encode("utf-8"), outfile) except IOError: # JVM close the socket pass except Exception: # Write the error to stderr if it happened while serializing - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() + print("PySpark worker failed with exception:", file=sys.stderr) + print(traceback.format_exc(), file=sys.stderr) exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) diff --git a/python/run-tests b/python/run-tests index f3a07d8aba562..ed3e819ef30c1 100755 --- a/python/run-tests +++ b/python/run-tests @@ -66,7 +66,7 @@ function run_core_tests() { function run_sql_tests() { echo "Run sql tests ..." - run_test "pyspark/sql/types.py" + run_test "pyspark/sql/_types.py" run_test "pyspark/sql/context.py" run_test "pyspark/sql/dataframe.py" run_test "pyspark/sql/functions.py" @@ -136,6 +136,19 @@ run_mllib_tests run_ml_tests run_streaming_tests +# Try to test with Python 3 +if [ $(which python3.4) ]; then + export PYSPARK_PYTHON="python3.4" + echo "Testing with Python3.4 version:" + $PYSPARK_PYTHON --version + + run_core_tests + run_sql_tests + run_mllib_tests + run_ml_tests + run_streaming_tests +fi + # Try to test with PyPy if [ $(which pypy) ]; then export PYSPARK_PYTHON="pypy" diff --git a/python/test_support/userlib-0.1-py2.7.egg b/python/test_support/userlib-0.1-py2.7.egg deleted file mode 100644 index 1674c9cb2227e160aec55c74e9ba1ddba850f580..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1945 zcmWIWW@Zs#U|`^2Sdq%-{OPC4zkVQZ3lQ@IacOaCQBG!(er{rBo?bzvZr?^OCPM+2 z-!(^Pyju5>Ifws9kif>ilh5?x&a}Ph`t|+UfpA{q4-(mHcXF|6ID3k0S`rmJ^Pu27 zX2pP|PDj;+_Dt5>5pw-nXZI!d?|Lntzt1q@eSGhRp0S;^U`hJyZ6+W7I>sGuc2o~I z;$-<*VantEcQTf&4@>aB-66eQMnKwrV%L&?hZb7x-!-%B+8>4hxZ7LO_?&A_9oL=$ zbn|W?2Kfl)_W1bByv&mLc%b`}lmDGLz4eV;d4LhKhymk9W&!bThE5TMq?8kh2`5rh zI+9Wnd=is5t|hQ}rTk-O;&oG4-6HJKBDP%2^0|tb`0vbyFVikRj(5)S6>%#*g z4>4Xznml`c(5%T>?3=boqzIt-qG4rZelQ~gLjn^6g8-5*pfQlVbi#eF!v-Slm#_J* zt#~NyvVK)UwfP*@J=O@3Im_WVolA58N~>g61`pSxe0_vE)< z^ZdsjUjEk5J^74tL*7Q7qYqkq3~ia@R@xk@{p5N=T`@m+{)dK?i}POB9N~!PDabP2 znxJE9d~uu8)sXNlS7bEW#Y9#FH7t}rbS~rCraZfArw&~@*7N!K)x8r0<{n>Yd*t

    waS|`|O?l3T$=A=7qdQyq0*MfDE zS-ZRgRd|K9^1{m+-Ws>d>^fX$@A>54M%x1?ijPX2D3<;<+wF7fk=$4Lh4pO9-@=?k}n_0dd;3x|M^$py6en$SghCm=SC!>>%c6)9BG#!3k-N^AQnc7 z0HkE(t$pfz=n3C5XM#_h(LJHB7099Mr(1N+Q%CO^6IygS{lCb@1vG^Nh{b@|)!kj! z)6dOcza+mPJ}0#-HAkN)2L?f8-Bp`(* z^5{v|nXIY_yAzriEjv{+f4c7E5CpoBkx7IZ6hzp|O`rrAY-t2hu#yOxhmkd7E4Uz9 zfrW769wg03=`R`G1oT1!VL}ry>7ZGUq8nR^N9bk-CO(*MB>T~=M^EGk0|J2tz!MQl zl1DcKJ*gwi=tnjKmhkau2c>%$*wT0qSv$5|fNm6eCO{bVAK56REP-wUdYVO;Fo^{z YJYXp}z?+o~q=XX)eSyWk87qhf04R`+K>z>% diff --git a/python/test_support/userlib-0.1.zip b/python/test_support/userlib-0.1.zip new file mode 100644 index 0000000000000000000000000000000000000000..496e1349aa9674cfa4f232ea8b5e17fcdb49f4fd GIT binary patch literal 668 zcmWIWW@h1H0D&_>m2O}Ll;B{HVJIz5Ey~GE(hm*cWMDr3PAF9Jy-;Xr1vdjD%U4DQ z1{M*Z+5or-kw6n(?`LOH28x2PFp>%J@tJv%$WKZ+A#ed zj*Lul%(&u00_Z;w5MX%g2%-@&!wQKRw17l54KrRKrU7G+VN0VEl4&Th1T+{+3_%RW k6#>WwZ$dH{9vwhKLD7NXT2?loY6fN?`~#%BKu%)-0QB3qpa1{> literal 0 HcmV?d00001 From 5fe43433529346788e8c343d338a5b7dc169cf58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=91=E5=B3=A4?= Date: Thu, 16 Apr 2015 17:32:42 -0700 Subject: [PATCH 792/817] SPARK-6927 [SQL] Sorting Error when codegen on MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix this error by adding BinaryType comparor in GenerateOrdering. JIRA https://issues.apache.org/jira/browse/SPARK-6927 Author: 云峤 Closes #5524 from kaka1992/fix-codegen-sort and squashes the following commits: d7e2afe [云峤] fix codegen sorting error --- .../codegen/GenerateOrdering.scala | 14 ++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 20 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 0db29eb404bd1..fc2a2b60703e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NumericType} +import org.apache.spark.sql.types.{BinaryType, StringType, NumericType} /** * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of @@ -43,6 +43,18 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit val evalB = expressionEvaluator(order.child) val compare = order.child.dataType match { + case BinaryType => + q""" + val x = ${if (order.direction == Ascending) evalA.primitiveTerm else evalB.primitiveTerm} + val y = ${if (order.direction != Ascending) evalB.primitiveTerm else evalA.primitiveTerm} + var i = 0 + while (i < x.length && i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + i = i+1 + } + return x.length - y.length + """ case _: NumericType => q""" val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index d739e550f3e56..9e02e69fda3f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -398,6 +398,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { setConf(SQLConf.EXTERNAL_SORT, before.toString) } + test("SPARK-6927 sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.EXTERNAL_SORT, "false") + setConf(SQLConf.CODEGEN_ENABLED, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + + test("SPARK-6927 external sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.CODEGEN_ENABLED, "true") + setConf(SQLConf.EXTERNAL_SORT, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + test("limit") { checkAnswer( sql("SELECT * FROM testData LIMIT 10"), From 6183b5e2caedd074073d0f6cb6609a634e2f5194 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 17:33:57 -0700 Subject: [PATCH 793/817] [SPARK-6911] [SQL] improve accessor for nested types Support access columns by index in Python: ``` >>> df[df[0] > 3].collect() [Row(age=5, name=u'Bob')] ``` Access items in ArrayType or MapType ``` >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() >>> df.select(df.l[0], df.d["key"]).show() ``` Access field in StructType ``` >>> df.select(df.r.getField("b")).show() >>> df.select(df.r.a).show() ``` Author: Davies Liu Closes #5513 from davies/access and squashes the following commits: e04d5a0 [Davies Liu] Update run-tests-jenkins 7ada9eb [Davies Liu] update timeout d125ac4 [Davies Liu] check column name, improve scala tests 6b62540 [Davies Liu] fix test db15b42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into access 6c32e79 [Davies Liu] add scala tests 11f1df3 [Davies Liu] improve accessor for nested types --- python/pyspark/sql/dataframe.py | 49 +++++++++++++++++-- python/pyspark/sql/tests.py | 18 +++++++ .../scala/org/apache/spark/sql/Column.scala | 7 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +++ .../scala/org/apache/spark/sql/TestData.scala | 9 ++-- 5 files changed, 76 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d76504f986270..b9a3e6cfe7f49 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -563,16 +563,23 @@ def __getitem__(self, item): [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df[ df.age > 3 ].collect() [Row(age=5, name=u'Bob')] + >>> df[df[0] > 3].collect() + [Row(age=5, name=u'Bob')] """ if isinstance(item, basestring): + if item not in self.columns: + raise IndexError("no such column: %s" % item) jc = self._jdf.apply(item) return Column(jc) elif isinstance(item, Column): return self.filter(item) - elif isinstance(item, list): + elif isinstance(item, (list, tuple)): return self.select(*item) + elif isinstance(item, int): + jc = self._jdf.apply(self.columns[item]) + return Column(jc) else: - raise IndexError("unexpected index: %s" % item) + raise TypeError("unexpected type: %s" % type(item)) def __getattr__(self, name): """Returns the :class:`Column` denoted by ``name``. @@ -580,8 +587,8 @@ def __getattr__(self, name): >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] """ - if name.startswith("__"): - raise AttributeError(name) + if name not in self.columns: + raise AttributeError("No such column: %s" % name) jc = self._jdf.apply(name) return Column(jc) @@ -1093,7 +1100,39 @@ def __init__(self, jc): # container operators __contains__ = _bin_op("contains") __getitem__ = _bin_op("getItem") - getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") + + def getItem(self, key): + """An expression that gets an item at position `ordinal` out of a list, + or gets an item by key out of a dict. + + >>> df = sc.parallelize([([1, 2], {"key": "value"})]).toDF(["l", "d"]) + >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() + l[0] d[key] + 1 value + >>> df.select(df.l[0], df.d["key"]).show() + l[0] d[key] + 1 value + """ + return self[key] + + def getField(self, name): + """An expression that gets a field by name in a StructField. + + >>> from pyspark.sql import Row + >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() + >>> df.select(df.r.getField("b")).show() + r.b + b + >>> df.select(df.r.a).show() + r.a + 1 + """ + return Column(self._jc.getField(name)) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + return self.getField(item) # string methods rlike = _bin_op("rlike") diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 7c09a0cfe30ab..6691e8c8dc44b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -426,6 +426,24 @@ def test_help_command(self): pydoc.render_doc(df.foo) pydoc.render_doc(df.take(1)) + def test_access_column(self): + df = self.df + self.assertTrue(isinstance(df.key, Column)) + self.assertTrue(isinstance(df['key'], Column)) + self.assertTrue(isinstance(df[0], Column)) + self.assertRaises(IndexError, lambda: df[2]) + self.assertRaises(IndexError, lambda: df["bad_key"]) + self.assertRaises(TypeError, lambda: df[{}]) + + def test_access_nested_types(self): + df = self.sc.parallelize([Row(l=[1], r=Row(a=1, b="b"), d={"k": "v"})]).toDF() + self.assertEqual(1, df.select(df.l[0]).first()[0]) + self.assertEqual(1, df.select(df.l.getItem(0)).first()[0]) + self.assertEqual(1, df.select(df.r.a).first()[0]) + self.assertEqual("b", df.select(df.r.getField("b")).first()[0]) + self.assertEqual("v", df.select(df.d["k"]).first()[0]) + self.assertEqual("v", df.select(df.d.getItem("k")).first()[0]) + def test_infer_long_type(self): longrow = [Row(f1='a', f2=100000000000000)] df = self.sc.parallelize(longrow).toDF() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 3cd7adf8cab5e..edb229c059e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -515,14 +515,15 @@ class Column(protected[sql] val expr: Expression) extends Logging { def rlike(literal: String): Column = RLike(expr, lit(literal).expr) /** - * An expression that gets an item at position `ordinal` out of an array. + * An expression that gets an item at position `ordinal` out of an array, + * or gets a value by key `key` in a [[MapType]]. * * @group expr_ops */ - def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + def getItem(key: Any): Column = GetItem(expr, Literal(key)) /** - * An expression that gets a field by name in a [[StructField]]. + * An expression that gets a field by name in a [[StructType]]. * * @group expr_ops */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b26e22f6229fe..34b2cb054a3e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -86,6 +86,12 @@ class DataFrameSuite extends QueryTest { TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } + test("access complex data") { + assert(complexData.filter(complexData("a").getItem(0) === 2).count() == 1) + assert(complexData.filter(complexData("m").getItem("1") === 1).count() == 1) + assert(complexData.filter(complexData("s").getField("key") === 1).count() == 1) + } + test("table scan") { checkAnswer( testData, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 637f59b2e68ca..225b51bd73d6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.test._ case class TestData(key: Int, value: String) @@ -199,11 +198,11 @@ object TestData { Salary(1, 1000.0) :: Nil).toDF() salary.registerTempTable("salary") - case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) + case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) val complexData = TestSQLContext.sparkContext.parallelize( - ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) - :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) + ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1), true) + :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2), false) :: Nil).toDF() complexData.registerTempTable("complexData") } From d96608674f6c2ff3abb13c65d80c1a3872206710 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 16 Apr 2015 17:35:51 -0700 Subject: [PATCH 794/817] [SQL][Minor] Fix foreachUp of treenode `foreachUp` should runs the given function recursively on [[children]] then on this node(just like transformUp). The current implementation does not follow this. This will leads to checkanalysis do not check from bottom of logical tree. Author: scwf Author: Fei Wang Closes #5518 from scwf/patch-1 and squashes the following commits: 18e28b2 [scwf] added a test case 1ccbfa8 [Fei Wang] fix foreachUp --- .../apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index a2df51e598a2b..97502ed3afe72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -85,7 +85,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param f the function to be applied to each node in the tree. */ def foreachUp(f: BaseType => Unit): Unit = { - children.foreach(_.foreach(f)) + children.foreach(_.foreachUp(f)) f(this) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 4eb8708335dcf..6b393327cc97a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -117,5 +117,17 @@ class TreeNodeSuite extends FunSuite { assert(transformed.origin.startPosition.isDefined) } + test("foreach up") { + val actual = new ArrayBuffer[String]() + val expected = Seq("1", "2", "3", "4", "-", "*", "+") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression foreachUp { + case b: BinaryExpression => actual.append(b.symbol); + case l: Literal => actual.append(l.toString); + } + + assert(expected === actual) + } + } From 1e43851d6455f65b850ea0327d0e92f65395d23f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 16 Apr 2015 17:50:20 -0700 Subject: [PATCH 795/817] [SPARK-6899][SQL] Fix type mismatch when using codegen with Average on DecimalType JIRA https://issues.apache.org/jira/browse/SPARK-6899 Author: Liang-Chi Hsieh Closes #5517 from viirya/fix_codegen_average and squashes the following commits: 8ae5f65 [Liang-Chi Hsieh] Add the case of DecimalType.Unlimited to Average. --- .../spark/sql/catalyst/expressions/aggregates.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 14a855054b94d..f3830c6d3bcf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -326,7 +326,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN override def asPartial: SplitEvaluation = { child.dataType match { - case DecimalType.Fixed(_, _) => + case DecimalType.Fixed(_, _) | DecimalType.Unlimited => // Turn the child to unlimited decimals for calculation, before going back to fixed val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 34b2cb054a3e7..44a7d1e7bbb6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -537,4 +537,13 @@ class DataFrameSuite extends QueryTest { val df = TestSQLContext.createDataFrame(rowRDD, schema) df.rdd.collect() } + + test("SPARK-6899") { + val originalValue = TestSQLContext.conf.codegenEnabled + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, "true") + checkAnswer( + decimalData.agg(avg('a)), + Row(new java.math.BigDecimal(2.0))) + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) + } } From e5949c287ed19e78b6eecc61c3e88a07ad452eb9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 17:59:49 -0700 Subject: [PATCH 796/817] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver Otherwise we cannot add jars with drivers after the fact. Author: Michael Armbrust Closes #5543 from marmbrus/jdbcClassloader and squashes the following commits: d9930f3 [Michael Armbrust] fix imports 73d0614 [Michael Armbrust] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver --- .../main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 99b755c9f25d0..5f480083d5a49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils /** * Data corresponding to one partition of a JDBCRDD. @@ -99,7 +100,7 @@ private[sql] class DefaultSource extends RelationProvider { val upperBound = parameters.getOrElse("upperBound", null) val numPartitions = parameters.getOrElse("numPartitions", null) - if (driver != null) Class.forName(driver) + if (driver != null) Utils.getContextOrSparkClassLoader.loadClass(driver) if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { From 8220d5265f1bbea9dfdaeec4f2d06d7fe24c0bc3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 21:49:26 -0500 Subject: [PATCH 797/817] [SPARK-6972][SQL] Add Coalesce to DataFrame Author: Michael Armbrust Closes #5545 from marmbrus/addCoalesce and squashes the following commits: 9fdf3f6 [Michael Armbrust] [SPARK-6972][SQL] Add Coalesce to DataFrame --- .../scala/org/apache/spark/sql/DataFrame.scala | 14 ++++++++++++++ .../main/scala/org/apache/spark/sql/RDDApi.scala | 2 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 3 files changed, 24 insertions(+) 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 3235f85d5bbd2..17c21f6e3a0e9 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 @@ -908,6 +908,20 @@ class DataFrame private[sql]( schema, needsConversion = false) } + /** + * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. + * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. + * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of + * the 100 new partitions will claim 10 of the current partitions. + * @group rdd + */ + override def coalesce(numPartitions: Int): DataFrame = { + sqlContext.createDataFrame( + queryExecution.toRdd.coalesce(numPartitions), + schema, + needsConversion = false) + } + /** * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. * @group dfops diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index ba4373f0124b4..63dbab19947c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -61,5 +61,7 @@ private[sql] trait RDDApi[T] { def repartition(numPartitions: Int): DataFrame + def coalesce(numPartitions: Int): DataFrame + def distinct: DataFrame } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 44a7d1e7bbb6a..3250ab476aeb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -178,6 +178,14 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } + test("coalesce") { + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) + + checkAnswer( + testData.select('key).coalesce(1).select('key), + testData.select('key).collect().toSeq) + } + test("groupBy") { checkAnswer( testData2.groupBy("a").agg($"a", sum($"b")), From f7a25644ed5b3b49fe7f33743bec3d95cdf7913e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 17 Apr 2015 11:02:31 +0100 Subject: [PATCH 798/817] SPARK-6846 [WEBUI] Stage kill URL easy to accidentally trigger and possibility for security issue kill endpoints now only accept a POST (kill stage, master kill app, master kill driver); kill link now POSTs Author: Sean Owen Closes #5528 from srowen/SPARK-6846 and squashes the following commits: 137ac9f [Sean Owen] Oops, fix scalastyle line length probelm 7c5f961 [Sean Owen] Add Imran's test of kill link 59f447d [Sean Owen] kill endpoints now only accept a POST (kill stage, master kill app, master kill driver); kill link now POSTs --- .../org/apache/spark/ui/static/webui.css | 6 +-- .../spark/deploy/master/ui/MasterPage.scala | 28 +++++++------ .../spark/deploy/master/ui/MasterWebUI.scala | 8 ++-- .../org/apache/spark/ui/JettyUtils.scala | 17 +++++++- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../org/apache/spark/ui/jobs/StageTable.scala | 27 ++++++------- .../org/apache/spark/ui/UISeleniumSuite.scala | 40 +++++++++++++------ 7 files changed, 78 insertions(+), 52 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 6c37cc8b98236..4910744d1d790 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -85,17 +85,13 @@ table.sortable td { filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } -span.kill-link { +a.kill-link { margin-right: 2px; margin-left: 20px; color: gray; float: right; } -span.kill-link a { - color: gray; -} - span.expand-details { font-size: 10pt; cursor: pointer; diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 399f07399a0aa..1f2c3fdbfb2bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -190,12 +190,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def appRow(app: ApplicationInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { - val killLinkUri = s"app/kill?id=${app.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill application ${app.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill application ${app.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) + }

    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 1b670418ab1ff..bb11e0642ddc6 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 @@ -43,10 +43,10 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler( - createRedirectHandler("/app/kill", "/", masterPage.handleAppKillRequest)) - attachHandler( - createRedirectHandler("/driver/kill", "/", masterPage.handleDriverKillRequest)) + attachHandler(createRedirectHandler( + "/app/kill", "/", masterPage.handleAppKillRequest, httpMethod = "POST")) + attachHandler(createRedirectHandler( + "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethod = "POST")) } /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 95f254a9ef22a..a091ca650c60c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -114,10 +114,23 @@ private[spark] object JettyUtils extends Logging { srcPath: String, destPath: String, beforeRedirect: HttpServletRequest => Unit = x => (), - basePath: String = ""): ServletContextHandler = { + basePath: String = "", + httpMethod: String = "GET"): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { - override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse): Unit = { + httpMethod match { + case "GET" => doRequest(request, response) + case _ => response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + override def doPost(request: HttpServletRequest, response: HttpServletResponse): Unit = { + httpMethod match { + case "POST" => doRequest(request, response) + case _ => response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + private def doRequest(request: HttpServletRequest, response: HttpServletResponse): Unit = { beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString 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 adfa6bbada256..580ab8b1325f8 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -55,8 +55,8 @@ private[spark] class SparkUI private ( attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) + attachHandler(createRedirectHandler( + "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, httpMethod = "POST")) } initialize() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5865850fa09b5..cb72890a0fd20 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -73,20 +73,21 @@ private[ui] class StageTableBase( } private def makeDescription(s: StageInfo): Seq[Node] = { - // scalastyle:off + val basePathUri = UIUtils.prependBaseUri(basePath) + val killLink = if (killEnabled) { - val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(basePath), s.stageId) - val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" - .format(s.stageId) - - (kill) - + val killLinkUri = s"$basePathUri/stages/stage/kill/" + val confirm = + s"if (window.confirm('Are you sure you want to kill stage ${s.stageId} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" + + + + (kill) + } - // scalastyle:on - val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) + val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -98,11 +99,9 @@ private[ui] class StageTableBase( 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 1cb594633f331..eb9db550fd74c 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui +import java.net.{HttpURLConnection, URL} import javax.servlet.http.HttpServletRequest import scala.collection.JavaConversions._ @@ -56,12 +57,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before * Create a test SparkContext with the SparkUI enabled. * It is safe to `get` the SparkUI directly from the SparkContext returned here. */ - private def newSparkContext(): SparkContext = { + private def newSparkContext(killEnabled: Boolean = true): SparkContext = { val conf = new SparkConf() .setMaster("local") .setAppName("test") .set("spark.ui.enabled", "true") .set("spark.ui.port", "0") + .set("spark.ui.killEnabled", killEnabled.toString) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -128,21 +130,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } test("spark.ui.killEnabled should properly control kill button display") { - def getSparkContext(killEnabled: Boolean): SparkContext = { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.killEnabled", killEnabled.toString) - new SparkContext(conf) - } - def hasKillLink: Boolean = find(className("kill-link")).isDefined def runSlowJob(sc: SparkContext) { sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } - withSpark(getSparkContext(killEnabled = true)) { sc => + withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") @@ -150,7 +143,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } - withSpark(getSparkContext(killEnabled = false)) { sc => + withSpark(newSparkContext(killEnabled = false)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") @@ -233,7 +226,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // because someone could change the error message and cause this test to pass by accident. // Instead, it's safer to check that each row contains a link to a stage details page. findAll(cssSelector("tbody tr")).foreach { row => - val link = row.underlying.findElement(By.xpath(".//a")) + val link = row.underlying.findElement(By.xpath("./td/div/a")) link.getAttribute("href") should include ("stage") } } @@ -356,4 +349,25 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("kill stage is POST only") { + def getResponseCode(url: URL, method: String): Int = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod(method) + connection.connect() + val code = connection.getResponseCode() + connection.disconnect() + code + } + + withSpark(newSparkContext(killEnabled = true)) { sc => + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + val url = new URL( + sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0&terminate=true") + getResponseCode(url, "GET") should be (405) + getResponseCode(url, "POST") should be (200) + } + } + } } From 4527761bcd6501c362baf2780905a0018b9a74ba Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 11:06:01 +0100 Subject: [PATCH 799/817] [SPARK-6046] [core] Reorganize deprecated config support in SparkConf. This change tries to follow the chosen way for handling deprecated configs in SparkConf: all values (old and new) are kept in the conf object, and newer names take precedence over older ones when retrieving the value. Warnings are logged when config options are set, which generally happens on the driver node (where the logs are most visible). Author: Marcelo Vanzin Closes #5514 from vanzin/SPARK-6046 and squashes the following commits: 9371529 [Marcelo Vanzin] Avoid math. 6cf3f11 [Marcelo Vanzin] Review feedback. 2445d48 [Marcelo Vanzin] Fix (and cleanup) update interval initialization. b6824be [Marcelo Vanzin] Clean up the other deprecated config use also. ab20351 [Marcelo Vanzin] Update FsHistoryProvider to only retrieve new config key. 2c93209 [Marcelo Vanzin] [SPARK-6046] [core] Reorganize deprecated config support in SparkConf. --- .../scala/org/apache/spark/SparkConf.scala | 174 ++++++++++-------- .../deploy/history/FsHistoryProvider.scala | 9 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../org/apache/spark/SparkConfSuite.scala | 22 +++ docs/monitoring.md | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 3 +- 6 files changed, 124 insertions(+), 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 390e631647bd6..b0186e9a007b8 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -68,6 +68,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (value == null) { throw new NullPointerException("null value for " + key) } + logDeprecationWarning(key) settings.put(key, value) this } @@ -134,13 +135,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Set multiple parameters together */ def setAll(settings: Traversable[(String, String)]): SparkConf = { - this.settings.putAll(settings.toMap.asJava) + settings.foreach { case (k, v) => set(k, v) } this } /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { - settings.putIfAbsent(key, value) + if (settings.putIfAbsent(key, value) == null) { + logDeprecationWarning(key) + } this } @@ -174,8 +177,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } - /** - * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no * suffix is provided then seconds are assumed. * @throws NoSuchElementException */ @@ -183,36 +186,35 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { Utils.timeStringAsSeconds(get(key)) } - /** - * Get a time parameter as seconds, falling back to a default if not set. If no + /** + * Get a time parameter as seconds, falling back to a default if not set. If no * suffix is provided then seconds are assumed. - * */ def getTimeAsSeconds(key: String, defaultValue: String): Long = { Utils.timeStringAsSeconds(get(key, defaultValue)) } - /** - * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no - * suffix is provided then milliseconds are assumed. + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. * @throws NoSuchElementException */ def getTimeAsMs(key: String): Long = { Utils.timeStringAsMs(get(key)) } - /** - * Get a time parameter as milliseconds, falling back to a default if not set. If no - * suffix is provided then milliseconds are assumed. + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. */ def getTimeAsMs(key: String, defaultValue: String): Long = { Utils.timeStringAsMs(get(key, defaultValue)) } - + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { - Option(settings.get(key)) + Option(settings.get(key)).orElse(getDeprecatedConfig(key, this)) } /** Get all parameters as a list of pairs */ @@ -379,13 +381,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } } - - // Warn against the use of deprecated configs - deprecatedConfigs.values.foreach { dc => - if (contains(dc.oldName)) { - dc.warn() - } - } } /** @@ -400,19 +395,44 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { private[spark] object SparkConf extends Logging { + /** + * Maps deprecated config keys to information about the deprecation. + * + * The extra information is logged as a warning when the config is present in the user's + * configuration. + */ private val deprecatedConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( - DeprecatedConfig("spark.files.userClassPathFirst", "spark.executor.userClassPathFirst", - "1.3"), - DeprecatedConfig("spark.yarn.user.classpath.first", null, "1.3", - "Use spark.{driver,executor}.userClassPathFirst instead."), - DeprecatedConfig("spark.history.fs.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead"), - DeprecatedConfig("spark.history.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead")) - configs.map { x => (x.oldName, x) }.toMap + DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", + "Please use spark.{driver,executor}.userClassPathFirst instead.")) + Map(configs.map { cfg => (cfg.key -> cfg) }:_*) + } + + /** + * Maps a current config key to alternate keys that were used in previous version of Spark. + * + * The alternates are used in the order defined in this map. If deprecated configs are + * present in the user's configuration, a warning is logged. + */ + private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( + "spark.executor.userClassPathFirst" -> Seq( + AlternateConfig("spark.files.userClassPathFirst", "1.3")), + "spark.history.fs.update.interval" -> Seq( + AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), + AlternateConfig("spark.history.fs.updateInterval", "1.3"), + AlternateConfig("spark.history.updateInterval", "1.3")) + ) + + /** + * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated + * config keys. + * + * Maps the deprecated config name to a 2-tuple (new config name, alternate config info). + */ + private val allAlternatives: Map[String, (String, AlternateConfig)] = { + configsWithAlternatives.keys.flatMap { key => + configsWithAlternatives(key).map { cfg => (cfg.key -> (key -> cfg)) } + }.toMap } /** @@ -443,61 +463,57 @@ private[spark] object SparkConf extends Logging { } /** - * Translate the configuration key if it is deprecated and has a replacement, otherwise just - * returns the provided key. - * - * @param userKey Configuration key from the user / caller. - * @param warn Whether to print a warning if the key is deprecated. Warnings will be printed - * only once for each key. + * Looks for available deprecated keys for the given config option, and return the first + * value available. */ - private def translateConfKey(userKey: String, warn: Boolean = false): String = { - deprecatedConfigs.get(userKey) - .map { deprecatedKey => - if (warn) { - deprecatedKey.warn() - } - deprecatedKey.newName.getOrElse(userKey) - }.getOrElse(userKey) + def getDeprecatedConfig(key: String, conf: SparkConf): Option[String] = { + configsWithAlternatives.get(key).flatMap { alts => + alts.collectFirst { case alt if conf.contains(alt.key) => + val value = conf.get(alt.key) + alt.translation.map(_(value)).getOrElse(value) + } + } } /** - * Holds information about keys that have been deprecated or renamed. + * Logs a warning message if the given config key is deprecated. + */ + def logDeprecationWarning(key: String): Unit = { + deprecatedConfigs.get(key).foreach { cfg => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"may be removed in the future. ${cfg.deprecationMessage}") + } + + allAlternatives.get(key).foreach { case (newKey, cfg) => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"and may be removed in the future. Please use the new key '$newKey' instead.") + } + } + + /** + * Holds information about keys that have been deprecated and do not have a replacement. * - * @param oldName Old configuration key. - * @param newName New configuration key, or `null` if key has no replacement, in which case the - * deprecated key will be used (but the warning message will still be printed). + * @param key The deprecated key. * @param version Version of Spark where key was deprecated. - * @param deprecationMessage Message to include in the deprecation warning; mandatory when - * `newName` is not provided. + * @param deprecationMessage Message to include in the deprecation warning. */ private case class DeprecatedConfig( - oldName: String, - _newName: String, + key: String, version: String, - deprecationMessage: String = null) { - - private val warned = new AtomicBoolean(false) - val newName = Option(_newName) + deprecationMessage: String) - if (newName == null && (deprecationMessage == null || deprecationMessage.isEmpty())) { - throw new IllegalArgumentException("Need new config name or deprecation message.") - } - - def warn(): Unit = { - if (warned.compareAndSet(false, true)) { - if (newName != null) { - val message = Option(deprecationMessage).getOrElse( - s"Please use the alternative '$newName' instead.") - logWarning( - s"The configuration option '$oldName' has been replaced as of Spark $version and " + - s"may be removed in the future. $message") - } else { - logWarning( - s"The configuration option '$oldName' has been deprecated as of Spark $version and " + - s"may be removed in the future. $deprecationMessage") - } - } - } + /** + * Information about an alternate configuration key that has been deprecated. + * + * @param key The deprecated config key. + * @param version The Spark version in which the key was deprecated. + * @param translation A translation function for converting old config values into new ones. + */ + private case class AlternateConfig( + key: String, + version: String, + translation: Option[String => String] = None) - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 9d40d8c8fd7a8..985545742df67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -49,11 +49,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val NOT_STARTED = "" // Interval between each check for event log updates - private val UPDATE_INTERVAL_MS = conf.getOption("spark.history.fs.update.interval.seconds") - .orElse(conf.getOption("spark.history.fs.updateInterval")) - .orElse(conf.getOption("spark.history.updateInterval")) - .map(_.toInt) - .getOrElse(10) * 1000 + private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") // Interval between each cleaner checks for event logs to delete private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", @@ -130,8 +126,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Disable the background thread during tests. if (!conf.contains("spark.testing")) { // A task that periodically checks for event log updates on disk. - pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS) if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { // A task that periodically cleans event logs on disk. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 1b5fdeba28ee2..327d155b38c22 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -89,10 +89,7 @@ private[spark] class Executor( ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) // Whether to load classes in user jars before those in Spark jars - private val userClassPathFirst: Boolean = { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) - } + private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index e08210ae60d17..7d87ba5fd2610 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -197,6 +197,28 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro serializer.newInstance().serialize(new StringBuffer()) } + test("deprecated configs") { + val conf = new SparkConf() + val newName = "spark.history.fs.update.interval" + + assert(!conf.contains(newName)) + + conf.set("spark.history.updateInterval", "1") + assert(conf.get(newName) === "1") + + conf.set("spark.history.fs.updateInterval", "2") + assert(conf.get(newName) === "2") + + conf.set("spark.history.fs.update.interval.seconds", "3") + assert(conf.get(newName) === "3") + + conf.set(newName, "4") + assert(conf.get(newName) === "4") + + val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size + assert(count === 4) + } + } class Class1 {} diff --git a/docs/monitoring.md b/docs/monitoring.md index 6816671ffbf46..2a130224591ca 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -86,10 +86,10 @@ follows: - - + + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1091ff54b0463..52e4dee46c535 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1052,8 +1052,7 @@ object Client extends Logging { if (isDriver) { conf.getBoolean("spark.driver.userClassPathFirst", false) } else { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) + conf.getBoolean("spark.executor.userClassPathFirst", false) } } From f6a9a57a72767f48fcc02e5fda4d6eafa67aebde Mon Sep 17 00:00:00 2001 From: Punya Biswal Date: Fri, 17 Apr 2015 11:08:37 +0100 Subject: [PATCH 800/817] [SPARK-6952] Handle long args when detecting PID reuse sbin/spark-daemon.sh used ps -p "$TARGET_PID" -o args= to figure out whether the process running with the expected PID is actually a Spark daemon. When running with a large classpath, the output of ps gets truncated and the check fails spuriously. This weakens the check to see if it's a java command (which is something we do in other parts of the script) rather than looking for the specific main class name. This means that SPARK-4832 might happen under a slightly broader range of circumstances (a java program happened to reuse the same PID), but it seems worthwhile compared to failing consistently with a large classpath. Author: Punya Biswal Closes #5535 from punya/feature/SPARK-6952 and squashes the following commits: 7ea12d1 [Punya Biswal] Handle long args when detecting PID reuse --- sbin/spark-daemon.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index d8e0facb81169..de762acc8fa0e 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -129,7 +129,7 @@ run_command() { if [ -f "$pid" ]; then TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then echo "$command running as process $TARGET_ID. Stop it first." exit 1 fi @@ -163,7 +163,7 @@ run_command() { echo "$newpid" > "$pid" sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then + if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then echo "failed to launch $command:" tail -2 "$log" | sed 's/^/ /' echo "full log in $log" From dc48ba9f9f7449dd2f12cbad288b65c8119d9284 Mon Sep 17 00:00:00 2001 From: linweizhong Date: Fri, 17 Apr 2015 12:04:02 +0100 Subject: [PATCH 801/817] [SPARK-6604][PySpark]Specify ip of python server scoket In driver now will start a server socket and use a wildcard ip, use 127.0.0.0 is more reasonable, as we only use it by local Python process. /cc davies Author: linweizhong Closes #5256 from Sephiroth-Lin/SPARK-6604 and squashes the following commits: 7b3c633 [linweizhong] rephrase --- core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b1ffba4c546bf..7409dc2d866f6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -604,7 +604,7 @@ private[spark] object PythonRDD extends Logging { * The thread will terminate after all the data are sent or any exceptions happen. */ private def serveIterator[T](items: Iterator[T], threadName: String): Int = { - val serverSocket = new ServerSocket(0, 1) + val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) // Close the socket if no connection in 3 seconds serverSocket.setSoTimeout(3000) From c84d91692aa25c01882bcc3f9fd5de3cfa786195 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 17 Apr 2015 11:29:27 -0500 Subject: [PATCH 802/817] [SPARK-6957] [SPARK-6958] [SQL] improve API compatibility to pandas ``` select(['cola', 'colb']) groupby(['colA', 'colB']) groupby([df.colA, df.colB]) df.sort('A', ascending=True) df.sort(['A', 'B'], ascending=True) df.sort(['A', 'B'], ascending=[1, 0]) ``` cc rxin Author: Davies Liu Closes #5544 from davies/compatibility and squashes the following commits: 4944058 [Davies Liu] add docstrings adb2816 [Davies Liu] Merge branch 'master' of github.com:apache/spark into compatibility bcbbcab [Davies Liu] support ascending as list 8dabdf0 [Davies Liu] improve API compatibility to pandas --- python/pyspark/sql/dataframe.py | 96 ++++++++++++++++++++++----------- python/pyspark/sql/functions.py | 11 ++-- python/pyspark/sql/tests.py | 2 +- 3 files changed, 70 insertions(+), 39 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b9a3e6cfe7f49..326d22e72f104 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -485,13 +485,17 @@ def join(self, other, joinExprs=None, joinType=None): return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix - def sort(self, *cols): + def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). - :param cols: list of :class:`Column` to sort by. + :param cols: list of :class:`Column` or column names to sort by. + :param ascending: sort by ascending order or not, could be bool, int + or list of bool, int (default: True). >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.sort("age", ascending=False).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> df.orderBy(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> from pyspark.sql.functions import * @@ -499,16 +503,42 @@ def sort(self, *cols): [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] >>> df.orderBy(desc("age"), "name").collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.orderBy(["age", "name"], ascending=[0, 1]).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] """ if not cols: raise ValueError("should sort by at least one column") - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + jcols = [_to_java_column(c) for c in cols] + ascending = kwargs.get('ascending', True) + if isinstance(ascending, (bool, int)): + if not ascending: + jcols = [jc.desc() for jc in jcols] + elif isinstance(ascending, list): + jcols = [jc if asc else jc.desc() + for asc, jc in zip(ascending, jcols)] + else: + raise TypeError("ascending can only be bool or list, but got %s" % type(ascending)) + + jdf = self._jdf.sort(self._jseq(jcols)) return DataFrame(jdf, self.sql_ctx) orderBy = sort + def _jseq(self, cols, converter=None): + """Return a JVM Seq of Columns from a list of Column or names""" + return _to_seq(self.sql_ctx._sc, cols, converter) + + def _jcols(self, *cols): + """Return a JVM Seq of Columns from a list of Column or column names + + If `cols` has only one list in it, cols[0] will be used as the list. + """ + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + return self._jseq(cols, _to_java_column) + def describe(self, *cols): """Computes statistics for numeric columns. @@ -523,9 +553,7 @@ def describe(self, *cols): min 2 max 5 """ - cols = ListConverter().convert(cols, - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) + jdf = self._jdf.describe(self._jseq(cols)) return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix @@ -607,9 +635,7 @@ def select(self, *cols): >>> df.select(df.name, (df.age + 10).alias('age')).collect() [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.select(self._jcols(*cols)) return DataFrame(jdf, self.sql_ctx) def selectExpr(self, *expr): @@ -620,8 +646,9 @@ def selectExpr(self, *expr): >>> df.selectExpr("age * 2", "abs(age)").collect() [Row((age * 2)=4, Abs(age)=2), Row((age * 2)=10, Abs(age)=5)] """ - jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) - jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) + if len(expr) == 1 and isinstance(expr[0], list): + expr = expr[0] + jdf = self._jdf.selectExpr(self._jseq(expr)) return DataFrame(jdf, self.sql_ctx) @ignore_unicode_prefix @@ -659,6 +686,8 @@ def groupBy(self, *cols): so we can run aggregation on them. See :class:`GroupedData` for all the available aggregate functions. + :func:`groupby` is an alias for :func:`groupBy`. + :param cols: list of columns to group by. Each element should be a column name (string) or an expression (:class:`Column`). @@ -668,12 +697,14 @@ def groupBy(self, *cols): [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] + >>> df.groupBy(['name', df.age]).count().collect() + [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.groupBy(self._jcols(*cols)) return GroupedData(jdf, self.sql_ctx) + groupby = groupBy + def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups (shorthand for ``df.groupBy.agg()``). @@ -744,9 +775,7 @@ def dropna(self, how='any', thresh=None, subset=None): if thresh is None: thresh = len(subset) if how == 'any' else 1 - cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) - cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) - return DataFrame(self._jdf.na().drop(thresh, cols), self.sql_ctx) + return DataFrame(self._jdf.na().drop(thresh, self._jseq(subset)), self.sql_ctx) def fillna(self, value, subset=None): """Replace null values, alias for ``na.fill()``. @@ -799,9 +828,7 @@ def fillna(self, value, subset=None): elif not isinstance(subset, (list, tuple)): raise ValueError("subset should be a list or tuple of column names") - cols = ListConverter().convert(subset, self.sql_ctx._sc._gateway._gateway_client) - cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) - return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + return DataFrame(self._jdf.na().fill(value, self._jseq(subset)), self.sql_ctx) @ignore_unicode_prefix def withColumn(self, colName, col): @@ -862,10 +889,8 @@ def _api(self): def df_varargs_api(f): def _api(self, *args): - jargs = ListConverter().convert(args, - self.sql_ctx._sc._gateway._gateway_client) name = f.__name__ - jdf = getattr(self._jdf, name)(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jargs)) + jdf = getattr(self._jdf, name)(_to_seq(self.sql_ctx._sc, args)) return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -912,9 +937,8 @@ def agg(self, *exprs): else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - jcols = ListConverter().convert([c._jc for c in exprs[1:]], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.agg(exprs[0]._jc, + _to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]])) return DataFrame(jdf, self.sql_ctx) @dfapi @@ -1006,6 +1030,19 @@ def _to_java_column(col): return jcol +def _to_seq(sc, cols, converter=None): + """ + Convert a list of Column (or names) into a JVM Seq of Column. + + An optional `converter` could be used to convert items in `cols` + into JVM Column objects. + """ + if converter: + cols = [converter(c) for c in cols] + jcols = ListConverter().convert(cols, sc._gateway._gateway_client) + return sc._jvm.PythonUtils.toSeq(jcols) + + def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): @@ -1177,8 +1214,7 @@ def inSet(self, *cols): cols = cols[0] cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols] sc = SparkContext._active_spark_context - jcols = ListConverter().convert(cols, sc._gateway._gateway_client) - jc = getattr(self._jc, "in")(sc._jvm.PythonUtils.toSeq(jcols)) + jc = getattr(self._jc, "in")(_to_seq(sc, cols)) return Column(jc) # order diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 1d6536952810f..bb47923f24b82 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -23,13 +23,11 @@ if sys.version < "3": from itertools import imap as map -from py4j.java_collections import ListConverter - from pyspark import SparkContext from pyspark.rdd import _prepare_for_python_RDD from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.sql.types import StringType -from pyspark.sql.dataframe import Column, _to_java_column +from pyspark.sql.dataframe import Column, _to_java_column, _to_seq __all__ = ['countDistinct', 'approxCountDistinct', 'udf'] @@ -87,8 +85,7 @@ def countDistinct(col, *cols): [Row(c=2)] """ sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], sc._gateway._gateway_client) - jc = sc._jvm.functions.countDistinct(_to_java_column(col), sc._jvm.PythonUtils.toSeq(jcols)) + jc = sc._jvm.functions.countDistinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column)) return Column(jc) @@ -138,9 +135,7 @@ def __del__(self): def __call__(self, *cols): sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) + jc = self._judf.apply(_to_seq(sc, cols, _to_java_column)) return Column(jc) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 6691e8c8dc44b..aa3aa1d164d9f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -282,7 +282,7 @@ def test_apply_schema(self): StructField("struct1", StructType([StructField("b", ShortType(), False)]), False), StructField("list1", ArrayType(ByteType(), False), False), StructField("null1", DoubleType(), True)]) - df = self.sqlCtx.applySchema(rdd, schema) + df = self.sqlCtx.createDataFrame(rdd, schema) results = df.map(lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) r = (127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), From 50ab8a6543ad5c31e89c16df374d0cb13222fd1e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 14:21:51 -0500 Subject: [PATCH 803/817] [SPARK-2669] [yarn] Distribute client configuration to AM. Currently, when Spark launches the Yarn AM, the process will use the local Hadoop configuration on the node where the AM launches, if one is present. A more correct approach is to use the same configuration used to launch the Spark job, since the user may have made modifications (such as adding app-specific configs). The approach taken here is to use the distributed cache to make all files in the Hadoop configuration directory available to the AM. This is a little overkill since only the AM needs them (the executors use the broadcast Hadoop configuration from the driver), but is the easier approach. Even though only a few files in that directory may end up being used, all of them are uploaded. This allows supporting use cases such as when auxiliary configuration files are used for SSL configuration, or when uploading a Hive configuration directory. Not all of these may be reflected in a o.a.h.conf.Configuration object, but may be needed when a driver in cluster mode instantiates, for example, a HiveConf object instead. Author: Marcelo Vanzin Closes #4142 from vanzin/SPARK-2669 and squashes the following commits: f5434b9 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 013f0fb [Marcelo Vanzin] Review feedback. f693152 [Marcelo Vanzin] Le sigh. ed45b7d [Marcelo Vanzin] Zip all config files and upload them as an archive. 5927b6b [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 cbb9fb3 [Marcelo Vanzin] Remove stale test. e3e58d0 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 e3d0613 [Marcelo Vanzin] Review feedback. 34bdbd8 [Marcelo Vanzin] Fix test. 022a688 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 a77ddd5 [Marcelo Vanzin] Merge branch 'master' into SPARK-2669 79221c7 [Marcelo Vanzin] [SPARK-2669] [yarn] Distribute client configuration to AM. --- docs/running-on-yarn.md | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 125 +++++++++++++++--- .../spark/deploy/yarn/ExecutorRunnable.scala | 2 +- .../spark/deploy/yarn/ClientSuite.scala | 29 ++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 6 +- 5 files changed, 132 insertions(+), 36 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 853c9f26b0ec9..0968fc5ad632b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -211,7 +211,11 @@ Most of the configs are the same for Spark on YARN as for other deployment modes # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. -These configs are used to write to the dfs and connect to the YARN ResourceManager. +These configs are used to write to the dfs and connect to the YARN ResourceManager. The +configuration contained in this directory will be distributed to the YARN cluster so that all +containers used by the application use the same configuration. If the configuration references +Java system properties or environment variables not managed by YARN, they should also be set in the +Spark application's configuration (driver, executors, and the AM when running in client mode). There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 52e4dee46c535..019afbd1a1743 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,15 +17,18 @@ package org.apache.spark.deploy.yarn +import java.io.{File, FileOutputStream} import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer +import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} import com.google.common.base.Objects +import com.google.common.io.Files import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration @@ -77,12 +80,6 @@ private[spark] class Client( def stop(): Unit = yarnClient.stop() - /* ------------------------------------------------------------------------------------- * - | The following methods have much in common in the stable and alpha versions of Client, | - | but cannot be implemented in the parent trait due to subtle API differences across | - | hadoop versions. | - * ------------------------------------------------------------------------------------- */ - /** * Submit an application running our ApplicationMaster to the ResourceManager. * @@ -223,6 +220,10 @@ private[spark] class Client( val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = getNameNodesToAccess(sparkConf) + dst + // Used to keep track of URIs added to the distributed cache. If the same URI is added + // multiple times, YARN will fail to launch containers for the app with an internal + // error. + val distributedUris = new HashSet[String] obtainTokensForNamenodes(nns, hadoopConf, credentials) obtainTokenForHiveMetastore(hadoopConf, credentials) @@ -241,6 +242,17 @@ private[spark] class Client( "for alternatives.") } + def addDistributedUri(uri: URI): Boolean = { + val uriStr = uri.toString() + if (distributedUris.contains(uriStr)) { + logWarning(s"Resource $uri added multiple times to distributed cache.") + false + } else { + distributedUris += uriStr + true + } + } + /** * Copy the given main resource to the distributed cache if the scheme is not "local". * Otherwise, set the corresponding key in our SparkConf to handle it downstream. @@ -258,11 +270,13 @@ private[spark] class Client( if (!localPath.isEmpty()) { val localURI = new URI(localPath) if (localURI.getScheme != LOCAL_SCHEME) { - val src = getQualifiedLocalPath(localURI, hadoopConf) - val destPath = copyFileToRemote(dst, src, replication) - val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource(destFs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, destName, statCache) + if (addDistributedUri(localURI)) { + val src = getQualifiedLocalPath(localURI, hadoopConf) + val destPath = copyFileToRemote(dst, src, replication) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) + } } else if (confKey != null) { // If the resource is intended for local use only, handle this downstream // by setting the appropriate property @@ -271,6 +285,13 @@ private[spark] class Client( } } + createConfArchive().foreach { file => + require(addDistributedUri(file.toURI())) + val destPath = copyFileToRemote(dst, new Path(file.toURI()), replication) + distCacheMgr.addResource(fs, hadoopConf, destPath, localResources, LocalResourceType.ARCHIVE, + LOCALIZED_HADOOP_CONF_DIR, statCache, appMasterOnly = true) + } + /** * Do the same for any additional resources passed in through ClientArguments. * Each resource category is represented by a 3-tuple of: @@ -288,13 +309,15 @@ private[spark] class Client( flist.split(',').foreach { file => val localURI = new URI(file.trim()) if (localURI.getScheme != LOCAL_SCHEME) { - val localPath = new Path(localURI) - val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) - distCacheMgr.addResource( - fs, hadoopConf, destPath, localResources, resType, linkname, statCache) - if (addToClasspath) { - cachedSecondaryJarLinks += linkname + if (addDistributedUri(localURI)) { + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) + if (addToClasspath) { + cachedSecondaryJarLinks += linkname + } } } else if (addToClasspath) { // Resource is intended for local use only and should be added to the class path @@ -310,6 +333,57 @@ private[spark] class Client( localResources } + /** + * Create an archive with the Hadoop config files for distribution. + * + * These are only used by the AM, since executors will use the configuration object broadcast by + * the driver. The files are zipped and added to the job as an archive, so that YARN will explode + * it when distributing to the AM. This directory is then added to the classpath of the AM + * process, just to make sure that everybody is using the same default config. + * + * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR + * shows up in the classpath before YARN_CONF_DIR. + * + * Currently this makes a shallow copy of the conf directory. If there are cases where a + * Hadoop config directory contains subdirectories, this code will have to be fixed. + */ + private def createConfArchive(): Option[File] = { + val hadoopConfFiles = new HashMap[String, File]() + Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey => + sys.env.get(envKey).foreach { path => + val dir = new File(path) + if (dir.isDirectory()) { + dir.listFiles().foreach { file => + if (!hadoopConfFiles.contains(file.getName())) { + hadoopConfFiles(file.getName()) = file + } + } + } + } + } + + if (!hadoopConfFiles.isEmpty) { + val hadoopConfArchive = File.createTempFile(LOCALIZED_HADOOP_CONF_DIR, ".zip", + new File(Utils.getLocalDir(sparkConf))) + + val hadoopConfStream = new ZipOutputStream(new FileOutputStream(hadoopConfArchive)) + try { + hadoopConfStream.setLevel(0) + hadoopConfFiles.foreach { case (name, file) => + hadoopConfStream.putNextEntry(new ZipEntry(name)) + Files.copy(file, hadoopConfStream) + hadoopConfStream.closeEntry() + } + } finally { + hadoopConfStream.close() + } + + Some(hadoopConfArchive) + } else { + None + } + } + /** * Set up the environment for launching our ApplicationMaster container. */ @@ -317,7 +391,7 @@ private[spark] class Client( logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, true, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -718,6 +792,9 @@ object Client extends Logging { // Distribution-defined classpath to add to processes val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH" + // Subdirectory where the user's hadoop config files will be placed. + val LOCALIZED_HADOOP_CONF_DIR = "__hadoop_conf__" + /** * Find the user-defined Spark jar if configured, or return the jar containing this * class if not. @@ -831,11 +908,19 @@ object Client extends Logging { conf: Configuration, sparkConf: SparkConf, env: HashMap[String, String], + isAM: Boolean, extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry( YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env ) + + if (isAM) { + addClasspathEntry( + YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + + LOCALIZED_HADOOP_CONF_DIR, env) + } + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { val userClassPath = if (args != null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index b06069c07f451..9d04d241dae9e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -277,7 +277,7 @@ class ExecutorRunnable( private def prepareEnvironment(container: Container): HashMap[String, String] = { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - Client.populateClasspath(null, yarnConf, sparkConf, env, extraCp) + Client.populateClasspath(null, yarnConf, sparkConf, env, false, extraCp) sparkConf.getExecutorEnv.foreach { case (key, value) => // This assumes each executor environment variable set here is a path diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index c1b94ac9c5bdd..a51c2005cb472 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -20,6 +20,11 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ HashMap => MutableHashMap } +import scala.reflect.ClassTag +import scala.util.Try + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig @@ -30,11 +35,6 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ HashMap => MutableHashMap } -import scala.reflect.ClassTag -import scala.util.Try - import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils @@ -93,7 +93,7 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - Client.populateClasspath(args, conf, sparkConf, env) + Client.populateClasspath(args, conf, sparkConf, env, true) val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => @@ -104,13 +104,16 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { cp should not contain (uri.getPath()) } }) - if (classOf[Environment].getMethods().exists(_.getName == "$$")) { - cp should contain("{{PWD}}") - } else if (Utils.isWindows) { - cp should contain("%PWD%") - } else { - cp should contain(Environment.PWD.$()) - } + val pwdVar = + if (classOf[Environment].getMethods().exists(_.getName == "$$")) { + "{{PWD}}" + } else if (Utils.isWindows) { + "%PWD%" + } else { + Environment.PWD.$() + } + cp should contain(pwdVar) + cp should contain (s"$pwdVar${Path.SEPARATOR}${Client.LOCALIZED_HADOOP_CONF_DIR}") cp should not contain (Client.SPARK_JAR) cp should not contain (Client.APP_JAR) } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index a18c94d4ab4a8..3877da4120e7c 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -77,6 +77,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ private var logConfDir: File = _ override def beforeAll() { @@ -120,6 +121,9 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_HADOOP_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) } override def afterAll() { @@ -258,7 +262,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit appArgs Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> tempDir.getAbsolutePath())) + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) } /** From a83571acc938582865efb41645aa1e414f339e46 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 17 Apr 2015 13:15:36 -0700 Subject: [PATCH 804/817] [SPARK-6113] [ml] Stabilize DecisionTree API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is a PR for cleaning up and finalizing the DecisionTree API. PRs for ensembles will follow once this is merged. ### Goal Here is the description copied from the JIRA (for both trees and ensembles): > **Issue**: The APIs for DecisionTree and ensembles (RandomForests and GradientBoostedTrees) have been experimental for a long time. The API has become very convoluted because trees and ensembles have many, many variants, some of which we have added incrementally without a long-term design. > **Proposal**: This JIRA is for discussing changes required to finalize the APIs. After we discuss, I will make a PR to update the APIs and make them non-Experimental. This will require making many breaking changes; see the design doc for details. > **[Design doc](https://docs.google.com/document/d/1rJ_DZinyDG3PkYkAKSsQlY0QgCeefn4hUv7GsPkzBP4)** : This outlines current issues and the proposed API. Overall code layout: * The old API in mllib.tree.* will remain the same. * The new API will reside in ml.classification.* and ml.regression.* ### Summary of changes Old API * Exactly the same, except I made 1 method in Loss private (but that is not a breaking change since that method was introduced after the Spark 1.3 release). New APIs * Under Pipeline API * The new API preserves functionality, except: * New API does NOT store prob (probability of label in classification). I want to have it store the full vector of probabilities but feel that should be in a later PR. * Use abstractions for parameters, estimators, and models to avoid code duplication * Limit parameters to relevant algorithms * For enum-like types, only expose Strings * We can make these pluggable later on by adding new parameters. That is a far-future item. Test suites * I organized DecisionTreeSuite, but I made absolutely no changes to the tests themselves. * The test suites for the new API only test (a) similarity with the results of the old API and (b) elements of the new API. * After code is moved to this new API, we should move the tests from the old suites which test the internals. ### Details #### Changed names Parameters * useNodeIdCache -> cacheNodeIds #### Other changes * Split: Changed categories to set instead of list #### Non-decision tree changes * AttributeGroup * Added parentheses to toMetadata, toStructField methods (These were removed in a previous PR, but I ran into 1 issue with the Scala compiler not being able to disambiguate between a toMetadata method with no parentheses and a toMetadata method which takes 1 argument.) * Attributes * Renamed: toMetadata -> toMetadataImpl * Added toMetadata methods which return ML metadata (keyed with “ML_ATTR”) * NominalAttribute: Added getNumValues method which examines both numValues and values. * Params.inheritValues: Checks whether the parent param really belongs to the child (to allow Estimator-Model pairs with different sets of parameters) ### Questions for reviewers * Is "DecisionTreeClassificationModel" too long a name? * Is this OK in the docs? ``` class DecisionTreeRegressor extends TreeRegressor[DecisionTreeRegressionModel] with DecisionTreeParams[DecisionTreeRegressor] with TreeRegressorParams[DecisionTreeRegressor] ``` ### Future We should open up the abstractions at some point. E.g., it would be useful to be able to set tree-related parameters in 1 place and then pass those to multiple tree-based algorithms. Follow-up JIRAs will be (in this order): * Tree ensembles * Deprecate old tree code * Move DecisionTree implementation code to new API. * Move tests from the old suites which test the internals. * Update programming guide * Python API * Change RandomForest* to always use bootstrapping, even when numTrees = 1 * Provide the probability of the predicted label for classification. After we move code to the new API and update it to maintain probabilities for all labels, then we can add the probabilities to the new API. CC: mengxr manishamde codedeft chouqin MechCoder Author: Joseph K. Bradley Closes #5530 from jkbradley/dt-api-dt and squashes the following commits: 6aae255 [Joseph K. Bradley] Changed tree abstractions not to take type parameters, and for setters to return this.type instead ec17947 [Joseph K. Bradley] Updates based on code review. Main changes were: moving public types from ml.impl.tree to ml.tree, modifying CategoricalSplit to take an Array of categories but store a Set internally, making more types sealed or final 5626c81 [Joseph K. Bradley] style fixes f8fbd24 [Joseph K. Bradley] imported reorg of DecisionTreeSuite from old PR. small cleanups 7ef63ed [Joseph K. Bradley] Added DecisionTreeRegressor, test suites, and example (for real this time) e11673f [Joseph K. Bradley] Added DecisionTreeRegressor, test suites, and example 119f407 [Joseph K. Bradley] added DecisionTreeClassifier example 0bdc486 [Joseph K. Bradley] fixed issues after param PR was merged f9fbb60 [Joseph K. Bradley] Done with DecisionTreeClassifier, but no save/load yet. Need to add example as well 2532c9a [Joseph K. Bradley] partial move to spark.ml API, not done yet c72c1a0 [Joseph K. Bradley] Copied changes for common items, plus DecisionTreeClassifier from original PR --- .../examples/ml/DecisionTreeExample.scala | 322 +++++++++++++++ .../spark/ml/attribute/AttributeGroup.scala | 10 +- .../spark/ml/attribute/attributes.scala | 43 +- .../DecisionTreeClassifier.scala | 155 ++++++++ .../spark/ml/feature/StringIndexer.scala | 2 +- .../spark/ml/impl/tree/treeParams.scala | 300 ++++++++++++++ .../scala/org/apache/spark/ml/package.scala | 12 + .../org/apache/spark/ml/param/params.scala | 3 +- .../ml/regression/DecisionTreeRegressor.scala | 145 +++++++ .../scala/org/apache/spark/ml/tree/Node.scala | 205 ++++++++++ .../org/apache/spark/ml/tree/Split.scala | 151 +++++++ .../org/apache/spark/ml/tree/treeModels.scala | 60 +++ .../apache/spark/ml/util/MetadataUtils.scala | 82 ++++ .../spark/mllib/tree/DecisionTree.scala | 5 +- .../mllib/tree/GradientBoostedTrees.scala | 12 +- .../spark/mllib/tree/RandomForest.scala | 2 +- .../tree/configuration/BoostingStrategy.scala | 10 +- .../spark/mllib/tree/loss/AbsoluteError.scala | 5 +- .../spark/mllib/tree/loss/LogLoss.scala | 5 +- .../apache/spark/mllib/tree/loss/Loss.scala | 4 +- .../spark/mllib/tree/loss/SquaredError.scala | 5 +- .../mllib/tree/model/DecisionTreeModel.scala | 4 +- .../apache/spark/mllib/tree/model/Node.scala | 2 +- .../mllib/tree/model/treeEnsembleModels.scala | 32 +- .../JavaDecisionTreeClassifierSuite.java | 98 +++++ .../JavaDecisionTreeRegressorSuite.java | 97 +++++ .../ml/attribute/AttributeGroupSuite.scala | 4 +- .../spark/ml/attribute/AttributeSuite.scala | 42 +- .../DecisionTreeClassifierSuite.scala | 274 +++++++++++++ .../spark/ml/feature/VectorIndexerSuite.scala | 2 +- .../org/apache/spark/ml/impl/TreeTests.scala | 132 +++++++ .../DecisionTreeRegressorSuite.scala | 91 +++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 373 +++++++++--------- 33 files changed, 2426 insertions(+), 263 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala new file mode 100644 index 0000000000000..d4cc8dede07ef --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +import scala.collection.mutable +import scala.language.reflectiveCalls + +import scopt.OptionParser + +import org.apache.spark.ml.tree.DecisionTreeModel +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.examples.mllib.AbstractParams +import org.apache.spark.ml.{Pipeline, PipelineStage} +import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} +import org.apache.spark.ml.feature.{VectorIndexer, StringIndexer} +import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.evaluation.{RegressionMetrics, MulticlassMetrics} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.{SQLContext, DataFrame} + + +/** + * An example runner for decision trees. Run with + * {{{ + * ./bin/run-example ml.DecisionTreeExample [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeExample { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "Classification", + maxDepth: Int = 5, + maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, + numTrees: Int = 1, + featureSubsetStrategy: String = "auto", + fracTest: Double = 0.2, + cacheNodeIds: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeExample") { + head("DecisionTreeExample: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (Classification, Regression), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) + opt[Double]("fracTest") + .text(s"fraction of data to hold out for testing. If given option testInput, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("cacheNodeIds") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.cacheNodeIds}") + .action((x, c) => c.copy(cacheNodeIds = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + }}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + s"default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) + opt[String]("testInput") + .text(s"input path to test dataset. If given, option fracTest is ignored." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.fracTest < 0 || params.fracTest > 1) { + failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1].") + } else { + success + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + /** Load a dataset from the given path, using the given format */ + private[ml] def loadData( + sc: SparkContext, + path: String, + format: String, + expectedNumFeatures: Option[Int] = None): RDD[LabeledPoint] = { + format match { + case "dense" => MLUtils.loadLabeledPoints(sc, path) + case "libsvm" => expectedNumFeatures match { + case Some(numFeatures) => MLUtils.loadLibSVMFile(sc, path, numFeatures) + case None => MLUtils.loadLibSVMFile(sc, path) + } + case _ => throw new IllegalArgumentException(s"Bad data format: $format") + } + } + + /** + * Load training and test data from files. + * @param input Path to input dataset. + * @param dataFormat "libsvm" or "dense" + * @param testInput Path to test dataset. + * @param algo Classification or Regression + * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. + * @return (training dataset, test dataset) + */ + private[ml] def loadDatasets( + sc: SparkContext, + input: String, + dataFormat: String, + testInput: String, + algo: String, + fracTest: Double): (DataFrame, DataFrame) = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Load training data + val origExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat) + + // Load or create test set + val splits: Array[RDD[LabeledPoint]] = if (testInput != "") { + // Load testInput. + val numFeatures = origExamples.take(1)(0).features.size + val origTestExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat, Some(numFeatures)) + Array(origExamples, origTestExamples) + } else { + // Split input into training, test. + origExamples.randomSplit(Array(1.0 - fracTest, fracTest), seed = 12345) + } + + // For classification, convert labels to Strings since we will index them later with + // StringIndexer. + def labelsToStrings(data: DataFrame): DataFrame = { + algo.toLowerCase match { + case "classification" => + data.withColumn("labelString", data("label").cast(StringType)) + case "regression" => + data + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + } + val dataframes = splits.map(_.toDF()).map(labelsToStrings).map(_.cache()) + + (dataframes(0), dataframes(1)) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DecisionTreeExample with $params") + val sc = new SparkContext(conf) + params.checkpointDir.foreach(sc.setCheckpointDir) + val algo = params.algo.toLowerCase + + println(s"DecisionTreeExample with parameters:\n$params") + + // Load training and test data and cache it. + val (training: DataFrame, test: DataFrame) = + loadDatasets(sc, params.input, params.dataFormat, params.testInput, algo, params.fracTest) + + val numTraining = training.count() + val numTest = test.count() + val numFeatures = training.select("features").first().getAs[Vector](0).size + println("Loaded data:") + println(s" numTraining = $numTraining, numTest = $numTest") + println(s" numFeatures = $numFeatures") + + // Set up Pipeline + val stages = new mutable.ArrayBuffer[PipelineStage]() + // (1) For classification, re-index classes. + val labelColName = if (algo == "classification") "indexedLabel" else "label" + if (algo == "classification") { + val labelIndexer = new StringIndexer().setInputCol("labelString").setOutputCol(labelColName) + stages += labelIndexer + } + // (2) Identify categorical features using VectorIndexer. + // Features with more than maxCategories values will be treated as continuous. + val featuresIndexer = new VectorIndexer().setInputCol("features") + .setOutputCol("indexedFeatures").setMaxCategories(10) + stages += featuresIndexer + // (3) Learn DecisionTree + val dt = algo match { + case "classification" => + new DecisionTreeClassifier().setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case "regression" => + new DecisionTreeRegressor().setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + stages += dt + val pipeline = new Pipeline().setStages(stages.toArray) + + // Fit the Pipeline + val startTime = System.nanoTime() + val pipelineModel = pipeline.fit(training) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") + + // Get the trained Decision Tree from the fitted PipelineModel + val treeModel: DecisionTreeModel = algo match { + case "classification" => + pipelineModel.getModel[DecisionTreeClassificationModel]( + dt.asInstanceOf[DecisionTreeClassifier]) + case "regression" => + pipelineModel.getModel[DecisionTreeRegressionModel](dt.asInstanceOf[DecisionTreeRegressor]) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + if (treeModel.numNodes < 20) { + println(treeModel.toDebugString) // Print full model. + } else { + println(treeModel) // Print model summary. + } + + // Predict on training + val trainingFullPredictions = pipelineModel.transform(training).cache() + val trainingPredictions = trainingFullPredictions.select("prediction") + .map(_.getDouble(0)) + val trainingLabels = trainingFullPredictions.select(labelColName).map(_.getDouble(0)) + // Predict on test data + val testFullPredictions = pipelineModel.transform(test).cache() + val testPredictions = testFullPredictions.select("prediction") + .map(_.getDouble(0)) + val testLabels = testFullPredictions.select(labelColName).map(_.getDouble(0)) + + // For classification, print number of classes for reference. + if (algo == "classification") { + val numClasses = + MetadataUtils.getNumClasses(trainingFullPredictions.schema(labelColName)) match { + case Some(n) => n + case None => throw new RuntimeException( + "DecisionTreeExample had unknown failure when indexing labels for classification.") + } + println(s"numClasses = $numClasses.") + } + + // Evaluate model on training, test data + algo match { + case "classification" => + val trainingAccuracy = + new MulticlassMetrics(trainingPredictions.zip(trainingLabels)).precision + println(s"Train accuracy = $trainingAccuracy") + val testAccuracy = + new MulticlassMetrics(testPredictions.zip(testLabels)).precision + println(s"Test accuracy = $testAccuracy") + case "regression" => + val trainingRMSE = + new RegressionMetrics(trainingPredictions.zip(trainingLabels)).rootMeanSquaredError + println(s"Training root mean squared error (RMSE) = $trainingRMSE") + val testRMSE = + new RegressionMetrics(testPredictions.zip(testLabels)).rootMeanSquaredError + println(s"Test root mean squared error (RMSE) = $testRMSE") + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index aa27a668f1695..d7dee8fed2a55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -117,12 +117,12 @@ class AttributeGroup private ( case numeric: NumericAttribute => // Skip default numeric attributes. if (numeric.withoutIndex != NumericAttribute.defaultAttr) { - numericMetadata += numeric.toMetadata(withType = false) + numericMetadata += numeric.toMetadataImpl(withType = false) } case nominal: NominalAttribute => - nominalMetadata += nominal.toMetadata(withType = false) + nominalMetadata += nominal.toMetadataImpl(withType = false) case binary: BinaryAttribute => - binaryMetadata += binary.toMetadata(withType = false) + binaryMetadata += binary.toMetadataImpl(withType = false) } val attrBldr = new MetadataBuilder if (numericMetadata.nonEmpty) { @@ -151,7 +151,7 @@ class AttributeGroup private ( } /** Converts to ML metadata */ - def toMetadata: Metadata = toMetadata(Metadata.empty) + def toMetadata(): Metadata = toMetadata(Metadata.empty) /** Converts to a StructField with some existing metadata. */ def toStructField(existingMetadata: Metadata): StructField = { @@ -159,7 +159,7 @@ class AttributeGroup private ( } /** Converts to a StructField. */ - def toStructField: StructField = toStructField(Metadata.empty) + def toStructField(): StructField = toStructField(Metadata.empty) override def equals(other: Any): Boolean = { other match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index 00b7566aab434..5717d6ec2eaec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -68,21 +68,32 @@ sealed abstract class Attribute extends Serializable { * Converts this attribute to [[Metadata]]. * @param withType whether to include the type info */ - private[attribute] def toMetadata(withType: Boolean): Metadata + private[attribute] def toMetadataImpl(withType: Boolean): Metadata /** * Converts this attribute to [[Metadata]]. For numeric attributes, the type info is excluded to * save space, because numeric type is the default attribute type. For nominal and binary * attributes, the type info is included. */ - private[attribute] def toMetadata(): Metadata = { + private[attribute] def toMetadataImpl(): Metadata = { if (attrType == AttributeType.Numeric) { - toMetadata(withType = false) + toMetadataImpl(withType = false) } else { - toMetadata(withType = true) + toMetadataImpl(withType = true) } } + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() + .withMetadata(existingMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl()) + .build() + } + + /** Converts to ML metadata */ + def toMetadata(): Metadata = toMetadata(Metadata.empty) + /** * Converts to a [[StructField]] with some existing metadata. * @param existingMetadata existing metadata to carry over @@ -90,7 +101,7 @@ sealed abstract class Attribute extends Serializable { def toStructField(existingMetadata: Metadata): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadata()) + .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadataImpl()) .build() StructField(name.get, DoubleType, nullable = false, newMetadata) } @@ -98,7 +109,7 @@ sealed abstract class Attribute extends Serializable { /** Converts to a [[StructField]]. */ def toStructField(): StructField = toStructField(Metadata.empty) - override def toString: String = toMetadata(withType = true).toString + override def toString: String = toMetadataImpl(withType = true).toString } /** Trait for ML attribute factories. */ @@ -210,7 +221,7 @@ class NumericAttribute private[ml] ( override def isNominal: Boolean = false /** Convert this attribute to metadata. */ - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -353,6 +364,20 @@ class NominalAttribute private[ml] ( /** Copy without the `numValues`. */ def withoutNumValues: NominalAttribute = copy(numValues = None) + /** + * Get the number of values, either from `numValues` or from `values`. + * Return None if unknown. + */ + def getNumValues: Option[Int] = { + if (numValues.nonEmpty) { + numValues + } else if (values.nonEmpty) { + Some(values.get.length) + } else { + None + } + } + /** Creates a copy of this attribute with optional changes. */ private def copy( name: Option[String] = name, @@ -363,7 +388,7 @@ class NominalAttribute private[ml] ( new NominalAttribute(name, index, isOrdinal, numValues, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -465,7 +490,7 @@ class BinaryAttribute private[ml] ( new BinaryAttribute(name, index, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder if (withType) bldr.putString(TYPE, attrType.name) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala new file mode 100644 index 0000000000000..3855e396b5534 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -0,0 +1,155 @@ +/* + * 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.classification + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{Predictor, PredictionModel} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassifier + extends Predictor[Vector, DecisionTreeClassifier, DecisionTreeClassificationModel] + with DecisionTreeParams + with TreeClassifierParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = + super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = + super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train( + dataset: DataFrame, + paramMap: ParamMap): DecisionTreeClassificationModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema(paramMap(featuresCol))) + val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema(paramMap(labelCol))) match { + case Some(n: Int) => n + case None => throw new IllegalArgumentException("DecisionTreeClassifier was given input" + + s" with invalid label column, without the number of classes specified.") + // TODO: Automatically index labels. + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, paramMap) + val strategy = getOldStrategy(categoricalFeatures, numClasses) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeClassificationModel.fromOld(oldModel, this, paramMap, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + override private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int): OldStrategy = { + val strategy = super.getOldStrategy(categoricalFeatures, numClasses) + strategy.algo = OldAlgo.Classification + strategy.setImpurity(getOldImpurity) + strategy + } +} + +object DecisionTreeClassifier { + /** Accessor for supported impurities */ + final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassificationModel private[ml] ( + override val parent: DecisionTreeClassifier, + override val fittingParamMap: ParamMap, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeClassificationModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override protected def copy(): DecisionTreeClassificationModel = { + val m = new DecisionTreeClassificationModel(parent, fittingParamMap, rootNode) + Params.inheritValues(this.extractParamMap(), this, m) + m + } + + override def toString: String = { + s"DecisionTreeClassificationModel of depth $depth with $numNodes nodes" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Classification) + } +} + +private[ml] object DecisionTreeClassificationModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeClassifier, + fittingParamMap: ParamMap, + categoricalFeatures: Map[Int, Int]): DecisionTreeClassificationModel = { + require(oldModel.algo == OldAlgo.Classification, + s"Cannot convert non-classification DecisionTreeModel (old API) to" + + s" DecisionTreeClassificationModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeClassificationModel(parent, fittingParamMap, rootNode) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 4d960df357fe9..23956c512c8a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -118,7 +118,7 @@ class StringIndexerModel private[ml] ( } val outputColName = map(outputCol) val metadata = NominalAttribute.defaultAttr - .withName(outputColName).withValues(labels).toStructField().metadata + .withName(outputColName).withValues(labels).toMetadata() dataset.select(col("*"), indexer(dataset(map(inputCol))).as(outputColName, metadata)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala new file mode 100644 index 0000000000000..6f4509f03d033 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala @@ -0,0 +1,300 @@ +/* + * 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.impl.tree + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.impurity.{Gini => OldGini, Entropy => OldEntropy, + Impurity => OldImpurity, Variance => OldVariance} + + +/** + * :: DeveloperApi :: + * Parameters for Decision Tree-based algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait DecisionTreeParams extends PredictorParams { + + /** + * Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (default = 5) + * @group param + */ + final val maxDepth: IntParam = + new IntParam(this, "maxDepth", "Maximum depth of the tree." + + " E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.") + + /** + * Maximum number of bins used for discretizing continuous features and for choosing how to split + * on features at each node. More bins give higher granularity. + * Must be >= 2 and >= number of categories in any categorical feature. + * (default = 32) + * @group param + */ + final val maxBins: IntParam = new IntParam(this, "maxBins", "Max number of bins for" + + " discretizing continuous features. Must be >=2 and >= number of categories for any" + + " categorical feature.") + + /** + * Minimum number of instances each child must have after split. + * If a split causes the left or right child to have fewer than minInstancesPerNode, + * the split will be discarded as invalid. + * Should be >= 1. + * (default = 1) + * @group param + */ + final val minInstancesPerNode: IntParam = new IntParam(this, "minInstancesPerNode", "Minimum" + + " number of instances each child must have after split. If a split causes the left or right" + + " child to have fewer than minInstancesPerNode, the split will be discarded as invalid." + + " Should be >= 1.") + + /** + * Minimum information gain for a split to be considered at a tree node. + * (default = 0.0) + * @group param + */ + final val minInfoGain: DoubleParam = new DoubleParam(this, "minInfoGain", + "Minimum information gain for a split to be considered at a tree node.") + + /** + * Maximum memory in MB allocated to histogram aggregation. + * (default = 256 MB) + * @group expertParam + */ + final val maxMemoryInMB: IntParam = new IntParam(this, "maxMemoryInMB", + "Maximum memory in MB allocated to histogram aggregation.") + + /** + * 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. + * (default = false) + * @group expertParam + */ + final val cacheNodeIds: BooleanParam = new BooleanParam(this, "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.") + + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be >= 1. + * (default = 10) + * @group expertParam + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "Specifies" + + " how often to checkpoint the cached node IDs. E.g. 10 means that the cache will get" + + " checkpointed every 10 iterations. This is only used if cacheNodeIds is true and if the" + + " checkpoint directory is set in the SparkContext. Must be >= 1.") + + setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, minInfoGain -> 0.0, + maxMemoryInMB -> 256, cacheNodeIds -> false, checkpointInterval -> 10) + + /** @group setParam */ + def setMaxDepth(value: Int): this.type = { + require(value >= 0, s"maxDepth parameter must be >= 0. Given bad value: $value") + set(maxDepth, value) + this.asInstanceOf[this.type] + } + + /** @group getParam */ + def getMaxDepth: Int = getOrDefault(maxDepth) + + /** @group setParam */ + def setMaxBins(value: Int): this.type = { + require(value >= 2, s"maxBins parameter must be >= 2. Given bad value: $value") + set(maxBins, value) + this + } + + /** @group getParam */ + def getMaxBins: Int = getOrDefault(maxBins) + + /** @group setParam */ + def setMinInstancesPerNode(value: Int): this.type = { + require(value >= 1, s"minInstancesPerNode parameter must be >= 1. Given bad value: $value") + set(minInstancesPerNode, value) + this + } + + /** @group getParam */ + def getMinInstancesPerNode: Int = getOrDefault(minInstancesPerNode) + + /** @group setParam */ + def setMinInfoGain(value: Double): this.type = { + set(minInfoGain, value) + this + } + + /** @group getParam */ + def getMinInfoGain: Double = getOrDefault(minInfoGain) + + /** @group expertSetParam */ + def setMaxMemoryInMB(value: Int): this.type = { + require(value > 0, s"maxMemoryInMB parameter must be > 0. Given bad value: $value") + set(maxMemoryInMB, value) + this + } + + /** @group expertGetParam */ + def getMaxMemoryInMB: Int = getOrDefault(maxMemoryInMB) + + /** @group expertSetParam */ + def setCacheNodeIds(value: Boolean): this.type = { + set(cacheNodeIds, value) + this + } + + /** @group expertGetParam */ + def getCacheNodeIds: Boolean = getOrDefault(cacheNodeIds) + + /** @group expertSetParam */ + def setCheckpointInterval(value: Int): this.type = { + require(value >= 1, s"checkpointInterval parameter must be >= 1. Given bad value: $value") + set(checkpointInterval, value) + this + } + + /** @group expertGetParam */ + def getCheckpointInterval: Int = getOrDefault(checkpointInterval) + + /** + * Create a Strategy instance to use with the old API. + * NOTE: The caller should set impurity and subsamplingRate (which is set to 1.0, + * the default for single trees). + */ + private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int): OldStrategy = { + val strategy = OldStrategy.defaultStategy(OldAlgo.Classification) + strategy.checkpointInterval = getCheckpointInterval + strategy.maxBins = getMaxBins + strategy.maxDepth = getMaxDepth + strategy.maxMemoryInMB = getMaxMemoryInMB + strategy.minInfoGain = getMinInfoGain + strategy.minInstancesPerNode = getMinInstancesPerNode + strategy.useNodeIdCache = getCacheNodeIds + strategy.numClasses = numClasses + strategy.categoricalFeaturesInfo = categoricalFeatures + strategy.subsamplingRate = 1.0 // default for individual trees + strategy + } +} + +/** + * (private trait) Parameters for Decision Tree-based classification algorithms. + */ +private[ml] trait TreeClassifierParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "entropy" and "gini". + * (default = gini) + * @group param + */ + val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeClassifierParams.supportedImpurities.mkString(", ")}") + + setDefault(impurity -> "gini") + + /** @group setParam */ + def setImpurity(value: String): this.type = { + val impurityStr = value.toLowerCase + require(TreeClassifierParams.supportedImpurities.contains(impurityStr), + s"Tree-based classifier was given unrecognized impurity: $value." + + s" Supported options: ${TreeClassifierParams.supportedImpurities.mkString(", ")}") + set(impurity, impurityStr) + this + } + + /** @group getParam */ + def getImpurity: String = getOrDefault(impurity) + + /** Convert new impurity to old impurity. */ + private[ml] def getOldImpurity: OldImpurity = { + getImpurity match { + case "entropy" => OldEntropy + case "gini" => OldGini + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeClassifierParams was given unrecognized impurity: $impurity.") + } + } +} + +private[ml] object TreeClassifierParams { + // These options should be lowercase. + val supportedImpurities: Array[String] = Array("entropy", "gini").map(_.toLowerCase) +} + +/** + * (private trait) Parameters for Decision Tree-based regression algorithms. + */ +private[ml] trait TreeRegressorParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "variance". + * (default = variance) + * @group param + */ + val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeRegressorParams.supportedImpurities.mkString(", ")}") + + setDefault(impurity -> "variance") + + /** @group setParam */ + def setImpurity(value: String): this.type = { + val impurityStr = value.toLowerCase + require(TreeRegressorParams.supportedImpurities.contains(impurityStr), + s"Tree-based regressor was given unrecognized impurity: $value." + + s" Supported options: ${TreeRegressorParams.supportedImpurities.mkString(", ")}") + set(impurity, impurityStr) + this + } + + /** @group getParam */ + def getImpurity: String = getOrDefault(impurity) + + /** Convert new impurity to old impurity. */ + protected def getOldImpurity: OldImpurity = { + getImpurity match { + case "variance" => OldVariance + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeRegressorParams was given unrecognized impurity: $impurity") + } + } +} + +private[ml] object TreeRegressorParams { + // These options should be lowercase. + val supportedImpurities: Array[String] = Array("variance").map(_.toLowerCase) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/package.scala b/mllib/src/main/scala/org/apache/spark/ml/package.scala index b45bd1499b72e..ac75e9de1a8f2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -32,6 +32,18 @@ package org.apache.spark * @groupname getParam Parameter getters * @groupprio getParam 6 * + * @groupname expertParam (expert-only) Parameters + * @groupdesc expertParam A list of advanced, expert-only (hyper-)parameter keys this algorithm can + * take. Users can set and get the parameter values through setters and getters, + * respectively. + * @groupprio expertParam 7 + * + * @groupname expertSetParam (expert-only) Parameter setters + * @groupprio expertSetParam 8 + * + * @groupname expertGetParam (expert-only) Parameter getters + * @groupprio expertGetParam 9 + * * @groupname Ungrouped Members * @groupprio Ungrouped 0 */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 849c60433c777..ddc5907e7facd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -296,8 +296,9 @@ private[spark] object Params { paramMap: ParamMap, parent: E, child: M): Unit = { + val childParams = child.params.map(_.name).toSet parent.params.foreach { param => - if (paramMap.contains(param)) { + if (paramMap.contains(param) && childParams.contains(param.name)) { child.set(child.getParam(param.name), paramMap(param)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala new file mode 100644 index 0000000000000..49a8b77acf960 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for regression. + * It supports both continuous and categorical features. + */ +@AlphaComponent +final class DecisionTreeRegressor + extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] + with DecisionTreeParams + with TreeRegressorParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = + super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train( + dataset: DataFrame, + paramMap: ParamMap): DecisionTreeRegressionModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema(paramMap(featuresCol))) + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, paramMap) + val strategy = getOldStrategy(categoricalFeatures) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeRegressionModel.fromOld(oldModel, this, paramMap, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + private[ml] def getOldStrategy(categoricalFeatures: Map[Int, Int]): OldStrategy = { + val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0) + strategy.algo = OldAlgo.Regression + strategy.setImpurity(getOldImpurity) + strategy + } +} + +object DecisionTreeRegressor { + /** Accessor for supported impurities */ + final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * It supports both continuous and categorical features. + * @param rootNode Root of the decision tree + */ +@AlphaComponent +final class DecisionTreeRegressionModel private[ml] ( + override val parent: DecisionTreeRegressor, + override val fittingParamMap: ParamMap, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeRegressionModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override protected def copy(): DecisionTreeRegressionModel = { + val m = new DecisionTreeRegressionModel(parent, fittingParamMap, rootNode) + Params.inheritValues(this.extractParamMap(), this, m) + m + } + + override def toString: String = { + s"DecisionTreeRegressionModel of depth $depth with $numNodes nodes" + } + + /** Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Regression) + } +} + +private[ml] object DecisionTreeRegressionModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeRegressor, + fittingParamMap: ParamMap, + categoricalFeatures: Map[Int, Int]): DecisionTreeRegressionModel = { + require(oldModel.algo == OldAlgo.Regression, + s"Cannot convert non-regression DecisionTreeModel (old API) to" + + s" DecisionTreeRegressionModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeRegressionModel(parent, fittingParamMap, rootNode) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala new file mode 100644 index 0000000000000..d6e2203d9f937 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -0,0 +1,205 @@ +/* + * 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.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.model.{InformationGainStats => OldInformationGainStats, + Node => OldNode, Predict => OldPredict} + + +/** + * Decision tree node interface. + */ +sealed abstract class Node extends Serializable { + + // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree + // code into the new API and deprecate the old API. + + /** Prediction this node makes (or would make, if it is an internal node) */ + def prediction: Double + + /** Impurity measure at this node (for training data) */ + def impurity: Double + + /** Recursive prediction helper method */ + private[ml] def predict(features: Vector): Double = prediction + + /** + * Get the number of nodes in tree below this node, including leaf nodes. + * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. + */ + private[tree] def numDescendants: Int + + /** + * Recursive print function. + * @param indentFactor The number of spaces to add to each level of indentation. + */ + private[tree] def subtreeToString(indentFactor: Int = 0): String + + /** + * Get depth of tree from this node. + * E.g.: Depth 0 means this is a leaf node. Depth 1 means 1 internal and 2 leaf nodes. + */ + private[tree] def subtreeDepth: Int + + /** + * Create a copy of this node in the old Node format, recursively creating child nodes as needed. + * @param id Node ID using old format IDs + */ + private[ml] def toOld(id: Int): OldNode +} + +private[ml] object Node { + + /** + * Create a new Node from the old Node format, recursively creating child nodes as needed. + */ + def fromOld(oldNode: OldNode, categoricalFeatures: Map[Int, Int]): Node = { + if (oldNode.isLeaf) { + // TODO: Once the implementation has been moved to this API, then include sufficient + // statistics here. + new LeafNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity) + } else { + val gain = if (oldNode.stats.nonEmpty) { + oldNode.stats.get.gain + } else { + 0.0 + } + new InternalNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity, + gain = gain, leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures), + rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures), + split = Split.fromOld(oldNode.split.get, categoricalFeatures)) + } + } +} + +/** + * Decision tree leaf node. + * @param prediction Prediction this node makes + * @param impurity Impurity measure at this node (for training data) + */ +final class LeafNode private[ml] ( + override val prediction: Double, + override val impurity: Double) extends Node { + + override def toString: String = s"LeafNode(prediction = $prediction, impurity = $impurity)" + + override private[ml] def predict(features: Vector): Double = prediction + + override private[tree] def numDescendants: Int = 0 + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"Predict: $prediction\n" + } + + override private[tree] def subtreeDepth: Int = 0 + + override private[ml] def toOld(id: Int): OldNode = { + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = true, + None, None, None, None) + } +} + +/** + * Internal Decision Tree node. + * @param prediction Prediction this node would make if it were a leaf node + * @param impurity Impurity measure at this node (for training data) + * @param gain Information gain value. + * Values < 0 indicate missing values; this quirk will be removed with future updates. + * @param leftChild Left-hand child node + * @param rightChild Right-hand child node + * @param split Information about the test used to split to the left or right child. + */ +final class InternalNode private[ml] ( + override val prediction: Double, + override val impurity: Double, + val gain: Double, + val leftChild: Node, + val rightChild: Node, + val split: Split) extends Node { + + override def toString: String = { + s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" + } + + override private[ml] def predict(features: Vector): Double = { + if (split.shouldGoLeft(features)) { + leftChild.predict(features) + } else { + rightChild.predict(features) + } + } + + override private[tree] def numDescendants: Int = { + 2 + leftChild.numDescendants + rightChild.numDescendants + } + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"If (${InternalNode.splitToString(split, left=true)})\n" + + leftChild.subtreeToString(indentFactor + 1) + + prefix + s"Else (${InternalNode.splitToString(split, left=false)})\n" + + rightChild.subtreeToString(indentFactor + 1) + } + + override private[tree] def subtreeDepth: Int = { + 1 + math.max(leftChild.subtreeDepth, rightChild.subtreeDepth) + } + + override private[ml] def toOld(id: Int): OldNode = { + assert(id.toLong * 2 < Int.MaxValue, "Decision Tree could not be converted from new to old API" + + " since the old API does not support deep trees.") + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = false, + Some(split.toOld), Some(leftChild.toOld(OldNode.leftChildIndex(id))), + Some(rightChild.toOld(OldNode.rightChildIndex(id))), + Some(new OldInformationGainStats(gain, impurity, leftChild.impurity, rightChild.impurity, + new OldPredict(leftChild.prediction, prob = 0.0), + new OldPredict(rightChild.prediction, prob = 0.0)))) + } +} + +private object InternalNode { + + /** + * Helper method for [[Node.subtreeToString()]]. + * @param split Split to print + * @param left Indicates whether this is the part of the split going to the left, + * or that going to the right. + */ + private def splitToString(split: Split, left: Boolean): String = { + val featureStr = s"feature ${split.featureIndex}" + split match { + case contSplit: ContinuousSplit => + if (left) { + s"$featureStr <= ${contSplit.threshold}" + } else { + s"$featureStr > ${contSplit.threshold}" + } + case catSplit: CategoricalSplit => + val categoriesStr = catSplit.getLeftCategories.mkString("{", ",", "}") + if (left) { + s"$featureStr in $categoriesStr" + } else { + s"$featureStr not in $categoriesStr" + } + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala new file mode 100644 index 0000000000000..cb940f62990ed --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -0,0 +1,151 @@ +/* + * 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.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} +import org.apache.spark.mllib.tree.model.{Split => OldSplit} + + +/** + * Interface for a "Split," which specifies a test made at a decision tree node + * to choose the left or right path. + */ +sealed trait Split extends Serializable { + + /** Index of feature which this split tests */ + def featureIndex: Int + + /** Return true (split to left) or false (split to right) */ + private[ml] def shouldGoLeft(features: Vector): Boolean + + /** Convert to old Split format */ + private[tree] def toOld: OldSplit +} + +private[ml] object Split { + + def fromOld(oldSplit: OldSplit, categoricalFeatures: Map[Int, Int]): Split = { + oldSplit.featureType match { + case OldFeatureType.Categorical => + new CategoricalSplit(featureIndex = oldSplit.feature, + leftCategories = oldSplit.categories.toArray, categoricalFeatures(oldSplit.feature)) + case OldFeatureType.Continuous => + new ContinuousSplit(featureIndex = oldSplit.feature, threshold = oldSplit.threshold) + } + } +} + +/** + * Split which tests a categorical feature. + * @param featureIndex Index of the feature to test + * @param leftCategories If the feature value is in this set of categories, then the split goes + * left. Otherwise, it goes right. + * @param numCategories Number of categories for this feature. + */ +final class CategoricalSplit( + override val featureIndex: Int, + leftCategories: Array[Double], + private val numCategories: Int) + extends Split { + + require(leftCategories.forall(cat => 0 <= cat && cat < numCategories), "Invalid leftCategories" + + s" (should be in range [0, $numCategories)): ${leftCategories.mkString(",")}") + + /** + * If true, then "categories" is the set of categories for splitting to the left, and vice versa. + */ + private val isLeft: Boolean = leftCategories.length <= numCategories / 2 + + /** Set of categories determining the splitting rule, along with [[isLeft]]. */ + private val categories: Set[Double] = { + if (isLeft) { + leftCategories.toSet + } else { + setComplement(leftCategories.toSet) + } + } + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + if (isLeft) { + categories.contains(features(featureIndex)) + } else { + !categories.contains(features(featureIndex)) + } + } + + override def equals(o: Any): Boolean = { + o match { + case other: CategoricalSplit => featureIndex == other.featureIndex && + isLeft == other.isLeft && categories == other.categories + case _ => false + } + } + + override private[tree] def toOld: OldSplit = { + val oldCats = if (isLeft) { + categories + } else { + setComplement(categories) + } + OldSplit(featureIndex, threshold = 0.0, OldFeatureType.Categorical, oldCats.toList) + } + + /** Get sorted categories which split to the left */ + def getLeftCategories: Array[Double] = { + val cats = if (isLeft) categories else setComplement(categories) + cats.toArray.sorted + } + + /** Get sorted categories which split to the right */ + def getRightCategories: Array[Double] = { + val cats = if (isLeft) setComplement(categories) else categories + cats.toArray.sorted + } + + /** [0, numCategories) \ cats */ + private def setComplement(cats: Set[Double]): Set[Double] = { + Range(0, numCategories).map(_.toDouble).filter(cat => !cats.contains(cat)).toSet + } +} + +/** + * Split which tests a continuous feature. + * @param featureIndex Index of the feature to test + * @param threshold If the feature value is <= this threshold, then the split goes left. + * Otherwise, it goes right. + */ +final class ContinuousSplit(override val featureIndex: Int, val threshold: Double) extends Split { + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + features(featureIndex) <= threshold + } + + override def equals(o: Any): Boolean = { + o match { + case other: ContinuousSplit => + featureIndex == other.featureIndex && threshold == other.threshold + case _ => + false + } + } + + override private[tree] def toOld: OldSplit = { + OldSplit(featureIndex, threshold, OldFeatureType.Continuous, List.empty[Double]) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala new file mode 100644 index 0000000000000..8e3bc3849dcf0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tree + +import org.apache.spark.annotation.AlphaComponent + + +/** + * :: AlphaComponent :: + * + * Abstraction for Decision Tree models. + * + * TODO: Add support for predicting probabilities and raw predictions + */ +@AlphaComponent +trait DecisionTreeModel { + + /** Root of the decision tree */ + def rootNode: Node + + /** Number of nodes in tree, including leaf nodes. */ + def numNodes: Int = { + 1 + rootNode.numDescendants + } + + /** + * Depth of the tree. + * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. + */ + lazy val depth: Int = { + rootNode.subtreeDepth + } + + /** Summary of the model */ + override def toString: String = { + // Implementing classes should generally override this method to be more descriptive. + s"DecisionTreeModel of depth $depth with $numNodes nodes" + } + + /** Full description of model */ + def toDebugString: String = { + val header = toString + "\n" + header + rootNode.subtreeToString(2) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala new file mode 100644 index 0000000000000..c84c8b4eb744f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import scala.collection.immutable.HashMap + +import org.apache.spark.annotation.Experimental +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, BinaryAttribute, NominalAttribute, + NumericAttribute} +import org.apache.spark.sql.types.StructField + + +/** + * :: Experimental :: + * + * Helper utilities for tree-based algorithms + */ +@Experimental +object MetadataUtils { + + /** + * Examine a schema to identify the number of classes in a label column. + * Returns None if the number of labels is not specified, or if the label column is continuous. + */ + def getNumClasses(labelSchema: StructField): Option[Int] = { + Attribute.fromStructField(labelSchema) match { + case numAttr: NumericAttribute => None + case binAttr: BinaryAttribute => Some(2) + case nomAttr: NominalAttribute => nomAttr.getNumValues + } + } + + /** + * Examine a schema to identify categorical (Binary and Nominal) features. + * + * @param featuresSchema Schema of the features column. + * If a feature does not have metadata, it is assumed to be continuous. + * If a feature is Nominal, then it must have the number of values + * specified. + * @return Map: feature index --> number of categories. + * The map's set of keys will be the set of categorical feature indices. + */ + def getCategoricalFeatures(featuresSchema: StructField): Map[Int, Int] = { + val metadata = AttributeGroup.fromStructField(featuresSchema) + if (metadata.attributes.isEmpty) { + HashMap.empty[Int, Int] + } else { + metadata.attributes.get.zipWithIndex.flatMap { case (attr, idx) => + if (attr == null) { + Iterator() + } else { + attr match { + case numAttr: NumericAttribute => Iterator() + case binAttr: BinaryAttribute => Iterator(idx -> 2) + case nomAttr: NominalAttribute => + nomAttr.getNumValues match { + case Some(numValues: Int) => Iterator(idx -> numValues) + case None => throw new IllegalArgumentException(s"Feature $idx is marked as" + + " Nominal (categorical), but it does not have the number of values specified.") + } + } + } + }.toMap + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b9d0c56dd1ea3..dfe3a0b6913ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -1147,7 +1147,10 @@ object DecisionTree extends Serializable with Logging { } } - assert(splits.length > 0) + // TODO: Do not fail; just ignore the useless feature. + assert(splits.length > 0, + s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + + " Please remove this feature and then try again.") // set number of splits accordingly metadata.setNumSplits(featureIndex, splits.length) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index c02c79f094b66..0e31c7ed58df8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -81,11 +81,11 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) /** * Method to validate a gradient boosting model * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @param validationInput Validation dataset: - RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - Should be different from and follow the same distribution as input. - e.g., these two datasets could be created from an original dataset - by using [[org.apache.spark.rdd.RDD.randomSplit()]] + * @param validationInput Validation dataset. + * This dataset should be different from the training dataset, + * but it should follow the same distribution. + * E.g., these two datasets could be created from an original dataset + * by using [[org.apache.spark.rdd.RDD.randomSplit()]] * @return a gradient boosted trees model that can be used for prediction */ def runWithValidation( @@ -194,8 +194,6 @@ object GradientBoostedTrees extends Logging { val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel baseLearnerWeights(0) = firstTreeWeight - val startingModel = new GradientBoostedTreesModel( - Regression, Array(firstTreeModel), baseLearnerWeights.slice(0, 1)) var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index db01f2e229e5a..055e60c7d9c95 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -249,7 +249,7 @@ private class RandomForest ( nodeIdCache.get.deleteAllCheckpoints() } catch { case e:IOException => - logWarning(s"delete all chackpoints failed. Error reason: ${e.getMessage}") + logWarning(s"delete all checkpoints failed. Error reason: ${e.getMessage}") } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 664c8df019233..2d6b01524ff3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -89,14 +89,14 @@ object BoostingStrategy { * @return Configuration for boosting algorithm */ def defaultParams(algo: Algo): BoostingStrategy = { - val treeStragtegy = Strategy.defaultStategy(algo) - treeStragtegy.maxDepth = 3 + val treeStrategy = Strategy.defaultStategy(algo) + treeStrategy.maxDepth = 3 algo match { case Algo.Classification => - treeStragtegy.numClasses = 2 - new BoostingStrategy(treeStragtegy, LogLoss) + treeStrategy.numClasses = 2 + new BoostingStrategy(treeStrategy, LogLoss) case Algo.Regression => - new BoostingStrategy(treeStragtegy, SquaredError) + new BoostingStrategy(treeStrategy, SquaredError) case _ => throw new IllegalArgumentException(s"$algo is not supported by boosting.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 6f570b4e09c79..2bdef73c4a8f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -45,9 +45,8 @@ object AbsoluteError extends Loss { if (label - prediction < 0) 1.0 else -1.0 } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = label - prediction math.abs(err) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 24ee9f3d51293..778c24526de70 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -47,10 +47,9 @@ object LogLoss extends Loss { - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val margin = 2.0 * label * prediction // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. 2.0 * MLUtils.log1pExp(-margin) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index d3b82b752fa0d..64ffccbce073f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -22,6 +22,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. @@ -57,6 +58,5 @@ trait Loss extends Serializable { * @param label True label. * @return Measure of model error on datapoint. */ - def computeError(prediction: Double, label: Double): Double - + private[mllib] def computeError(prediction: Double, label: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index 58857ae15e93e..a5582d3ef3324 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -45,9 +45,8 @@ object SquaredError extends Loss { 2.0 * (prediction - label) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = prediction - label err * err } - } 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 c9bafd60fba4d..331af428533de 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 @@ -113,11 +113,13 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = DecisionTreeModel.formatVersion } object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { + private[spark] def formatVersion: String = "1.0" + private[tree] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 4f72bb8014cc0..708ba04b567d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -175,7 +175,7 @@ class Node ( } } -private[tree] object Node { +private[spark] object Node { /** * Return a node with the given node id (but nothing else set). 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 fef3d2acb202a..8341219bfa71c 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 @@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils + /** * :: Experimental :: * Represents a random forest model. @@ -47,7 +48,7 @@ import org.apache.spark.util.Utils */ @Experimental class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) - extends TreeEnsembleModel(algo, trees, Array.fill(trees.size)(1.0), + extends TreeEnsembleModel(algo, trees, Array.fill(trees.length)(1.0), combiningStrategy = if (algo == Classification) Vote else Average) with Saveable { @@ -58,11 +59,13 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis RandomForestModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override protected def formatVersion: String = RandomForestModel.formatVersion } object RandomForestModel extends Loader[RandomForestModel] { + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): RandomForestModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -102,15 +105,13 @@ class GradientBoostedTreesModel( extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) with Saveable { - require(trees.size == treeWeights.size) + require(trees.length == treeWeights.length) override def save(sc: SparkContext, path: String): Unit = { TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, GradientBoostedTreesModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion - /** * Method to compute error or loss for every iteration of gradient boosting. * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] @@ -138,7 +139,7 @@ class GradientBoostedTreesModel( evaluationArray(0) = predictionAndError.values.mean() val broadcastTrees = sc.broadcast(trees) - (1 until numIterations).map { nTree => + (1 until numIterations).foreach { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) val currentTreeWeight = localTreeWeights(nTree) @@ -155,6 +156,7 @@ class GradientBoostedTreesModel( evaluationArray } + override protected def formatVersion: String = GradientBoostedTreesModel.formatVersion } object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { @@ -200,17 +202,17 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { loss: Loss): RDD[(Double, Double)] = { val newPredError = data.zip(predictionAndError).mapPartitions { iter => - iter.map { - case (lp, (pred, error)) => { - val newPred = pred + tree.predict(lp.features) * treeWeight - val newError = loss.computeError(newPred, lp.label) - (newPred, newError) - } + iter.map { case (lp, (pred, error)) => + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) } } newPredError } + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -340,12 +342,12 @@ private[tree] sealed class TreeEnsembleModel( } /** - * Get number of trees in forest. + * Get number of trees in ensemble. */ - def numTrees: Int = trees.size + def numTrees: Int = trees.length /** - * Get total number of nodes, summed over all trees in the forest. + * Get total number of nodes, summed over all trees in the ensemble. */ def totalNumNodes: Int = trees.map(_.numNodes).sum } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java new file mode 100644 index 0000000000000..43b8787f9dd7e --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification; + +import java.io.File; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.util.Utils; + + +public class JavaDecisionTreeClassifierSuite implements Serializable { + + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeClassifierSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeClassifier dt = new DecisionTreeClassifier() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (int i = 0; i < DecisionTreeClassifier.supportedImpurities().length; ++i) { + dt.setImpurity(DecisionTreeClassifier.supportedImpurities()[i]); + } + DecisionTreeClassificationModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model3.save(sc.sc(), path); + DecisionTreeClassificationModel sameModel = + DecisionTreeClassificationModel.load(sc.sc(), path); + TreeTests.checkEqual(model3, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java new file mode 100644 index 0000000000000..a3a339004f31c --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -0,0 +1,97 @@ +/* + * 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.regression; + +import java.io.File; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.util.Utils; + + +public class JavaDecisionTreeRegressorSuite implements Serializable { + + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeRegressorSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeRegressor dt = new DecisionTreeRegressor() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (int i = 0; i < DecisionTreeRegressor.supportedImpurities().length; ++i) { + dt.setImpurity(DecisionTreeRegressor.supportedImpurities()[i]); + } + DecisionTreeRegressionModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model2.save(sc.sc(), path); + DecisionTreeRegressionModel sameModel = DecisionTreeRegressionModel.load(sc.sc(), path); + TreeTests.checkEqual(model2, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala index 0dcfe5a2002dc..17ddd335deb6d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -44,7 +44,7 @@ class AttributeGroupSuite extends FunSuite { group("abc") } assert(group === AttributeGroup.fromMetadata(group.toMetadataImpl, group.name)) - assert(group === AttributeGroup.fromStructField(group.toStructField)) + assert(group === AttributeGroup.fromStructField(group.toStructField())) } test("attribute group without attributes") { @@ -54,7 +54,7 @@ class AttributeGroupSuite extends FunSuite { assert(group0.size === 10) assert(group0.attributes.isEmpty) assert(group0 === AttributeGroup.fromMetadata(group0.toMetadataImpl, group0.name)) - assert(group0 === AttributeGroup.fromStructField(group0.toStructField)) + assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) val group1 = new AttributeGroup("item") assert(group1.name === "item") diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index 6ec35b03656f9..3e1a7196e37cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -36,9 +36,9 @@ class AttributeSuite extends FunSuite { assert(attr.max.isEmpty) assert(attr.std.isEmpty) assert(attr.sparsity.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) intercept[NoSuchElementException] { @@ -59,9 +59,9 @@ class AttributeSuite extends FunSuite { assert(!attr.isNominal) assert(attr.name === Some(name)) assert(attr.index === Some(index)) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) val field = attr.toStructField() @@ -81,7 +81,7 @@ class AttributeSuite extends FunSuite { assert(attr2.max === Some(1.0)) assert(attr2.std === Some(0.5)) assert(attr2.sparsity === Some(0.3)) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) } test("bad numeric attributes") { @@ -105,9 +105,9 @@ class AttributeSuite extends FunSuite { assert(attr.values.isEmpty) assert(attr.numValues.isEmpty) assert(attr.isOrdinal.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -135,9 +135,9 @@ class AttributeSuite extends FunSuite { assert(attr.values === Some(values)) assert(attr.indexOf("medium") === 1) assert(attr.getValue(1) === "medium") - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) @@ -147,8 +147,8 @@ class AttributeSuite extends FunSuite { assert(attr2.index.isEmpty) assert(attr2.values.get === Array("small", "medium", "large", "x-large")) assert(attr2.indexOf("x-large") === 3) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) - assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadata(withType = false))) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) + assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadataImpl(withType = false))) } test("bad nominal attributes") { @@ -168,9 +168,9 @@ class AttributeSuite extends FunSuite { assert(attr.name.isEmpty) assert(attr.index.isEmpty) assert(attr.values.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -196,9 +196,9 @@ class AttributeSuite extends FunSuite { assert(attr.name === Some(name)) assert(attr.index === Some(index)) assert(attr.values.get === values) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala new file mode 100644 index 0000000000000..af88595df5245 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -0,0 +1,274 @@ +/* + * 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.classification + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeClassifierSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeClassifierSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel0RDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel1RDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var continuousDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassForOrderedFeaturesRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + orderedLabeledPointsWithLabel0RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + orderedLabeledPointsWithLabel1RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + categoricalDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + continuousDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( + OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Binary classification stump with ordered categorical features") { + val dt = new DecisionTreeClassifier() + .setImpurity("gini") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + val numClasses = 2 + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with fixed labels 0,1 for Entropy,Gini") { + val dt = new DecisionTreeClassifier() + .setMaxDepth(3) + .setMaxBins(100) + val numClasses = 2 + Array(orderedLabeledPointsWithLabel0RDD, orderedLabeledPointsWithLabel1RDD).foreach { rdd => + DecisionTreeClassifier.supportedImpurities.foreach { impurity => + dt.setImpurity(impurity) + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + } + } + + test("Multiclass classification stump with 3-ary (unordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 3 + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Binary classification stump with 2 continuous features") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(maxBins) + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with continuous features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with continuous + unordered categorical features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with 10-ary (ordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(10) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min instances per node requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("do not choose split that does not satisfy min instance per node requirements") { + // if a split does not satisfy min instances per node requirements, + // this split is invalid, even though the information gain of split is large. + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxBins(2) + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min info gain requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInfoGain(1.0) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val oldModel = OldDecisionTreeSuite.createModel(OldAlgo.Classification) + val newModel = DecisionTreeClassificationModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = DecisionTreeClassificationModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private[ml] object DecisionTreeClassifierSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeClassifier, + categoricalFeatures: Map[Int, Int], + numClasses: Int): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeClassificationModel.fromOld(oldTree, newTree.parent, + newTree.fittingParamMap, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 81ef831c42e55..1b261b2643854 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -228,7 +228,7 @@ class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { } val attrGroup = new AttributeGroup("features", featureAttributes) val densePoints1WithMeta = - densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata)) + densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata())) val vectorIndexer = getIndexer.setMaxCategories(2) val model = vectorIndexer.fit(densePoints1WithMeta) // Check that ML metadata are preserved. diff --git a/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala new file mode 100644 index 0000000000000..2e57d4ce37f1d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala @@ -0,0 +1,132 @@ +/* + * 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.impl + +import scala.collection.JavaConverters._ + +import org.scalatest.FunSuite + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.tree.{DecisionTreeModel, InternalNode, LeafNode, Node} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, DataFrame} + + +private[ml] object TreeTests extends FunSuite { + + /** + * Convert the given data to a DataFrame, and set the features and label metadata. + * @param data Dataset. Categorical features and labels must already have 0-based indices. + * This must be non-empty. + * @param categoricalFeatures Map: categorical feature index -> number of distinct values + * @param numClasses Number of classes label can take. If 0, mark as continuous. + * @return DataFrame with metadata + */ + def setMetadata( + data: RDD[LabeledPoint], + categoricalFeatures: Map[Int, Int], + numClasses: Int): DataFrame = { + val sqlContext = new SQLContext(data.sparkContext) + import sqlContext.implicits._ + val df = data.toDF() + val numFeatures = data.first().features.size + val featuresAttributes = Range(0, numFeatures).map { feature => + if (categoricalFeatures.contains(feature)) { + NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) + } else { + NumericAttribute.defaultAttr.withIndex(feature) + } + }.toArray + val featuresMetadata = new AttributeGroup("features", featuresAttributes).toMetadata() + val labelAttribute = if (numClasses == 0) { + NumericAttribute.defaultAttr.withName("label") + } else { + NominalAttribute.defaultAttr.withName("label").withNumValues(numClasses) + } + val labelMetadata = labelAttribute.toMetadata() + df.select(df("features").as("features", featuresMetadata), + df("label").as("label", labelMetadata)) + } + + /** Java-friendly version of [[setMetadata()]] */ + def setMetadata( + data: JavaRDD[LabeledPoint], + categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer], + numClasses: Int): DataFrame = { + setMetadata(data.rdd, categoricalFeatures.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numClasses) + } + + /** + * Check if the two trees are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + * If the trees are not equal, this prints the two trees and throws an exception. + */ + def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + try { + checkEqual(a.rootNode, b.rootNode) + } catch { + case ex: Exception => + throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + "TREE A:\n" + a.toDebugString + "\n" + + "TREE B:\n" + b.toDebugString + "\n", ex) + } + } + + /** + * Return true iff the two nodes and their descendants are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + */ + private def checkEqual(a: Node, b: Node): Unit = { + assert(a.prediction === b.prediction) + assert(a.impurity === b.impurity) + (a, b) match { + case (aye: InternalNode, bee: InternalNode) => + assert(aye.split === bee.split) + checkEqual(aye.leftChild, bee.leftChild) + checkEqual(aye.rightChild, bee.rightChild) + case (aye: LeafNode, bee: LeafNode) => // do nothing + case _ => + throw new AssertionError("Found mismatched nodes") + } + } + + // TODO: Reinstate after adding ensembles + /** + * Check if the two models are exactly the same. + * If the models are not equal, this throws an exception. + */ + /* + def checkEqual(a: TreeEnsembleModel, b: TreeEnsembleModel): Unit = { + try { + a.getTrees.zip(b.getTrees).foreach { case (treeA, treeB) => + TreeTests.checkEqual(treeA, treeB) + } + assert(a.getTreeWeights === b.getTreeWeights) + } catch { + case ex: Exception => throw new AssertionError( + "checkEqual failed since the two tree ensembles were not identical") + } + } + */ +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala new file mode 100644 index 0000000000000..0b40fe33fae9d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeRegressorSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeRegressorSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Regression stump with 3-ary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + test("Regression stump with binary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: test("model save/load") +} + +private[ml] object DecisionTreeRegressorSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeRegressor, + categoricalFeatures: Map[Int, Int]): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeRegressionModel.fromOld(oldTree, newTree.parent, + newTree.fittingParamMap, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 4c162df810bb2..249b8eae19b17 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -36,6 +36,10 @@ import org.apache.spark.util.Utils class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { + ///////////////////////////////////////////////////////////////////////////// + // Tests examining individual elements of training + ///////////////////////////////////////////////////////////////////////////// + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -254,6 +258,165 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(bins(0).length === 0) } + test("Avoid aggregation on the last level") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue leaf nodes into node queue + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Avoid aggregation if impurity is 0.0") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue a node into node queue if its impurity is 0.0 + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Second level node building with vs. without groups") { + val arr = DecisionTreeSuite.generateOrderedLabeledPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, + numClasses = 2, maxBins = 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val rootNode1 = modelOneNode.topNode.deepCopy() + val rootNode2 = modelOneNode.topNode.deepCopy() + assert(rootNode1.leftNode.nonEmpty) + assert(rootNode1.rightNode.nonEmpty) + + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + // Single group second level tree construction. + val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) + val treeToNodeToIndexInfo = Map((0, Map( + (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), + (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + val children1 = new Array[Node](2) + children1(0) = rootNode1.leftNode.get + children1(1) = rootNode1.rightNode.get + + // Train one second-level node at a time. + val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) + val treeToNodeToIndexInfoA = Map((0, Map( + (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) + val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) + val treeToNodeToIndexInfoB = Map((0, Map( + (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) + val children2 = new Array[Node](2) + children2(0) = rootNode2.leftNode.get + children2(1) = rootNode2.rightNode.get + + // Verify whether the splits obtained using single group and multiple group level + // construction strategies are the same. + for (i <- 0 until 2) { + assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) + assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) + val stats1 = children1(i).stats.get + val stats2 = children2(i).stats.get + assert(stats1.gain === stats2.gain) + assert(stats1.impurity === stats2.impurity) + assert(stats1.leftImpurity === stats2.leftImpurity) + assert(stats1.rightImpurity === stats2.rightImpurity) + assert(children1(i).predict.predict === children2(i).predict.predict) + } + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() @@ -438,76 +601,6 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(rootNode.predict.predict === 1) } - test("Second level node building with vs. without groups") { - val arr = DecisionTreeSuite.generateOrderedLabeledPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - - // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClasses = 2, maxBins = 100) - val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNode1 = modelOneNode.topNode.deepCopy() - val rootNode2 = modelOneNode.topNode.deepCopy() - assert(rootNode1.leftNode.nonEmpty) - assert(rootNode1.rightNode.nonEmpty) - - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - // Single group second level tree construction. - val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) - val treeToNodeToIndexInfo = Map((0, Map( - (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), - (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - val children1 = new Array[Node](2) - children1(0) = rootNode1.leftNode.get - children1(1) = rootNode1.rightNode.get - - // Train one second-level node at a time. - val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) - val treeToNodeToIndexInfoA = Map((0, Map( - (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) - val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) - val treeToNodeToIndexInfoB = Map((0, Map( - (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) - val children2 = new Array[Node](2) - children2(0) = rootNode2.leftNode.get - children2(1) = rootNode2.rightNode.get - - // Verify whether the splits obtained using single group and multiple group level - // construction strategies are the same. - for (i <- 0 until 2) { - assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) - assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) - assert(children1(i).split === children2(i).split) - assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) - val stats1 = children1(i).stats.get - val stats2 = children2(i).stats.get - assert(stats1.gain === stats2.gain) - assert(stats1.impurity === stats2.impurity) - assert(stats1.leftImpurity === stats2.leftImpurity) - assert(stats1.rightImpurity === stats2.rightImpurity) - assert(children1(i).predict.predict === children2(i).predict.predict) - } - } - test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) @@ -528,11 +621,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) - arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 2) @@ -544,11 +637,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 2 continuous features") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -668,11 +761,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min instances per node requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) - + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, numClasses = 2, minInstancesPerNode = 2) @@ -695,11 +787,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { test("do not choose split that does not satisfy min instance per node requirements") { // if a split does not satisfy min instances per node requirements, // this split is invalid, even though the information gain of split is large. - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) - arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, @@ -715,10 +807,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min info gain requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, @@ -739,91 +831,9 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(gain == InformationGainStats.invalidInformationGainStats) } - test("Avoid aggregation on the last level") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue leaf nodes into node queue - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } - - test("Avoid aggregation if impurity is 0.0") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue a node into node queue if its impurity is 0.0 - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// test("Node.subtreeIterator") { val model = DecisionTreeSuite.createModel(Classification) @@ -996,8 +1006,9 @@ object DecisionTreeSuite extends FunSuite { /** * Create a tree model. This is deterministic and contains a variety of node and feature types. + * TODO: Update this to be a correct tree (with matching probabilities, impurities, etc.) */ - private[tree] def createModel(algo: Algo): DecisionTreeModel = { + private[mllib] def createModel(algo: Algo): DecisionTreeModel = { val topNode = createInternalNode(id = 1, Continuous) val (node2, node3) = (createLeafNode(id = 2), createInternalNode(id = 3, Categorical)) val (node6, node7) = (createLeafNode(id = 6), createLeafNode(id = 7)) @@ -1017,7 +1028,7 @@ object DecisionTreeSuite extends FunSuite { * make mistakes such as creating loops of Nodes. * If the trees are not equal, this prints the two trees and throws an exception. */ - private[tree] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + private[mllib] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { try { assert(a.algo === b.algo) checkEqual(a.topNode, b.topNode) From 59e206deb7346148412bbf5ba4ab626718fadf18 Mon Sep 17 00:00:00 2001 From: cafreeman Date: Fri, 17 Apr 2015 13:42:19 -0700 Subject: [PATCH 805/817] [SPARK-6807] [SparkR] Merge recent SparkR-pkg changes This PR pulls in recent changes in SparkR-pkg, including cartesian, intersection, sampleByKey, subtract, subtractByKey, except, and some API for StructType and StructField. Author: cafreeman Author: Davies Liu Author: Zongheng Yang Author: Shivaram Venkataraman Author: Shivaram Venkataraman Author: Sun Rui Closes #5436 from davies/R3 and squashes the following commits: c2b09be [Davies Liu] SQLTypes -> schema a5a02f2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R3 168b7fe [Davies Liu] sort generics b1fe460 [Davies Liu] fix conflict in README.md e74c04e [Davies Liu] fix schema.R 4f5ac09 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R5 41f8184 [Davies Liu] rm man ae78312 [Davies Liu] Merge pull request #237 from sun-rui/SPARKR-154_3 1bdcb63 [Zongheng Yang] Updates to README.md. 5a553e7 [cafreeman] Use object attribute instead of argument 71372d9 [cafreeman] Update docs and examples 8526d2e71 [cafreeman] Remove `tojson` functions 6ef5f2d [cafreeman] Fix spacing 7741d66 [cafreeman] Rename the SQL DataType function 141efd8 [Shivaram Venkataraman] Merge pull request #245 from hqzizania/upstream 9387402 [Davies Liu] fix style 40199eb [Shivaram Venkataraman] Move except into sorted position 07d0dbc [Sun Rui] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD. 7e8caa3 [Shivaram Venkataraman] Merge pull request #246 from hlin09/fixCombineByKey ed66c81 [cafreeman] Update `subtract` to work with `generics.R` f3ba785 [cafreeman] Fixed duplicate export 275deb4 [cafreeman] Update `NAMESPACE` and tests 1a3b63d [cafreeman] new version of `CreateDF` 836c4bf [cafreeman] Update `createDataFrame` and `toDF` be5d5c1 [cafreeman] refactor schema functions 40338a4 [Zongheng Yang] Merge pull request #244 from sun-rui/SPARKR-154_5 20b97a6 [Zongheng Yang] Merge pull request #234 from hqzizania/assist ba54e34 [Shivaram Venkataraman] Merge pull request #238 from sun-rui/SPARKR-154_4 c9497a3 [Shivaram Venkataraman] Merge pull request #208 from lythesia/master b317aa7 [Zongheng Yang] Merge pull request #243 from hqzizania/master 136a07e [Zongheng Yang] Merge pull request #242 from hqzizania/stats cd66603 [cafreeman] new line at EOF 8b76e81 [Shivaram Venkataraman] Merge pull request #233 from redbaron/fail-early-on-missing-dep 7dd81b7 [cafreeman] Documentation 0e2a94f [cafreeman] Define functions for schema and fields --- R/pkg/DESCRIPTION | 2 +- R/pkg/NAMESPACE | 20 +- R/pkg/R/DataFrame.R | 18 +- R/pkg/R/RDD.R | 205 ++++++++++++------ R/pkg/R/SQLContext.R | 44 +--- R/pkg/R/SQLTypes.R | 64 ------ R/pkg/R/column.R | 2 +- R/pkg/R/generics.R | 46 +++- R/pkg/R/group.R | 2 +- R/pkg/R/pairRDD.R | 192 +++++++++++++--- R/pkg/R/schema.R | 162 ++++++++++++++ R/pkg/R/serialize.R | 9 +- R/pkg/R/utils.R | 80 +++++++ R/pkg/inst/tests/test_rdd.R | 193 ++++++++++++++--- R/pkg/inst/tests/test_shuffle.R | 12 + R/pkg/inst/tests/test_sparkSQL.R | 35 +-- R/pkg/inst/worker/worker.R | 59 ++++- .../scala/org/apache/spark/api/r/RRDD.scala | 131 +++++------ .../scala/org/apache/spark/api/r/SerDe.scala | 14 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 32 ++- 20 files changed, 971 insertions(+), 351 deletions(-) delete mode 100644 R/pkg/R/SQLTypes.R create mode 100644 R/pkg/R/schema.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 052f68c6c24e2..1c1779a763c7e 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -19,7 +19,7 @@ Collate: 'jobj.R' 'RDD.R' 'pairRDD.R' - 'SQLTypes.R' + 'schema.R' 'column.R' 'group.R' 'DataFrame.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index a354cdce74afa..80283643861ac 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -5,6 +5,7 @@ exportMethods( "aggregateByKey", "aggregateRDD", "cache", + "cartesian", "checkpoint", "coalesce", "cogroup", @@ -28,6 +29,7 @@ exportMethods( "fullOuterJoin", "glom", "groupByKey", + "intersection", "join", "keyBy", "keys", @@ -52,11 +54,14 @@ exportMethods( "reduceByKeyLocally", "repartition", "rightOuterJoin", + "sampleByKey", "sampleRDD", "saveAsTextFile", "saveAsObjectFile", "sortBy", "sortByKey", + "subtract", + "subtractByKey", "sumRDD", "take", "takeOrdered", @@ -95,6 +100,7 @@ exportClasses("DataFrame") exportMethods("columns", "distinct", "dtypes", + "except", "explain", "filter", "groupBy", @@ -118,7 +124,6 @@ exportMethods("columns", "show", "showDF", "sortDF", - "subtract", "toJSON", "toRDD", "unionAll", @@ -178,5 +183,14 @@ export("cacheTable", "toDF", "uncacheTable") -export("print.structType", - "print.structField") +export("sparkRSQL.init", + "sparkRHive.init") + +export("structField", + "structField.jobj", + "structField.character", + "print.structField", + "structType", + "structType.jobj", + "structType.structField", + "print.structType") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 044fdb4d01223..861fe1c78b0db 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") @@ -1141,15 +1141,15 @@ setMethod("intersect", dataFrame(intersected) }) -#' Subtract +#' except #' #' Return a new DataFrame containing rows in this DataFrame #' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. #' #' @param x A Spark DataFrame #' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the subtract operation. -#' @rdname subtract +#' @return A DataFrame containing the result of the except operation. +#' @rdname except #' @export #' @examples #'\dontrun{ @@ -1157,13 +1157,15 @@ setMethod("intersect", #' sqlCtx <- sparkRSQL.init(sc) #' df1 <- jsonFile(sqlCtx, path) #' df2 <- jsonFile(sqlCtx, path2) -#' subtractDF <- subtract(df, df2) +#' exceptDF <- except(df, df2) #' } -setMethod("subtract", +#' @rdname except +#' @export +setMethod("except", signature(x = "DataFrame", y = "DataFrame"), function(x, y) { - subtracted <- callJMethod(x@sdf, "except", y@sdf) - dataFrame(subtracted) + excepted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(excepted) }) #' Save the contents of the DataFrame to a data source diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 820027ef67e3b..128431334ca52 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -730,6 +730,7 @@ setMethod("take", index <- -1 jrdd <- getJRDD(x) numPartitions <- numPartitions(x) + serializedModeRDD <- getSerializedMode(x) # TODO(shivaram): Collect more than one partition based on size # estimates similar to the scala version of `take`. @@ -748,13 +749,14 @@ setMethod("take", elems <- convertJListToRList(partition, flatten = TRUE, logicalUpperBound = size, - serializedMode = getSerializedMode(x)) - # TODO: Check if this append is O(n^2)? + serializedMode = serializedModeRDD) + resList <- append(resList, elems) } resList }) + #' First #' #' Return the first element of an RDD @@ -1092,21 +1094,42 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { if (num < length(part)) { # R limitation: order works only on primitive types! ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) - list(part[ord[1:num]]) + part[ord[1:num]] } else { - list(part) + part } } - reduceFunc <- function(elems, part) { - newElems <- append(elems, part) - # R limitation: order works only on primitive types! - ord <- order(unlist(newElems, recursive = FALSE), decreasing = !ascending) - newElems[ord[1:num]] - } - newRdd <- mapPartitions(x, partitionFunc) - reduce(newRdd, reduceFunc) + + resList <- list() + index <- -1 + jrdd <- getJRDD(newRdd) + numPartitions <- numPartitions(newRdd) + serializedModeRDD <- getSerializedMode(newRdd) + + while (TRUE) { + index <- index + 1 + + if (index >= numPartitions) { + ord <- order(unlist(resList, recursive = FALSE), decreasing = !ascending) + resList <- resList[ord[1:num]] + break + } + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + # elems is capped to have at most `num` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = num, + serializedMode = serializedModeRDD) + + resList <- append(resList, elems) + } + resList } #' Returns the first N elements from an RDD in ascending order. @@ -1465,67 +1488,105 @@ setMethod("zipRDD", stop("Can only zip RDDs which have the same number of partitions.") } - if (getSerializedMode(x) != getSerializedMode(other) || - getSerializedMode(x) == "byte") { - # Append the number of elements in each partition to that partition so that we can later - # check if corresponding partitions of both RDDs have the same number of elements. - # - # Note that this appending also serves the purpose of reserialization, because even if - # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded - # as a single byte array. For example, partitions of an RDD generated from partitionBy() - # may be encoded as multiple byte arrays. - appendLength <- function(part) { - part[[length(part) + 1]] <- length(part) + 1 - part - } - x <- lapplyPartition(x, appendLength) - other <- lapplyPartition(other, appendLength) - } + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "zip", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) - zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) - # The zippedRDD's elements are of scala Tuple2 type. The serialized - # flag Here is used for the elements inside the tuples. - serializerMode <- getSerializedMode(x) - zippedRDD <- RDD(zippedJRDD, serializerMode) + mergePartitions(rdd, TRUE) + }) + +#' Cartesian product of this RDD and another one. +#' +#' Return the Cartesian product of this RDD and another one, +#' that is, the RDD of all pairs of elements (a, b) where a +#' is in this and b is in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @return A new RDD which is the Cartesian product of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2) +#' sortByKey(cartesian(rdd, rdd)) +#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#'} +#' @rdname cartesian +#' @aliases cartesian,RDD,RDD-method +setMethod("cartesian", + signature(x = "RDD", other = "RDD"), + function(x, other) { + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "cartesian", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) - partitionFunc <- function(split, part) { - len <- length(part) - if (len > 0) { - if (serializerMode == "byte") { - lengthOfValues <- part[[len]] - lengthOfKeys <- part[[len - lengthOfValues]] - stopifnot(len == lengthOfKeys + lengthOfValues) - - # check if corresponding partitions of both RDDs have the same number of elements. - if (lengthOfKeys != lengthOfValues) { - stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") - } - - if (lengthOfKeys > 1) { - keys <- part[1 : (lengthOfKeys - 1)] - values <- part[(lengthOfKeys + 1) : (len - 1)] - } else { - keys <- list() - values <- list() - } - } else { - # Keys, values must have same length here, because this has - # been validated inside the JavaRDD.zip() function. - keys <- part[c(TRUE, FALSE)] - values <- part[c(FALSE, TRUE)] - } - mapply( - function(k, v) { - list(k, v) - }, - keys, - values, - SIMPLIFY = FALSE, - USE.NAMES = FALSE) - } else { - part - } + mergePartitions(rdd, FALSE) + }) + +#' Subtract an RDD with another RDD. +#' +#' Return an RDD with the elements from this that are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the elements from this that are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +#' rdd2 <- parallelize(sc, list(2, 4)) +#' collect(subtract(rdd1, rdd2)) +#' # list(1, 1, 3) +#'} +#' @rdname subtract +#' @aliases subtract,RDD +setMethod("subtract", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + mapFunction <- function(e) { list(e, NA) } + rdd1 <- map(x, mapFunction) + rdd2 <- map(other, mapFunction) + keys(subtractByKey(rdd1, rdd2, numPartitions)) + }) + +#' Intersection of this RDD and another one. +#' +#' Return the intersection of this RDD and another one. +#' The output will not contain any duplicate elements, +#' even if the input RDDs did. Performs a hash partition +#' across the cluster. +#' Note that this method performs a shuffle internally. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions The number of partitions in the result RDD. +#' @return An RDD which is the intersection of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' # list(1, 2, 3) +#'} +#' @rdname intersection +#' @aliases intersection,RDD +setMethod("intersection", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + rdd1 <- map(x, function(v) { list(v, NA) }) + rdd2 <- map(other, function(v) { list(v, NA) }) + + filterFunction <- function(elem) { + iters <- elem[[2]] + all(as.vector( + lapply(iters, function(iter) { length(iter) > 0 }), mode = "logical")) } - - PipelinedRDD(zippedRDD, partitionFunc) + + keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 930ada22f4c38..4f05ba524a01a 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -54,9 +54,9 @@ infer_type <- function(x) { # StructType types <- lapply(x, infer_type) fields <- lapply(1:length(x), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + structField(names[[i]], types[[i]], TRUE) }) - list(type = "struct", fields = fields) + do.call(structType, fields) } } else if (length(x) > 1) { list(type = "array", elementType = type, containsNull = TRUE) @@ -65,30 +65,6 @@ infer_type <- function(x) { } } -#' dump the schema into JSON string -tojson <- function(x) { - if (is.list(x)) { - names <- names(x) - if (!is.null(names)) { - items <- lapply(names, function(n) { - safe_n <- gsub('"', '\\"', n) - paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') - }) - d <- paste(items, collapse = ', ') - paste('{', d, '}', sep = '') - } else { - l <- paste(lapply(x, tojson), collapse = ', ') - paste('[', l, ']', sep = '') - } - } else if (is.character(x)) { - paste('"', x, '"', sep = '') - } else if (is.logical(x)) { - if (x) "true" else "false" - } else { - stop(paste("unexpected type:", class(x))) - } -} - #' Create a DataFrame from an RDD #' #' Converts an RDD to a DataFrame by infer the types. @@ -134,7 +110,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || is.null(names(schema))) { + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { row <- first(rdd) names <- if (is.null(schema)) { names(row) @@ -143,7 +119,7 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { } if (is.null(names)) { names <- lapply(1:length(row), function(x) { - paste("_", as.character(x), sep = "") + paste("_", as.character(x), sep = "") }) } @@ -159,20 +135,18 @@ createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { types <- lapply(row, infer_type) fields <- lapply(1:length(row), function(i) { - list(name = names[[i]], type = types[[i]], nullable = TRUE) + structField(names[[i]], types[[i]], TRUE) }) - schema <- list(type = "struct", fields = fields) + schema <- do.call(structType, fields) } - stopifnot(class(schema) == "list") - stopifnot(schema$type == "struct") - stopifnot(class(schema$fields) == "list") - schemaString <- tojson(schema) + stopifnot(class(schema) == "structType") + # schemaString <- tojson(schema) jrdd <- getJRDD(lapply(rdd, function(x) x), "row") srdd <- callJMethod(jrdd, "rdd") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schemaString, sqlCtx) + srdd, schema$jobj, sqlCtx) dataFrame(sdf) } diff --git a/R/pkg/R/SQLTypes.R b/R/pkg/R/SQLTypes.R deleted file mode 100644 index 962fba5b3cf03..0000000000000 --- a/R/pkg/R/SQLTypes.R +++ /dev/null @@ -1,64 +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. -# - -# Utility functions for handling SparkSQL DataTypes. - -# Handler for StructType -structType <- function(st) { - obj <- structure(new.env(parent = emptyenv()), class = "structType") - obj$jobj <- st - obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) } - obj -} - -#' Print a Spark StructType. -#' -#' This function prints the contents of a StructType returned from the -#' SparkR JVM backend. -#' -#' @param x A StructType object -#' @param ... further arguments passed to or from other methods -print.structType <- function(x, ...) { - fieldsList <- lapply(x$fields(), function(i) { i$print() }) - print(fieldsList) -} - -# Handler for StructField -structField <- function(sf) { - obj <- structure(new.env(parent = emptyenv()), class = "structField") - obj$jobj <- sf - obj$name <- function() { callJMethod(sf, "name") } - obj$dataType <- function() { callJMethod(sf, "dataType") } - obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } - obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } - obj$nullable <- function() { callJMethod(sf, "nullable") } - obj$print <- function() { paste("StructField(", - paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "), - ")", sep = "") } - obj -} - -#' Print a Spark StructField. -#' -#' This function prints the contents of a StructField returned from the -#' SparkR JVM backend. -#' -#' @param x A StructField object -#' @param ... further arguments passed to or from other methods -print.structField <- function(x, ...) { - cat(x$print()) -} diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index b282001d8b6b5..95fb9ff0887b6 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R SQLTypes.R +#' @include generics.R jobj.R schema.R NULL setOldClass("jobj") diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 5fb1ccaa84ee2..6c6233390134c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -230,6 +230,10 @@ setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") } ############ Binary Functions ############# +#' @rdname cartesian +#' @export +setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) + #' @rdname countByKey #' @export setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) @@ -238,6 +242,11 @@ setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) #' @export setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) +#' @rdname intersection +#' @export +setGeneric("intersection", function(x, other, numPartitions = 1L) { + standardGeneric("intersection") }) + #' @rdname keys #' @export setGeneric("keys", function(x) { standardGeneric("keys") }) @@ -250,12 +259,18 @@ setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) #' @export setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) +#' @rdname sampleByKey +#' @export +setGeneric("sampleByKey", + function(x, withReplacement, fractions, seed) { + standardGeneric("sampleByKey") + }) + #' @rdname values #' @export setGeneric("values", function(x) { standardGeneric("values") }) - ############ Shuffle Functions ############ #' @rdname aggregateByKey @@ -330,9 +345,24 @@ setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("ri #' @rdname sortByKey #' @export -setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { - standardGeneric("sortByKey") -}) +setGeneric("sortByKey", + function(x, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortByKey") + }) + +#' @rdname subtract +#' @export +setGeneric("subtract", + function(x, other, numPartitions = 1L) { + standardGeneric("subtract") + }) + +#' @rdname subtractByKey +#' @export +setGeneric("subtractByKey", + function(x, other, numPartitions = 1L) { + standardGeneric("subtractByKey") + }) ################### Broadcast Variable Methods ################# @@ -357,6 +387,10 @@ setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) #' @export setGeneric("explain", function(x, ...) { standardGeneric("explain") }) +#' @rdname except +#' @export +setGeneric("except", function(x, y) { standardGeneric("except") }) + #' @rdname filter #' @export setGeneric("filter", function(x, condition) { standardGeneric("filter") }) @@ -434,10 +468,6 @@ setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) #' @export setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) -#' @rdname subtract -#' @export -setGeneric("subtract", function(x, y) { standardGeneric("subtract") }) - #' @rdname tojson #' @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 855fbdfc7c4ca..02237b3672d6b 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,7 +17,7 @@ # group.R - GroupedData class and methods implemented in S4 OO classes -#' @include generics.R jobj.R SQLTypes.R column.R +#' @include generics.R jobj.R schema.R column.R NULL setOldClass("jobj") diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 5d64822859d1f..13efebc11c46e 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -430,7 +430,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) }) convertEnvsToList(keys, combiners) @@ -443,7 +443,7 @@ setMethod("combineByKey", pred <- function(item) exists(item$hash, keys) lapply(part, function(item) { - item$hash <- as.character(item[[1]]) + item$hash <- as.character(hashCode(item[[1]])) updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) }) convertEnvsToList(keys, combiners) @@ -452,19 +452,19 @@ setMethod("combineByKey", }) #' Aggregate a pair RDD by each key. -#' +#' #' Aggregate the values of each key in an RDD, using given combine functions #' and a neutral "zero value". This function can return a different result type, #' U, than the type of the values in this RDD, V. Thus, we need one operation -#' for merging a V into a U and one operation for merging two U's, The former -#' operation is used for merging values within a partition, and the latter is -#' used for merging values between partitions. To avoid memory allocation, both -#' of these functions are allowed to modify and return their first argument +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument #' instead of creating a new U. -#' +#' #' @param x An RDD. #' @param zeroValue A neutral "zero value". -#' @param seqOp A function to aggregate the values of each key. It may return +#' @param seqOp A function to aggregate the values of each key. It may return #' a different result type from the type of the values. #' @param combOp A function to aggregate results of seqOp. #' @return An RDD containing the aggregation result. @@ -476,7 +476,7 @@ setMethod("combineByKey", #' zeroValue <- list(0, 0) #' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } #' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) #' # list(list(1, list(3, 2)), list(2, list(7, 2))) #'} #' @rdname aggregateByKey @@ -493,12 +493,12 @@ setMethod("aggregateByKey", }) #' Fold a pair RDD by each key. -#' +#' #' Aggregate the values of each key in an RDD, using an associative function "func" -#' and a neutral "zero value" which may be added to the result an arbitrary -#' number of times, and must not change the result (e.g., 0 for addition, or +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or #' 1 for multiplication.). -#' +#' #' @param x An RDD. #' @param zeroValue A neutral "zero value". #' @param func An associative function for folding values of each key. @@ -548,11 +548,11 @@ setMethod("join", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), doJoin) }) @@ -568,8 +568,8 @@ setMethod("join", #' @param y An RDD to be joined. Should be an RDD where each element is #' list(K, V). #' @param numPartitions Number of partitions to create. -#' @return For each element (k, v) in x, the resulting RDD will either contain -#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) #' if no elements in rdd2 have key k. #' @examples #'\dontrun{ @@ -586,11 +586,11 @@ setMethod("leftOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(FALSE, TRUE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) @@ -623,18 +623,18 @@ setMethod("rightOuterJoin", function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) - + doJoin <- function(v) { joinTaggedList(v, list(TRUE, FALSE)) } - + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) #' Full outer join two RDDs #' #' @description -#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). #' The key types of the two RDDs should be the same. #' #' @param x An RDD to be joined. Should be an RDD where each element is @@ -644,7 +644,7 @@ setMethod("rightOuterJoin", #' @param numPartitions Number of partitions to create. #' @return For each element (k, v) in x and (k, w) in y, the resulting RDD #' will contain all pairs (k, (v, w)) for both (k, v) in x and -#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements #' in x/y have key k. #' @examples #'\dontrun{ @@ -683,7 +683,7 @@ setMethod("fullOuterJoin", #' sc <- sparkR.init() #' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) #' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' cogroup(rdd1, rdd2, numPartitions = 2L) #' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) #'} #' @rdname cogroup @@ -694,7 +694,7 @@ setMethod("cogroup", rdds <- list(...) rddsLen <- length(rdds) for (i in 1:rddsLen) { - rdds[[i]] <- lapply(rdds[[i]], + rdds[[i]] <- lapply(rdds[[i]], function(x) { list(x[[1]], list(i, x[[2]])) }) } union.rdd <- Reduce(unionRDD, rdds) @@ -719,7 +719,7 @@ setMethod("cogroup", } }) } - cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), group.func) }) @@ -741,18 +741,18 @@ setMethod("sortByKey", signature(x = "RDD"), function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { rangeBounds <- list() - + if (numPartitions > 1) { rddSize <- count(x) # constant from Spark's RangePartitioner maxSampleSize <- numPartitions * 20 fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) - + # Note: the built-in R sort() function only works on atomic vectors samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) - + if (length(samples) > 0) { rangeBounds <- lapply(seq_len(numPartitions - 1), function(i) { @@ -764,24 +764,146 @@ setMethod("sortByKey", rangePartitionFunc <- function(key) { partition <- 0 - + # TODO: Use binary search instead of linear search, similar with Spark while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { partition <- partition + 1 } - + if (ascending) { partition } else { numPartitions - partition - 1 } } - + partitionFunc <- function(part) { sortKeyValueList(part, decreasing = !ascending) } - + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) +#' Subtract a pair RDD with another pair RDD. +#' +#' Return an RDD with the pairs from x whose keys are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the pairs from x whose keys are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +#' list("b", 5), list("a", 2))) +#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +#' collect(subtractByKey(rdd1, rdd2)) +#' # list(list("b", 4), list("b", 5)) +#'} +#' @rdname subtractByKey +#' @aliases subtractByKey,RDD +setMethod("subtractByKey", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + filterFunction <- function(elem) { + iters <- elem[[2]] + (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) + } + + flatMapValues(filterRDD(cogroup(x, + other, + numPartitions = numPartitions), + filterFunction), + function (v) { v[[1]] }) + }) + +#' Return a subset of this RDD sampled by key. +#' +#' @description +#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates +#' for different keys as specified by fractions, a key to sampling rate map. +#' +#' @param x The RDD to sample elements by key, where each element is +#' list(K, V) or c(K, V). +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3000) +#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +#' fractions <- list(a = 0.2, b = 0.1) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#'} +#' @rdname sampleByKey +#' @aliases sampleByKey,RDD-method +setMethod("sampleByKey", + signature(x = "RDD", withReplacement = "logical", + fractions = "vector", seed = "integer"), + function(x, withReplacement, fractions, seed) { + + for (elem in fractions) { + if (elem < 0.0) { + stop(paste("Negative fraction value ", fractions[which(fractions == elem)])) + } + } + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(split, part) { + set.seed(bitwXor(seed, split)) + res <- vector("list", length(part)) + len <- 0 + + # mixing because the initial seeds are close to each other + runif(10) + + for (elem in part) { + if (elem[[1]] %in% names(fractions)) { + frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) + if (withReplacement) { + count <- rpois(1, frac) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < frac) { + len <- len + 1 + res[[len]] <- elem + } + } + } else { + stop("KeyError: \"", elem[[1]], "\"") + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. (duplicated from sampleRDD) + if (len > 0) { + res[1:len] + } else { + list() + } + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R new file mode 100644 index 0000000000000..e442119086b17 --- /dev/null +++ b/R/pkg/R/schema.R @@ -0,0 +1,162 @@ +# +# 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. +# + +# A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField +# datatypes. These are used to create and interact with DataFrame schemas. + +#' structType +#' +#' Create a structType object that contains the metadata for a DataFrame. Intended for +#' use with createDataFrame and toDF. +#' +#' @param x a structField object (created with the field() function) +#' @param ... additional structField objects +#' @return a structType object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' schema <- structType(structField("a", "integer"), structField("b", "string")) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } +structType <- function(x, ...) { + UseMethod("structType", x) +} + +structType.jobj <- function(x) { + obj <- structure(list(), class = "structType") + obj$jobj <- x + obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } + obj +} + +structType.structField <- function(x, ...) { + fields <- list(x, ...) + if (!all(sapply(fields, inherits, "structField"))) { + stop("All arguments must be structField objects.") + } + sfObjList <- lapply(fields, function(field) { + field$jobj + }) + stObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructType", + listToSeq(sfObjList)) + structType(stObj) +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + cat("StructType\n", + sapply(x$fields(), function(field) { paste("|-", "name = \"", field$name(), + "\", type = \"", field$dataType.toString(), + "\", nullable = ", field$nullable(), "\n", + sep = "") }) + , sep = "") +} + +#' structField +#' +#' Create a structField object that contains the metadata for a single field in a schema. +#' +#' @param x The name of the field +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @return a structField object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' field1 <- structField("a", "integer", TRUE) +#' field2 <- structField("b", "string", TRUE) +#' schema <- structType(field1, field2) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } + +structField <- function(x, ...) { + UseMethod("structField", x) +} + +structField.jobj <- function(x) { + obj <- structure(list(), class = "structField") + obj$jobj <- x + obj$name <- function() { callJMethod(x, "name") } + obj$dataType <- function() { callJMethod(x, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(x, "nullable") } + obj +} + +structField.character <- function(x, type, nullable = TRUE) { + if (class(x) != "character") { + stop("Field name must be a string.") + } + if (class(type) != "character") { + stop("Field type must be a string.") + } + if (class(nullable) != "logical") { + stop("nullable must be either TRUE or FALSE") + } + options <- c("byte", + "integer", + "double", + "numeric", + "character", + "string", + "binary", + "raw", + "logical", + "boolean", + "timestamp", + "date") + dataType <- if (type %in% options) { + type + } else { + stop(paste("Unsupported type for Dataframe:", type)) + } + sfObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructField", + x, + dataType, + nullable) + structField(sfObj) +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat("StructField(name = \"", x$name(), + "\", type = \"", x$dataType.toString(), + "\", nullable = ", x$nullable(), + ")", + sep = "") +} diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 8a9c0c652ce24..c53d0a961016f 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -69,8 +69,9 @@ writeJobj <- function(con, value) { } writeString <- function(con, value) { - writeInt(con, as.integer(nchar(value) + 1)) - writeBin(value, con, endian = "big") + utfVal <- enc2utf8(value) + writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) + writeBin(utfVal, con, endian = "big") } writeInt <- function(con, value) { @@ -189,7 +190,3 @@ writeArgs <- function(con, args) { } } } - -writeStrings <- function(con, stringList) { - writeLines(unlist(stringList), con) -} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c337fb0751e72..23305d3c67074 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -465,3 +465,83 @@ cleanClosure <- function(func, checkedFuncs = new.env()) { } func } + +# Append partition lengths to each partition in two input RDDs if needed. +# param +# x An RDD. +# Other An RDD. +# return value +# A list of two result RDDs. +appendPartitionLengths <- function(x, other) { + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # know the boundary of elements from x and other. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + len <- length(part) + part[[len + 1]] <- len + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + list (x, other) +} + +# Perform zip or cartesian between elements from two RDDs in each partition +# param +# rdd An RDD. +# zip A boolean flag indicating this call is for zip operation or not. +# return value +# A result RDD. +mergePartitions <- function(rdd, zip) { + serializerMode <- getSerializedMode(rdd) + partitionFunc <- function(split, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + if (zip && lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + } else { + keys <- list() + } + if (lengthOfValues > 1) { + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + values <- list() + } + + if (!zip) { + return(mergeCompactLists(keys, values)) + } + } else { + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { list(k, v) }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(rdd, partitionFunc) +} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index b76e4db03e715..3ba7d1716302a 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -35,7 +35,7 @@ test_that("get number of partitions in RDD", { test_that("first on RDD", { expect_true(first(rdd) == 1) newrdd <- lapply(rdd, function(x) x + 1) - expect_true(first(newrdd) == 2) + expect_true(first(newrdd) == 2) }) test_that("count and length on RDD", { @@ -48,7 +48,7 @@ test_that("count by values and keys", { actual <- countByValue(mods) expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - + actual <- countByKey(intRdd) expected <- list(list(2L, 2L), list(1L, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -82,11 +82,11 @@ test_that("filterRDD on RDD", { filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) actual <- collect(filtered.rdd) expect_equal(actual, list(2, 4, 6, 8, 10)) - + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) actual <- collect(filtered.rdd) expect_equal(actual, list(list(1L, -1))) - + # Filter out all elements. filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) actual <- collect(filtered.rdd) @@ -96,7 +96,7 @@ test_that("filterRDD on RDD", { test_that("lookup on RDD", { vals <- lookup(intRdd, 1L) expect_equal(vals, list(-1, 200)) - + vals <- lookup(intRdd, 3L) expect_equal(vals, list()) }) @@ -110,7 +110,7 @@ test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { }) rdd2 <- lapply(rdd2, function(x) x + x) actual <- collect(rdd2) - expected <- list(24, 24, 24, 24, 24, + expected <- list(24, 24, 24, 24, 24, 168, 170, 172, 174, 176) expect_equal(actual, expected) }) @@ -248,10 +248,10 @@ test_that("flatMapValues() on pairwise RDDs", { l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) actual <- collect(flatMapValues(l, function(x) { x })) expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) - + # Generate x to x+1 for every value actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) - expect_equal(actual, + expect_equal(actual, list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) }) @@ -348,7 +348,7 @@ test_that("top() on RDDs", { rdd <- parallelize(sc, l) actual <- top(rdd, 6L) expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) - + l <- list("e", "d", "c", "d", "a") rdd <- parallelize(sc, l) actual <- top(rdd, 3L) @@ -358,7 +358,7 @@ test_that("top() on RDDs", { test_that("fold() on RDDs", { actual <- fold(rdd, 0, "+") expect_equal(actual, Reduce("+", nums, 0)) - + rdd <- parallelize(sc, list()) actual <- fold(rdd, 0, "+") expect_equal(actual, 0) @@ -371,7 +371,7 @@ test_that("aggregateRDD() on RDDs", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) expect_equal(actual, list(10, 4)) - + rdd <- parallelize(sc, list()) actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) expect_equal(actual, list(0, 0)) @@ -380,13 +380,13 @@ test_that("aggregateRDD() on RDDs", { test_that("zipWithUniqueId() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 3), list("c", 1), + expected <- list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) expect_equal(actual, expected) - + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -394,13 +394,13 @@ test_that("zipWithUniqueId() on RDDs", { test_that("zipWithIndex() on RDDs", { rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) - + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), + expected <- list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) expect_equal(actual, expected) }) @@ -427,12 +427,12 @@ test_that("pipeRDD() on RDDs", { actual <- collect(pipeRDD(rdd, "more")) expected <- as.list(as.character(1:10)) expect_equal(actual, expected) - + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) actual <- collect(pipeRDD(trailed.rdd, "sort")) expected <- list("", "1", "2", "3") expect_equal(actual, expected) - + rev.nums <- 9:0 rev.rdd <- parallelize(sc, rev.nums, 2L) actual <- collect(pipeRDD(rev.rdd, "sort")) @@ -446,11 +446,11 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd1, rdd2)) expect_equal(actual, list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) - + mockFile = c("Spark is pretty.", "Spark is awesome.") fileName <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName) - + rdd <- textFile(sc, fileName, 1) actual <- collect(zipRDD(rdd, rdd)) expected <- lapply(mockFile, function(x) { list(x ,x) }) @@ -465,10 +465,125 @@ test_that("zipRDD() on RDDs", { actual <- collect(zipRDD(rdd, rdd1)) expected <- lapply(mockFile, function(x) { list(x, x) }) expect_equal(actual, expected) - + + unlink(fileName) +}) + +test_that("cartesian() on RDDs", { + rdd <- parallelize(sc, 1:3) + actual <- collect(cartesian(rdd, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(1, 1), list(1, 2), list(1, 3), + list(2, 1), list(2, 2), list(2, 3), + list(3, 1), list(3, 2), list(3, 3))) + + # test case where one RDD is empty + emptyRdd <- parallelize(sc, list()) + actual <- collect(cartesian(rdd, emptyRdd)) + expect_equal(actual, list()) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + actual <- collect(cartesian(rdd, rdd)) + expected <- list( + list("Spark is awesome.", "Spark is pretty."), + list("Spark is awesome.", "Spark is awesome."), + list("Spark is pretty.", "Spark is pretty."), + list("Spark is pretty.", "Spark is awesome.")) + expect_equal(sortKeyValueList(actual), expected) + + rdd1 <- parallelize(sc, 0:1) + actual <- collect(cartesian(rdd1, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(0, "Spark is pretty."), + list(0, "Spark is awesome."), + list(1, "Spark is pretty."), + list(1, "Spark is awesome."))) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(cartesian(rdd, rdd1)) + expect_equal(sortKeyValueList(actual), expected) + unlink(fileName) }) +test_that("subtract() on RDDs", { + l <- list(1, 1, 2, 2, 3, 4) + rdd1 <- parallelize(sc, l) + + # subtract by itself + actual <- collect(subtract(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtract by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + l) + + rdd2 <- parallelize(sc, list(2, 4)) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + list(1, 1, 3)) + + l <- list("a", "a", "b", "b", "c", "d") + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list("b", "d")) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="character"))), + list("a", "a", "c")) +}) + +test_that("subtractByKey() on pairwise RDDs", { + l <- list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2)) + rdd1 <- parallelize(sc, l) + + # subtractByKey by itself + actual <- collect(subtractByKey(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtractByKey by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(l)) + + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list("b", 4), list("b", 5))) + + l <- list(list(1, 1), list(2, 4), + list(2, 5), list(1, 2)) + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list(2, 4), list(2, 5))) +}) + +test_that("intersection() on RDDs", { + # intersection with self + actual <- collect(intersection(rdd, rdd)) + expect_equal(sort(as.integer(actual)), nums) + + # intersection with an empty RDD + emptyRdd <- parallelize(sc, list()) + actual <- collect(intersection(rdd, emptyRdd)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) + rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) + actual <- collect(intersection(rdd1, rdd2)) + expect_equal(sort(as.integer(actual)), 1:3) +}) + test_that("join() on pairwise RDDs", { rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) @@ -596,9 +711,9 @@ test_that("sortByKey() on pairwise RDDs", { sortedRdd3 <- sortByKey(rdd3) actual <- collect(sortedRdd3) expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) - + # test on the boundary cases - + # boundary case 1: the RDD to be sorted has only 1 partition rdd4 <- parallelize(sc, l, 1L) sortedRdd4 <- sortByKey(rdd4) @@ -623,7 +738,7 @@ test_that("sortByKey() on pairwise RDDs", { rdd7 <- parallelize(sc, l3, 2L) sortedRdd7 <- sortByKey(rdd7) actual <- collect(sortedRdd7) - expect_equal(actual, l3) + expect_equal(actual, l3) }) test_that("collectAsMap() on a pairwise RDD", { @@ -634,12 +749,36 @@ test_that("collectAsMap() on a pairwise RDD", { rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) vals <- collectAsMap(rdd) expect_equal(vals, list(a = 1, b = 2)) - + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) vals <- collectAsMap(rdd) expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) - + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) vals <- collectAsMap(rdd) expect_equal(vals, list(`1` = "a", `2` = "b")) }) + +test_that("sampleByKey() on pairwise RDDs", { + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) + fractions <- list(a = 0.2, b = 0.1) + sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) + expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) + expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) + expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) + expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) + expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) + + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) + fractions <- list(`2` = 0.2, `3` = 0.1) + sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) + expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) + expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) + expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) + expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) + expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) +}) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R index d1da8232aea81..d7dedda553c56 100644 --- a/R/pkg/inst/tests/test_shuffle.R +++ b/R/pkg/inst/tests/test_shuffle.R @@ -87,6 +87,18 @@ test_that("combineByKey for doubles", { expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) }) +test_that("combineByKey for characters", { + stringKeyRDD <- parallelize(sc, + list(list("max", 1L), list("min", 2L), + list("other", 3L), list("max", 4L)), 2L) + reduced <- combineByKey(stringKeyRDD, + function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + test_that("aggregateByKey", { # test aggregateByKey for int keys rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index cf5cf6d1692af..25831ae2d9e18 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -44,9 +44,8 @@ test_that("infer types", { expect_equal(infer_type(list(1L, 2L)), list(type = 'array', elementType = "integer", containsNull = TRUE)) expect_equal(infer_type(list(a = 1L, b = "2")), - list(type = "struct", - fields = list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)))) + structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE))) e <- new.env() assign("a", 1L, envir = e) expect_equal(infer_type(e), @@ -54,6 +53,18 @@ test_that("infer types", { valueContainsNull = TRUE)) }) +test_that("structType and structField", { + testField <- structField("a", "string") + expect_true(inherits(testField, "structField")) + expect_true(testField$name() == "a") + expect_true(testField$nullable()) + + testSchema <- structType(testField, structField("b", "integer")) + expect_true(inherits(testSchema, "structType")) + expect_true(inherits(testSchema$fields()[[2]], "structField")) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "StringType") +}) + test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) df <- createDataFrame(sqlCtx, rdd, list("a", "b")) @@ -66,9 +77,8 @@ test_that("create DataFrame from RDD", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- createDataFrame(sqlCtx, rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -94,9 +104,8 @@ test_that("toDF", { expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("_1", "_2")) - fields <- list(list(name = "a", type = "integer", nullable = TRUE), - list(name = "b", type = "string", nullable = TRUE)) - schema <- list(type = "struct", fields = fields) + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) df <- toDF(rdd, schema) expect_true(inherits(df, "DataFrame")) expect_equal(columns(df), c("a", "b")) @@ -635,7 +644,7 @@ test_that("isLocal()", { expect_false(isLocal(df)) }) -test_that("unionAll(), subtract(), and intersect() on a DataFrame", { +test_that("unionAll(), except(), and intersect() on a DataFrame", { df <- jsonFile(sqlCtx, jsonPath) lines <- c("{\"name\":\"Bob\", \"age\":24}", @@ -650,10 +659,10 @@ test_that("unionAll(), subtract(), and intersect() on a DataFrame", { expect_true(count(unioned) == 6) expect_true(first(unioned)$name == "Michael") - subtracted <- sortDF(subtract(df, df2), desc(df$age)) + excepted <- sortDF(except(df, df2), desc(df$age)) expect_true(inherits(unioned, "DataFrame")) - expect_true(count(subtracted) == 2) - expect_true(first(subtracted)$name == "Justin") + expect_true(count(excepted) == 2) + expect_true(first(excepted)$name == "Justin") intersected <- sortDF(intersect(df, df2), df$age) expect_true(inherits(unioned, "DataFrame")) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c6542928e8ddd..014bf7bd7b3fe 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -17,6 +17,23 @@ # Worker class +# Get current system time +currentTimeSecs <- function() { + as.numeric(Sys.time()) +} + +# Get elapsed time +elapsedSecs <- function() { + proc.time()[3] +} + +# Constants +specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) + +# Timing R process boot +bootTime <- currentTimeSecs() +bootElap <- elapsedSecs() + rLibDir <- Sys.getenv("SPARKR_RLIBDIR") # Set libPaths to include SparkR package as loadNamespace needs this # TODO: Figure out if we can avoid this by not loading any objects that require @@ -37,7 +54,7 @@ serializer <- SparkR:::readString(inputCon) # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { - suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) + suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) } # read function dependencies @@ -46,6 +63,9 @@ computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) env <- environment(computeFunc) parent.env(env) <- .GlobalEnv # Attach under global environment. +# Timing init envs for computing +initElap <- elapsedSecs() + # Read and set broadcast variables numBroadcastVars <- SparkR:::readInt(inputCon) if (numBroadcastVars > 0) { @@ -56,6 +76,9 @@ if (numBroadcastVars > 0) { } } +# Timing broadcast +broadcastElap <- elapsedSecs() + # If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) @@ -73,14 +96,23 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- computeFunc(partition, data) + # Timing computing + computeElap <- elapsedSecs() + if (serializer == "byte") { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { SparkR:::writeRowSerialize(outputCon, output) } else { - SparkR:::writeStrings(outputCon, output) + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) } + # Timing output + outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -90,6 +122,8 @@ if (isEmpty != 0) { } else if (deserializer == "row") { data <- SparkR:::readDeserializeRows(inputCon) } + # Timing reading input data for execution + inputElap <- elapsedSecs() res <- new.env() @@ -107,6 +141,8 @@ if (isEmpty != 0) { res[[bucket]] <- acc } invisible(lapply(data, hashTupleToEnvir)) + # Timing computing + computeElap <- elapsedSecs() # Step 2: write out all of the environment as key-value pairs. for (name in ls(res)) { @@ -116,13 +152,26 @@ if (isEmpty != 0) { length(res[[name]]$data) <- res[[name]]$counter SparkR:::writeRawSerialize(outputCon, res[[name]]$data) } + # Timing output + outputElap <- elapsedSecs() } +} else { + inputElap <- broadcastElap + computeElap <- broadcastElap + outputElap <- broadcastElap } +# Report timing +SparkR:::writeInt(outputCon, specialLengths$TIMING_DATA) +SparkR:::writeDouble(outputCon, bootTime) +SparkR:::writeDouble(outputCon, initElap - bootElap) # init +SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast +SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input +SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute +SparkR:::writeDouble(outputCon, outputElap - computeElap) # output + # End of output -if (serializer %in% c("byte", "row")) { - SparkR:::writeInt(outputCon, 0L) -} +SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) close(outputCon) close(inputCon) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 5fa4d483b8342..6fea5e1144f2f 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -42,10 +42,15 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( rLibDir: String, broadcastVars: Array[Broadcast[Object]]) extends RDD[U](parent) with Logging { + protected var dataStream: DataInputStream = _ + private var bootTime: Double = _ override def getPartitions: Array[Partition] = parent.partitions override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) @@ -69,7 +74,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // the socket used to receive the output of task val outSocket = serverSocket.accept() val inputStream = new BufferedInputStream(outSocket.getInputStream) - val dataStream = openDataStream(inputStream) + dataStream = new DataInputStream(inputStream) serverSocket.close() try { @@ -155,6 +160,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( } else if (deserializer == SerializationFormats.ROW) { dataOut.write(elem.asInstanceOf[Array[Byte]]) } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) printOut.println(elem) } } @@ -180,9 +186,41 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( }.start() } - protected def openDataStream(input: InputStream): Closeable + protected def readData(length: Int): U - protected def read(): U + protected def read(): U = { + try { + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length >= 0 => + readData(length) + } + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } + } } /** @@ -202,31 +240,16 @@ private class PairwiseRRDD[T: ClassTag]( SerializationFormats.BYTE, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): (Int, Array[Byte]) = { - try { - val length = dataStream.readInt() - - length match { - case length if length == 2 => - val hashedKey = dataStream.readInt() - val contentPairsLength = dataStream.readInt() - val contentPairs = new Array[Byte](contentPairsLength) - dataStream.readFully(contentPairs) - (hashedKey, contentPairs) - case _ => null // End of input - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } - } + override protected def readData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null + } } lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) @@ -247,28 +270,13 @@ private class RRDD[T: ClassTag]( parent, -1, func, deserializer, serializer, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: DataInputStream = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new DataInputStream(input) - dataStream - } - - override protected def read(): Array[Byte] = { - try { - val length = dataStream.readInt() - - length match { - case length if length > 0 => - val obj = new Array[Byte](length) - dataStream.readFully(obj, 0, length) - obj - case _ => null - } - } catch { - case eof: EOFException => { - throw new SparkException("R worker exited unexpectedly (crashed)", eof) - } + override protected def readData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null } } @@ -289,26 +297,21 @@ private class StringRRDD[T: ClassTag]( parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - private var dataStream: BufferedReader = _ - - override protected def openDataStream(input: InputStream): Closeable = { - dataStream = new BufferedReader(new InputStreamReader(input)) - dataStream - } - - override protected def read(): String = { - try { - dataStream.readLine() - } catch { - case e: IOException => { - throw new SparkException("R worker exited unexpectedly (crashed)", e) - } + override protected def readData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null } } lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } +private object SpecialLengths { + val TIMING_DATA = -1 +} + private[r] class BufferedStreamThread( in: InputStream, name: String, diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index ccb2a371f4e48..371dfe454d1a2 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -85,13 +85,17 @@ private[spark] object SerDe { in.readDouble() } + def readStringBytes(in: DataInputStream, len: Int): String = { + val bytes = new Array[Byte](len) + in.readFully(bytes) + assert(bytes(len - 1) == 0) + val str = new String(bytes.dropRight(1), "UTF-8") + str + } + def readString(in: DataInputStream): String = { val len = in.readInt() - val asciiBytes = new Array[Byte](len) - in.readFully(asciiBytes) - assert(asciiBytes(len - 1) == 0) - val str = new String(asciiBytes.dropRight(1).map(_.toChar)) - str + readStringBytes(in, len) } def readBoolean(in: DataInputStream): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index d1ea7cc3e9162..ae77f72998a22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -23,7 +23,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.r.SerDe import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} private[r] object SQLUtils { @@ -39,8 +39,34 @@ private[r] object SQLUtils { arr.toSeq } - def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { - val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + def createStructType(fields : Seq[StructField]): StructType = { + StructType(fields) + } + + def getSQLDataType(dataType: String): DataType = { + dataType match { + case "byte" => org.apache.spark.sql.types.ByteType + case "integer" => org.apache.spark.sql.types.IntegerType + case "double" => org.apache.spark.sql.types.DoubleType + case "numeric" => org.apache.spark.sql.types.DoubleType + case "character" => org.apache.spark.sql.types.StringType + case "string" => org.apache.spark.sql.types.StringType + case "binary" => org.apache.spark.sql.types.BinaryType + case "raw" => org.apache.spark.sql.types.BinaryType + case "logical" => org.apache.spark.sql.types.BooleanType + case "boolean" => org.apache.spark.sql.types.BooleanType + case "timestamp" => org.apache.spark.sql.types.TimestampType + case "date" => org.apache.spark.sql.types.DateType + case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + } + } + + def createStructField(name: String, dataType: String, nullable: Boolean): StructField = { + val dtObj = getSQLDataType(dataType) + StructField(name, dtObj, nullable) + } + + def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { val num = schema.fields.size val rowRDD = rdd.map(bytesToRow) sqlContext.createDataFrame(rowRDD, schema) From d305e686b3d73213784bd75cdad7d168b22a1dc4 Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Fri, 17 Apr 2015 16:23:10 -0500 Subject: [PATCH 806/817] SPARK-6988 : Fix documentation regarding DataFrames using the Java API This patch includes : * adding how to use map after an sql query using javaRDD * fixing the first few java examples that were written in Scala Thank you for your time, Olivier. Author: Olivier Girardot Closes #5564 from ogirardot/branch-1.3 and squashes the following commits: 9f8d60e [Olivier Girardot] SPARK-6988 : Fix documentation regarding DataFrames using the Java API (cherry picked from commit 6b528dc139da594ef2e651d84bd91fe0f738a39d) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 03500867df70f..d49233714a0bb 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -193,8 +193,8 @@ df.groupBy("age").count().show()
    {% highlight java %} -val sc: JavaSparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +JavaSparkContext sc // An existing SparkContext. +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc) // Create the DataFrame DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); @@ -308,8 +308,8 @@ val df = sqlContext.sql("SELECT * FROM table")
    {% highlight java %} -val sqlContext = ... // An existing SQLContext -val df = sqlContext.sql("SELECT * FROM table") +SQLContext sqlContext = ... // An existing SQLContext +DataFrame df = sqlContext.sql("SELECT * FROM table") {% endhighlight %}
    @@ -435,7 +435,7 @@ DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AN // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -599,7 +599,7 @@ DataFrame results = sqlContext.sql("SELECT name FROM people"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. -List names = results.map(new Function() { +List names = results.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -860,7 +860,7 @@ DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } From a452c59210cf2c8ff8601cdb11401eea6dc14973 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 Apr 2015 16:30:13 -0500 Subject: [PATCH 807/817] Minor fix to SPARK-6958: Improve Python docstring for DataFrame.sort. As a follow up PR to #5544. cc davies Author: Reynold Xin Closes #5558 from rxin/sort-doc-improvement and squashes the following commits: f4c276f [Reynold Xin] Review feedback. d2dcf24 [Reynold Xin] Minor fix to SPARK-6958: Improve Python docstring for DataFrame.sort. --- python/pyspark/sql/dataframe.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 326d22e72f104..d70c5b0a6930c 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -489,8 +489,9 @@ def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). :param cols: list of :class:`Column` or column names to sort by. - :param ascending: sort by ascending order or not, could be bool, int - or list of bool, int (default: True). + :param ascending: boolean or list of boolean (default True). + Sort ascending vs. descending. Specify list for multiple sort orders. + If a list is specified, length of the list must equal length of the `cols`. >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] @@ -519,7 +520,7 @@ def sort(self, *cols, **kwargs): jcols = [jc if asc else jc.desc() for asc, jc in zip(ascending, jcols)] else: - raise TypeError("ascending can only be bool or list, but got %s" % type(ascending)) + raise TypeError("ascending can only be boolean or list, but got %s" % type(ascending)) jdf = self._jdf.sort(self._jseq(jcols)) return DataFrame(jdf, self.sql_ctx) From c5ed510135aee3a1a0402057b3b5229892aa6f3a Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Fri, 17 Apr 2015 18:28:42 -0700 Subject: [PATCH 808/817] [SPARK-6703][Core] Provide a way to discover existing SparkContext's I've added a static getOrCreate method to the static SparkContext object that allows one to either retrieve a previously created SparkContext or to instantiate a new one with the provided config. The method accepts an optional SparkConf to make usage intuitive. Still working on a test for this, basically want to create a new context from scratch, then ensure that subsequent calls don't overwrite that. Author: Ilya Ganelin Closes #5501 from ilganeli/SPARK-6703 and squashes the following commits: db9a963 [Ilya Ganelin] Closing second spark context 1dc0444 [Ilya Ganelin] Added ref equality check 8c884fa [Ilya Ganelin] Made getOrCreate synchronized cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup 270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes 15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude 0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference 733ec9f [Ilya Ganelin] Fixed some bugs in test code 8be2f83 [Ilya Ganelin] Replaced match with if e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired a99032f [Ilya Ganelin] Spacing fix d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation --- .../scala/org/apache/spark/SparkContext.scala | 49 ++++++++++++++++--- .../org/apache/spark/SparkContextSuite.scala | 20 ++++++++ project/MimaExcludes.scala | 4 ++ 3 files changed, 66 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e106c5c4bef60..86269eac52db0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -23,7 +23,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID import scala.collection.{Map, Set} @@ -1887,11 +1887,12 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ - private var activeContext: Option[SparkContext] = None + private val activeContext: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null) /** * Points to a partially-constructed SparkContext if some thread is in the SparkContext @@ -1926,7 +1927,8 @@ object SparkContext extends Logging { logWarning(warnMsg) } - activeContext.foreach { ctx => + if (activeContext.get() != null) { + val ctx = activeContext.get() val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" @@ -1941,6 +1943,39 @@ object SparkContext extends Logging { } } + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(config: SparkConf): SparkContext = { + // Synchronize to ensure that multiple create requests don't trigger an exception + // from assertNoOtherContextIsRunning within setActiveContext + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + if (activeContext.get() == null) { + setActiveContext(new SparkContext(config), allowMultipleContexts = false) + } + activeContext.get() + } + } + + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * This method allows not passing a SparkConf (useful if just retrieving). + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(): SparkContext = { + getOrCreate(new SparkConf()) + } + /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is * running. Throws an exception if a running context is detected and logs a warning if another @@ -1967,7 +2002,7 @@ object SparkContext extends Logging { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { assertNoOtherContextIsRunning(sc, allowMultipleContexts) contextBeingConstructed = None - activeContext = Some(sc) + activeContext.set(sc) } } @@ -1978,7 +2013,7 @@ object SparkContext extends Logging { */ private[spark] def clearActiveContext(): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - activeContext = None + activeContext.set(null) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 94be1c6d6397c..728558a424780 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -67,6 +67,26 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } } + test("Test getOrCreate") { + var sc2: SparkContext = null + SparkContext.clearActiveContext() + val conf = new SparkConf().setAppName("test").setMaster("local") + + sc = SparkContext.getOrCreate(conf) + + assert(sc.getConf.get("spark.app.name").equals("test")) + sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) + assert(sc2.getConf.get("spark.app.name").equals("test")) + assert(sc === sc2) + assert(sc eq sc2) + + // Try creating second context to confirm that it's still possible, if desired + sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true")) + + sc2.stop() + } + test("BytesWritable implicit conversion is correct") { // Regression test for SPARK-3121 val bytesWritable = new BytesWritable() diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1564babefa62f..7ef363a2f07ad 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,6 +68,10 @@ object MimaExcludes { // SPARK-6693 add tostring with max lines and width for matrix ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.toString") + )++ Seq( + // SPARK-6703 Add getOrCreate method to SparkContext + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkContext.org$apache$spark$SparkContext$$activeContext") ) case v if v.startsWith("1.3") => From 6fbeb82e13db7117d8f216e6148632490a4bc5be Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Fri, 17 Apr 2015 18:30:55 -0700 Subject: [PATCH 809/817] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md Author: Jongyoul Lee Closes #5063 from jongyoul/SPARK-6350 and squashes the following commits: 9238d6e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs - Changed configuration name - Made mesosExecutorCores private 2d41241 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 89edb4f [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 8ba7694 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Fixed docs 7549314 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed docs 4ae7b0c [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Removed TODO c27efce [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Fixed Mesos*Suite for supporting integer WorkerOffers - Fixed Documentation 1fe4c03 [Jongyoul Lee] [SPARK-6453][Mesos] Some Mesos*Suite have a different package with their classes - Change available resources of cpus to integer value beacuse WorkerOffer support the amount cpus as integer value 5f3767e [Jongyoul Lee] Revert "[SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode" 4b7c69e [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Changed configruation name and description from "spark.mesos.executor.cores" to "spark.executor.frameworkCores" 0556792 [Jongyoul Lee] [SPARK-6350][Mesos] Make mesosExecutorCores configurable in mesos "fine-grained" mode - Defined executorCores from "spark.mesos.executor.cores" - Changed the amount of mesosExecutor's cores to executorCores. - Added new configuration option on running-on-mesos.md --- .../cluster/mesos/MesosSchedulerBackend.scala | 14 +++++++------- .../cluster/mesos/MesosSchedulerBackendSuite.scala | 4 ++-- docs/running-on-mesos.md | 10 ++++++++++ 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index b381436839227..d9d62b0e287ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -67,6 +67,8 @@ private[spark] class MesosSchedulerBackend( // The listener bus to publish executor added/removed events. val listenerBus = sc.listenerBus + + private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) @volatile var appId: String = _ @@ -139,7 +141,7 @@ private[spark] class MesosSchedulerBackend( .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder() - .setValue(scheduler.CPUS_PER_TASK).build()) + .setValue(mesosExecutorCores).build()) .build() val memory = Resource.newBuilder() .setName("mem") @@ -220,10 +222,9 @@ private[spark] class MesosSchedulerBackend( val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task - cpus >= 2 * scheduler.CPUS_PER_TASK) || + cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)) || (slaveIdsWithExecutors.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) } @@ -232,10 +233,9 @@ private[spark] class MesosSchedulerBackend( val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { - // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? - getResource(o.getResourcesList, "cpus").toInt - - scheduler.CPUS_PER_TASK + // If the Mesos executor has not been started on this slave yet, set aside a few + // cores for the Mesos executor by offering fewer cores to the Spark executor + (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt } new WorkerOffer( o.getSlaveId.getValue, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index a311512e82c5e..cdd7be0fbe5dd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -118,12 +118,12 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(0).getSlaveId.getValue, mesosOffers.get(0).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(2).getSlaveId.getValue, mesosOffers.get(2).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index c984639bd34cf..594bf78b67713 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -210,6 +210,16 @@ See the [configuration page](configuration.html) for information on Spark config Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. +
    + + + + From 1991337336596f94698e79c2366f065c374128ab Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Apr 2015 19:02:07 -0700 Subject: [PATCH 810/817] [SPARK-5933] [core] Move config deprecation warnings to SparkConf. I didn't find many deprecated configs after a grep-based search, but the ones I could find were moved to the centralized location in SparkConf. While there, I deprecated a couple more HS configs that mentioned time units. Author: Marcelo Vanzin Closes #5562 from vanzin/SPARK-5933 and squashes the following commits: dcb617e7 [Marcelo Vanzin] [SPARK-5933] [core] Move config deprecation warnings to SparkConf. --- .../main/scala/org/apache/spark/SparkConf.scala | 17 ++++++++++++++--- .../main/scala/org/apache/spark/SparkEnv.scala | 10 ++-------- .../deploy/history/FsHistoryProvider.scala | 15 +++------------ .../scala/org/apache/spark/SparkConfSuite.scala | 3 +++ docs/monitoring.md | 15 +++++++-------- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +-------- 6 files changed, 30 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b0186e9a007b8..e3a649d755450 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -403,6 +403,9 @@ private[spark] object SparkConf extends Logging { */ private val deprecatedConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( + DeprecatedConfig("spark.cache.class", "0.8", + "The spark.cache.class property is no longer being used! Specify storage levels using " + + "the RDD.persist() method instead."), DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", "Please use spark.{driver,executor}.userClassPathFirst instead.")) Map(configs.map { cfg => (cfg.key -> cfg) }:_*) @@ -420,7 +423,15 @@ private[spark] object SparkConf extends Logging { "spark.history.fs.update.interval" -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), AlternateConfig("spark.history.fs.updateInterval", "1.3"), - AlternateConfig("spark.history.updateInterval", "1.3")) + AlternateConfig("spark.history.updateInterval", "1.3")), + "spark.history.fs.cleaner.interval" -> Seq( + AlternateConfig("spark.history.fs.cleaner.interval.seconds", "1.4")), + "spark.history.fs.cleaner.maxAge" -> Seq( + AlternateConfig("spark.history.fs.cleaner.maxAge.seconds", "1.4")), + "spark.yarn.am.waitTime" -> Seq( + AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", + // Translate old value to a duration, with 10s wait time per try. + translation = s => s"${s.toLong * 10}s")) ) /** @@ -470,7 +481,7 @@ private[spark] object SparkConf extends Logging { configsWithAlternatives.get(key).flatMap { alts => alts.collectFirst { case alt if conf.contains(alt.key) => val value = conf.get(alt.key) - alt.translation.map(_(value)).getOrElse(value) + if (alt.translation != null) alt.translation(value) else value } } } @@ -514,6 +525,6 @@ private[spark] object SparkConf extends Logging { private case class AlternateConfig( key: String, version: String, - translation: Option[String => String] = None) + translation: String => String = null) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 0171488e09562..959aefabd8de4 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -103,7 +103,7 @@ class SparkEnv ( // actorSystem.awaitTermination() // Note that blockTransferService is stopped by BlockManager since it is started by it. - + // If we only stop sc, but the driver process still run as a services then we need to delete // the tmp dir, if not, it will create too many tmp dirs. // We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the @@ -375,12 +375,6 @@ object SparkEnv extends Logging { "." } - // Warn about deprecated spark.cache.class property - if (conf.contains("spark.cache.class")) { - logWarning("The spark.cache.class property is no longer being used! Specify storage " + - "levels using the RDD.persist() method instead.") - } - val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { new OutputCommitCoordinator(conf) } @@ -406,7 +400,7 @@ object SparkEnv extends Logging { shuffleMemoryManager, outputCommitCoordinator, conf) - + // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is // called, and we only need to do it for driver. Because driver may run as a service, and if we // don't delete this tmp dir when sc is stopped, then will create too many tmp dirs. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 985545742df67..47bdd7749ec3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -52,8 +52,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") // Interval between each cleaner checks for event logs to delete - private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", - DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S) * 1000 + private val CLEAN_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.cleaner.interval", "1d") private val logDir = conf.getOption("spark.history.fs.logDirectory") .map { d => Utils.resolveURI(d).toString } @@ -130,8 +129,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { // A task that periodically cleans event logs on disk. - pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS) } } } @@ -270,8 +268,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) .getOrElse(Seq[FileStatus]()) - val maxAge = conf.getLong("spark.history.fs.cleaner.maxAge.seconds", - DEFAULT_SPARK_HISTORY_FS_MAXAGE_S) * 1000 + val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 val now = System.currentTimeMillis() val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() @@ -417,12 +414,6 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private object FsHistoryProvider { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - - // One day - val DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S = Duration(1, TimeUnit.DAYS).toSeconds - - // One week - val DEFAULT_SPARK_HISTORY_FS_MAXAGE_S = Duration(7, TimeUnit.DAYS).toSeconds } private class FsApplicationHistoryInfo( diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 7d87ba5fd2610..8e6c200c4ba00 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -217,6 +217,9 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size assert(count === 4) + + conf.set("spark.yarn.applicationMaster.waitTries", "42") + assert(conf.getTimeAsSeconds("spark.yarn.am.waitTime") === 420) } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 2a130224591ca..8a85928d6d44d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -153,19 +153,18 @@ follows: - - + + - - + +
    @@ -223,12 +225,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { (driver.state == DriverState.RUNNING || driver.state == DriverState.SUBMITTED || driver.state == DriverState.RELAUNCHING)) { - val killLinkUri = s"driver/kill?id=${driver.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill driver ${driver.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill driver ${driver.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) +
    }
    {driver.id} {killLink}
    spark.history.fs.update.interval.seconds10spark.history.fs.update.interval10s - The period, in seconds, at which information displayed by this history server is updated. + The period at which information displayed by this history server is updated. Each update checks for any changes made to the event logs in persisted storage.
    spark.mesos.mesosExecutor.cores1.0 + (Fine-grained mode only) Number of cores to give each Mesos executor. This does not + include the cores used to run the Spark tasks. In other words, even if no Spark task + is being run, each Mesos executor will occupy the number of cores configured here. + The value can be a floating point number. +
    spark.mesos.executor.home driver side SPARK_HOME
    spark.history.fs.cleaner.interval.seconds86400spark.history.fs.cleaner.interval1d - How often the job history cleaner checks for files to delete, in seconds. Defaults to 86400 (one day). - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.seconds. + How often the job history cleaner checks for files to delete. + Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.
    spark.history.fs.cleaner.maxAge.seconds3600 * 24 * 7spark.history.fs.cleaner.maxAge7d - Job history files older than this many seconds will be deleted when the history cleaner runs. - Defaults to 3600 * 24 * 7 (1 week). + Job history files older than this will be deleted when the history cleaner runs.
    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 c357b7ae9d4da..f7a84207e9da6 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 @@ -373,14 +373,7 @@ private[spark] class ApplicationMaster( private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") sparkContextRef.synchronized { - val waitTries = sparkConf.getOption("spark.yarn.applicationMaster.waitTries") - .map(_.toLong * 10000L) - if (waitTries.isDefined) { - logWarning( - "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") - } - val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", - s"${waitTries.getOrElse(100000L)}ms") + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { From d850b4bd3a294dd245881e03f7f94bf970a7ee79 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 17 Apr 2015 19:17:06 -0700 Subject: [PATCH 811/817] [SPARK-6975][Yarn] Fix argument validation error `numExecutors` checking is failed when dynamic allocation is enabled with default configuration. Details can be seen is [SPARK-6975](https://issues.apache.org/jira/browse/SPARK-6975). sryza, please help me to review this, not sure is this the correct way, I think previous you change this part :) Author: jerryshao Closes #5551 from jerryshao/SPARK-6975 and squashes the following commits: 4335da1 [jerryshao] Change according to the comments 77bdcbd [jerryshao] Fix argument validation error --- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index da6798cb1b279..1423533470fc0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -103,9 +103,13 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) * This is intended to be called only after the provided arguments have been parsed. */ private def validateArgs(): Unit = { - if (numExecutors <= 0) { + if (numExecutors < 0 || (!isDynamicAllocationEnabled && numExecutors == 0)) { throw new IllegalArgumentException( - "You must specify at least 1 executor!\n" + getUsageMessage()) + s""" + |Number of executors was $numExecutors, but must be at least 1 + |(or 0 if dynamic executor allocation is enabled). + |${getUsageMessage()} + """.stripMargin) } if (executorCores < sparkConf.getInt("spark.task.cpus", 1)) { throw new SparkException("Executor cores must not be less than " + From 5f095d56054d57c54d81db1d36cd46312810fb6a Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Sat, 18 Apr 2015 00:31:01 -0700 Subject: [PATCH 812/817] SPARK-6992 : Fix documentation example for Spark SQL on StructType This patch is fixing the Java examples for Spark SQL when defining programmatically a Schema and mapping Rows. Author: Olivier Girardot Closes #5569 from ogirardot/branch-1.3 and squashes the following commits: c29e58d [Olivier Girardot] SPARK-6992 : Fix documentation example for Spark SQL on StructType (cherry picked from commit c9b1ba4b16a7afe93d45bf75b128cc0dd287ded0) Signed-off-by: Reynold Xin --- docs/sql-programming-guide.md | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d49233714a0bb..b2022546268a7 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -555,13 +555,16 @@ by `SQLContext`. For example: {% highlight java %} -// Import factory methods provided by DataType. -import org.apache.spark.sql.types.DataType; +import org.apache.spark.api.java.function.Function; +// Import factory methods provided by DataTypes. +import org.apache.spark.sql.types.DataTypes; // Import StructType and StructField import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.StructField; // Import Row. import org.apache.spark.sql.Row; +// Import RowFactory. +import org.apache.spark.sql.RowFactory; // sc is an existing JavaSparkContext. SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); @@ -575,16 +578,16 @@ String schemaString = "name age"; // Generate the schema based on the string of schema List fields = new ArrayList(); for (String fieldName: schemaString.split(" ")) { - fields.add(DataType.createStructField(fieldName, DataType.StringType, true)); + fields.add(DataTypes.createStructField(fieldName, DataTypes.StringType, true)); } -StructType schema = DataType.createStructType(fields); +StructType schema = DataTypes.createStructType(fields); // Convert records of the RDD (people) to Rows. JavaRDD rowRDD = people.map( new Function() { public Row call(String record) throws Exception { String[] fields = record.split(","); - return Row.create(fields[0], fields[1].trim()); + return RowFactory.create(fields[0], fields[1].trim()); } }); From 327ebf0cb5e236579bece057eda27b21aed0e2dc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 18 Apr 2015 10:14:56 +0100 Subject: [PATCH 813/817] [core] [minor] Make sure ConnectionManager stops. My previous fix (force a selector wakeup) didn't seem to work since I ran into the hang again. So change the code a bit to be more explicit about the condition when the selector thread should exit. Author: Marcelo Vanzin Closes #5566 from vanzin/conn-mgr-hang and squashes the following commits: ddb2c03 [Marcelo Vanzin] [core] [minor] Make sure ConnectionManager stops. --- .../spark/network/nio/ConnectionManager.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 5a74c13b38bf7..1a68e621eaee7 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -188,6 +188,7 @@ private[nio] class ConnectionManager( private val writeRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() + @volatile private var isActive = true private val selectorThread = new Thread("connection-manager-thread") { override def run(): Unit = ConnectionManager.this.run() } @@ -342,7 +343,7 @@ private[nio] class ConnectionManager( def run() { try { - while(!selectorThread.isInterrupted) { + while (isActive) { while (!registerRequests.isEmpty) { val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) @@ -398,7 +399,7 @@ private[nio] class ConnectionManager( } catch { // Explicitly only dealing with CancelledKeyException here since other exceptions // should be dealt with differently. - case e: CancelledKeyException => { + case e: CancelledKeyException => // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -420,8 +421,11 @@ private[nio] class ConnectionManager( } } } - } - 0 + 0 + + case e: ClosedSelectorException => + logDebug("Failed select() as selector is closed.", e) + return } if (selectedKeysCount == 0) { @@ -988,11 +992,11 @@ private[nio] class ConnectionManager( } def stop() { + isActive = false ackTimeoutMonitor.stop() - selector.wakeup() + selector.close() selectorThread.interrupt() selectorThread.join() - selector.close() val connections = connectionsByKey.values connections.foreach(_.close()) if (connectionsByKey.size != 0) { From 28683b4df5de06373b867068b9b8adfbcaf93176 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 18 Apr 2015 16:46:28 -0700 Subject: [PATCH 814/817] [SPARK-6219] Reuse pep8.py Per the discussion in the comments on [this commit](https://github.com/apache/spark/commit/f17d43b033d928dbc46aef8e367aa08902e698ad#commitcomment-10780649), this PR allows the Python lint script to reuse `pep8.py` when possible. Author: Nicholas Chammas Closes #5561 from nchammas/save-dem-pep8-bytes and squashes the following commits: b7c91e6 [Nicholas Chammas] reuse pep8.py --- dev/.gitignore | 1 + dev/lint-python | 21 +++++++++++---------- 2 files changed, 12 insertions(+), 10 deletions(-) create mode 100644 dev/.gitignore diff --git a/dev/.gitignore b/dev/.gitignore new file mode 100644 index 0000000000000..4a6027429e0d3 --- /dev/null +++ b/dev/.gitignore @@ -0,0 +1 @@ +pep8*.py diff --git a/dev/lint-python b/dev/lint-python index fded654893a7c..f50d149dc4d44 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -32,18 +32,19 @@ compile_status="${PIPESTATUS[0]}" #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 #+ TODOs: #+ - Download pep8 from PyPI. It's more "official". -PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" -PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.6.2/pep8.py" +PEP8_VERSION="1.6.2" +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8-$PEP8_VERSION.py" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/$PEP8_VERSION/pep8.py" -# if [ ! -e "$PEP8_SCRIPT_PATH" ]; then -curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" -curl_status="$?" +if [ ! -e "$PEP8_SCRIPT_PATH" ]; then + curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" + curl_status="$?" -if [ "$curl_status" -ne 0 ]; then - echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." - exit "$curl_status" + if [ "$curl_status" -ne 0 ]; then + echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." + exit "$curl_status" + fi fi -# fi # There is no need to write this output to a file #+ first, but we do so so that the check status can @@ -65,7 +66,7 @@ else echo "Python lint checks passed." fi -rm "$PEP8_SCRIPT_PATH" +# rm "$PEP8_SCRIPT_PATH" rm "$PYTHON_LINT_REPORT_PATH" exit "$lint_status" From 729885ec6b4be61144d04821f1a6e8d2134eea00 Mon Sep 17 00:00:00 2001 From: Gaurav Nanda Date: Sat, 18 Apr 2015 17:20:46 -0700 Subject: [PATCH 815/817] Fixed doc Just fixed a doc. Author: Gaurav Nanda Closes #5576 from gaurav324/master and squashes the following commits: 8a7323f [Gaurav Nanda] Fixed doc --- docs/mllib-linear-methods.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 9270741d439d9..2b2be4d9d0273 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -377,7 +377,7 @@ references. Here is an [detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). -For multiclass classification problems, the algorithm will outputs a multinomial logistic regression +For multiclass classification problems, the algorithm will output a multinomial logistic regression model, which contains $K - 1$ binary logistic regression models regressed against the first class. Given a new data points, $K - 1$ models will be run, and the class with largest probability will be chosen as the predicted class. From 8fbd45c74e762dd6b071ea58a60f5bb649f74042 Mon Sep 17 00:00:00 2001 From: Olivier Girardot Date: Sat, 18 Apr 2015 18:21:44 -0700 Subject: [PATCH 816/817] SPARK-6993 : Add default min, max methods for JavaDoubleRDD The default method will use Guava's Ordering instead of java.util.Comparator.naturalOrder() because it's not available in Java 7, only in Java 8. Author: Olivier Girardot Closes #5571 from ogirardot/master and squashes the following commits: 7fe2e9e [Olivier Girardot] SPARK-6993 : Add default min, max methods for JavaDoubleRDD --- .../org/apache/spark/api/java/JavaDoubleRDD.scala | 14 ++++++++++++++ .../test/java/org/apache/spark/JavaAPISuite.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 79e4ebf2db578..61af867b11b9c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -163,6 +163,20 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) /** Add up the elements in this RDD. */ def sum(): JDouble = srdd.sum() + /** + * Returns the minimum element from this RDD as defined by + * the default comparator natural order. + * @return the minimum of the RDD + */ + def min(): JDouble = min(com.google.common.collect.Ordering.natural()) + + /** + * Returns the maximum element from this RDD as defined by + * the default comparator natural order. + * @return the maximum of the RDD + */ + def max(): JDouble = max(com.google.common.collect.Ordering.natural()) + /** * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and * count of the RDD's elements in one operation. diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index d4b5bb519157c..8a4f2a08fe701 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -761,6 +761,20 @@ public void min() { Assert.assertEquals(1.0, max, 0.001); } + @Test + public void naturalMax() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(); + Assert.assertTrue(4.0 == max); + } + + @Test + public void naturalMin() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(); + Assert.assertTrue(1.0 == max); + } + @Test public void takeOrdered() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); From 0424da68d4c81dc3a9944d8485feb1233c6633c4 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 19 Apr 2015 09:37:09 +0100 Subject: [PATCH 817/817] [SPARK-6963][CORE]Flaky test: o.a.s.ContextCleanerSuite automatically cleanup checkpoint cc andrewor14 Author: GuoQiang Li Closes #5548 from witgo/SPARK-6963 and squashes the following commits: 964aea7 [GuoQiang Li] review commits b08b3c9 [GuoQiang Li] Flaky test: o.a.s.ContextCleanerSuite automatically cleanup checkpoint --- .../org/apache/spark/ContextCleaner.scala | 2 ++ .../apache/spark/ContextCleanerSuite.scala | 21 +++++++++++++------ 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 715b259057569..37198d887b07b 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -236,6 +236,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { try { logDebug("Cleaning rdd checkpoint data " + rddId) RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + listeners.foreach(_.checkpointCleaned(rddId)) logInfo("Cleaned rdd checkpoint data " + rddId) } catch { @@ -260,4 +261,5 @@ private[spark] trait CleanerListener { def shuffleCleaned(shuffleId: Int) def broadcastCleaned(broadcastId: Long) def accumCleaned(accId: Long) + def checkpointCleaned(rddId: Long) } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 097e7076e5391..c7868ddcf770f 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -224,7 +224,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(fs.exists(path)) // the checkpoint is not cleaned by default (without the configuration set) - var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Nil) rdd = null // Make RDD out of scope runGC() postGCTester.assertCleanup() @@ -245,7 +245,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) // Test that GC causes checkpoint data cleanup after dereferencing the RDD - postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) rdd = null // Make RDD out of scope runGC() postGCTester.assertCleanup() @@ -406,12 +406,14 @@ class CleanerTester( sc: SparkContext, rddIds: Seq[Int] = Seq.empty, shuffleIds: Seq[Int] = Seq.empty, - broadcastIds: Seq[Long] = Seq.empty) + broadcastIds: Seq[Long] = Seq.empty, + checkpointIds: Seq[Long] = Seq.empty) extends Logging { val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds + val toBeCheckpointIds = new HashSet[Long] with SynchronizedSet[Long] ++= checkpointIds val isDistributed = !sc.isLocal val cleanerListener = new CleanerListener { @@ -427,12 +429,17 @@ class CleanerTester( def broadcastCleaned(broadcastId: Long): Unit = { toBeCleanedBroadcstIds -= broadcastId - logInfo("Broadcast" + broadcastId + " cleaned") + logInfo("Broadcast " + broadcastId + " cleaned") } def accumCleaned(accId: Long): Unit = { logInfo("Cleaned accId " + accId + " cleaned") } + + def checkpointCleaned(rddId: Long): Unit = { + toBeCheckpointIds -= rddId + logInfo("checkpoint " + rddId + " cleaned") + } } val MAX_VALIDATION_ATTEMPTS = 10 @@ -456,7 +463,8 @@ class CleanerTester( /** Verify that RDDs, shuffles, etc. occupy resources */ private def preCleanupValidate() { - assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty || + checkpointIds.nonEmpty, "Nothing to cleanup") // Verify the RDDs have been persisted and blocks are present rddIds.foreach { rddId => @@ -547,7 +555,8 @@ class CleanerTester( private def isAllCleanedUp = toBeCleanedRDDIds.isEmpty && toBeCleanedShuffleIds.isEmpty && - toBeCleanedBroadcstIds.isEmpty + toBeCleanedBroadcstIds.isEmpty && + toBeCheckpointIds.isEmpty private def getRDDBlocks(rddId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match {